diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
index 227e5e5af3..77078046dd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
@@ -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,122 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
}
+ 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-state-rows-timeline",
+ "aggregated-num-total-state-rows-histogram",
+ numRowsTotalData,
+ minBatchTime,
+ maxBatchTime,
+ 0,
+ maxNumRowsTotal,
+ "records")
+ graphUIDataForNumberTotalRows.generateDataJs(jsCollector)
+
+ val graphUIDataForNumberUpdatedRows =
+ new GraphUIData(
+ "aggregated-num-updated-state-rows-timeline",
+ "aggregated-num-updated-state-rows-histogram",
+ numRowsUpdatedData,
+ minBatchTime,
+ maxBatchTime,
+ 0,
+ maxNumRowsUpdated,
+ "records")
+ graphUIDataForNumberUpdatedRows.generateDataJs(jsCollector)
+
+ val graphUIDataForMemoryUsedBytes =
+ new GraphUIData(
+ "aggregated-state-memory-used-bytes-timeline",
+ "aggregated-state-memory-used-bytes-histogram",
+ memoryUsedBytesData,
+ minBatchTime,
+ maxBatchTime,
+ 0,
+ maxMemoryUsedBytes,
+ "bytes")
+ graphUIDataForMemoryUsedBytes.generateDataJs(jsCollector)
+
+ val graphUIDataForNumRowsDroppedByWatermark =
+ new GraphUIData(
+ "aggregated-num-state-rows-dropped-by-watermark-timeline",
+ "aggregated-num-state-rows-dropped-by-watermark-histogram",
+ numRowsDroppedByWatermarkData,
+ minBatchTime,
+ maxBatchTime,
+ 0,
+ maxNumRowsDroppedByWatermark,
+ "records")
+ graphUIDataForNumRowsDroppedByWatermark.generateDataJs(jsCollector)
+
+ // scalastyle:off
+