holdenk commented on a change in pull request #29014:
URL: https://github.com/apache/spark/pull/29014#discussion_r457705293



##########
File path: 
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
##########
@@ -912,13 +914,39 @@ private[spark] class TaskSchedulerImpl(
     }
   }
 
-  override def executorDecommission(executorId: String): Unit = {
+  override def executorDecommission(
+      executorId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = {
+    synchronized {
+      if (!executorsPendingDecommission.contains(executorId)) {
+        executorsPendingDecommission(executorId) = decommissionInfo
+      }
+    }
     rootPool.executorDecommission(executorId)
     backend.reviveOffers()
   }
 
-  override def executorLost(executorId: String, reason: ExecutorLossReason): 
Unit = {
+  override def getExecutorDecommissionInfo(executorId: String)
+    : Option[ExecutorDecommissionInfo] = synchronized {
+      executorsPendingDecommission.get(executorId)
+  }
+
+  override def executorLost(executorId: String, givenReason: 
ExecutorLossReason): Unit = {
     var failedExecutor: Option[String] = None
+    val reason = givenReason match {
+      // Handle slave loss due to decommissioning

Review comment:
       s/slave/ExecutorProcess/

##########
File path: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
##########
@@ -1767,10 +1767,18 @@ private[spark] class DAGScheduler(
 
           // TODO: mark the executor as failed only if there were lots of 
fetch failures on it
           if (bmAddress != null) {
-            val hostToUnregisterOutputs = if 
(env.blockManager.externalShuffleServiceEnabled &&
-              unRegisterOutputOnHostOnFetchFailure) {
-              // We had a fetch failure with the external shuffle service, so 
we
-              // assume all shuffle data on the node is bad.
+            val externalShuffleServiceEnabled = 
env.blockManager.externalShuffleServiceEnabled
+            val isHostDecommissioned = taskScheduler
+              .getExecutorDecommissionInfo(bmAddress.executorId)
+              .exists(_.isHostDecommissioned)
+            // Host shuffle data is considered lost if:
+            // - If we know that the host was decommissioned
+            // - Or when `unRegisterOutputOnHostOnFetchFailure` is enabled and 
we had
+            //   a fetch failure with the external shuffle service, so we 
assume all
+            //   shuffle data on the node is bad.
+            val hostLost = isHostDecommissioned || 
(externalShuffleServiceEnabled &&

Review comment:
       I think for #1 updating the description text is probably the best path 
forward, I don't think we want to use that config flag.

##########
File path: 
core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala
##########
@@ -245,8 +249,8 @@ class AppClientSuite
       execRemovedList.add(id)
     }
 
-    def executorDecommissioned(id: String, message: String): Unit = {
-      execDecommissionedList.add(id)
+    def executorDecommissioned(id: String, decommissionInfo: 
ExecutorDecommissionInfo): Unit = {
+      execDecommissionedMap.putIfAbsent(id, decommissionInfo)

Review comment:
       If we get a new decommission message would we want to overwrite the old 
one?

##########
File path: 
core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.deploy
+
+import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue}
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration._
+
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.concurrent.Eventually._
+import org.scalatest.time.Span
+
+import org.apache.spark._
+import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, 
RequestMasterState, WorkerDecommission}
+import org.apache.spark.deploy.master.{ApplicationInfo, Master, WorkerInfo}
+import org.apache.spark.deploy.worker.Worker
+import org.apache.spark.internal.{config, Logging}
+import org.apache.spark.network.TransportContext
+import org.apache.spark.network.netty.SparkTransportConf
+import org.apache.spark.network.shuffle.ExternalBlockHandler
+import org.apache.spark.rpc.{RpcAddress, RpcEnv}
+import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd, 
SparkListenerStageSubmitted, SparkListenerTaskEnd, SparkListenerTaskStart, 
TaskInfo}
+import org.apache.spark.shuffle.FetchFailedException
+import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.util.Utils
+
+class DecommissionWorkerSuite
+  extends SparkFunSuite
+    with Logging
+    with LocalSparkContext
+    with BeforeAndAfterEach {
+
+  private val conf = new SparkConf()
+  private val securityManager = new SecurityManager(conf)
+
+  private var masterRpcEnv: RpcEnv = null
+  private var master: Master = null
+  private val workerIdToRpcEnvs: mutable.HashMap[String, RpcEnv] = 
mutable.HashMap.empty
+  private val workers: mutable.ArrayBuffer[Worker] = mutable.ArrayBuffer.empty
+
+  override def beforeEach(): Unit = {
+    super.beforeEach()
+    masterRpcEnv = RpcEnv.create(Master.SYSTEM_NAME, "localhost", 0, conf, 
securityManager)
+    master = makeMaster()
+  }
+
+  override def afterEach(): Unit = {
+    try {
+      masterRpcEnv.shutdown()
+      workerIdToRpcEnvs.values.foreach(_.shutdown())
+      workerIdToRpcEnvs.clear()
+      master.stop()
+      workers.foreach(_.stop())
+      workers.clear()
+      masterRpcEnv = null
+    } finally {
+      super.afterEach()
+    }
+  }
+
+  test("decommission workers should not result in job failure") {
+    val maxTaskFailures = conf.get(config.TASK_MAX_FAILURES)
+    val numTimesToKillWorkers = maxTaskFailures + 1
+    val numWorkers = numTimesToKillWorkers + 1
+    makeWorkers(numWorkers)
+    sc = createSparkContext(appConf)
+    val executorIdToWorkerInfo = getExecutorToWorkerAssignments
+    val taskIdsKilled = new ConcurrentHashMap[Long, Boolean]
+    val listener = new RootStageAwareListener {
+      override def handleRootTaskStart(taskStart: SparkListenerTaskStart): 
Unit = {
+        val taskInfo = taskStart.taskInfo
+        if (taskInfo.index == 0) {
+          if (taskIdsKilled.size() < numTimesToKillWorkers) {
+            val workerInfo = executorIdToWorkerInfo(taskInfo.executorId)
+            decommissionWorkerOnMaster(workerInfo, "partition 0 must die")
+            killWorkerAfterTimeout(workerInfo, 1)
+            taskIdsKilled.put(taskInfo.taskId, true)
+          }
+        } else {
+          assert(false, s"Unknown task index ${taskInfo.index}")
+        }
+      }
+
+      override def handleRootTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
+        val taskInfo = taskEnd.taskInfo
+        if (taskInfo.index == 0) {
+          if (taskIdsKilled.get(taskInfo.taskId)) {
+            assert(taskInfo.successful === false)
+          } else {
+            assert(taskInfo.successful && taskInfo.attemptNumber === 0)

Review comment:
       So if we keep killing the task for partition 0 the task attempt number 
does not go up and we just keep giving it new taskIds? Just wanting to make 
sure I understand the logic of the test.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to