-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-26329][CORE] Faster polling of executor memory metrics. #23767
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
Changes from 1 commit
9f5b195
03e41a8
7f6bd74
7397897
e1aeafc
75ba39d
ea2ff0d
0cbfc04
8cb30a8
077abb0
7a3c90d
3ed583a
0a4828a
9530b75
38a397c
e062e60
20b4b7e
b898ad2
fbb55bf
99addf1
7331b27
7556d6a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
Fix some bugs in ExecutorSuite as well.
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,14 +22,12 @@ import java.util.concurrent.atomic.{AtomicLong, AtomicLongArray} | |
|
|
||
| import scala.collection.mutable.HashMap | ||
|
|
||
| import org.apache.spark.annotation.DeveloperApi | ||
| 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} | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * A class that polls executor metrics, and tracks their peaks per task and per stage. | ||
| * Each executor keeps an instance of this class. | ||
| * The poll method polls the executor metrics, and is either run in its own thread or | ||
|
|
@@ -38,19 +36,31 @@ import org.apache.spark.util.{ThreadUtils, Utils} | |
| * executor's task runner threads concurrently with the polling thread. One thread may | ||
| * update one of these maps while another reads it, so the reading thread may not get | ||
| * the latest metrics, but this is ok. | ||
| * One ConcurrentHashMap tracks the number of running tasks and the executor metric | ||
| * peaks for each stage. A positive task count means the stage is active. When the task | ||
| * count reaches zero for a stage, we remove the entry from the map. That way, the map | ||
| * only contains entries for active stages and does not grow without bound. On every | ||
| * heartbeat, the executor gets the per-stage metric peaks from this class and sends | ||
| * them and the peaks are reset. | ||
| * The other ConcurrentHashMap tracks the executor metric peaks for each task (the peaks | ||
| * seen while each task is running). At task end, these peaks are sent with the task | ||
| * result by the task runner. | ||
| * The reason we track executor metric peaks per task in addition to per stage is: | ||
| * If between heartbeats, a stage completes, so there are no more running tasks for that | ||
| * stage, then in the next heartbeat, there are no metrics sent for that stage; however, | ||
| * at the end of a task that belonged to that stage, the metrics would have been sent | ||
| * in the task result, so we do not lose those peaks. | ||
| * | ||
wypoon marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| * @param memoryManager the memory manager used by the executor. | ||
wypoon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| * @param pollingInterval the polling interval in milliseconds. | ||
| */ | ||
| @DeveloperApi | ||
| private[spark] class ExecutorMetricsPoller( | ||
|
||
| memoryManager: MemoryManager, | ||
| pollingInterval: Long) | ||
| extends Logging { | ||
| pollingInterval: Long) extends Logging { | ||
|
|
||
| type StageKey = (Int, Int) | ||
| // tuple for Task Count and Metric Peaks | ||
| type TCMP = (AtomicLong, AtomicLongArray) | ||
| // Task Count and Metric Peaks | ||
| case class TCMP(count: AtomicLong, peaks: AtomicLongArray) | ||
|
|
||
| // Map of (stageId, stageAttemptId) to (count of running tasks, executor metric peaks) | ||
| private val stageTCMP = new ConcurrentHashMap[StageKey, TCMP] | ||
|
|
@@ -85,7 +95,7 @@ private[spark] class ExecutorMetricsPoller( | |
| } | ||
|
|
||
| // for each active stage, update the peaks | ||
| stageTCMP.forEachValue(LONG_MAX_VALUE, v => updatePeaks(v._2)) | ||
| stageTCMP.forEachValue(LONG_MAX_VALUE, v => updatePeaks(v.peaks)) | ||
|
|
||
| // for each running task, update the peaks | ||
| taskMetricPeaks.forEachValue(LONG_MAX_VALUE, updatePeaks) | ||
|
|
@@ -101,38 +111,29 @@ private[spark] class ExecutorMetricsPoller( | |
|
|
||
| /** | ||
| * Called by TaskRunner#run. | ||
| * | ||
| * @param taskId the id of the task being run. | ||
| * @param stageId the id of the stage the task belongs to. | ||
| * @param stageAttemptId the attempt number of the stage the task belongs to. | ||
| */ | ||
| def onTaskStart(taskId: Long, stageId: Int, stageAttemptId: Int): Unit = { | ||
| // Put an entry in taskMetricPeaks for the task. | ||
| taskMetricPeaks.put(taskId, new AtomicLongArray(ExecutorMetricType.numMetrics)) | ||
|
|
||
| // Put a new entry in stageTCMP for the stage if there isn't one already. | ||
| // Increment the task count. | ||
| val (count, _) = stageTCMP.computeIfAbsent((stageId, stageAttemptId), | ||
| _ => (new AtomicLong(0), new AtomicLongArray(ExecutorMetricType.numMetrics))) | ||
| val stageCount = count.incrementAndGet() | ||
| val countAndPeaks = stageTCMP.computeIfAbsent((stageId, stageAttemptId), | ||
| _ => TCMP(new AtomicLong(0), new AtomicLongArray(ExecutorMetricType.numMetrics))) | ||
| val stageCount = countAndPeaks.count.incrementAndGet() | ||
| logDebug(s"stageTCMP: ($stageId, $stageAttemptId) -> $stageCount") | ||
| } | ||
|
|
||
| /** | ||
| * Called by TaskRunner#run. It should only be called if onTaskStart has been called with | ||
| * the same arguments. | ||
| * | ||
| * @param taskId the id of the task that was run. | ||
| * @param stageId the id of the stage the task belongs to. | ||
| * @param stageAttemptId the attempt number of the stage the task belongs to. | ||
| */ | ||
| def onTaskCompletion(taskId: Long, stageId: Int, stageAttemptId: Int): Unit = { | ||
| // Decrement the task count. | ||
| // Remove the entry from stageTCMP if the task count reaches zero. | ||
|
|
||
| def decrementCount(stage: StageKey, countAndPeaks: TCMP): TCMP = { | ||
| val count = countAndPeaks._1 | ||
| val countValue = count.decrementAndGet() | ||
| val countValue = countAndPeaks.count.decrementAndGet() | ||
| if (countValue == 0L) { | ||
| logDebug(s"removing (${stage._1}, ${stage._2}) from stageTCMP") | ||
| null | ||
|
|
@@ -150,8 +151,6 @@ private[spark] class ExecutorMetricsPoller( | |
|
|
||
| /** | ||
| * Called by TaskRunner#run. | ||
| * | ||
| * @param taskId the id of the task that was run. | ||
| */ | ||
| def getTaskMetricPeaks(taskId: Long): Array[Long] = { | ||
| // If this is called with an invalid taskId or a valid taskId but the task was killed and | ||
|
|
@@ -177,8 +176,8 @@ private[spark] class ExecutorMetricsPoller( | |
| val executorUpdates = new HashMap[StageKey, ExecutorMetrics] | ||
|
|
||
| def getUpdateAndResetPeaks(k: StageKey, v: TCMP): TCMP = { | ||
| executorUpdates.put(k, new ExecutorMetrics(v._2)) | ||
| (v._1, new AtomicLongArray(ExecutorMetricType.numMetrics)) | ||
| executorUpdates.put(k, new ExecutorMetrics(v.peaks)) | ||
| TCMP(v.count, new AtomicLongArray(ExecutorMetricType.numMetrics)) | ||
| } | ||
|
|
||
| stageTCMP.replaceAll(getUpdateAndResetPeaks) | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.