attilapiros commented on a change in pull request #29014: URL: https://github.com/apache/spark/pull/29014#discussion_r460525105
########## File path: core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala ########## @@ -0,0 +1,424 @@ +/* + * 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().set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) + 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 = 2 + conf.set(config.TASK_MAX_FAILURES, maxTaskFailures) + val numTimesToKillWorkers = maxTaskFailures + 1 + val numWorkers = numTimesToKillWorkers + 1 + createWorkers(numWorkers) + + // Here we will have a single task job and we will keep decommissioning (and killing) the + // worker running that task K times. Where K is more than the maxTaskFailures. Since the worker + // is notified of the decommissioning, the task failures can be ignored and not fail + // the job. + + 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 + delayedAssert(taskInfo.index == 0, s"Unknown task index ${taskInfo.index}") + if (taskIdsKilled.size() < numTimesToKillWorkers) { + val workerInfo = executorIdToWorkerInfo(taskInfo.executorId) + decommissionWorkerOnMaster(workerInfo, "partition 0 must die") + killWorkerAfterTimeout(workerInfo, 1) + taskIdsKilled.put(taskInfo.taskId, true) + } + } + + override def handleRootTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + val taskInfo = taskEnd.taskInfo + delayedAssert(taskInfo.index === 0, s"Expected task index ${taskInfo.index} to be 0") + // If a task has been killed then it shouldn't be successful + val taskSuccessExpected = !taskIdsKilled.getOrDefault(taskInfo.taskId, false) + val taskSuccessActual = taskInfo.successful + delayedAssert(taskSuccessActual === taskSuccessExpected, + s"Expected task success $taskSuccessActual == $taskSuccessExpected") + } + } + sc.addSparkListener(listener) + // single task job + val jobResult = sc.parallelize(1 to 1, 1).map { _ => + Thread.sleep(5 * 1000L); 1 + }.count() + assert(jobResult === 1) + assert(listener.getTasksFinished().size === numTimesToKillWorkers + 1) + } + + test("decommission workers ensure that shuffle output is regenerated even with shuffle service") { + val conf = appConf + conf.set(config.Tests.TEST_NO_STAGE_RETRY, true) + conf.set(config.SHUFFLE_MANAGER, "sort") + conf.set(config.SHUFFLE_SERVICE_ENABLED, true) + createWorkers(2) + val ss = new ExternalShuffleServiceHolder(conf) + sc = createSparkContext(conf) + + // Here we will create a 2 stage job: The first stage will have two tasks and the second stage + // will have one task. The two tasks in the first stage will be long and short. We decommission + // and kill the worker after the short task is done. Eventually the driver should get the + // executor lost signal for the short task executor. This should trigger regenerating + // the shuffle output since we cleanly decommissioned the executor, despite running with an + // external shuffle service. + try { + val executorIdToWorkerInfo = getExecutorToWorkerAssignments + val workerForTask0Decommissioned = new AtomicBoolean(false) + // single task job + val listener = new RootStageAwareListener { + override def handleRootTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + val taskInfo = taskEnd.taskInfo + delayedAssert(taskInfo.index <= 1, s"Expected ${taskInfo.index} <= 1") + delayedAssert(taskInfo.successful, s"Task ${taskInfo.index} should be successful") + if (taskInfo.index == 0) { + if (workerForTask0Decommissioned.compareAndSet(false, true)) { + // Since this task hasn't been killed before, it should still be at its first attempt. + delayedAssert(taskInfo.attemptNumber === 0, "Should have succeeded in 1st attempt") + val workerInfo = executorIdToWorkerInfo(taskInfo.executorId) + decommissionWorkerOnMaster(workerInfo, "Kill early done map worker") + killWorkerAfterTimeout(workerInfo, 0) + logInfo(s"Killed the node ${workerInfo.hostPort} that was running the early task") + } else { + // The task should have been rerun since the worker was decommissioned just after + // it was finished. + // either the task attempt or the stage attempt number should be more than 0. + val attemptNumber = taskInfo.attemptNumber + val stageAttempt = taskEnd.stageAttemptId + delayedAssert(attemptNumber > 0 || stageAttempt > 0, + s"The task should have been rerun after decommissioning worker:" + + s" ($attemptNumber, $stageAttempt)") + } + } else { + delayedAssert(taskInfo.attemptNumber === 0, "2nd task should succeed on 1st attempt") + } + } + } + sc.addSparkListener(listener) + val jobResult = sc.parallelize(1 to 2, 2).mapPartitionsWithIndex((pid, _) => { + val sleepTimeSeconds = if (pid == 0) 1 else 10 + Thread.sleep(sleepTimeSeconds * 1000L) + List(1).iterator + }, preservesPartitioning = true).repartition(1).sum() + assert(jobResult === 2) + // 4 tasks: 2 from first stage, one retry due to decom, one more in the second stage. Review comment: This line explains the 4 tasks quite well but the assert 2 lines below checks `>= 4` so please mention what could be the additional task ---------------------------------------------------------------- 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: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
