-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-33223][SS][UI]Structured Streaming Web UI state information #30151
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 5 commits
eb581b8
ba23c1a
4adc856
85b1c08
3d11793
291cf8a
02a14fc
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,7 +22,7 @@ import java.lang.{Long => JLong} | |
| import java.util.UUID | ||
| import javax.servlet.http.HttpServletRequest | ||
|
|
||
| import scala.xml.{Node, Unparsed} | ||
| import scala.xml.{Node, NodeBuffer, Unparsed} | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.sql.streaming.ui.UIUtils._ | ||
|
|
@@ -126,6 +126,123 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) | |
| <br /> | ||
| } | ||
|
|
||
| def generateAggregatedStateOperators( | ||
| query: StreamingQueryUIData, | ||
| minBatchTime: Long, | ||
| maxBatchTime: Long, | ||
| jsCollector: JsCollector | ||
| ): NodeBuffer = { | ||
| // This is made sure on caller side but put it here to be defensive | ||
| require(query.lastProgress != null) | ||
| if (query.lastProgress.stateOperators.nonEmpty) { | ||
| val numRowsTotalData = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp), | ||
| p.stateOperators.map(_.numRowsTotal).sum.toDouble)) | ||
| val maxNumRowsTotal = numRowsTotalData.maxBy(_._2)._2 | ||
|
|
||
| val numRowsUpdatedData = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp), | ||
| p.stateOperators.map(_.numRowsUpdated).sum.toDouble)) | ||
| val maxNumRowsUpdated = numRowsUpdatedData.maxBy(_._2)._2 | ||
|
|
||
| val memoryUsedBytesData = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp), | ||
| p.stateOperators.map(_.memoryUsedBytes).sum.toDouble)) | ||
| val maxMemoryUsedBytes = memoryUsedBytesData.maxBy(_._2)._2 | ||
|
|
||
| val numRowsDroppedByWatermarkData = query.recentProgress | ||
| .map(p => (parseProgressTimestamp(p.timestamp), | ||
| p.stateOperators.map(_.numRowsDroppedByWatermark).sum.toDouble)) | ||
| val maxNumRowsDroppedByWatermark = numRowsDroppedByWatermarkData.maxBy(_._2)._2 | ||
|
|
||
| val graphUIDataForNumberTotalRows = | ||
| new GraphUIData( | ||
| "aggregated-num-total-rows-timeline", | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| "aggregated-num-total-rows-histogram", | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| numRowsTotalData, | ||
| minBatchTime, | ||
| maxBatchTime, | ||
| 0, | ||
| maxNumRowsTotal, | ||
| "records") | ||
| graphUIDataForNumberTotalRows.generateDataJs(jsCollector) | ||
|
|
||
| val graphUIDataForNumberUpdatedRows = | ||
| new GraphUIData( | ||
| "aggregated-num-updated-rows-timeline", | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| "aggregated-num-updated-rows-histogram", | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| numRowsUpdatedData, | ||
| minBatchTime, | ||
| maxBatchTime, | ||
| 0, | ||
| maxNumRowsUpdated, | ||
| "records") | ||
| graphUIDataForNumberUpdatedRows.generateDataJs(jsCollector) | ||
|
|
||
| val graphUIDataForMemoryUsedBytes = | ||
| new GraphUIData( | ||
| "aggregated-memory-used-bytes-timeline", | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| "aggregated-memory-used-bytes-histogram", | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| memoryUsedBytesData, | ||
| minBatchTime, | ||
| maxBatchTime, | ||
| 0, | ||
| maxMemoryUsedBytes, | ||
| "bytes") | ||
| graphUIDataForMemoryUsedBytes.generateDataJs(jsCollector) | ||
|
|
||
| val graphUIDataForNumRowsDroppedByWatermark = | ||
| new GraphUIData( | ||
| "aggregated-num-rows-dropped-by-watermark-timeline", | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| "aggregated-num-rows-dropped-by-watermark-histogram", | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| numRowsDroppedByWatermarkData, | ||
| minBatchTime, | ||
| maxBatchTime, | ||
| 0, | ||
| maxNumRowsDroppedByWatermark, | ||
| "records") | ||
| graphUIDataForNumRowsDroppedByWatermark.generateDataJs(jsCollector) | ||
|
|
||
| // scalastyle:off | ||
| <tr> | ||
| <td style="vertical-align: middle;"> | ||
| <div style="width: 160px;"> | ||
| <div><strong>Aggregated Number Of Total State Rows {SparkUIUtils.tooltip("Aggregated number of total state rows.", "right")}</strong></div> | ||
| </div> | ||
| </td> | ||
| <td class={"aggregated-num-total-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td> | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| <td class={"aggregated-num-total-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td> | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| </tr> | ||
| <tr> | ||
| <td style="vertical-align: middle;"> | ||
| <div style="width: 160px;"> | ||
| <div><strong>Aggregated Number Of Updated State Rows {SparkUIUtils.tooltip("Aggregated number of updated state rows.", "right")}</strong></div> | ||
| </div> | ||
| </td> | ||
| <td class={"aggregated-num-updated-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td> | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| <td class={"aggregated-num-updated-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td> | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| </tr> | ||
| <tr> | ||
| <td style="vertical-align: middle;"> | ||
| <div style="width: 160px;"> | ||
| <div><strong>Aggregated State Memory Used In Bytes {SparkUIUtils.tooltip("Aggregated state memory used in bytes.", "right")}</strong></div> | ||
| </div> | ||
| </td> | ||
| <td class={"aggregated-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td> | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| <td class={"aggregated-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td> | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| </tr> | ||
| <tr> | ||
| <td style="vertical-align: middle;"> | ||
| <div style="width: 160px;"> | ||
| <div><strong>Aggregated Number Of State Rows Dropped By Watermark {SparkUIUtils.tooltip("Aggregated number of state rows dropped by watermark.", "right")}</strong></div> | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I figured out during work on mine - the rows dropped by watermark are not from state. It's a bit confusing, but they're input rows for "stateful operators". I'll make a follow-up PR to correct this.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
| </div> | ||
| </td> | ||
| <td class={"aggregated-num-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td> | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| <td class={"aggregated-num-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td> | ||
gaborgsomogyi marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| </tr> | ||
| // scalastyle:on | ||
| } else { | ||
| new NodeBuffer() | ||
| } | ||
| } | ||
|
|
||
| def generateStatTable(query: StreamingQueryUIData): Seq[Node] = { | ||
| val batchToTimestamps = withNoProgress(query, | ||
| query.recentProgress.map(p => (p.batchId, parseProgressTimestamp(p.timestamp))), | ||
|
|
@@ -284,6 +401,7 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) | |
| </td> | ||
| <td class="duration-area-stack" colspan="2">{graphUIDataForDuration.generateAreaStackHtmlWithData(jsCollector, operationDurationData)}</td> | ||
| </tr> | ||
| {generateAggregatedStateOperators(query, minBatchTime, maxBatchTime, jsCollector)} | ||
| </tbody> | ||
| </table> | ||
| } else { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.