diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8ebe2bb1e8b57..23b914e779b83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2875,14 +2875,15 @@ object SQLConf { .stringConf .createWithDefault("") - val DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST = - buildConf("spark.sql.streaming.disabledUICustomMetricsList") + val DISABLED_STREAMING_UI_CUSTOM_METRIC_LIST = + buildConf("spark.sql.streaming.disabledUICustomMetricList") .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 - .createWithDefault("loadedMapCacheHitCount") + .toSequence + .createWithDefault(Seq("loadedMapCacheHitCount")) /** * Holds information about keys that have been deprecated. @@ -3521,9 +3522,6 @@ class SQLConf extends Serializable with Logging { def disabledJdbcConnectionProviders: String = getConf(SQLConf.DISABLED_JDBC_CONN_PROVIDER_LIST) - def disabledStreamingCustomMetrics: String = - getConf(SQLConf.DISABLED_STREAMING_UI_CUSTOM_METRICS_LIST) - /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala index a2fcc68552c9b..691328f064a33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala @@ -250,15 +250,17 @@ private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) } def generateAggregatedCustomMetrics( - query: StreamingQueryUIData, - minBatchTime: Long, - maxBatchTime: Long, - jsCollector: JsCollector): NodeBuffer = { + 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) + val disabledCustomMetrics = + SQLConf.get.getConf(SQLConf.DISABLED_STREAMING_UI_CUSTOM_METRIC_LIST) + logDebug(s"Disabled custom metrics: $disabledCustomMetrics") query.lastProgress.stateOperators.head.customMetrics.keySet().asScala .filterNot(disabledCustomMetrics.contains(_)).map { metricName => val data = query.recentProgress.map(p => (parseProgressTimestamp(p.timestamp),