You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by at...@apache.org on 2021/04/02 05:14:53 UTC

[spark] branch master updated: [SPARK-34779][CORE] ExecutorMetricsPoller should keep stage entry in stageTCMP until a heartbeat occurs

This is an automated email from the ASF dual-hosted git repository.

attilapiros pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new f03c7c0  [SPARK-34779][CORE] ExecutorMetricsPoller should keep stage entry in stageTCMP until a heartbeat occurs
f03c7c0 is described below

commit f03c7c0e9dd6b6b87049b8546460b7f21c086749
Author: Baohe Zhang <ba...@verizonmedia.com>
AuthorDate: Fri Apr 2 07:14:18 2021 +0200

    [SPARK-34779][CORE] ExecutorMetricsPoller should keep stage entry in 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 <ba...@verizonmedia.com>
    Signed-off-by: “attilapiros” <pi...@gmail.com>
---
 .../spark/executor/ExecutorMetricsPoller.scala     | 29 ++++++++-----
 .../executor/ExecutorMetricsPollerSuite.scala      | 48 ++++++++++++++++++++++
 2 files changed, 67 insertions(+), 10 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala
index 1c1a1ca..0cdb306 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala
@@ -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]
@@ -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)
@@ -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
   }
 
diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorMetricsPollerSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorMetricsPollerSuite.scala
new file mode 100644
index 0000000..e471864
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/executor/ExecutorMetricsPollerSuite.scala
@@ -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
+  }
+}

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org