You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/11/11 12:53:37 UTC

[GitHub] [spark] gaborgsomogyi opened a new pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

gaborgsomogyi opened a new pull request #30336:
URL: https://github.com/apache/spark/pull/30336


   ### What changes were proposed in this pull request?
   Structured Streaming UI is not containing state custom metrics information. In this PR I've added it.
   
   ### Why are the changes needed?
   Missing state custom metrics information.
   
   ### Does this PR introduce _any_ user-facing change?
   Additional UI elements appear.
   
   ### How was this patch tested?
   Existing unit tests + manual test.
   <img width="1128" alt="Screenshot 2020-11-11 at 10 44 55" src="https://user-images.githubusercontent.com/18561820/98813627-c7890380-2424-11eb-8ac3-d80a466d28da.png">
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725459670






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522658041



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +202,98 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+   query: StreamingQueryUIData,
+   minBatchTime: Long,
+   maxBatchTime: Long,
+   jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val disabledCustomMetrics = Utils.stringToSeq(SQLConf.get.disabledStreamingCustomMetrics)

Review comment:
       If there's a custom metric about "latency", avg is probably the better aggregation instead of sum, though in such case we actually want the histogram with various percentiles. Let's restrict the aggregation to only "sum" with explanation on the config.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r526560274



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,11 +178,15 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
+  def unit: String
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
+case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)

