-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API #21221
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API #21221
Changes from 25 commits
c8e8abe
5d6ae1c
ad10d28
10ed328
2d20367
f904f1e
c502ec4
7879e66
2662f6f
2871335
da83f2e
f25a44b
ca85c82
8b74ba8
036148c
91fb1db
2d8894a
99044e6
263c8c8
812fdcf
7ed42a5
8d9acdf
20799d2
8905d23
a0eed11
03cd5bc
10e7f15
a14b82a
2897281
ee4aa1d
571285b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,71 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark | ||
|
|
||
| import java.util.concurrent.TimeUnit | ||
|
|
||
| import org.apache.spark.executor.ExecutorMetrics | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.memory.MemoryManager | ||
| import org.apache.spark.metrics.ExecutorMetricType | ||
| import org.apache.spark.util.{ThreadUtils, Utils} | ||
|
|
||
| /** | ||
| * Creates a heartbeat thread which will call the specified reportHeartbeat function at | ||
| * intervals of intervalMs. | ||
| * | ||
| * @param memoryManager the memory manager for execution and storage memory. | ||
| * @param reportHeartbeat the heartbeat reporting function to call. | ||
| * @param name the thread name for the heartbeater. | ||
| * @param intervalMs the interval between heartbeats. | ||
| */ | ||
| private[spark] class Heartbeater( | ||
| memoryManager: MemoryManager, | ||
| reportHeartbeat: () => Unit, | ||
| name: String, | ||
| intervalMs: Long) extends Logging { | ||
| // Executor for the heartbeat task | ||
| private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor(name) | ||
|
|
||
| /** Schedules a task to report a heartbeat. */ | ||
| def start(): Unit = { | ||
| // Wait a random interval so the heartbeats don't end up in sync | ||
| val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] | ||
|
|
||
| val heartbeatTask = new Runnable() { | ||
| override def run(): Unit = Utils.logUncaughtExceptions(reportHeartbeat()) | ||
| } | ||
| heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) | ||
| } | ||
|
|
||
| /** Stops the heartbeat thread. */ | ||
| def stop(): Unit = { | ||
| heartbeater.shutdown() | ||
| heartbeater.awaitTermination(10, TimeUnit.SECONDS) | ||
| } | ||
|
|
||
| /** | ||
| * Get the current executor level metrics. These are returned as an array, with the index | ||
| * determined by MetricGetter.values | ||
| */ | ||
| def getCurrentMetrics(): ExecutorMetrics = { | ||
| val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray | ||
| new ExecutorMetrics(metrics) | ||
| } | ||
| } | ||
|
|
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,7 +38,7 @@ import org.apache.spark.internal.Logging | |
| import org.apache.spark.internal.config._ | ||
| import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} | ||
| import org.apache.spark.rpc.RpcTimeout | ||
| import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription} | ||
| import org.apache.spark.scheduler._ | ||
| import org.apache.spark.shuffle.FetchFailedException | ||
| import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} | ||
| import org.apache.spark.util._ | ||
|
|
@@ -148,7 +148,8 @@ private[spark] class Executor( | |
| private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] | ||
|
|
||
| // Executor for the heartbeat task. | ||
| private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") | ||
| private val heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat, | ||
| "executor-heartbeater", conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) | ||
|
|
||
| // must be initialized before running startDriverHeartbeat() | ||
| private val heartbeatReceiverRef = | ||
|
|
@@ -167,7 +168,7 @@ private[spark] class Executor( | |
| */ | ||
| private var heartbeatFailures = 0 | ||
|
|
||
| startDriverHeartbeater() | ||
| heartbeater.start() | ||
|
|
||
| private[executor] def numRunningTasks: Int = runningTasks.size() | ||
|
|
||
|
|
@@ -216,8 +217,7 @@ private[spark] class Executor( | |
|
|
||
| def stop(): Unit = { | ||
| env.metricsSystem.report() | ||
| heartbeater.shutdown() | ||
| heartbeater.awaitTermination(10, TimeUnit.SECONDS) | ||
| heartbeater.stop() | ||
|
||
| threadPool.shutdown() | ||
| if (!isLocal) { | ||
| env.stop() | ||
|
|
@@ -787,6 +787,9 @@ private[spark] class Executor( | |
| val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]() | ||
| val curGCTime = computeTotalGcTime() | ||
|
|
||
| // get executor level memory metrics | ||
| val executorUpdates = heartbeater.getCurrentMetrics() | ||
|
|
||
| for (taskRunner <- runningTasks.values().asScala) { | ||
| if (taskRunner.task != null) { | ||
| taskRunner.task.metrics.mergeShuffleReadMetrics() | ||
|
|
@@ -795,7 +798,8 @@ private[spark] class Executor( | |
| } | ||
| } | ||
|
|
||
| val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId) | ||
| val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId, | ||
| executorUpdates) | ||
| try { | ||
| val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( | ||
| message, RpcTimeout(conf, "spark.executor.heartbeatInterval", "10s")) | ||
|
|
@@ -815,21 +819,6 @@ private[spark] class Executor( | |
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Schedules a task to report heartbeat and partial metrics for active tasks to driver. | ||
| */ | ||
| private def startDriverHeartbeater(): Unit = { | ||
| val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") | ||
|
|
||
| // Wait a random interval so the heartbeats don't end up in sync | ||
| val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] | ||
|
|
||
| val heartbeatTask = new Runnable() { | ||
| override def run(): Unit = Utils.logUncaughtExceptions(reportHeartBeat()) | ||
| } | ||
| heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) | ||
| } | ||
| } | ||
|
|
||
| private[spark] object Executor { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,81 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.spark.executor | ||
|
|
||
| import org.apache.spark.annotation.DeveloperApi | ||
| import org.apache.spark.metrics.ExecutorMetricType | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * Metrics tracked for executors and the driver. | ||
| * | ||
| * Executor-level metrics are sent from each executor to the driver as part of the Heartbeat. | ||
| */ | ||
| @DeveloperApi | ||
| class ExecutorMetrics private[spark] extends Serializable { | ||
|
|
||
| // Metrics are indexed by MetricGetter.values | ||
| private val metrics = new Array[Long](ExecutorMetricType.values.length) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Out of curiosity, why are we using an array here with index-based fetching? We could use a struct / case class to represent these metrics. But I suppose the size of the payload we send is smaller if we use an Array, and we don't want to pay serialization costs?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I suggested this earlier in the reviews. Most of the operations for dealing with this data want to iterate over all the fields. its much easier this way vs. having a bazillion if (x.fizz > y.fizz) {
y.fizz = x.fizz
}
if (x.buzz > y.buzz) {
y.buzz = x.buzz
}
...
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yup that's fine - I did some googling, unfortunately there isn't a great way to iterate over fields of a case class. You could create a thin wrapper object around the array instead though, if we really think the nicer API is worthwhile: Or even this: The latter which would be better because you'd be guaranteed to create the struct with the right number of metrics. Though such abstractions are not necessary by any means.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is it likely that users would want to access the individual fields, rather than iterating through all? The 1st option would be a bit nicer if so.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Unclear - if we expose these metrics to some external consumer via an API for example, then we almost certainly want to have a schema labelling these fields for consumption by e.g. dashboards. I think what we have here is fine for now. |
||
|
|
||
| // the first element is initialized to -1, indicating that the values for the array | ||
| // haven't been set yet. | ||
| metrics(0) = -1 | ||
|
|
||
| /** Returns the value for the specified metricType. */ | ||
| def getMetricValue(metricType: ExecutorMetricType): Long = { | ||
| metrics(ExecutorMetricType.metricIdxMap(metricType)) | ||
| } | ||
|
|
||
| /** Returns true if the values for the metrics have been set, false otherwise. */ | ||
| def isSet(): Boolean = metrics(0) > -1 | ||
|
|
||
| private[spark] def this(metrics: Array[Long]) { | ||
| this() | ||
| Array.copy(metrics, 0, this.metrics, 0, Math.min(metrics.size, this.metrics.size)) | ||
| } | ||
|
|
||
| /** | ||
| * Constructor: create the ExecutorMetrics with the values specified. | ||
| * | ||
| * @param executorMetrics map of executor metric name to value | ||
| */ | ||
| private[spark] def this(executorMetrics: Map[String, Long]) { | ||
| this() | ||
| (0 until ExecutorMetricType.values.length).foreach { idx => | ||
| metrics(idx) = executorMetrics.getOrElse(ExecutorMetricType.values(idx).name, 0L) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Compare the specified executor metrics values with the current executor metric values, | ||
| * and update the value for any metrics where the new value for the metric is larger. | ||
| * | ||
| * @param executorMetrics the executor metrics to compare | ||
| * @return if there is a new peak value for any metric | ||
| */ | ||
| private[spark] def compareAndUpdatePeakValues(executorMetrics: ExecutorMetrics): Boolean = { | ||
| var updated: Boolean = false | ||
|
||
|
|
||
| (0 until ExecutorMetricType.values.length).foreach { idx => | ||
| if ( executorMetrics.metrics(idx) > metrics(idx)) { | ||
|
||
| updated = true | ||
| metrics(idx) = executorMetrics.metrics(idx) | ||
| } | ||
| } | ||
| updated | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -69,6 +69,11 @@ package object config { | |
| .bytesConf(ByteUnit.KiB) | ||
| .createWithDefaultString("100k") | ||
|
|
||
| private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS = | ||
| ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled") | ||
| .booleanConf | ||
| .createWithDefault(true) | ||
|
||
|
|
||
| private[spark] val EVENT_LOG_OVERWRITE = | ||
| ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -180,6 +180,26 @@ private[spark] abstract class MemoryManager( | |
| onHeapStorageMemoryPool.memoryUsed + offHeapStorageMemoryPool.memoryUsed | ||
| } | ||
|
|
||
| /** | ||
| * On heap execution memory currently in use, in bytes. | ||
| */ | ||
| final def onHeapExecutionMemoryUsed: Long = onHeapExecutionMemoryPool.memoryUsed | ||
|
||
|
|
||
| /** | ||
| * Off heap execution memory currently in use, in bytes. | ||
| */ | ||
| final def offHeapExecutionMemoryUsed: Long = offHeapExecutionMemoryPool.memoryUsed | ||
|
||
|
|
||
| /** | ||
| * On heap storage memory currently in use, in bytes. | ||
| */ | ||
| final def onHeapStorageMemoryUsed: Long = onHeapStorageMemoryPool.memoryUsed | ||
|
|
||
| /** | ||
| * Off heap storage memory currently in use, in bytes. | ||
| */ | ||
| final def offHeapStorageMemoryUsed: Long = offHeapStorageMemoryPool.memoryUsed | ||
|
|
||
| /** | ||
| * Returns the execution memory consumption, in bytes, for the given task. | ||
| */ | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: remove extra spaces for better indent