attilapiros commented on a change in pull request #29014: URL: https://github.com/apache/spark/pull/29014#discussion_r458119261
########## File path: core/src/test/scala/org/apache/spark/deploy/DecommissionWorkerSuite.scala ########## @@ -0,0 +1,405 @@ +/* + * 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) + + // 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 should be treated as benign and not + // 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 + 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 a task has been killed then it shouldn't be successful + assert(taskInfo.successful === !taskIdsKilled.get(taskInfo.taskId)) Review comment: I know this is working as it is (I checked it) but it is hard to see why we get back `false` instead of `null` as the item is not there as it is stated in the [method documentation](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ConcurrentHashMap.html): > Returns: the value to which the specified key is mapped, or null if this map contains no mapping for the key So I suggest this: ```suggestion assert(taskInfo.successful === !taskIdsKilled.getOrDefault(taskInfo.taskId, false)) ``` ---------------------------------------------------------------- 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