Skip to content

Commit

Permalink
[SPARK-34779][CORE] ExecutorMetricsPoller should keep stage entry in …
Browse files Browse the repository at this point in the history
…stageTCMP until a heartbeat occurs

### What changes were proposed in this pull request?
Allow ExecutorMetricsPoller to keep stage entries in stageTCMP until a heartbeat occurs even if the entries have task count = 0.

### Why are the changes needed?
This is an improvement.

The current implementation of ExecutorMetricsPoller keeps a map, stageTCMP of (stageId, stageAttemptId) to (count of running tasks, executor metric peaks). The entry for the stage is removed on task completion if the task count decreases to 0. In the case of an executor with a single core, this leads to unnecessary removal and insertion of entries for a given stage.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
A new unit test is added.

Closes #31871 from baohe-zhang/SPARK-34779.

Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: “attilapiros” <piros.attila.zsolt@gmail.com>
  • Loading branch information
Baohe Zhang authored and attilapiros committed Apr 2, 2021
1 parent 8724f2b commit f03c7c0
Show file tree
Hide file tree
Showing 2 changed files with 67 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -53,10 +53,10 @@ private[spark] class ExecutorMetricsPoller(

type StageKey = (Int, Int)
// Task Count and Metric Peaks
private case class TCMP(count: AtomicLong, peaks: AtomicLongArray)
private[executor] 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]
private[executor] val stageTCMP = new ConcurrentHashMap[StageKey, TCMP]

// Map of taskId to executor metric peaks
private val taskMetricPeaks = new ConcurrentHashMap[Long, AtomicLongArray]
Expand Down Expand Up @@ -124,17 +124,12 @@ private[spark] class ExecutorMetricsPoller(
*/
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 countValue = countAndPeaks.count.decrementAndGet()
if (countValue == 0L) {
logDebug(s"removing (${stage._1}, ${stage._2}) from stageTCMP")
null
} else {
logDebug(s"stageTCMP: (${stage._1}, ${stage._2}) -> " + countValue)
countAndPeaks
}
assert(countValue >= 0, "task count shouldn't below 0")
logDebug(s"stageTCMP: (${stage._1}, ${stage._2}) -> " + countValue)
countAndPeaks
}

stageTCMP.computeIfPresent((stageId, stageAttemptId), decrementCount)
Expand Down Expand Up @@ -176,6 +171,20 @@ private[spark] class ExecutorMetricsPoller(

stageTCMP.replaceAll(getUpdateAndResetPeaks)

def removeIfInactive(k: StageKey, v: TCMP): TCMP = {
if (v.count.get == 0) {
logDebug(s"removing (${k._1}, ${k._2}) from stageTCMP")
null
} else {
v
}
}

// Remove the entry from stageTCMP if the task count reaches zero.
executorUpdates.foreach { case (k, _) =>
stageTCMP.computeIfPresent(k, removeIfInactive)
}

executorUpdates
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.{SparkConf, SparkFunSuite}
import org.apache.spark.memory.TestMemoryManager

class ExecutorMetricsPollerSuite extends SparkFunSuite {

test("SPARK-34779: stage entry shouldn't be removed before a heartbeat occurs") {
val testMemoryManager = new TestMemoryManager(new SparkConf())
val poller = new ExecutorMetricsPoller(testMemoryManager, 1000, None)

poller.onTaskStart(0L, 0, 0)
// stage (0, 0) has an active task, so it remains on stageTCMP after heartbeat.
assert(poller.getExecutorUpdates.size === 1)
assert(poller.stageTCMP.size === 1)
assert(poller.stageTCMP.get((0, 0)).count.get === 1)

poller.onTaskCompletion(0L, 0, 0)
// stage (0, 0) doesn't have active tasks, but its entry will be kept until next
// heartbeat.
assert(poller.stageTCMP.size === 1)
assert(poller.stageTCMP.get((0, 0)).count.get === 0)

// the next heartbeat will report the peak metrics of stage (0, 0) during the
// previous heartbeat interval, then remove it from stageTCMP.
assert(poller.getExecutorUpdates.size === 1)
assert(poller.stageTCMP.size === 0)

poller.stop
}
}

0 comments on commit f03c7c0

Please sign in to comment.