Skip to content

Commit 729f43f

Browse files
LucaCanalisquito
authored andcommitted
[SPARK-27189][CORE] Add Executor metrics and memory usage instrumentation to the metrics system
## What changes were proposed in this pull request? This PR proposes to add instrumentation of memory usage via the Spark Dropwizard/Codahale metrics system. Memory usage metrics are available via the Executor metrics, recently implemented as detailed in https://issues.apache.org/jira/browse/SPARK-23206. Additional notes: This takes advantage of the metrics poller introduced in #23767. ## Why are the changes needed? Executor metrics bring have many useful insights on memory usage, in particular on the usage of storage memory and executor memory. This is useful for troubleshooting. Having the information in the metrics systems allows to add those metrics to Spark performance dashboards and study memory usage as a function of time, as in the example graph https://issues.apache.org/jira/secure/attachment/12962810/Example_dashboard_Spark_Memory_Metrics.PNG ## Does this PR introduce any user-facing change? Adds `ExecutorMetrics` source to publish executor metrics via the Dropwizard metrics system. Details of the available metrics in docs/monitoring.md Adds configuration parameter `spark.metrics.executormetrics.source.enabled` ## How was this patch tested? Tested on YARN cluster and with an existing setup for a Spark dashboard based on InfluxDB and Grafana. Closes #24132 from LucaCanali/memoryMetricsSource. Authored-by: Luca Canali <luca.canali@cern.ch> Signed-off-by: Imran Rashid <irashid@cloudera.com>
1 parent a717d21 commit 729f43f

File tree

7 files changed

+167
-6
lines changed

7 files changed

+167
-6
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 13 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,7 @@ import org.apache.spark.annotation.DeveloperApi
4242
import org.apache.spark.broadcast.Broadcast
4343
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
4444
import org.apache.spark.deploy.StandaloneResourceUtils._
45-
import org.apache.spark.executor.ExecutorMetrics
45+
import org.apache.spark.executor.{ExecutorMetrics, ExecutorMetricsSource}
4646
import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat}
4747
import org.apache.spark.internal.Logging
4848
import org.apache.spark.internal.config._
@@ -551,9 +551,16 @@ class SparkContext(config: SparkConf) extends Logging {
551551
_dagScheduler = new DAGScheduler(this)
552552
_heartbeatReceiver.ask[Boolean](TaskSchedulerIsSet)
553553

554+
val _executorMetricsSource =
555+
if (_conf.get(METRICS_EXECUTORMETRICS_SOURCE_ENABLED)) {
556+
Some(new ExecutorMetricsSource)
557+
} else {
558+
None
559+
}
560+
554561
// create and start the heartbeater for collecting memory metrics
555562
_heartbeater = new Heartbeater(
556-
() => SparkContext.this.reportHeartBeat(),
563+
() => SparkContext.this.reportHeartBeat(_executorMetricsSource),
557564
"driver-heartbeater",
558565
conf.get(EXECUTOR_HEARTBEAT_INTERVAL))
559566
_heartbeater.start()
@@ -622,6 +629,7 @@ class SparkContext(config: SparkConf) extends Logging {
622629
_env.metricsSystem.registerSource(_dagScheduler.metricsSource)
623630
_env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager))
624631
_env.metricsSystem.registerSource(new JVMCPUSource())
632+
_executorMetricsSource.foreach(_.register(_env.metricsSystem))
625633
_executorAllocationManager.foreach { e =>
626634
_env.metricsSystem.registerSource(e.executorAllocationManagerSource)
627635
}
@@ -2473,8 +2481,10 @@ class SparkContext(config: SparkConf) extends Logging {
24732481
}
24742482