Review comment:
       How about setting empty string as the default value for `unit` ? This can resolve the compatibility issue.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729902709


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35900/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729860594


   retest this please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726780931


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35668/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731028284


   **[Test build #131412 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131412/testReport)** for PR 30336 at commit [`9a87255`](https://github.com/apache/spark/commit/9a872550908d4824db0272a6301d45fda62f6cbf).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728853290


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522908513



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -2875,6 +2875,15 @@ object SQLConf {
     .stringConf
     .createWithDefault("")
 
+  val DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST =
+    buildConf("spark.sql.streaming.disabledUICustomMetricsList")
+      .internal()
+      .doc("Configures a list of custom metrics on Structured Streaming UI, which are disabled. " +
+        "The list contains the name of the custom metrics separated by comma.")
+      .version("3.1.0")
+      .stringConf

Review comment:
       OK I found an example.
   
   ```
     val IMAGE_PULL_SECRETS =
       ConfigBuilder("spark.kubernetes.container.image.pullSecrets")
         .doc("Comma separated list of the Kubernetes secrets used " +
           "to access private image registries.")
         .version("2.4.0")
         .stringConf
         .toSequence
         .createWithDefault(Nil)
   ```
   
   `.toSequence` will do the trick.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729858484


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35897/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xuanyuanking commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r526564813



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,11 +178,15 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
+  def unit: String
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
+case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)

Review comment:
       Yeah agree, empty string can also resolve the issue.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729807283


   Hmm, I'll wipe my whole .m2 repo because something get stuck. Anyway, adding the fix.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726794834


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35668/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729946621


   No worries, I'll make sure I retrigger the build again before merging, and if it fails I won't merge and ask about fixing the failure.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xuanyuanking commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r521791021



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -3512,6 +3521,9 @@ class SQLConf extends Serializable with Logging {
 
   def disabledJdbcConnectionProviders: String = getConf(SQLConf.DISABLED_JDBC_CONN_PROVIDER_LIST)
 
+  def disabledStreamingCustomMetrics: String =

Review comment:
       nit: If there's only one place to get the conf, maybe we can directly get the conf on the caller side.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
zsxwing commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r527288900



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,14 +178,14 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
-  def unit: String
+  def unit: String = ""
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)
+case class StateStoreCustomSumMetric(name: String, desc: String, override val unit: String)

Review comment:
       Adding a parameter to a case class is neither source compatible nor binary compatible. I think it's okay to ignore unit on the graph since the metric name has this information. We can re-consider to add unit when we hear the voice from the user.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728841740


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35817/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729864277


   **[Test build #131296 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131296/testReport)** for PR 30336 at commit [`5700455`](https://github.com/apache/spark/commit/57004553bf8d7259c49d51d99f83eb93131a2fce).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729787089


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525175321



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       I agree w/ this! Thanks @sarutak 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733061143


   **[Test build #131657 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131657/testReport)** for PR 30336 at commit [`115de28`](https://github.com/apache/spark/commit/115de28e7e959061b415abc6063f73e7926fd0b0).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r526885211



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,11 +178,15 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
+  def unit: String
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
+case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)

Review comment:
       Ping me here if somebody doesn't agree on the default value, closing it for now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728156611






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729864277


   **[Test build #131296 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131296/testReport)** for PR 30336 at commit [`5700455`](https://github.com/apache/spark/commit/57004553bf8d7259c49d51d99f83eb93131a2fce).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729867640


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728073686






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522837788



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -2875,6 +2875,15 @@ object SQLConf {
     .stringConf
     .createWithDefault("")
 
+  val DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST =
+    buildConf("spark.sql.streaming.disabledUICustomMetricsList")
+      .internal()
+      .doc("Configures a list of custom metrics on Structured Streaming UI, which are disabled. " +
+        "The list contains the name of the custom metrics separated by comma.")
+      .version("3.1.0")
+      .stringConf

Review comment:
       Not sure what you exactly mean, I've just found `USE_V1_SOURCE_LIST`, `DISABLED_V2_STREAMING_WRITERS`, `DISABLED_V2_STREAMING_MICROBATCH_READERS`, `NESTED_PREDICATE_PUSHDOWN_FILE_SOURCE_LIST`, etc. Can you elaborate please?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524998946



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729354586


   > If you meant about the "way" to get the list of custom metrics, sure it'd be ideal to document it. Directly listing the available custom metrics isn't feasible, and even requires further sync between code and doc if we somehow change the HDFS state store provider.
   
   O.K, it seems to be better to document how to get the list of custom metrics.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729502108


   > O.K, it seems to be better to document how to get the list of custom metrics.
   
   I'll add it to the doc of the new parameter.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729890413


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35900/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729821334


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35894/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525185287



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       I think the way could be to introduce metric unit in the mentioned structure.
   Since it doesn't exist I can't see possibilities for now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731084641


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36018/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525205747



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")
+        graphUIData.generateDataJs(jsCollector)
+
+        result ++=
+          // scalastyle:off
+          <tr>
+            <td style="vertical-align: middle;">
+              <div style="width: 240px;">
+                <div><strong>Aggregated Custom Metric {s"$metricName"} {SparkUIUtils.tooltip("Custom metric.", "right")}</strong></div>

Review comment:
       All we could potentially add is the name itself in the tooltip too: `Aggregated Custom Metric {s"$metricName"}`
   WDYT?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733230957






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524027024



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       Does this work if there are multiple state operators in the query plan and they have different custom metrics?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-727959570


   **[Test build #131158 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131158/testReport)** for PR 30336 at commit [`a788234`](https://github.com/apache/spark/commit/a7882349eef386933ced1d1f6cc587b0d25ce94c).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525132716



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")
+        graphUIData.generateDataJs(jsCollector)
+
+        result ++=
+          // scalastyle:off
+          <tr>
+            <td style="vertical-align: middle;">
+              <div style="width: 240px;">
+                <div><strong>Aggregated Custom Metric {s"$metricName"} {SparkUIUtils.tooltip("Custom metric.", "right")}</strong></div>

Review comment:
       All the tooltips for custom metrics say "Custom metric.".
   It seems meaningless. Can we show users more meaningful message?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       How about leaving it as is and focusing the UI for now?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733068651






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733100214


   **[Test build #131676 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131676/testReport)** for PR 30336 at commit [`115de28`](https://github.com/apache/spark/commit/115de28e7e959061b415abc6063f73e7926fd0b0).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729809934


   **[Test build #131292 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131292/testReport)** for PR 30336 at commit [`5700455`](https://github.com/apache/spark/commit/57004553bf8d7259c49d51d99f83eb93131a2fce).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726837049


   **[Test build #131074 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131074/testReport)** for PR 30336 at commit [`19e2da1`](https://github.com/apache/spark/commit/19e2da17a86e04d0d3e6c767c30f57440a0263e5).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731096036


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36018/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728910755


   By the way, how can users know what is the name of custom metrics they can list?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726932486


   **[Test build #131072 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131072/testReport)** for PR 30336 at commit [`f11bc22`](https://github.com/apache/spark/commit/f11bc2264a39a565089471074446e28824bd532b).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524217589



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)

Review comment:
       Closing this and we can track the mentioned comment if that satisfies everybody.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731600520






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733274719


   Thanks all for reviewing and thanks @gaborgsomogyi for the contribution. Merged to master.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r528226415



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -22,15 +22,25 @@ import java.lang.{Long => JLong}
 import java.util.UUID
 import javax.servlet.http.HttpServletRequest
 
+import scala.collection.JavaConverters._
 import scala.xml.{Node, NodeBuffer, Unparsed}
 
 import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.state.StateStoreProvider
+import org.apache.spark.sql.internal.SQLConf.STATE_STORE_PROVIDER_CLASS
+import org.apache.spark.sql.internal.StaticSQLConf.ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST
 import org.apache.spark.sql.streaming.ui.UIUtils._
 import org.apache.spark.ui.{GraphUIData, JsCollector, UIUtils => SparkUIUtils, WebUIPage}
 
 private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
   extends WebUIPage("statistics") with Logging {
 
+  // State store provider implementation mustn't do any heavyweight initialiation in constructor
+  // but in its init method.
+  private val supportedCustomMetrics = StateStoreProvider.create(
+    parent.parent.conf.get(STATE_STORE_PROVIDER_CLASS)).supportedCustomMetrics
+  logDebug(s"Supported custom metrics: $supportedCustomMetrics")

Review comment:
       How about moving this into `generateAggregatedCustomMetrics` and only having it when `enabledCustomMetrics` isn't empty?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525131477



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       > What is the final call what satisfies everybody? Should we make StateStoreWriter#getProgress as final to enforce this or we're good as-is?
   
   How about leaving it as is and focusing the UI for now?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733324374






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729794400


   @sarutak it will be merged only on Friday so we have plenty of time if fix needed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729781576


   **[Test build #131278 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131278/testReport)** for PR 30336 at commit [`791011c`](https://github.com/apache/spark/commit/791011c7015399b62a2a2dd26467d3d178538c89).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)`
     * `case class StateStoreCustomSizeMetric(name: String, desc: String, unit: String)`
     * `case class StateStoreCustomTimingMetric(name: String, desc: String, unit: String)`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726832566


   **[Test build #131072 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131072/testReport)** for PR 30336 at commit [`f11bc22`](https://github.com/apache/spark/commit/f11bc2264a39a565089471074446e28824bd532b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525707868



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")
+        graphUIData.generateDataJs(jsCollector)
+
+        result ++=
+          // scalastyle:off
+          <tr>
+            <td style="vertical-align: middle;">
+              <div style="width: 240px;">
+                <div><strong>Aggregated Custom Metric {s"$metricName"} {SparkUIUtils.tooltip("Custom metric.", "right")}</strong></div>

Review comment:
       FYI showing desc looks to be coupled with this comment - https://github.com/apache/spark/pull/30336#discussion_r525207223




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r523025938



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +202,98 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+   query: StreamingQueryUIData,
+   minBatchTime: Long,
+   maxBatchTime: Long,
+   jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val disabledCustomMetrics = Utils.stringToSeq(SQLConf.get.disabledStreamingCustomMetrics)

Review comment:
       I've moved the config to the UI section because of my previous comment.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522937215



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -2875,6 +2875,15 @@ object SQLConf {
     .stringConf
     .createWithDefault("")
 
+  val DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST =
+    buildConf("spark.sql.streaming.disabledUICustomMetricsList")
+      .internal()
+      .doc("Configures a list of custom metrics on Structured Streaming UI, which are disabled. " +
+        "The list contains the name of the custom metrics separated by comma.")
+      .version("3.1.0")
+      .stringConf

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522787893



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +202,98 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+   query: StreamingQueryUIData,
+   minBatchTime: Long,
+   maxBatchTime: Long,
+   jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val disabledCustomMetrics = Utils.stringToSeq(SQLConf.get.disabledStreamingCustomMetrics)

Review comment:
       Makes sense, we can come back to this when user need is more clear.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729882157






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729902736






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-732919629


   **[Test build #131657 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131657/testReport)** for PR 30336 at commit [`115de28`](https://github.com/apache/spark/commit/115de28e7e959061b415abc6063f73e7926fd0b0).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733148850


   **[Test build #131690 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131690/testReport)** for PR 30336 at commit [`e2f1594`](https://github.com/apache/spark/commit/e2f15947ef6e97fbcee5f9f328e6e8315a2eea66).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xuanyuanking commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r526557807



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,11 +178,15 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
+  def unit: String
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
+case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)

Review comment:
       Sorry for the comment at the last minute. I think the PR is ready for merge. Only one concern for the API change.
   This is an API change and will break extended StateStore if they have custom metrics. Seems now the unit is the UI only requirement, how about we generate the unit in `StreamingQueryStatisticsPage.scala`? Or we can add the unit info in the `desc` string. e.g "StateStoreCustomSizeMetric("uncompressedFileSize", "uncompressed file size - file bytes")" WDYT?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729973563


   **[Test build #131292 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131292/testReport)** for PR 30336 at commit [`5700455`](https://github.com/apache/spark/commit/57004553bf8d7259c49d51d99f83eb93131a2fce).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730412245


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35953/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733490462


   Thank you all for taking care!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730371251


   **[Test build #131349 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131349/testReport)** for PR 30336 at commit [`d43b1b9`](https://github.com/apache/spark/commit/d43b1b9af2e70283d7d06b276f1ee9ea5401fceb).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525704616



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")
+        graphUIData.generateDataJs(jsCollector)
+
+        result ++=
+          // scalastyle:off
+          <tr>
+            <td style="vertical-align: middle;">
+              <div style="width: 240px;">
+                <div><strong>Aggregated Custom Metric {s"$metricName"} {SparkUIUtils.tooltip("Custom metric.", "right")}</strong></div>

Review comment:
       It'd be ideal to show the desc, but the information isn't available in `StateOperatorProgress`. Unless we'd like to bring a major change here, the best try looks to be just showing metric name as a tooltip.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729974685






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729666468


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35882/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522928366



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -2875,6 +2875,15 @@ object SQLConf {
     .stringConf
     .createWithDefault("")
 
+  val DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST =
+    buildConf("spark.sql.streaming.disabledUICustomMetricsList")
+      .internal()
+      .doc("Configures a list of custom metrics on Structured Streaming UI, which are disabled. " +
+        "The list contains the name of the custom metrics separated by comma.")
+      .version("3.1.0")
+      .stringConf

Review comment:
       Ah, it's in the k8s area. Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733110222






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-732919629


   **[Test build #131657 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131657/testReport)** for PR 30336 at commit [`115de28`](https://github.com/apache/spark/commit/115de28e7e959061b415abc6063f73e7926fd0b0).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726913462


   **[Test build #131074 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131074/testReport)** for PR 30336 at commit [`19e2da1`](https://github.com/apache/spark/commit/19e2da17a86e04d0d3e6c767c30f57440a0263e5).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729799867


   > The interesting part is that locally works. Just executed it. Doing a full re-compile and let's see.
   What the test complains about is used later on: parent.parent.conf which was worked before.
   
   Yes, this change itself works well. The error seems that the mock of `StreamingQueryTab` is not implemented enough. Only the following 3 members seems to be imeplemented.
   ```
       when(tab.appName).thenReturn("testing")
       when(tab.headerTabs).thenReturn(Seq.empty)
       when(tab.statusListener).thenReturn(statusListener)
   ```
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r526870945



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,11 +178,15 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
+  def unit: String
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
+case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)

Review comment:
       More eyes, better code. I've just added default value as empty string.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730396501


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35953/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522980423



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +202,98 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+   query: StreamingQueryUIData,
+   minBatchTime: Long,
+   maxBatchTime: Long,
+   jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val disabledCustomMetrics = Utils.stringToSeq(SQLConf.get.disabledStreamingCustomMetrics)

Review comment:
       Hmm, now I see what's the matter. SparkUI doesn't have active session which is fine because it's common but w/o active session `SQLConf` parameters not propagated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729858517


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35897/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731600520






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731568627


   **[Test build #131463 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131463/testReport)** for PR 30336 at commit [`864062f`](https://github.com/apache/spark/commit/864062ffa7e6eda3ac4dc18a47cd806200e93f7a).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728156611






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r529756560



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +209,100 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-      </tr>
-      <tr>
-        <td style="vertical-align: middle;">
-          <div style="width: 160px;">
-            <div><strong>Aggregated Number Of Rows Dropped By Watermark {SparkUIUtils.tooltip("Accumulates all input rows being dropped in stateful operators by watermark. 'Inputs' are relative to operators.", "right")}</strong></div>
-          </div>
-        </td>
-        <td class={"aggregated-num-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
-      </tr>
-      // scalastyle:on
+      val result =
+        // 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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
+        </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+        </tr>
+        <tr>
+          <td style="vertical-align: middle;">
+            <div style="width: 160px;">
+              <div><strong>Aggregated Number Of Rows Dropped By Watermark {SparkUIUtils.tooltip("Accumulates all input rows being dropped in stateful operators by watermark. 'Inputs' are relative to operators.", "right")}</strong></div>
+            </div>
+          </td>
+          <td class={"aggregated-num-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+        </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>

Review comment:
       For matching user input to configs, usually we compare in case-insensitive way, I am not sure if custom metrics are also in this way.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733073458


   **[Test build #131676 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131676/testReport)** for PR 30336 at commit [`115de28`](https://github.com/apache/spark/commit/115de28e7e959061b415abc6063f73e7926fd0b0).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728939457






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729632780


   **[Test build #131278 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131278/testReport)** for PR 30336 at commit [`791011c`](https://github.com/apache/spark/commit/791011c7015399b62a2a2dd26467d3d178538c89).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726756276


   **[Test build #131064 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131064/testReport)** for PR 30336 at commit [`6654934`](https://github.com/apache/spark/commit/66549340ca156c2678ffdb833384257bc3df3a05).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729782561


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/131278/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731568627


   **[Test build #131463 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131463/testReport)** for PR 30336 at commit [`864062f`](https://github.com/apache/spark/commit/864062ffa7e6eda3ac4dc18a47cd806200e93f7a).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729867423


   **[Test build #131290 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131290/testReport)** for PR 30336 at commit [`791011c`](https://github.com/apache/spark/commit/791011c7015399b62a2a2dd26467d3d178538c89).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)`
     * `case class StateStoreCustomSizeMetric(name: String, desc: String, unit: String)`
     * `case class StateStoreCustomTimingMetric(name: String, desc: String, unit: String)`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729662154


   I built the latest change and confirmed that tooltip and unit are shown properly.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728660542


   @xuanyuanking @sarutak 
   Appreciate your next round of reviews. Thanks in advance.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729952216


   Hmm... Weird. This passes in my local.
   
   ```
   build/sbt -Phive -Phive-thriftserver "testOnly org.apache.spark.sql.streaming.ui.StreamingQueryPageSuite"
   ```
   
   I've had to fix the conflict during rebase with master, but Jenkins doesn't complain with conflicts, so does Github.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524216650



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)

Review comment:
       There is another way what I'm currently evaluating w/ `StaticSQLConf` Maybe we can use that. Please see https://github.com/apache/spark/pull/30336#discussion_r523829732




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r529477301



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +209,100 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-      </tr>
-      <tr>
-        <td style="vertical-align: middle;">
-          <div style="width: 160px;">
-            <div><strong>Aggregated Number Of Rows Dropped By Watermark {SparkUIUtils.tooltip("Accumulates all input rows being dropped in stateful operators by watermark. 'Inputs' are relative to operators.", "right")}</strong></div>
-          </div>
-        </td>
-        <td class={"aggregated-num-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
-      </tr>
-      // scalastyle:on
+      val result =
+        // 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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
+        </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+        </tr>
+        <tr>
+          <td style="vertical-align: middle;">
+            <div style="width: 160px;">
+              <div><strong>Aggregated Number Of Rows Dropped By Watermark {SparkUIUtils.tooltip("Accumulates all input rows being dropped in stateful operators by watermark. 'Inputs' are relative to operators.", "right")}</strong></div>
+            </div>
+          </td>
+          <td class={"aggregated-num-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+        </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>

Review comment:
       I haven't seen that this is a custom on list related configs. Not sure we need to add something new.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525254553



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       Never mind, these are `lazy val`s in the provider implementation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r521729031



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +202,98 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+   query: StreamingQueryUIData,

Review comment:
       nit: indentation

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +202,98 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+   query: StreamingQueryUIData,
+   minBatchTime: Long,
+   maxBatchTime: Long,
+   jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val disabledCustomMetrics = Utils.stringToSeq(SQLConf.get.disabledStreamingCustomMetrics)

Review comment:
       I think it'd be better to do opposite, disable everything and let end users enable it, as in many cases default metrics would be sufficient to get the picture of current status. Custom metrics are more likely auxiliary.
   
   Given this is from custom metrics, we don't know which metrics are available under the state store provider end users use, and we don't know how to aggregate (here we simply assume "sum" is the right aggregation) which will end up with providing odd values.
   
   So the ideal approach would be providing metric name with aggregation method (sum, avg, min, max), but if it sounds to be complicated just for this functionality, we could simply restrict to sum and explain it to config doc.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -2875,6 +2875,15 @@ object SQLConf {
     .stringConf
     .createWithDefault("")
 
+  val DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST =
+    buildConf("spark.sql.streaming.disabledUICustomMetricsList")
+      .internal()
+      .doc("Configures a list of custom metrics on Structured Streaming UI, which are disabled. " +
+        "The list contains the name of the custom metrics separated by comma.")
+      .version("3.1.0")
+      .stringConf

Review comment:
       I think there's a kind of conf we can get list of string separated by comma, instead of doing it manually.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -2875,6 +2875,15 @@ object SQLConf {
     .stringConf
     .createWithDefault("")
 
+  val DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST =
+    buildConf("spark.sql.streaming.disabledUICustomMetricsList")

Review comment:
       nit: as `metrics` is already plural, `List` looks to be redundant.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729837159


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35897/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525724645



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       The idea looks promising. I see the concern from @gaborgsomogyi but given it has separate `init` method to bootstrap, more likely the implementation wouldn't do something in constructor.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726943658


   **[Test build #131064 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131064/testReport)** for PR 30336 at commit [`6654934`](https://github.com/apache/spark/commit/66549340ca156c2678ffdb833384257bc3df3a05).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524844943



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       OK you found a good point I was missing.
   
   Regarding subclassing I think it's not intended to - Spark doesn't have any actual implementation for overriding the method, as well as we don't expect user code to implement `StateStoreWriter` and kick in to physical execution node. So as you said that's technically possible, that looks to be beyond intention. (Seems to be missing guard, but not sure we have been strictly doing it.)
   
   And regarding the possibility of having different custom metrics among stat store provider instances like I mentioned:
   
   https://github.com/apache/spark/blob/5af5aa146ecbff38b809127b5eb9805441627ed2/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala#L127-L137
   
   Here the list of custom metrics are being picked up, and it is populated from dummy StateStoreProvider instance. That said, Spark "expects" the same value of `supportedCustomMetrics` across StateStoreProvider instances created from same provider class.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xuanyuanking commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r523976903



##########
File path: core/src/main/scala/org/apache/spark/internal/config/UI.scala
##########
@@ -203,4 +203,15 @@ private[spark] object UI {
     .stringConf
     .transform(_.toUpperCase(Locale.ROOT))
     .createWithDefault("LOCAL")
+
+  val ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST =
+    ConfigBuilder("spark.ui.sql.streaming.enabledCustomMetricList")
+      .internal()
+      .doc("Configures a list of custom metrics on Structured Streaming UI, which are enabled. " +
+        "The list contains the name of the custom metrics separated by comma. In aggregation" +
+        "only sum used.")
+      .version("3.1.0")
+      .stringConf
+      .toSequence
+      .createWithDefault(Nil)

Review comment:
       Do we need to add some common custom metrics here by default?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")
+        graphUIData.generateDataJs(jsCollector)
+
+        result ++=
+          // scalastyle:off
+          <tr>
+            <td style="vertical-align: middle;">
+              <div style="width: 240px;">
+                <div><strong>Aggregated Custom Metric {s"$metricName"} {SparkUIUtils.tooltip("Custom metric.", "right")}</strong></div>
+              </div>
+            </td>
+            <td class={s"aggregated-$metricName-timeline"}>{graphUIData.generateTimelineHtml(jsCollector)}</td>
+            <td class={s"aggregated-$metricName-histogram"}>{graphUIData.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on

Review comment:
       indent

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)

Review comment:
       How about `val enabledCustomMetrics = SQLConf.get.getConf(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <tr>

Review comment:
       indent: 2 extra space




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524053548



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       One additional thing, please see that the timeline and histogram aggregated...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726914734


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525177407



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")
+        graphUIData.generateDataJs(jsCollector)
+
+        result ++=
+          // scalastyle:off
+          <tr>
+            <td style="vertical-align: middle;">
+              <div style="width: 240px;">
+                <div><strong>Aggregated Custom Metric {s"$metricName"} {SparkUIUtils.tooltip("Custom metric.", "right")}</strong></div>

Review comment:
       We can add anything but since these are fully custom I couldn't come up anything more meaningful.
   Do you have something to suggest?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728219046


   **[Test build #131158 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131158/testReport)** for PR 30336 at commit [`a788234`](https://github.com/apache/spark/commit/a7882349eef386933ced1d1f6cc587b0d25ce94c).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `trait HasMaxBlockSizeInMB extends Params `
     * `    >>> class VectorAccumulatorParam(AccumulatorParam):`
     * `            fully qualified classname of key Writable class (e.g. \"org.apache.hadoop.io.Text\")`
     * `            fully qualified classname of key Writable class (e.g. \"org.apache.hadoop.io.Text\")`
     * `            fully qualified classname of key Writable class (e.g. \"org.apache.hadoop.io.Text\")`
     * `            fully qualified classname of key Writable class (e.g. \"org.apache.hadoop.io.Text\")`
     * `class HasMaxBlockSizeInMB(Params):`
     * `case class UnresolvedPartitionSpec(`
     * `case class ResolvedPartitionSpec(`
     * `case class ElementAt(`
     * `case class GetArrayItem(`
     * `case class Elt(`
     * `class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging `
     * `abstract class AbstractSqlParser extends ParserInterface with Logging `
     * `class CatalystSqlParser extends AbstractSqlParser `
     * `case class AlterTableAddPartition(`
     * `case class AlterTableDropPartition(`
     * `  implicit class PartitionSpecsHelper(partSpecs: Seq[PartitionSpec]) `
     * `class SparkSqlParser extends AbstractSqlParser `
     * `class SparkSqlAstBuilder extends AstBuilder `
     * `case class CoalesceShufflePartitions(session: SparkSession) extends Rule[SparkPlan] `
     * `class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] `
     * `class FallBackFileSourceV2(sparkSession: SparkSession) extends Rule[LogicalPlan] `
     * `class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] `
     * `case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[LogicalPlan] `
     * `case class AlterTableAddPartitionExec(`
     * `case class AlterTableDropPartitionExec(`
     * `case class PlanDynamicPruningFilters(sparkSession: SparkSession)`
     * `case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] `
     * `class VariableSubstitution `
     * `class ResolveHiveSerdeTable(session: SparkSession) extends Rule[LogicalPlan] `
     * `class DetermineTableStats(session: SparkSession) extends Rule[LogicalPlan] `


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r526572871



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,11 +178,15 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
+  def unit: String
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
+case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)

Review comment:
       Nice finding! Glad I wait for 2 days to figure out any missing like this.
   
   Btw, I have a deeper question. Why the state store (+provider) interfaces are in sql/execution which is considered as "private"? We know there're several 3rd party implementations, and placing these APIs in sql/execution package means that we don't guarantee the compatibility as public API. e.g. MiMa didn't complain this.
   
   What would be the way to go? Could we move the interface with aliasing the old one, and deprecate the old one? Otherwise, should we make a breaking change to get better result on future? 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728939457






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731096062






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524159464



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       > This is working because all state operator must have the exact same number of custom metrics. 
   
   @HeartSaVioR you added `customMetrics` to `StateOperatorProgress` in #21469 .
   Did you intend that all the state operators in a query should have the same number / type of custom metrics?
   In other words, each state operator should not have its own custom metrics right?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524249906



##########
File path: core/src/main/scala/org/apache/spark/internal/config/UI.scala
##########
@@ -203,4 +203,15 @@ private[spark] object UI {
     .stringConf
     .transform(_.toUpperCase(Locale.ROOT))
     .createWithDefault("LOCAL")
+
+  val ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST =
+    ConfigBuilder("spark.ui.sql.streaming.enabledCustomMetricList")
+      .internal()
+      .doc("Configures a list of custom metrics on Structured Streaming UI, which are enabled. " +
+        "The list contains the name of the custom metrics separated by comma. In aggregation" +
+        "only sum used.")
+      .version("3.1.0")
+      .stringConf
+      .toSequence
+      .createWithDefault(Nil)

Review comment:
       The main reason was that these are full custom so any default is hard to come up with.
   If anybody thinks we can add defaults, please add comments and name them.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730002423


   **[Test build #131296 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131296/testReport)** for PR 30336 at commit [`5700455`](https://github.com/apache/spark/commit/57004553bf8d7259c49d51d99f83eb93131a2fce).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726855404


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35676/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733064181


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726933889






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522825136



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -3512,6 +3521,9 @@ class SQLConf extends Serializable with Logging {
 
   def disabledJdbcConnectionProviders: String = getConf(SQLConf.DISABLED_JDBC_CONN_PROVIDER_LIST)
 
+  def disabledStreamingCustomMetrics: String =

Review comment:
       Yeah, we can inline it. Removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522820019



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +202,98 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+   query: StreamingQueryUIData,

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728938258


   **[Test build #131215 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131215/testReport)** for PR 30336 at commit [`7787764`](https://github.com/apache/spark/commit/7787764ddb450a23d6f8aa0fcc85ed599285f993).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730412238






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731096062






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733068651






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733230957






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729836016






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729666480






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726914746


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/131074/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522943114



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +202,98 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+   query: StreamingQueryUIData,
+   minBatchTime: Long,
+   maxBatchTime: Long,
+   jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val disabledCustomMetrics = Utils.stringToSeq(SQLConf.get.disabledStreamingCustomMetrics)

Review comment:
       I'm leaving this open because something is not 100% with the SQL conf reading from the UI side and test fails when I try to use `withSQLConf`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525168120



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       Hmm, though users might possibly infer the unit for the current available custom metrics,  we can't easily make the label of y-axis for custom metrics...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524159464



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       > This is working because all state operator must have the exact same number of custom metrics. 
   
   @HeartSaVioR you added `customMetrics` to `StateOperatorProgress` in #21469 .
   Did you intended that all the state operators in a query should have the same number / type of custom metrics?
   In other words, each state operator should not have its own custom metrics right?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729882128


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35899/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726871312






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726832566


   **[Test build #131072 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131072/testReport)** for PR 30336 at commit [`f11bc22`](https://github.com/apache/spark/commit/f11bc2264a39a565089471074446e28824bd532b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731164627






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r526029940



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       I've added it and working fine so closing this thread.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728853290






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725448362


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35544/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731600279


   **[Test build #131463 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131463/testReport)** for PR 30336 at commit [`864062f`](https://github.com/apache/spark/commit/864062ffa7e6eda3ac4dc18a47cd806200e93f7a).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  case class GetShufflePushMergerLocations(numMergersNeeded: Int, hostsToFilter: Set[String])`
     * `  case class RemoveShufflePushMergerLocation(host: String) extends ToBlockManagerMaster`
     * `case class ParseUrl(children: Seq[Expression], failOnError: Boolean = SQLConf.get.ansiEnabled)`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729902736






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725480002


   cc @gengliangwang, @HeartSaVioR and @xuanyuanking FYI


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726756276


   **[Test build #131064 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131064/testReport)** for PR 30336 at commit [`6654934`](https://github.com/apache/spark/commit/66549340ca156c2678ffdb833384257bc3df3a05).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728802333


   **[Test build #131215 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131215/testReport)** for PR 30336 at commit [`7787764`](https://github.com/apache/spark/commit/7787764ddb450a23d6f8aa0fcc85ed599285f993).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729858509


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524042187



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       This is working because all state operator must have the exact same number of custom metrics. The names are coming from `head` and the values from `query.recentProgress`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725487881


   Thank pinging peoples, just wanted to wait until tests pass.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi edited a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi edited a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725487881


   Thanks pinging peoples, just wanted to wait until tests pass.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733110222






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728220162






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728969992


   > By the way, how can users know what is the name of custom metrics they can list?
   
   It in the query progress info level log entry:
   ```
   20/11/17 15:36:06 INFO MicroBatchExecution: Streaming query made progress: {
   ...
     "stateOperators" : [ {
       "numRowsTotal" : 30,
       "numRowsUpdated" : 30,
       "memoryUsedBytes" : 13144,
       "numRowsDroppedByWatermark" : 0,
       "customMetrics" : {
         "loadedMapCacheHitCount" : 2,
         "loadedMapCacheMissCount" : 0,
         "stateOnCurrentVersionSizeBytes" : 6296
       }
     } ],
   ...
   ```
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-727959570


   **[Test build #131158 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131158/testReport)** for PR 30336 at commit [`a788234`](https://github.com/apache/spark/commit/a7882349eef386933ced1d1f6cc587b0d25ce94c).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524242454



##########
File path: core/src/main/scala/org/apache/spark/internal/config/UI.scala
##########
@@ -203,4 +203,15 @@ private[spark] object UI {
     .stringConf
     .transform(_.toUpperCase(Locale.ROOT))
     .createWithDefault("LOCAL")
+
+  val ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST =

Review comment:
       After several tests done it looks like `StaticSQLConf` is the right solution (though reaching the value not w/ `SQLConf.get` but w/ `SparkConf`) which fits into the existing config set. Thanks for notifying, this way much better and unit test works also.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r527252370



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,14 +178,14 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
-  def unit: String
+  def unit: String = ""
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)
+case class StateStoreCustomSumMetric(name: String, desc: String, override val unit: String)

Review comment:
       We also would like to not break these three classes as well, either providing default value or providing two params constructor which would set `unit` as default value (`""`).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525182879



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       It would be definitely good to add something better, but I have similar problem like in tooltip. Every custom metric can have its own unit.
   I've just double checked and custom metrics stored in the following way: `java.util.Map[String, java.lang.Long]`
   Since this structure doesn't contain anything what the metric unit is I'm not sure from where it could come from.
   Any suggestion?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xuanyuanking commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525634623



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")
+        graphUIData.generateDataJs(jsCollector)
+
+        result ++=
+          // scalastyle:off
+          <tr>
+            <td style="vertical-align: middle;">
+              <div style="width: 240px;">
+                <div><strong>Aggregated Custom Metric {s"$metricName"} {SparkUIUtils.tooltip("Custom metric.", "right")}</strong></div>

Review comment:
       Is it possible to show the `StateStoreCustomMetric.desc` here? They should be more meaningful messages.
   https://github.com/apache/spark/blob/5e8549973dc8aeb484d06be988b04ac2ed4e7add/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala#L304-L314




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730003357






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728065114


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-727684998


   Oh, @sarutak too FYI


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729995265






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525207223



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       How about following idea?
   
   1. Add `unit` to `StateStoreCustomMetric`
   2. Call `StateStoreProvider.create(conf.get(STATE_STORE_PROVIDER_CLASS)).supportedCustomMetrics` and get the `Seq` of `StateStoreCustomMetric` in `StreamingQueryStatisticsPage`.
   3. Refer `desc` and `unit` from the `StateStoreCustomMetric`
   
   @HeartSaVioR  What do you tihnk?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733258139






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730570427






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729994137


   **[Test build #131295 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131295/testReport)** for PR 30336 at commit [`5700455`](https://github.com/apache/spark/commit/57004553bf8d7259c49d51d99f83eb93131a2fce).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524841642



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       `StateStoreWriter#getProgress` is not marked as `final` so I wondered it's intended to have sub classes which override `getProgress` where custom metrics can be created for a operator which is implemented as a sub class of `StateStoreWriter`.
   But now I understand that custom metrics are not for operators.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728064974


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35761/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525182879



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       It would be definitely good to add something better, but I have similar problem like in tooltip. Every custom metric can have its own unit.
   I've just double checked and custom metrics stored in the following way: `java.util.Map[String, java.lang.Long]`
   Since this structure doesn't contain any unit what the metric unit is I'm not sure from where it could come from.
   Any suggestion?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729868866


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35899/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729974685






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak edited a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak edited a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-727861148


   Hi @gaborgsomogyi , you'd like to add this new feature but you say you tested this feature manually so could you explain how you tested to us in the description? It helps reviewers.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi edited a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi edited a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729789606


   The interesting part is that locally works. Just executed it. Doing a full re-compile and let's see.
   What the test complains about is used later on: `parent.parent.conf` which was worked before.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730003357


   Merged build finished. Test PASSed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725411666


   **[Test build #130939 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130939/testReport)** for PR 30336 at commit [`2eefeb2`](https://github.com/apache/spark/commit/2eefeb2e391db034c28e560c44998fcb0ab00c85).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731576313


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36069/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi edited a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi edited a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728969992


   > By the way, how can users know what is the name of custom metrics they can list?
   
   It's in the query progress info level log entry:
   ```
   20/11/17 15:36:06 INFO MicroBatchExecution: Streaming query made progress: {
   ...
     "stateOperators" : [ {
       "numRowsTotal" : 30,
       "numRowsUpdated" : 30,
       "memoryUsedBytes" : 13144,
       "numRowsDroppedByWatermark" : 0,
       "customMetrics" : {
         "loadedMapCacheHitCount" : 2,
         "loadedMapCacheMissCount" : 0,
         "stateOnCurrentVersionSizeBytes" : 6296
       }
     } ],
   ...
   ```
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726933901


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/131072/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733396536






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525207223



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       How about following idea?
   
   1. Add `unit` to `StateStoreCustomMetric`
   2. Call `StateStoreProvider.create(conf.get(STATE_STORE_PROVIDER_CLASS)).supportedCustomMetrics` and get the `Seq` of `StateStoreCustomMetric` from `StreamingQueryStatisticsPage`.
   3. Refer `desc` and `unit` from the `StateStoreCustomMetric`
   
   @HeartSaVioR  What do you tihnk?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524844943



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       OK you found a good point I was missing.
   
   https://github.com/apache/spark/blob/5af5aa146ecbff38b809127b5eb9805441627ed2/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala#L127-L137
   
   Here the list of custom metrics are being picked up, and it is populated from dummy StateStoreProvider instance. That said, Spark "expects" the same value of `supportedCustomMetrics` across StateStoreProvider instances created from same provider class.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725565333


   **[Test build #130939 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130939/testReport)** for PR 30336 at commit [`2eefeb2`](https://github.com/apache/spark/commit/2eefeb2e391db034c28e560c44998fcb0ab00c85).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729791254


   **[Test build #131290 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131290/testReport)** for PR 30336 at commit [`791011c`](https://github.com/apache/spark/commit/791011c7015399b62a2a2dd26467d3d178538c89).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524246316



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       Solved all other comments but leaving this open because this can be a major issue w/ big influence on the PR.
   Let's wait on the answer and adjust the code/tests based on that (as said this case not tested because of maybe bad assumption).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-727956153


   **[Test build #131157 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131157/testReport)** for PR 30336 at commit [`34cb99f`](https://github.com/apache/spark/commit/34cb99ff32e2d9b16760c94576f399eb3436c1e2).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729632780


   **[Test build #131278 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131278/testReport)** for PR 30336 at commit [`791011c`](https://github.com/apache/spark/commit/791011c7015399b62a2a2dd26467d3d178538c89).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729838090


   **[Test build #131295 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131295/testReport)** for PR 30336 at commit [`5700455`](https://github.com/apache/spark/commit/57004553bf8d7259c49d51d99f83eb93131a2fce).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731164627






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-727891093


   @sarutak added how I've tested.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak edited a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak edited a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-727923705


   @gaborgsomogyi I expected what the streaming query is (code snippet may be enough) and what you confirmed by the manual test... With those information, we can understand what you did (and did not) test.
   Here is quoted from the PR template.
   
   ```
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524097943



##########
File path: core/src/main/scala/org/apache/spark/internal/config/UI.scala
##########
@@ -203,4 +203,15 @@ private[spark] object UI {
     .stringConf
     .transform(_.toUpperCase(Locale.ROOT))
     .createWithDefault("LOCAL")
+
+  val ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST =

Review comment:
       I've just had a deeper look at `StaticSQLConf` which works when it's read from `SparkConf`. We can do the exact same approach like `spark.sql.streaming.ui.retainedQueries` but I think it can't be really unit tested (at least not found a way to do).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729867651


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/131290/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733396536






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728853303


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35817/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728073715


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35760/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725459638


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35544/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730003366


   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/131296/
   Test PASSed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730371251


   **[Test build #131349 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131349/testReport)** for PR 30336 at commit [`d43b1b9`](https://github.com/apache/spark/commit/d43b1b9af2e70283d7d06b276f1ee9ea5401fceb).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726794855


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725459670






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728154949


   **[Test build #131157 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131157/testReport)** for PR 30336 at commit [`34cb99f`](https://github.com/apache/spark/commit/34cb99ff32e2d9b16760c94576f399eb3436c1e2).
    * This patch passes all tests.
    * This patch **does not merge cleanly**.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730412210


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35953/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-727923705


   @gaborgsomogyi I expected what the streaming query is (code snippet may be enough) and what you confirmed by the manual test... With those information, we can understand what you did (and did not) test.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726933889


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725566500






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728026801


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35761/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524850658



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,

Review comment:
       nit: indentation on parameters (now they're 3 spaces indentation)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726837049


   **[Test build #131074 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131074/testReport)** for PR 30336 at commit [`19e2da1`](https://github.com/apache/spark/commit/19e2da17a86e04d0d3e6c767c30f57440a0263e5).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729351175


   If you meant about the "way" to get the list of custom metrics, sure it'd be ideal to document it. Directly listing the available custom metrics isn't feasible, and even requires further sync between code and doc if we somehow change the HDFS state store provider.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728220162






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733073458


   **[Test build #131676 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131676/testReport)** for PR 30336 at commit [`115de28`](https://github.com/apache/spark/commit/115de28e7e959061b415abc6063f73e7926fd0b0).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728065155


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35761/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xuanyuanking commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524075705



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)

Review comment:
       Ah, got it. Thanks for the explanation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728976594


   > It's in the query progress info level log entry:
   
   It seems difficult for users to notice that those are the custom metrics which users can list in `spark.sql.streaming.ui.enabledCustomMetricList`.
   
   I think it's better to document about `spark.sql.streaming.ui.enabledCustomMetricList` and the current available custom metrics in this PR or a followup PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r526560274



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,11 +178,15 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
+  def unit: String
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
+case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)

Review comment:
       How about set empty string as the default value for `unit` ? This can resolve the compatibility issue.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r522820413



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -2875,6 +2875,15 @@ object SQLConf {
     .stringConf
     .createWithDefault("")
 
+  val DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST =
+    buildConf("spark.sql.streaming.disabledUICustomMetricsList")

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730570427






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726914734






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729666480






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731573503


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36069/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731028284


   **[Test build #131412 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131412/testReport)** for PR 30336 at commit [`9a87255`](https://github.com/apache/spark/commit/9a872550908d4824db0272a6301d45fda62f6cbf).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-733324374






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729858509






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728073571


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35760/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525131477



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       > What is the final call what satisfies everybody? Should we make StateStoreWriter#getProgress as final to enforce this or we're good as-is?
   
   How about leaving it as is and focusing on UI part for now?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726945516






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524204409



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       @sarutak nice catch, that's definitely not tested and the code will blow up.
   @HeartSaVioR If operator `O1` has custom metric `C1` and operator `O2` has custom metric `C2` then the currently implemented aggregation is not fine.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728035700


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35760/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729650978


   791011c looks to address current review comments so far, but would like to get confirmed by @sarutak and @xuanyuanking .


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-725566500






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729789606


   The interesting part is that locally works. Just executed it. Doing a full re-compile and let's see.
   What the test complains about is used later on: `parent.parent.conf`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r526606880



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala
##########
@@ -178,11 +178,15 @@ object StateStoreMetrics {
 trait StateStoreCustomMetric {
   def name: String
   def desc: String
+  def unit: String
 }
 
-case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomSizeMetric(name: String, desc: String) extends StateStoreCustomMetric
-case class StateStoreCustomTimingMetric(name: String, desc: String) extends StateStoreCustomMetric
+case class StateStoreCustomSumMetric(name: String, desc: String, unit: String)

Review comment:
       > Btw, I have a deeper question. Why the state store (+provider) interfaces are in sql/execution which is considered as "private"? We know there're several 3rd party implementations, and placing these APIs in sql/execution package means that we don't guarantee the compatibility as public API. e.g. MiMa didn't complain this.
   
   I wondered why MiMa didn't find the compatibility issue too. As @HeartSaVioR says `sql/execution` seems to be considered as "private", but I guess it may not be considered that the package is private at that time those interfaces were introduced to the package.
   Anyway, it seems that we need to discuss the guarantee of the compatibility in another place.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729802331


   After full recompile the test passed properly. Just curious, do you have the same behaviour on your side?
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729809934


   **[Test build #131292 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131292/testReport)** for PR 30336 at commit [`5700455`](https://github.com/apache/spark/commit/57004553bf8d7259c49d51d99f83eb93131a2fce).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728073686


   Build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729791254


   **[Test build #131290 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131290/testReport)** for PR 30336 at commit [`791011c`](https://github.com/apache/spark/commit/791011c7015399b62a2a2dd26467d3d178538c89).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729803854


   > After full recompile the test passed properly. Just curious, do you have the same behaviour on your side?
   
   Yes, the test fails on my laptop too.
   
   Did you run the suite with like the following command?
   ```
   build/sbt -Phive -Phive-thriftserver "testOnly org.apache.spark.sql.streaming.ui.StreamingQueryPageSuite"
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726794855






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r525270932



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -236,12 +239,61 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
           <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>
+        val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
+          p.stateOperators.map(_.customMetrics.get(metricName).toDouble).sum))
+        val max = data.maxBy(_._2)._2
+
+        val graphUIData =
+          new GraphUIData(
+            s"aggregated-$metricName-timeline",
+            s"aggregated-$metricName-histogram",
+            data,
+            minBatchTime,
+            maxBatchTime,
+            0,
+            max,
+            "")

Review comment:
       The only concern what I see is that all `StateStoreProvider` implementation must support lightweight initialization which can be done from UI thread w/o exception. Not sure this implicit requirement is or will be known to all `StateStoreProvider` implementor.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524042187



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       This is working because all state operator must have the exact same number of custom metrics. The names are coming from head and the values from `query.recentProgress`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729836016






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r523967561



##########
File path: core/src/main/scala/org/apache/spark/internal/config/UI.scala
##########
@@ -203,4 +203,15 @@ private[spark] object UI {
     .stringConf
     .transform(_.toUpperCase(Locale.ROOT))
     .createWithDefault("LOCAL")
+
+  val ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST =

Review comment:
       Please see my previous comment above. `SQLConf` work only of there is an active session, otherwise the config not propagated but silently gives back the default. Since in the webserver there is no active session (because it's common) I don't think we want to itroduce such. Is `StaticSQLConf` propagated w/o session?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729631217


   Updated the attached image too.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731576317






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729867640






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r529772096



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +209,100 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-      </tr>
-      <tr>
-        <td style="vertical-align: middle;">
-          <div style="width: 160px;">
-            <div><strong>Aggregated Number Of Rows Dropped By Watermark {SparkUIUtils.tooltip("Accumulates all input rows being dropped in stateful operators by watermark. 'Inputs' are relative to operators.", "right")}</strong></div>
-          </div>
-        </td>
-        <td class={"aggregated-num-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
-      </tr>
-      // scalastyle:on
+      val result =
+        // 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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
+        </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+        </tr>
+        <tr>
+          <td style="vertical-align: middle;">
+            <div style="width: 160px;">
+              <div><strong>Aggregated Number Of Rows Dropped By Watermark {SparkUIUtils.tooltip("Accumulates all input rows being dropped in stateful operators by watermark. 'Inputs' are relative to operators.", "right")}</strong></div>
+            </div>
+          </td>
+          <td class={"aggregated-num-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+        </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>

Review comment:
       OK, if that's so we I'll add it. I've checked similar list based configs where it's not done like that.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-726871274


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35676/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524205934



##########
File path: core/src/main/scala/org/apache/spark/internal/config/UI.scala
##########
@@ -203,4 +203,15 @@ private[spark] object UI {
     .stringConf
     .transform(_.toUpperCase(Locale.ROOT))
     .createWithDefault("LOCAL")
+
+  val ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST =
+    ConfigBuilder("spark.ui.sql.streaming.enabledCustomMetricList")
+      .internal()
+      .doc("Configures a list of custom metrics on Structured Streaming UI, which are enabled. " +
+        "The list contains the name of the custom metrics separated by comma. In aggregation" +
+        "only sum used.")
+      .version("3.1.0")
+      .stringConf
+      .toSequence
+      .createWithDefault(Nil)

Review comment:
       We've discussed w/ @HeartSaVioR that by default it must be empty and in need can be added.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r528225669



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +209,100 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-      </tr>
-      <tr>
-        <td style="vertical-align: middle;">
-          <div style="width: 160px;">
-            <div><strong>Aggregated Number Of Rows Dropped By Watermark {SparkUIUtils.tooltip("Accumulates all input rows being dropped in stateful operators by watermark. 'Inputs' are relative to operators.", "right")}</strong></div>
-          </div>
-        </td>
-        <td class={"aggregated-num-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
-      </tr>
-      // scalastyle:on
+      val result =
+        // 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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
+        </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+        </tr>
+        <tr>
+          <td style="vertical-align: middle;">
+            <div style="width: 160px;">
+              <div><strong>Aggregated Number Of Rows Dropped By Watermark {SparkUIUtils.tooltip("Accumulates all input rows being dropped in stateful operators by watermark. 'Inputs' are relative to operators.", "right")}</strong></div>
+            </div>
+          </td>
+          <td class={"aggregated-num-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+        </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+      query: StreamingQueryUIData,
+      minBatchTime: Long,
+      maxBatchTime: Long,
+      jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
+      .filter(enabledCustomMetrics.contains(_)).map { metricName =>

Review comment:
       Should we compare them by lowercase?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-728802333


   **[Test build #131215 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131215/testReport)** for PR 30336 at commit [`7787764`](https://github.com/apache/spark/commit/7787764ddb450a23d6f8aa0fcc85ed599285f993).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR edited a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-729952216


   ~~Hmm... Weird. This passes in my local.~~
   
   ```
   build/sbt -Phive -Phive-thriftserver "testOnly org.apache.spark.sql.streaming.ui.StreamingQueryPageSuite"
   ```
   
   ~~I've had to fix the conflict during rebase with master, but Jenkins doesn't complain with conflicts, so does Github.~~
   
   EDIT: Never mind. Just seen the new commit.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-731576317






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30336:
URL: https://github.com/apache/spark/pull/30336#discussion_r524638775



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala
##########
@@ -199,49 +201,99 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
           "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-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
-        <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
-      </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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
-        </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
-        </tr>
+      val result =
+        // scalastyle:off
         <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>
+              <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-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
-          <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-timeline"}>{graphUIDataForNumberTotalRows.generateTimelineHtml(jsCollector)}</td>
+          <td class={"aggregated-num-total-state-rows-histogram"}>{graphUIDataForNumberTotalRows.generateHistogramHtml(jsCollector)}</td>
         </tr>
-      // scalastyle:on
+          <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-state-rows-timeline"}>{graphUIDataForNumberUpdatedRows.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-updated-state-rows-histogram"}>{graphUIDataForNumberUpdatedRows.generateHistogramHtml(jsCollector)}</td>
+          </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-state-memory-used-bytes-timeline"}>{graphUIDataForMemoryUsedBytes.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-state-memory-used-bytes-histogram"}>{graphUIDataForMemoryUsedBytes.generateHistogramHtml(jsCollector)}</td>
+          </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>
+              </div>
+            </td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-timeline"}>{graphUIDataForNumRowsDroppedByWatermark.generateTimelineHtml(jsCollector)}</td>
+            <td class={"aggregated-num-state-rows-dropped-by-watermark-histogram"}>{graphUIDataForNumRowsDroppedByWatermark.generateHistogramHtml(jsCollector)}</td>
+          </tr>
+        // scalastyle:on
+
+      result ++= generateAggregatedCustomMetrics(query, minBatchTime, maxBatchTime, jsCollector)
+      result
     } else {
       new NodeBuffer()
     }
   }
 
+  def generateAggregatedCustomMetrics(
+     query: StreamingQueryUIData,
+     minBatchTime: Long,
+     maxBatchTime: Long,
+     jsCollector: JsCollector): NodeBuffer = {
+    val result: NodeBuffer = new NodeBuffer
+
+    // This is made sure on caller side but put it here to be defensive
+    require(query.lastProgress.stateOperators.nonEmpty)
+    val enabledCustomMetrics = parent.parent.conf.get(ENABLED_STREAMING_UI_CUSTOM_METRIC_LIST)
+    logDebug(s"Enabled custom metrics: $enabledCustomMetrics")
+    query.lastProgress.stateOperators.head.customMetrics.keySet().asScala

Review comment:
       The custom metrics is bound to the `state store provider` instead of `operator`.
   
   Given the provider instances are created per state store id, it's not technically impossible for provider to provide different custom metrics, but at least that's not a purpose and you'll wonder how to leverage the flexibility. These metrics are for state itself, not related to the operator.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR closed pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
HeartSaVioR closed pull request #30336:
URL: https://github.com/apache/spark/pull/30336


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30336: [SPARK-33287][SS][UI]Expose state custom metrics information on SS UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30336:
URL: https://github.com/apache/spark/pull/30336#issuecomment-730412238


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org