Skip to content

Commit

Permalink
Partial review fixes
Browse files Browse the repository at this point in the history
  • Loading branch information
gaborgsomogyi committed Nov 13, 2020
1 parent 2eefeb2 commit 6654934
Show file tree
Hide file tree
Showing 2 changed files with 11 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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),
Expand Down

0 comments on commit 6654934

Please sign in to comment.