24752483
/** Reports heartbeat metrics for the driver. */
2476-
private def reportHeartBeat(): Unit = {
2484+
private def reportHeartBeat(executorMetricsSource: Option[ExecutorMetricsSource]): Unit = {
24772485
val currentMetrics = ExecutorMetrics.getCurrentMetrics(env.memoryManager)
2486+
executorMetricsSource.foreach(_.updateMetricsSnapshot(currentMetrics))
2487+
24782488
val driverUpdates = new HashMap[(Int, Int), ExecutorMetrics]
24792489
// In the driver, we do not track per-stage metrics, so use a dummy stage for the key
24802490
driverUpdates.put(EventLoggingListener.DRIVER_STAGE_KEY, new ExecutorMetrics(currentMetrics))

core/src/main/scala/org/apache/spark/executor/Executor.scala

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -113,10 +113,18 @@ private[spark] class Executor(
113113
// create. The map key is a task id.
114114
private val taskReaperForTask: HashMap[Long, TaskReaper] = HashMap[Long, TaskReaper]()
115115

116+
val executorMetricsSource =
117+
if (conf.get(METRICS_EXECUTORMETRICS_SOURCE_ENABLED)) {
118+
Some(new ExecutorMetricsSource)
119+
} else {
120+
None
121+
}
122+
116123
if (!isLocal) {
117124
env.blockManager.initialize(conf.getAppId)
118125
env.metricsSystem.registerSource(executorSource)
119126
env.metricsSystem.registerSource(new JVMCPUSource())
127+
executorMetricsSource.foreach(_.register(env.metricsSystem))
120128
env.metricsSystem.registerSource(env.blockManager.shuffleMetricsSource)
121129
}
122130

@@ -181,7 +189,8 @@ private[spark] class Executor(
181189
// Poller for the memory metrics. Visible for testing.
182190
private[executor] val metricsPoller = new ExecutorMetricsPoller(
183191
env.memoryManager,
184-
METRICS_POLLING_INTERVAL_MS)
192+
METRICS_POLLING_INTERVAL_MS,
193+
executorMetricsSource)
185194

186195
// Executor for the heartbeat task.
187196
private val heartbeater = new Heartbeater(

core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,8 @@ import org.apache.spark.util.{ThreadUtils, Utils}
4848
*/
4949
private[spark] class ExecutorMetricsPoller(
5050
memoryManager: MemoryManager,
51-
pollingInterval: Long) extends Logging {
51+
pollingInterval: Long,
52+
executorMetricsSource: Option[ExecutorMetricsSource]) extends Logging {
5253

5354
type StageKey = (Int, Int)
5455
// Task Count and Metric Peaks
@@ -79,6 +80,7 @@ private[spark] class ExecutorMetricsPoller(
7980

8081
// get the latest values for the metrics
8182
val latestMetrics = ExecutorMetrics.getCurrentMetrics(memoryManager)
83+
executorMetricsSource.foreach(_.updateMetricsSnapshot(latestMetrics))
8284

8385
def updatePeaks(metrics: AtomicLongArray): Unit = {
8486
(0 until metrics.length).foreach { i =>
Lines changed: 65 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,65 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.executor
19+
20+
import com.codahale.metrics.{Gauge, MetricRegistry}
21+
22+
import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem}
23+
import org.apache.spark.metrics.source.Source
24+
25+
/**
26+
* Expose executor metrics from [[ExecutorMetricsType]] using the Dropwizard metrics system.
27+
*
28+
* Metrics related to the memory system can be expensive to gather, therefore
29+
* we implement some optimizations:
30+
* (1) Metrics values are cached, updated at each heartbeat (default period is 10 seconds).
31+
* An alternative faster polling mechanism is used, only if activated, by setting
32+
* spark.executor.metrics.pollingInterval=<interval in ms>.
33+
* (2) Procfs metrics are gathered all in one-go and only conditionally:
34+
* if the /proc filesystem exists
35+
* and spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true
36+
* and spark.eventLog.logStageExecutorMetrics.enabled=true.
37+
*/
38+
private[spark] class ExecutorMetricsSource extends Source {
39+
40+
override val metricRegistry = new MetricRegistry()
41+
override val sourceName = "ExecutorMetrics"
42+
@volatile var metricsSnapshot: Array[Long] = Array.fill(ExecutorMetricType.numMetrics)(0L)
43+
44+
// called by ExecutorMetricsPoller
45+
def updateMetricsSnapshot(metricsUpdates: Array[Long]): Unit = {
46+
metricsSnapshot = metricsUpdates
47+
}
48+
49+
private class ExecutorMetricGauge(idx: Int) extends Gauge[Long] {
50+
def getValue: Long = metricsSnapshot(idx)
51+
}
52+
53+
def register(metricsSystem: MetricsSystem): Unit = {
54+
val gauges: IndexedSeq[ExecutorMetricGauge] = (0 until ExecutorMetricType.numMetrics).map {
55+
idx => new ExecutorMetricGauge(idx)
56+
}.toIndexedSeq
57+
58+
ExecutorMetricType.metricToOffset.foreach {
59+
case (name, idx) =>
60+
metricRegistry.register(MetricRegistry.name(name), gauges(idx))
61+
}
62+
63+
metricsSystem.registerSource(this)
64+
}
65+
}

core/src/main/scala/org/apache/spark/internal/config/package.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -637,6 +637,12 @@ package object config {
637637
.stringConf
638638
.createOptional
639639

640+
private[spark] val METRICS_EXECUTORMETRICS_SOURCE_ENABLED =
641+
ConfigBuilder("spark.metrics.executorMetricsSource.enabled")
642+
.doc("Whether to register the ExecutorMetrics source with the metrics system.")
643+
.booleanConf
644+
.createWithDefault(true)
645+
640646
private[spark] val METRICS_STATIC_SOURCES_ENABLED =
641647
ConfigBuilder("spark.metrics.staticSources.enabled")
642648
.doc("Whether to register static sources with the metrics system.")

core/src/test/scala/org/apache/spark/metrics/source/SourceConfigSuite.scala

Lines changed: 29 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@
1818
package org.apache.spark.metrics.source
1919

2020
import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
21-
import org.apache.spark.internal.config.METRICS_STATIC_SOURCES_ENABLED
21+
import org.apache.spark.internal.config.{METRICS_EXECUTORMETRICS_SOURCE_ENABLED, METRICS_STATIC_SOURCES_ENABLED}
2222

2323
class SourceConfigSuite extends SparkFunSuite with LocalSparkContext {
2424

@@ -52,4 +52,32 @@ class SourceConfigSuite extends SparkFunSuite with LocalSparkContext {
5252
}
5353
}
5454

55+
test("Test configuration for adding ExecutorMetrics source registration") {
56+
val conf = new SparkConf()
57+
conf.set(METRICS_EXECUTORMETRICS_SOURCE_ENABLED, true)
58+
val sc = new SparkContext("local", "test", conf)
59+
try {
60+
val metricsSystem = sc.env.metricsSystem
61+
62+
// ExecutorMetrics source should be registered
63+
assert (metricsSystem.getSourcesByName("ExecutorMetrics").nonEmpty)
64+
} finally {
65+
sc.stop()
66+
}
67+
}
68+
69+
test("Test configuration for skipping ExecutorMetrics source registration") {
70+
val conf = new SparkConf()
71+
conf.set(METRICS_EXECUTORMETRICS_SOURCE_ENABLED, false)
72+
val sc = new SparkContext("local", "test", conf)
73+
try {
74+
val metricsSystem = sc.env.metricsSystem
75+
76+
// ExecutorMetrics source should not be registered
77+
assert (metricsSystem.getSourcesByName("ExecutorMetrics").isEmpty)
78+
} finally {
79+
sc.stop()
80+
}
81+
}
82+
5583
}

docs/monitoring.md

Lines changed: 41 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -995,6 +995,12 @@ This is the component with the largest amount of instrumented metrics
995995
- namespace=JVMCPU
996996
- jvmCpuTime
997997

998+
- namespace=ExecutorMetrics
999+
- **note:** these metrics are conditional to a configuration parameter:
1000+
`spark.metrics.executorMetricsSource.enabled` (default is true)
1001+
- This source contains memory-related metrics. A full list of available metrics in this
1002+
namespace can be found in the corresponding entry for the Executor component instance.
1003+
9981004
- namespace=plugin.\<Plugin Class Name>
9991005
- Optional namespace(s). Metrics in this namespace are defined by user-supplied code, and
10001006
configured using the Spark plugin API. See "Advanced Instrumentation" below for how to load
@@ -1046,6 +1052,41 @@ when running in local mode.
10461052
- threadpool.maxPool_size
10471053
- threadpool.startedTasks
10481054

1055+
- namespace=ExecutorMetrics
1056+
- **notes:**
1057+
- These metrics are conditional to a configuration parameter:
1058+
`spark.metrics.executorMetricsSource.enabled` (default value is true)
1059+
- ExecutorMetrics are updated as part of heartbeat processes scheduled
1060+
for the executors and for the driver at regular intervals: `spark.executor.heartbeatInterval` (default value is 10 seconds)
1061+
- An optional faster polling mechanism is available for executor memory metrics,
1062+
it can be activated by setting a polling interval (in milliseconds) using the configuration parameter `spark.executor.metrics.pollingInterval`
1063+
- JVMHeapMemory
1064+
- JVMOffHeapMemory
1065+
- OnHeapExecutionMemory
1066+
- OnHeapStorageMemory
1067+
- OnHeapUnifiedMemory
1068+
- OffHeapExecutionMemory
1069+
- OffHeapStorageMemory
1070+
- OffHeapUnifiedMemory
1071+
- DirectPoolMemory
1072+
- MappedPoolMemory
1073+
- MinorGCCount
1074+
- MinorGCTime
1075+
- MajorGCCount
1076+
- MajorGCTime
1077+
- "ProcessTree*" metric counters:
1078+
- ProcessTreeJVMVMemory
1079+
- ProcessTreeJVMRSSMemory
1080+
- ProcessTreePythonVMemory
1081+
- ProcessTreePythonRSSMemory
1082+
- ProcessTreeOtherVMemory
1083+
- ProcessTreeOtherRSSMemory
1084+
- **note:** "ProcessTree*" metrics are collected only under certain conditions.
1085+
The conditions are the logical AND of the following: `/proc` filesystem exists,
1086+
`spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true`,
1087+
`spark.eventLog.logStageExecutorMetrics.enabled=true`.
1088+
"ProcessTree*" metrics report 0 when those conditions are not met.
1089+
10491090
- namespace=JVMCPU
10501091
- jvmCpuTime
10511092

0 commit comments

Comments
 (0)