File tree Expand file tree Collapse file tree
catalyst/src/main/scala/org/apache/spark/sql/internal
core/src/main/scala/org/apache/spark/sql/streaming/ui Expand file tree Collapse file tree Original file line number Diff line number Diff line change @@ -2875,14 +2875,15 @@ object SQLConf {
28752875 .stringConf
28762876 .createWithDefault(" " )
28772877
2878- val DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST =
2879- buildConf(" spark.sql.streaming.disabledUICustomMetricsList " )
2878+ val DISABLED_STREAMING_UI_CUSTOM_METRIC_LIST =
2879+ buildConf(" spark.sql.streaming.disabledUICustomMetricList " )
28802880 .internal()
28812881 .doc(" Configures a list of custom metrics on Structured Streaming UI, which are disabled. " +
28822882 " The list contains the name of the custom metrics separated by comma." )
28832883 .version(" 3.1.0" )
28842884 .stringConf
2885- .createWithDefault(" loadedMapCacheHitCount" )
2885+ .toSequence
2886+ .createWithDefault(Seq (" loadedMapCacheHitCount" ))
28862887
28872888 /**
28882889 * Holds information about keys that have been deprecated.
@@ -3521,9 +3522,6 @@ class SQLConf extends Serializable with Logging {
35213522
35223523 def disabledJdbcConnectionProviders : String = getConf(SQLConf .DISABLED_JDBC_CONN_PROVIDER_LIST )
35233524
3524- def disabledStreamingCustomMetrics : String =
3525- getConf(SQLConf .DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST )
3526-
35273525 /** ********************** SQLConf functionality methods ************ */
35283526
35293527 /** Set Spark SQL configuration properties. */
Original file line number Diff line number Diff line change @@ -250,15 +250,17 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab)
250250 }
251251
252252 def generateAggregatedCustomMetrics (
253- query : StreamingQueryUIData ,
254- minBatchTime : Long ,
255- maxBatchTime : Long ,
256- jsCollector : JsCollector ): NodeBuffer = {
253+ query : StreamingQueryUIData ,
254+ minBatchTime : Long ,
255+ maxBatchTime : Long ,
256+ jsCollector : JsCollector ): NodeBuffer = {
257257 val result : NodeBuffer = new NodeBuffer
258258
259259 // This is made sure on caller side but put it here to be defensive
260260 require(query.lastProgress.stateOperators.nonEmpty)
261- val disabledCustomMetrics = Utils .stringToSeq(SQLConf .get.disabledStreamingCustomMetrics)
261+ val disabledCustomMetrics =
262+ SQLConf .get.getConf(SQLConf .DISABLED_STREAMING_UI_CUSTOM_METRIC_LIST )
263+ logDebug(s " Disabled custom metrics: $disabledCustomMetrics" )
262264 query.lastProgress.stateOperators.head.customMetrics.keySet().asScala
263265 .filterNot(disabledCustomMetrics.contains(_)).map { metricName =>
264266 val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),
You can’t perform that action at this time.
0 commit comments