From 848ca6d1873b9fda62e6601eb307f9967fae3c75 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 8 Aug 2014 13:51:25 -0700 Subject: [PATCH 01/19] [SPARK-2933] [yarn] Refactor and cleanup Yarn AM code. This change modifies the Yarn module so that all the logic related to running the ApplicationMaster is localized. Instead of, previously, 4 different classes with mostly identical code, now we have: - A single, shared ApplicationMaster class, which can operate both in client and cluster mode, and substitutes the old ApplicationMaster (for cluster mode) and ExecutorLauncher (for client mode). The benefit here is that all different execution modes for all supported yarn versions use the same shared code for monitoring executor allocation, setting up configuration, and monitoring the process's lifecycle. - A new YarnRMClient interface, which defines basic RM functionality needed by the ApplicationMaster. This interface has concrete implementations for each supported Yarn version. - A new YarnAllocator interface, which just abstracts the existing interface of the YarnAllocationHandler class. This is to avoid having to touch the allocator code too much in this change, although it might benefit from a similar effort in the future. The end result is much easier to understand code, with much less duplication, making it much easier to fix bugs, add features, and test everything knowing that all supported versions will behave the same. --- .../spark/deploy/yarn/ApplicationMaster.scala | 45 +- .../spark/deploy/yarn/ExecutorLauncher.scala | 12 +- .../deploy/yarn/YarnAllocationHandler.scala | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 441 ++++++++++++++++++ .../yarn/ApplicationMasterArguments.scala | 26 +- .../spark/deploy/yarn/ClientArguments.scala | 9 +- .../apache/spark/deploy/yarn/ClientBase.scala | 46 +- .../spark/deploy/yarn/YarnAllocator.scala | 31 ++ .../spark/deploy/yarn/YarnRMClient.scala | 63 +++ .../deploy/yarn/YarnSparkHadoopUtil.scala | 45 ++ .../cluster/YarnClientClusterScheduler.scala | 11 +- .../cluster/YarnClientSchedulerBackend.scala | 7 +- .../cluster/YarnClusterScheduler.scala | 4 +- .../spark/deploy/yarn/ApplicationMaster.scala | 423 ----------------- .../spark/deploy/yarn/ExecutorLauncher.scala | 291 ------------ .../deploy/yarn/YarnAllocationHandler.scala | 178 ++----- .../spark/deploy/yarn/YarnRMClientImpl.scala | 82 ++++ 17 files changed, 760 insertions(+), 956 deletions(-) create mode 100644 yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala create mode 100644 yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala create mode 100644 yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala delete mode 100644 yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala delete mode 100644 yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala create mode 100644 yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 62b5c3bc5f0f3..709d1da35b127 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -42,19 +42,14 @@ import org.apache.spark.util.{SignalLogger, Utils} /** * An application master that runs the users driver program and allocates executors. */ -class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, - sparkConf: SparkConf) extends Logging { +class ApplicationMaster(args: ApplicationMasterArguments) extends Logging { - def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = - this(args, new Configuration(), sparkConf) - - def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) - - private val rpc: YarnRPC = YarnRPC.create(conf) + private val yarnConf: YarnConfiguration = new YarnConfiguration(new Configuration()) + private val sparkConf = new SparkConf() + private val rpc: YarnRPC = YarnRPC.create(yarnConf) private var resourceManager: AMRMProtocol = _ private var appAttemptId: ApplicationAttemptId = _ private var userThread: Thread = _ - private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private val fs = FileSystem.get(yarnConf) private var yarnAllocator: YarnAllocationHandler = _ @@ -348,24 +343,14 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } */ - def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") { - synchronized { - if (isFinished) { - return - } - isFinished = true - - logInfo("finishApplicationMaster with " + status) - if (registered) { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - finishReq.setDiagnostics(diagnostics) - finishReq.setTrackingUrl(uiHistoryAddress) - resourceManager.finishApplicationMaster(finishReq) - } - } + override protected def finish() = { + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + finishReq.setDiagnostics(diagnostics) + finishReq.setTrackingUrl(uiHistoryAddress) + resourceManager.finishApplicationMaster(finishReq) } /** @@ -404,6 +389,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } object ApplicationMaster extends Logging { + + private var master: ApplicationMaster = _ + // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. private val ALLOCATE_HEARTBEAT_INTERVAL = 100 @@ -454,7 +442,8 @@ object ApplicationMaster extends Logging { SignalLogger.register(log) val args = new ApplicationMasterArguments(argStrings) SparkHadoopUtil.get.runAsSparkUser { () => - new ApplicationMaster(args).run() + master = new ApplicationMaster(args) + master.run() } } } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 184e2ad6c82cd..c28be22a81aad 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -41,15 +41,11 @@ import org.apache.spark.deploy.SparkHadoopUtil * * This is used only in yarn-client mode. */ -class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) - extends Logging { +class ExecutorLauncher(args: ApplicationMasterArguments) extends Logging { - def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = - this(args, new Configuration(), sparkConf) - - def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) - - private val rpc: YarnRPC = YarnRPC.create(conf) + private val sparkConf = new SparkConf() + private val yarnConf: YarnConfiguration = new YarnConfiguration(new Configuration()) + private val rpc: YarnRPC = YarnRPC.create(yarnConf) private var resourceManager: AMRMProtocol = _ private var appAttemptId: ApplicationAttemptId = _ private var reporterThread: Thread = _ diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 80e0162e9f277..f5eb17e82189d 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -68,7 +68,7 @@ private[yarn] class YarnAllocationHandler( val preferredHostToCount: Map[String, Int], val preferredRackToCount: Map[String, Int], val sparkConf: SparkConf) - extends Logging { + extends YarnAllocator with Logging { // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala new file mode 100644 index 0000000000000..64051c39d4f3a --- /dev/null +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -0,0 +1,441 @@ +/* + * 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.yarn + +import java.io.IOException +import java.net.Socket +import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} + +import scala.collection.JavaConversions._ +import scala.util.Try + +import akka.actor._ +import akka.remote._ +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.conf.YarnConfiguration + +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter +import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} + +/** + * Common application master functionality for Spark on Yarn. + */ +private[spark] class ApplicationMaster(args: ApplicationMasterArguments, + client: YarnRMClient) extends Logging { + // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be + // optimal as more containers are available. Might need to handle this better. + private val ALLOCATE_HEARTBEAT_INTERVAL = 100 + + private val sparkConf = new SparkConf() + private val yarnConf: YarnConfiguration = new YarnConfiguration(new Configuration()) + private val isDriver = args.userClass != null + + // Default to numExecutors * 2, with minimum of 3 + private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", + sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) + + private var finished = false + private var registered = false + private var reporterThread: Thread = _ + private var allocator: YarnAllocator = _ + + // Fields used in client mode. + private var actorSystem: ActorSystem = null + + // Fields used in cluster mode. + private var userThread: Thread = _ + private val sparkContextRef = new AtomicReference[SparkContext](null) + private val userResult = new AtomicBoolean(false) + + final def run() { + // Setup the directories so things go to YARN approved directories rather + // than user specified and /tmp. + System.setProperty("spark.local.dir", getLocalDirs()) + + if (isDriver) { + // Set the web ui port to be ephemeral for yarn so we don't conflict with + // other spark processes running on the same box + System.setProperty("spark.ui.port", "0") + + // When running the AM, the Spark master is always "yarn-cluster" + System.setProperty("spark.master", "yarn-cluster") + } + + val attemptId = client.getAttemptId() + logInfo("ApplicationAttemptId: " + attemptId) + + // Call this to force generation of secret so it gets populated into the + // Hadoop UGI. This has to happen before the startUserClass which does a + // doAs in order for the credentials to be passed on to the executor containers. + val securityMgr = new SecurityManager(sparkConf) + + val (uiAddress, uiHistoryAddress) = if (isDriver) { + // Start the user's JAR + userThread = startUserClass() + + // This a bit hacky, but we need to wait until the spark.driver.port property has + // been set by the Thread executing the user class. + waitForSparkContextInitialized() + + val sc = sparkContextRef.get() + (sc.ui.appUIHostPort, YarnSparkHadoopUtil.getUIHistoryAddress(sc, sparkConf)) + } else { + actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, + conf = sparkConf, securityManager = securityMgr)._1 + waitForSparkMaster() + (sparkConf.get("spark.driver.appUIAddress", ""), "") + } + + Utils.logUncaughtExceptions { + val sc = sparkContextRef.get() + allocator = client.register(yarnConf, + if (sc != null) sc.getConf else sparkConf, + if (sc != null) sc.preferredNodeLocationData else Map(), + uiAddress, + uiHistoryAddress) + registered = true + } + + addAmIpFilter() + + if (registered) { + // Launch thread that will heartbeat to the RM so it won't think the app has died. + reporterThread = launchReporterThread() + + // Allocate all containers + allocateExecutors() + } + + val success = + if (isDriver) { + try { + userThread.join() + userResult.get() + } finally { + // In cluster mode, ask the reporter thread to stop since the user app is finished. + reporterThread.interrupt() + } + } else { + // In client mode the actor will stop the reporter thread. + reporterThread.join() + true + } + + finish(if (success) FinalApplicationStatus.SUCCEEDED else FinalApplicationStatus.FAILED) + + val shouldCleanup = + if (success) { + true + } else { + val maxAppAttempts: Int = yarnConf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, + YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) + attemptId.getAttemptId() >= maxAppAttempts + } + + if (shouldCleanup) { + cleanupStagingDir() + } + } + + final def finish(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { + if (!finished) { + logInfo(s"Finishing ApplicationMaster with $status") + finished = true + reporterThread.interrupt() + reporterThread.join() + client.shutdown(status, diagnostics) + } + } + + private[spark] def sparkContextInitialized(sc: SparkContext) = { + var modified = false + sparkContextRef.synchronized { + modified = sparkContextRef.compareAndSet(null, sc) + sparkContextRef.notifyAll() + } + + // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do + // System.exit. + // Should not really have to do this, but it helps YARN to evict resources earlier. + // Not to mention, prevent the Client from declaring failure even though we exited properly. + // Note that this will unfortunately not properly clean up the staging files because it gets + // called too late, after the filesystem is already shutdown. + if (modified) { + Runtime.getRuntime().addShutdownHook(new Thread with Logging { + // This is not only logs, but also ensures that log system is initialized for this instance + // when we are actually 'run'-ing. + logInfo("Adding shutdown hook for context " + sc) + + override def run() { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() + finish(FinalApplicationStatus.SUCCEEDED) + } + }) + } + } + + /** Get the Yarn approved local directories. */ + private def getLocalDirs(): String = { + // Hadoop 0.23 and 2.x have different Environment variable names for the + // local dirs, so lets check both. We assume one of the 2 is set. + // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X + val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) + .orElse(Option(System.getenv("LOCAL_DIRS"))) + + localDirs match { + case None => throw new Exception("Yarn Local dirs can't be empty") + case Some(l) => l + } + } + + private def launchReporterThread(): Thread = { + // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses. + val expiryInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) + + // we want to be reasonably responsive without causing too many requests to RM. + val schedulerInterval = + sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) + + // must be <= timeoutInterval / 2. + val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) + + val t = new Thread { + override def run() { + while (!finished) { + checkNumExecutorsFailed() + allocateMissingExecutor() + logDebug("Sending progress") + allocator.allocateResources() + Try(Thread.sleep(interval)) + } + } + } + // setting to daemon status, though this is usually not a good idea. + t.setDaemon(true) + t.setName("Reporter") + t.start() + logInfo("Started progress reporter thread - sleep time : " + interval) + t + } + + /** + * Clean up the staging directory. + */ + private def cleanupStagingDir() { + val fs = FileSystem.get(yarnConf) + var stagingDirPath: Path = null + try { + val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean + if (!preserveFiles) { + stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) + if (stagingDirPath == null) { + logError("Staging directory is null") + return + } + logInfo("Deleting staging directory " + stagingDirPath) + fs.delete(stagingDirPath, true) + } + } catch { + case ioe: IOException => + logError("Failed to cleanup staging dir " + stagingDirPath, ioe) + } + } + + // Note: this need to happen before allocateExecutors. + private def waitForSparkContextInitialized() { + logInfo("Waiting for spark context initialization") + try { + var sparkContext: SparkContext = null + sparkContextRef.synchronized { + var count = 0 + val waitTime = 10000L + val numTries = sparkConf.getInt("spark.yarn.ApplicationMaster.waitTries", 10) + while (sparkContextRef.get() == null && count < numTries && !finished) { + logInfo("Waiting for spark context initialization ... " + count) + count = count + 1 + sparkContextRef.wait(waitTime) + } + sparkContext = sparkContextRef.get() + assert(sparkContext != null || count >= numTries) + + if (sparkContext == null) { + throw new IllegalStateException( + "Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d".format( + count * waitTime, numTries)) + } + } + } + } + + private def waitForSparkMaster() { + logInfo("Waiting for Spark driver to be reachable.") + var driverUp = false + val hostport = args.userArgs(0) + val (driverHost, driverPort) = Utils.parseHostPort(hostport) + while(!driverUp) { + try { + val socket = new Socket(driverHost, driverPort) + socket.close() + logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) + driverUp = true + } catch { + case e: Exception => + logError("Failed to connect to driver at %s:%s, retrying ...". + format(driverHost, driverPort)) + Thread.sleep(100) + } + } + sparkConf.set("spark.driver.host", driverHost) + sparkConf.set("spark.driver.port", driverPort.toString) + + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( + driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + + actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") + } + + private def allocateExecutors() { + try { + logInfo("Requesting" + args.numExecutors + " executors.") + // Wait until all containers have launched + allocator.addResourceRequests(args.numExecutors) + allocator.allocateResources() + // Exits the loop if the user thread exits. + + var iters = 0 + while (allocator.getNumExecutorsRunning < args.numExecutors && !finished) { + checkNumExecutorsFailed() + allocateMissingExecutor() + allocator.allocateResources() + Thread.sleep(ALLOCATE_HEARTBEAT_INTERVAL) + iters += 1 + } + } + logInfo("All executors have launched.") + } + + private def allocateMissingExecutor() { + val missingExecutorCount = args.numExecutors - allocator.getNumExecutorsRunning - + allocator.getNumPendingAllocate + if (missingExecutorCount > 0) { + logInfo("Allocating %d containers to make up for (potentially) lost containers". + format(missingExecutorCount)) + allocator.addResourceRequests(missingExecutorCount) + } + } + + private def checkNumExecutorsFailed() { + if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + finish(FinalApplicationStatus.FAILED, "Max number of executor failures reached.") + } + } + + // add the yarn amIpFilter that Yarn requires for properly securing the UI + private def addAmIpFilter() { + val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + System.setProperty("spark.ui.filters", amFilter) + val proxy = client.getProxyHostAndPort(yarnConf) + val parts : Array[String] = proxy.split(":") + val uriBase = "http://" + proxy + + System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) + + val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase + System.setProperty( + "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", params) + } + + private def startUserClass(): Thread = { + logInfo("Starting the user JAR in a separate Thread") + System.setProperty("spark.executor.instances", args.numExecutors.toString) + val mainMethod = Class.forName( + args.userClass, + false, + Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) + + val t = new Thread { + override def run() { + try { + // Copy + val mainArgs = new Array[String](args.userArgs.size) + args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size) + mainMethod.invoke(null, mainArgs) + // Some apps have "System.exit(0)" at the end. The user thread will stop here unless + // it has an uncaught exception thrown out. It needs a shutdown hook to set SUCCEEDED. + userResult.set(true) + } finally { + logDebug("Finishing main") + } + } + } + t.setName("Driver") + t.start() + t + } + + // Actor used to monitor the driver when running in client deploy mode. + private class MonitorActor(driverUrl: String) extends Actor { + + var driver: ActorSelection = _ + + override def preStart() { + logInfo("Listen to driver: " + driverUrl) + driver = context.actorSelection(driverUrl) + // Send a hello message to establish the connection, after which + // we can monitor Lifecycle Events. + driver ! "Hello" + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + } + + override def receive = { + case x: DisassociatedEvent => + logInfo(s"Driver terminated or disconnected! Shutting down. $x") + finish(FinalApplicationStatus.SUCCEEDED) + case x: AddWebUIFilter => + logInfo(s"Add WebUI Filter. $x") + driver ! x + } + + } + +} + +object ApplicationMaster extends Logging { + + private var master: ApplicationMaster = _ + + def main(argStrings: Array[String]) { + SignalLogger.register(log) + val args = new ApplicationMasterArguments(argStrings) + SparkHadoopUtil.get.runAsSparkUser { () => + master = new ApplicationMaster(args, new YarnRMClientImpl(args)) + master.run() + } + } + + private[spark] def sparkContextInitialized(sc: SparkContext) = { + master.sparkContextInitialized(sc) + } + +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 424b0fb0936f2..3e6b96fb63cea 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -63,11 +63,6 @@ class ApplicationMasterArguments(val args: Array[String]) { executorCores = value args = tail - case Nil => - if (userJar == null || userClass == null) { - printUsageAndExit(1) - } - case _ => printUsageAndExit(1, args) } @@ -80,16 +75,17 @@ class ApplicationMasterArguments(val args: Array[String]) { if (unknownParam != null) { System.err.println("Unknown/unsupported param " + unknownParam) } - System.err.println( - "Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options] \n" + - "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required)\n" + - " --class CLASS_NAME Name of your application's main class (required)\n" + - " --args ARGS Arguments to be passed to your application's main class.\n" + - " Mutliple invocations are possible, each will be passed in order.\n" + - " --num-executors NUM Number of executors to start (Default: 2)\n" + - " --executor-cores NUM Number of cores for the executors (Default: 1)\n" + - " --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G)\n") + System.err.println(""" + |Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options] + |Options: + | --jar JAR_PATH Path to your application's JAR file + | --class CLASS_NAME Name of your application's main class + | --args ARGS Arguments to be passed to your application's main class. + | Mutliple invocations are possible, each will be passed in order. + | --num-executors NUM Number of executors to start (Default: 2) + | --executor-cores NUM Number of cores for the executors (Default: 1) + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G) + """.stripMargin) System.exit(exitCode) } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 62f9b3cf5ab88..9eab191db59d5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -37,7 +37,6 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var numExecutors = 2 var amQueue = sparkConf.get("QUEUE", "default") var amMemory: Int = 512 // MB - var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" var inputFormatInfo: List[InputFormatInfo] = null var priority = 0 @@ -80,10 +79,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { args = tail case ("--master-class" | "--am-class") :: value :: tail => - if (args(0) == "--master-class") { - println("--master-class is deprecated. Use --am-class instead.") - } - amClass = value + println(s"${args(0)} is deprecated and is not used anymore.") args = tail case ("--master-memory" | "--driver-memory") :: MemoryParam(value) :: tail => @@ -135,9 +131,6 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { args = tail case Nil => - if (userClass == null) { - throw new IllegalArgumentException(getUsageMessage()) - } case _ => throw new IllegalArgumentException(getUsageMessage(args)) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 3897b3a373a8c..9e6507cc7187a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -42,12 +42,6 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, Spar /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The * Client submits an application to the YARN ResourceManager. - * - * Depending on the deployment mode this will launch one of two application master classes: - * 1. In cluster mode, it will launch an [[org.apache.spark.deploy.yarn.ApplicationMaster]] - * which launches a driver program inside of the cluster. - * 2. In client mode, it will launch an [[org.apache.spark.deploy.yarn.ExecutorLauncher]] to - * request executors on behalf of a driver running outside of the cluster. */ trait ClientBase extends Logging { val args: ClientArguments @@ -67,14 +61,11 @@ trait ClientBase extends Logging { // Additional memory overhead - in mb. protected def memoryOverhead: Int = sparkConf.getInt("spark.yarn.driver.memoryOverhead", - YarnAllocationHandler.MEMORY_OVERHEAD) + YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { Map( - ((args.userJar == null && args.amClass == classOf[ApplicationMaster].getName) -> - "Error: You must specify a user jar when running in standalone mode!"), - (args.userClass == null) -> "Error: You must specify a user class!", (args.numExecutors <= 0) -> "Error: You must specify at least 1 executor!", (args.amMemory <= memoryOverhead) -> ("Error: AM memory size must be" + "greater than: " + memoryOverhead), @@ -329,7 +320,7 @@ trait ClientBase extends Logging { // Note that to warn the user about the deprecation in cluster mode, some code from // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition // described above). - if (args.amClass == classOf[ApplicationMaster].getName) { + if (args.userClass != null) { sys.env.get("SPARK_JAVA_OPTS").foreach { value => val warning = s""" @@ -389,7 +380,7 @@ trait ClientBase extends Logging { javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } - if (args.amClass == classOf[ApplicationMaster].getName) { + if (args.userClass != null) { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) .foreach(opts => javaOpts += opts) @@ -397,22 +388,31 @@ trait ClientBase extends Logging { .foreach(p => javaOpts += s"-Djava.library.path=$p") } - // Command for the ApplicationMaster - val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ - javaOpts ++ - Seq(args.amClass, "--class", YarnSparkHadoopUtil.escapeForShell(args.userClass), - "--jar ", YarnSparkHadoopUtil.escapeForShell(args.userJar), - userArgsToString(args), - "--executor-memory", args.executorMemory.toString, + val userClass = + if (args.userClass != null) { + Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass)) + } else { + Nil + } + val amArgs = + Seq(classOf[ApplicationMaster].getName()) ++ userClass ++ + (if (args.userJar != null) Seq("--jar", args.userJar) else Nil) ++ + Seq("--executor-memory", args.executorMemory.toString, "--executor-cores", args.executorCores.toString, "--num-executors ", args.numExecutors.toString, + userArgsToString(args)) + + // Command for the ApplicationMaster + val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ + javaOpts ++ amArgs ++ + Seq( "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") logInfo("Yarn AM launch context:") - logInfo(s" class: ${args.amClass}") - logInfo(s" env: $env") - logInfo(s" command: ${commands.mkString(" ")}") + logInfo(s" user class: ${args.userClass}") + logInfo(s" env: $env") + logInfo(s" command: ${commands.mkString(" ")}") // TODO: it would be nicer to just make sure there are no null commands here val printableCommands = commands.map(s => if (s == null) "null" else s).toList @@ -623,7 +623,7 @@ object ClientBase extends Logging { YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, path, File.pathSeparator) - /** + /** * Get the list of namenodes the user may access. */ private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala new file mode 100644 index 0000000000000..9d6a3f61d1bb9 --- /dev/null +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -0,0 +1,31 @@ +/* + * 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.yarn + +/** + * Interface that defines a Yarn allocator. + */ +trait YarnAllocator { + + def addResourceRequests(numExecutors: Int): Unit + def allocateResources(): Unit + def getNumExecutorsFailed: Int + def getNumExecutorsRunning: Int + def getNumPendingAllocate: Int + +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala new file mode 100644 index 0000000000000..db11e43768f9c --- /dev/null +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -0,0 +1,63 @@ +/* + * 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.yarn + +import scala.collection.{Map, Set} + +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.api.records._ + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.scheduler.SplitInfo + +/** + * Interface that defines a Yarn RM client. + */ +trait YarnRMClient { + + /** + * Registers the application master with the RM. + * + * @param conf The Yarn configuration. + * @param sparkConf The Spark configuration. + * @param preferredNodeLocations Map with hints about where to allocate containers. + * @param uiAddress Address of the SparkUI. + * @param uiHistoryAddress Address of the application on the History Server. + */ + def register( + conf: YarnConfiguration, + sparkConf: SparkConf, + preferredNodeLocations: Map[String, Set[SplitInfo]], + uiAddress: String, + uiHistoryAddress: String): YarnAllocator + + /** + * Shuts down the AM. Guaranteed to only be called once. + * + * @param registered Whether the AM was successfully registered with the RM. + * @param status The final status of the AM. + */ + def shutdown(status: FinalApplicationStatus, diagnostics: String = ""): Unit + + /** Returns the attempt ID. */ + def getAttemptId(): ApplicationAttemptId + + /** Returns the RM's proxy host and port. */ + def getProxyHostAndPort(conf: YarnConfiguration): String + +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 10aef5eb2486f..0738164f74283 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -17,8 +17,11 @@ package org.apache.spark.deploy.yarn +import java.lang.{Boolean => JBoolean} +import java.util.{Collections, Set => JSet} import java.util.regex.Matcher import java.util.regex.Pattern +import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.HashMap @@ -29,11 +32,13 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.StringInterner import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants +import org.apache.hadoop.yarn.util.RackResolver import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.util.Utils /** * Contains util methods to interact with Hadoop from spark. @@ -79,6 +84,15 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } object YarnSparkHadoopUtil { + // Additional memory overhead - in mb. + val DEFAULT_MEMORY_OVERHEAD = 384 + + // Host to rack map - saved from allocation requests. We are expecting this not to change. + // Note that it is possible for this to change : and ResurceManager will indicate that to us via + // update response to allocate. But we are punting on handling that for now. + private val hostToRack = new ConcurrentHashMap[String, String]() + private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() + def addToEnvironment( env: HashMap[String, String], variable: String, @@ -173,4 +187,35 @@ object YarnSparkHadoopUtil { } } + def lookupRack(conf: Configuration, host: String): String = { + if (!hostToRack.contains(host)) { + populateRackInfo(conf, host) + } + hostToRack.get(host) + } + + def populateRackInfo(conf: Configuration, hostname: String) { + Utils.checkHost(hostname) + + if (!hostToRack.containsKey(hostname)) { + // If there are repeated failures to resolve, all to an ignore list. + val rackInfo = RackResolver.resolve(conf, hostname) + if (rackInfo != null && rackInfo.getNetworkLocation != null) { + val rack = rackInfo.getNetworkLocation + hostToRack.put(hostname, rack) + if (! rackToHostSet.containsKey(rack)) { + rackToHostSet.putIfAbsent(rack, + Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]())) + } + rackToHostSet.get(rack).add(hostname) + + // TODO(harvey): Figure out what this comment means... + // Since RackResolver caches, we are disabling this for now ... + } /* else { + // right ? Else we will keep calling rack resolver in case we cant resolve rack info ... + hostToRack.put(hostname, null) + } */ + } + } + } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala index 3474112ded5d7..d162b4c433f46 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -19,22 +19,21 @@ package org.apache.spark.scheduler.cluster import org.apache.spark._ import org.apache.hadoop.conf.Configuration -import org.apache.spark.deploy.yarn.YarnAllocationHandler +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.Utils /** - * - * This scheduler launches executors through Yarn - by calling into Client to launch ExecutorLauncher as AM. + * This scheduler launches executors through Yarn - by calling into Client to launch the Spark AM. */ -private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) { +private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) + extends TaskSchedulerImpl(sc) { def this(sc: SparkContext) = this(sc, new Configuration()) // By default, rack is unknown override def getRackForHost(hostPort: String): Option[String] = { val host = Utils.parseHostPort(hostPort)._1 - val retval = YarnAllocationHandler.lookupRack(conf, host) - if (retval != null) Some(retval) else None + Option(YarnSparkHadoopUtil.lookupRack(conf, host)) } } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 833e249f9f612..4eba6c42658b0 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher, YarnSparkHadoopUtil} +import org.apache.spark.deploy.yarn.{ApplicationMaster, Client, ClientArguments, YarnSparkHadoopUtil} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -60,10 +60,7 @@ private[spark] class YarnClientSchedulerBackend( val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( - "--class", "notused", - "--jar", null, // The primary jar will be added dynamically in SparkContext. - "--args", hostport, - "--am-class", classOf[ExecutorLauncher].getName + "--args", hostport ) // process any optional arguments, given either as environment variables diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala index 9aeca4a637d38..d4b4097527780 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark._ -import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler} +import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnSparkHadoopUtil} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -42,7 +42,7 @@ private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) // By default, rack is unknown override def getRackForHost(hostPort: String): Option[String] = { val host = Utils.parseHostPort(hostPort)._1 - val retval = YarnAllocationHandler.lookupRack(conf, host) + val retval = YarnSparkHadoopUtil.lookupRack(conf, host) if (retval != null) Some(retval) else None } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala deleted file mode 100644 index 035356d390c80..0000000000000 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ /dev/null @@ -1,423 +0,0 @@ -/* - * 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.yarn - -import java.io.IOException -import java.util.concurrent.CopyOnWriteArrayList -import java.util.concurrent.atomic.AtomicReference - -import scala.collection.JavaConversions._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.util.ShutdownHookManager -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.ConverterUtils -import org.apache.hadoop.yarn.webapp.util.WebAppUtils - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.{SignalLogger, Utils} - - -/** - * An application master that runs the user's driver program and allocates executors. - */ -class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, - sparkConf: SparkConf) extends Logging { - - def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = - this(args, new Configuration(), sparkConf) - - def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) - - private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - private var appAttemptId: ApplicationAttemptId = _ - private var userThread: Thread = _ - private val fs = FileSystem.get(yarnConf) - - private var yarnAllocator: YarnAllocationHandler = _ - private var isFinished: Boolean = false - private var uiAddress: String = _ - private var uiHistoryAddress: String = _ - private val maxAppAttempts: Int = conf.getInt( - YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) - private var isLastAMRetry: Boolean = true - private var amClient: AMRMClient[ContainerRequest] = _ - - // Default to numExecutors * 2, with minimum of 3 - private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", - sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) - - private var registered = false - - def run() { - // Setup the directories so things go to YARN approved directories rather - // than user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - - // Set the web ui port to be ephemeral for yarn so we don't conflict with - // other spark processes running on the same box - System.setProperty("spark.ui.port", "0") - - // When running the AM, the Spark master is always "yarn-cluster" - System.setProperty("spark.master", "yarn-cluster") - - // Use priority 30 as it's higher than HDFS. It's the same priority MapReduce is using. - ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) - - appAttemptId = ApplicationMaster.getApplicationAttemptId() - logInfo("ApplicationAttemptId: " + appAttemptId) - isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts - amClient = AMRMClient.createAMRMClient() - amClient.init(yarnConf) - amClient.start() - - // setup AmIpFilter for the SparkUI - do this before we start the UI - addAmIpFilter() - - ApplicationMaster.register(this) - - // Call this to force generation of secret so it gets populated into the - // Hadoop UGI. This has to happen before the startUserClass which does a - // doAs in order for the credentials to be passed on to the executor containers. - val securityMgr = new SecurityManager(sparkConf) - - // Start the user's JAR - userThread = startUserClass() - - // This a bit hacky, but we need to wait until the spark.driver.port property has - // been set by the Thread executing the user class. - waitForSparkContextInitialized() - - // Do this after Spark master is up and SparkContext is created so that we can register UI Url. - synchronized { - if (!isFinished) { - registerApplicationMaster() - registered = true - } - } - - // Allocate all containers - allocateExecutors() - - // Launch thread that will heartbeat to the RM so it won't think the app has died. - launchReporterThread() - - // Wait for the user class to finish - userThread.join() - - System.exit(0) - } - - // add the yarn amIpFilter that Yarn requires for properly securing the UI - private def addAmIpFilter() { - val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" - System.setProperty("spark.ui.filters", amFilter) - val proxy = WebAppUtils.getProxyHostAndPort(conf) - val parts : Array[String] = proxy.split(":") - val uriBase = "http://" + proxy + - System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) - - val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase - System.setProperty( - "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", params) - } - - // Get the Yarn approved local directories. - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn local dirs can't be empty") - case Some(l) => l - } - } - - private def registerApplicationMaster(): RegisterApplicationMasterResponse = { - logInfo("Registering the ApplicationMaster") - amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) - } - - private def startUserClass(): Thread = { - logInfo("Starting the user JAR in a separate Thread") - System.setProperty("spark.executor.instances", args.numExecutors.toString) - val mainMethod = Class.forName( - args.userClass, - false, - Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) - val t = new Thread { - override def run() { - var succeeded = false - try { - // Copy - val mainArgs = new Array[String](args.userArgs.size) - args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size) - mainMethod.invoke(null, mainArgs) - // Some apps have "System.exit(0)" at the end. The user thread will stop here unless - // it has an uncaught exception thrown out. It needs a shutdown hook to set SUCCEEDED. - succeeded = true - } finally { - logDebug("Finishing main") - isLastAMRetry = true - if (succeeded) { - ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - } else { - ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) - } - } - } - } - t.setName("Driver") - t.start() - t - } - - // This needs to happen before allocateExecutors() - private def waitForSparkContextInitialized() { - logInfo("Waiting for Spark context initialization") - try { - var sparkContext: SparkContext = null - ApplicationMaster.sparkContextRef.synchronized { - var numTries = 0 - val waitTime = 10000L - val maxNumTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10) - while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries - && !isFinished) { - logInfo("Waiting for Spark context initialization ... " + numTries) - numTries = numTries + 1 - ApplicationMaster.sparkContextRef.wait(waitTime) - } - sparkContext = ApplicationMaster.sparkContextRef.get() - assert(sparkContext != null || numTries >= maxNumTries) - - if (sparkContext != null) { - uiAddress = sparkContext.ui.appUIHostPort - uiHistoryAddress = YarnSparkHadoopUtil.getUIHistoryAddress(sparkContext, sparkConf) - this.yarnAllocator = YarnAllocationHandler.newAllocator( - yarnConf, - amClient, - appAttemptId, - args, - sparkContext.preferredNodeLocationData, - sparkContext.getConf) - } else { - logWarning("Unable to retrieve SparkContext in spite of waiting for %d, maxNumTries = %d". - format(numTries * waitTime, maxNumTries)) - this.yarnAllocator = YarnAllocationHandler.newAllocator( - yarnConf, - amClient, - appAttemptId, - args, - sparkContext.getConf) - } - } - } - } - - private def allocateExecutors() { - try { - logInfo("Requesting" + args.numExecutors + " executors.") - // Wait until all containers have launched - yarnAllocator.addResourceRequests(args.numExecutors) - yarnAllocator.allocateResources() - // Exits the loop if the user thread exits. - - var iters = 0 - while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { - checkNumExecutorsFailed() - allocateMissingExecutor() - yarnAllocator.allocateResources() - Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) - iters += 1 - } - } - logInfo("All executors have launched.") - } - - private def allocateMissingExecutor() { - val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning - - yarnAllocator.getNumPendingAllocate - if (missingExecutorCount > 0) { - logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingExecutorCount)) - yarnAllocator.addResourceRequests(missingExecutorCount) - } - } - - private def checkNumExecutorsFailed() { - if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of executor failures reached") - } - } - - private def launchReporterThread(): Thread = { - // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses. - val expiryInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) - - // we want to be reasonably responsive without causing too many requests to RM. - val schedulerInterval = - sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) - - // must be <= timeoutInterval / 2. - val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) - - val t = new Thread { - override def run() { - while (userThread.isAlive) { - checkNumExecutorsFailed() - allocateMissingExecutor() - logDebug("Sending progress") - yarnAllocator.allocateResources() - Thread.sleep(interval) - } - } - } - // Setting to daemon status, though this is usually not a good idea. - t.setDaemon(true) - t.start() - logInfo("Started progress reporter thread - heartbeat interval : " + interval) - t - } - - def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") { - synchronized { - if (isFinished) { - return - } - isFinished = true - - logInfo("Unregistering ApplicationMaster with " + status) - if (registered) { - amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) - } - } - } - - /** - * Clean up the staging directory. - */ - private def cleanupStagingDir() { - var stagingDirPath: Path = null - try { - val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean - if (!preserveFiles) { - stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) - if (stagingDirPath == null) { - logError("Staging directory is null") - return - } - logInfo("Deleting staging directory " + stagingDirPath) - fs.delete(stagingDirPath, true) - } - } catch { - case ioe: IOException => - logError("Failed to cleanup staging dir " + stagingDirPath, ioe) - } - } - - // The shutdown hook that runs when a signal is received AND during normal close of the JVM. - class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable { - - def run() { - logInfo("AppMaster received a signal.") - // We need to clean up staging dir before HDFS is shut down - // make sure we don't delete it until this is the last AM - if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir() - } - } - -} - -object ApplicationMaster extends Logging { - // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be - // optimal as more containers are available. Might need to handle this better. - private val ALLOCATE_HEARTBEAT_INTERVAL = 100 - - private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() - - val sparkContextRef: AtomicReference[SparkContext] = - new AtomicReference[SparkContext](null) - - def register(master: ApplicationMaster) { - applicationMasters.add(master) - } - - /** - * Called from YarnClusterScheduler to notify the AM code that a SparkContext has been - * initialized in the user code. - */ - def sparkContextInitialized(sc: SparkContext): Boolean = { - var modified = false - sparkContextRef.synchronized { - modified = sparkContextRef.compareAndSet(null, sc) - sparkContextRef.notifyAll() - } - - // Add a shutdown hook - as a best effort in case users do not call sc.stop or do - // System.exit. - // Should not really have to do this, but it helps YARN to evict resources earlier. - // Not to mention, prevent the Client from declaring failure even though we exited properly. - // Note that this will unfortunately not properly clean up the staging files because it gets - // called too late, after the filesystem is already shutdown. - if (modified) { - Runtime.getRuntime().addShutdownHook(new Thread with Logging { - // This is not only logs, but also ensures that log system is initialized for this instance - // when we are actually 'run'-ing. - logInfo("Adding shutdown hook for context " + sc) - - override def run() { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() - // Best case ... - for (master <- applicationMasters) { - master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - } - } - }) - } - - // Wait for initialization to complete and at least 'some' nodes to get allocated. - modified - } - - def getApplicationAttemptId(): ApplicationAttemptId = { - val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) - val containerId = ConverterUtils.toContainerId(containerIdString) - val appAttemptId = containerId.getApplicationAttemptId() - appAttemptId - } - - def main(argStrings: Array[String]) { - SignalLogger.register(log) - val args = new ApplicationMasterArguments(argStrings) - SparkHadoopUtil.get.runAsSparkUser { () => - new ApplicationMaster(args).run() - } - } -} diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala deleted file mode 100644 index fc7b8320d734d..0000000000000 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ /dev/null @@ -1,291 +0,0 @@ -/* - * 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.yarn - -import java.net.Socket -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.ApplicationConstants -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.conf.YarnConfiguration -import akka.actor._ -import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter -import org.apache.spark.scheduler.SplitInfo -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.yarn.webapp.util.WebAppUtils - -/** - * An application master that allocates executors on behalf of a driver that is running outside - * the cluster. - * - * This is used only in yarn-client mode. - */ -class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) - extends Logging { - - def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = - this(args, new Configuration(), sparkConf) - - def this(args: ApplicationMasterArguments) = this(args, new SparkConf()) - - private var appAttemptId: ApplicationAttemptId = _ - private var reporterThread: Thread = _ - private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - - private var yarnAllocator: YarnAllocationHandler = _ - private var driverClosed: Boolean = false - private var isFinished: Boolean = false - private var registered: Boolean = false - - private var amClient: AMRMClient[ContainerRequest] = _ - - // Default to numExecutors * 2, with minimum of 3 - private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", - sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) - - val securityManager = new SecurityManager(sparkConf) - val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = sparkConf, securityManager = securityManager)._1 - var actor: ActorRef = _ - - // This actor just working as a monitor to watch on Driver Actor. - class MonitorActor(driverUrl: String) extends Actor { - - var driver: ActorSelection = _ - - override def preStart() { - logInfo("Listen to driver: " + driverUrl) - driver = context.actorSelection(driverUrl) - // Send a hello message to establish the connection, after which - // we can monitor Lifecycle Events. - driver ! "Hello" - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - } - - override def receive = { - case x: DisassociatedEvent => - logInfo(s"Driver terminated or disconnected! Shutting down. $x") - driverClosed = true - case x: AddWebUIFilter => - logInfo(s"Add WebUI Filter. $x") - driver ! x - } - } - - def run() { - - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - - amClient = AMRMClient.createAMRMClient() - amClient.init(yarnConf) - amClient.start() - - appAttemptId = ApplicationMaster.getApplicationAttemptId() - synchronized { - if (!isFinished) { - registerApplicationMaster() - registered = true - } - } - - waitForSparkMaster() - addAmIpFilter() - - // Allocate all containers - allocateExecutors() - - // Launch a progress reporter thread, else app will get killed after expiration - // (def: 10mins) timeout ensure that progress is sent before - // YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. - - val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) - // we want to be reasonably responsive without causing too many requests to RM. - val schedulerInterval = - System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong - // must be <= timeoutInterval / 2. - val interval = math.min(timeoutInterval / 2, schedulerInterval) - - reporterThread = launchReporterThread(interval) - - - // Wait for the reporter thread to Finish. - reporterThread.join() - - finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - actorSystem.shutdown() - - logInfo("Exited") - System.exit(0) - } - - /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn Local dirs can't be empty") - case Some(l) => l - } - } - - private def registerApplicationMaster(): RegisterApplicationMasterResponse = { - val appUIAddress = sparkConf.get("spark.driver.appUIAddress", "") - logInfo(s"Registering the ApplicationMaster with appUIAddress: $appUIAddress") - amClient.registerApplicationMaster(Utils.localHostName(), 0, appUIAddress) - } - - // add the yarn amIpFilter that Yarn requires for properly securing the UI - private def addAmIpFilter() { - val proxy = WebAppUtils.getProxyHostAndPort(conf) - val parts = proxy.split(":") - val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) - val uriBase = "http://" + proxy + proxyBase - val amFilter = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase - val amFilterName = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" - actor ! AddWebUIFilter(amFilterName, amFilter, proxyBase) - } - - private def waitForSparkMaster() { - logInfo("Waiting for Spark driver to be reachable.") - var driverUp = false - val hostport = args.userArgs(0) - val (driverHost, driverPort) = Utils.parseHostPort(hostport) - while(!driverUp) { - try { - val socket = new Socket(driverHost, driverPort) - socket.close() - logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) - driverUp = true - } catch { - case e: Exception => - logError("Failed to connect to driver at %s:%s, retrying ...". - format(driverHost, driverPort)) - Thread.sleep(100) - } - } - sparkConf.set("spark.driver.host", driverHost) - sparkConf.set("spark.driver.port", driverPort.toString) - - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) - - actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") - } - - - private def allocateExecutors() { - // TODO: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. - val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = - scala.collection.immutable.Map() - - yarnAllocator = YarnAllocationHandler.newAllocator( - yarnConf, - amClient, - appAttemptId, - args, - preferredNodeLocationData, - sparkConf) - - logInfo("Requesting " + args.numExecutors + " executors.") - // Wait until all containers have launched - yarnAllocator.addResourceRequests(args.numExecutors) - yarnAllocator.allocateResources() - while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { - checkNumExecutorsFailed() - allocateMissingExecutor() - yarnAllocator.allocateResources() - Thread.sleep(100) - } - - logInfo("All executors have launched.") - } - - private def allocateMissingExecutor() { - val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning - - yarnAllocator.getNumPendingAllocate - if (missingExecutorCount > 0) { - logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingExecutorCount)) - yarnAllocator.addResourceRequests(missingExecutorCount) - } - } - - private def checkNumExecutorsFailed() { - if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of executor failures reached") - } - } - - private def launchReporterThread(_sleepTime: Long): Thread = { - val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime - - val t = new Thread { - override def run() { - while (!driverClosed) { - checkNumExecutorsFailed() - allocateMissingExecutor() - logDebug("Sending progress") - yarnAllocator.allocateResources() - Thread.sleep(sleepTime) - } - } - } - // setting to daemon status, though this is usually not a good idea. - t.setDaemon(true) - t.start() - logInfo("Started progress reporter thread - sleep time : " + sleepTime) - t - } - - def finishApplicationMaster(status: FinalApplicationStatus, appMessage: String = "") { - synchronized { - if (isFinished) { - return - } - logInfo("Unregistering ApplicationMaster with " + status) - if (registered) { - val trackingUrl = sparkConf.get("spark.yarn.historyServer.address", "") - amClient.unregisterApplicationMaster(status, appMessage, trackingUrl) - } - isFinished = true - } - } - -} - -object ExecutorLauncher { - def main(argStrings: Array[String]) { - val args = new ApplicationMasterArguments(argStrings) - SparkHadoopUtil.get.runAsSparkUser { () => - new ExecutorLauncher(args).run() - } - } -} diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 29ccec2adcac3..a58456bb39da6 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy.yarn -import java.lang.{Boolean => JBoolean} -import java.util.{Collections, Set => JSet} import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} import java.util.concurrent.atomic.AtomicInteger @@ -61,16 +59,19 @@ object AllocationType extends Enumeration { * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ private[yarn] class YarnAllocationHandler( - val conf: Configuration, - val amClient: AMRMClient[ContainerRequest], - val appAttemptId: ApplicationAttemptId, - val maxExecutors: Int, - val executorMemory: Int, - val executorCores: Int, - val preferredHostToCount: Map[String, Int], - val preferredRackToCount: Map[String, Int], - val sparkConf: SparkConf) - extends Logging { + conf: Configuration, + sparkConf: SparkConf, + amClient: AMRMClient[ContainerRequest], + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments, + map: collection.Map[String, collection.Set[SplitInfo]]) + extends YarnAllocator with Logging { + + private val ANY_HOST = "*" + // All requests are issued with same priority : we do not (yet) have any distinction between + // request types (like map/reduce in hadoop for example) + private val PRIORITY = 1 + // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. @@ -92,7 +93,7 @@ private[yarn] class YarnAllocationHandler( // Additional memory overhead - in mb. private def memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnAllocationHandler.MEMORY_OVERHEAD) + YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) // Number of container requests that have been sent to, but not yet allocated by the // ApplicationMaster. @@ -103,6 +104,11 @@ private[yarn] class YarnAllocationHandler( private val lastResponseId = new AtomicInteger() private val numExecutorsFailed = new AtomicInteger() + private val maxExecutors = args.numExecutors + private val executorMemory = args.executorMemory + private val executorCores = args.executorCores + private val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, map) + def getNumPendingAllocate: Int = numPendingAllocate.intValue def getNumExecutorsRunning: Int = numExecutorsRunning.intValue @@ -204,7 +210,7 @@ private[yarn] class YarnAllocationHandler( // For rack local containers if (remainingContainers != null) { - val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) if (rack != null) { val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - @@ -272,7 +278,7 @@ private[yarn] class YarnAllocationHandler( // To be safe, remove the container from `pendingReleaseContainers`. pendingReleaseContainers.remove(containerId) - val rack = YarnAllocationHandler.lookupRack(conf, executorHostname) + val rack = YarnSparkHadoopUtil.lookupRack(conf, executorHostname) allocatedHostToContainersMap.synchronized { val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]()) @@ -359,7 +365,7 @@ private[yarn] class YarnAllocationHandler( allocatedContainerToHostMap.remove(containerId) // TODO: Move this part outside the synchronized block? - val rack = YarnAllocationHandler.lookupRack(conf, host) + val rack = YarnSparkHadoopUtil.lookupRack(conf, host) if (rack != null) { val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 if (rackCount > 0) { @@ -392,9 +398,9 @@ private[yarn] class YarnAllocationHandler( for (container <- hostContainers) { val candidateHost = container.getNodes.last - assert(YarnAllocationHandler.ANY_HOST != candidateHost) + assert(ANY_HOST != candidateHost) - val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) if (rack != null) { var count = rackToCounts.getOrElse(rack, 0) count += 1 @@ -408,7 +414,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.RACK, rack, count, - YarnAllocationHandler.PRIORITY) + PRIORITY) } requestedContainers @@ -439,9 +445,9 @@ private[yarn] class YarnAllocationHandler( AllocationType.ANY, resource = null, numExecutors, - YarnAllocationHandler.PRIORITY).toList + PRIORITY).toList } else { - // Request for all hosts in preferred nodes and for numExecutors - + // Request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) for ((candidateHost, candidateCount) <- preferredHostToCount) { @@ -452,7 +458,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.HOST, candidateHost, requiredCount, - YarnAllocationHandler.PRIORITY) + PRIORITY) } } val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( @@ -462,7 +468,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.ANY, resource = null, numExecutors, - YarnAllocationHandler.PRIORITY) + PRIORITY) val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size) @@ -511,7 +517,7 @@ private[yarn] class YarnAllocationHandler( // There must be a third request, which is ANY. That will be specially handled. requestType match { case AllocationType.HOST => { - assert(YarnAllocationHandler.ANY_HOST != resource) + assert(ANY_HOST != resource) val hostname = resource val nodeLocal = constructContainerRequests( Array(hostname), @@ -520,7 +526,7 @@ private[yarn] class YarnAllocationHandler( priority) // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. - YarnAllocationHandler.populateRackInfo(conf, hostname) + YarnSparkHadoopUtil.populateRackInfo(conf, hostname) nodeLocal } case AllocationType.RACK => { @@ -553,88 +559,6 @@ private[yarn] class YarnAllocationHandler( } requests } -} - -object YarnAllocationHandler { - - val ANY_HOST = "*" - // All requests are issued with same priority : we do not (yet) have any distinction between - // request types (like map/reduce in hadoop for example) - val PRIORITY = 1 - - // Additional memory overhead - in mb. - val MEMORY_OVERHEAD = 384 - - // Host to rack map - saved from allocation requests. We are expecting this not to change. - // Note that it is possible for this to change : and ResurceManager will indicate that to us via - // update response to allocate. But we are punting on handling that for now. - private val hostToRack = new ConcurrentHashMap[String, String]() - private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() - - - def newAllocator( - conf: Configuration, - amClient: AMRMClient[ContainerRequest], - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - sparkConf: SparkConf - ): YarnAllocationHandler = { - new YarnAllocationHandler( - conf, - amClient, - appAttemptId, - args.numExecutors, - args.executorMemory, - args.executorCores, - Map[String, Int](), - Map[String, Int](), - sparkConf) - } - - def newAllocator( - conf: Configuration, - amClient: AMRMClient[ContainerRequest], - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - map: collection.Map[String, - collection.Set[SplitInfo]], - sparkConf: SparkConf - ): YarnAllocationHandler = { - val (hostToSplitCount, rackToSplitCount) = generateNodeToWeight(conf, map) - new YarnAllocationHandler( - conf, - amClient, - appAttemptId, - args.numExecutors, - args.executorMemory, - args.executorCores, - hostToSplitCount, - rackToSplitCount, - sparkConf) - } - - def newAllocator( - conf: Configuration, - amClient: AMRMClient[ContainerRequest], - appAttemptId: ApplicationAttemptId, - maxExecutors: Int, - executorMemory: Int, - executorCores: Int, - map: collection.Map[String, collection.Set[SplitInfo]], - sparkConf: SparkConf - ): YarnAllocationHandler = { - val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) - new YarnAllocationHandler( - conf, - amClient, - appAttemptId, - maxExecutors, - executorMemory, - executorCores, - hostToCount, - rackToCount, - sparkConf) - } // A simple method to copy the split info map. private def generateNodeToWeight( @@ -653,7 +577,7 @@ object YarnAllocationHandler { val hostCount = hostToCount.getOrElse(host, 0) hostToCount.put(host, hostCount + splits.size) - val rack = lookupRack(conf, host) + val rack = YarnSparkHadoopUtil.lookupRack(conf, host) if (rack != null){ val rackCount = rackToCount.getOrElse(host, 0) rackToCount.put(host, rackCount + splits.size) @@ -663,42 +587,4 @@ object YarnAllocationHandler { (hostToCount.toMap, rackToCount.toMap) } - def lookupRack(conf: Configuration, host: String): String = { - if (!hostToRack.contains(host)) { - populateRackInfo(conf, host) - } - hostToRack.get(host) - } - - def fetchCachedHostsForRack(rack: String): Option[Set[String]] = { - Option(rackToHostSet.get(rack)).map { set => - val convertedSet: collection.mutable.Set[String] = set - // TODO: Better way to get a Set[String] from JSet. - convertedSet.toSet - } - } - - def populateRackInfo(conf: Configuration, hostname: String) { - Utils.checkHost(hostname) - - if (!hostToRack.containsKey(hostname)) { - // If there are repeated failures to resolve, all to an ignore list. - val rackInfo = RackResolver.resolve(conf, hostname) - if (rackInfo != null && rackInfo.getNetworkLocation != null) { - val rack = rackInfo.getNetworkLocation - hostToRack.put(hostname, rack) - if (! rackToHostSet.containsKey(rack)) { - rackToHostSet.putIfAbsent(rack, - Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]())) - } - rackToHostSet.get(rack).add(hostname) - - // TODO(harvey): Figure out what this comment means... - // Since RackResolver caches, we are disabling this for now ... - } /* else { - // right ? Else we will keep calling rack resolver in case we cant resolve rack info ... - hostToRack.put(hostname, null) - } */ - } - } } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala new file mode 100644 index 0000000000000..7a3fd3e1a2520 --- /dev/null +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -0,0 +1,82 @@ +/* + * 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.yarn + +import java.io.IOException +import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.atomic.AtomicReference + +import scala.collection.JavaConversions._ +import scala.collection.{Map, Set} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.util.ShutdownHookManager +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.ConverterUtils +import org.apache.hadoop.yarn.webapp.util.WebAppUtils + +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.SplitInfo +import org.apache.spark.util.{SignalLogger, Utils} + + +/** + * YarnRMClient implementation for the Yarn stable API. + */ +private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMClient with Logging { + + private var amClient: AMRMClient[ContainerRequest] = _ + private var uiHistoryAddress: String = _ + + override def register( + conf: YarnConfiguration, + sparkConf: SparkConf, + preferredNodeLocations: Map[String, Set[SplitInfo]], + uiAddress: String, + uiHistoryAddress: String) = { + amClient = AMRMClient.createAMRMClient() + amClient.init(conf) + amClient.start() + this.uiHistoryAddress = uiHistoryAddress + + logInfo("Registering the ApplicationMaster") + amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args, + preferredNodeLocations) + } + + override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") = + amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) + + override def getAttemptId() = { + val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) + val containerId = ConverterUtils.toContainerId(containerIdString) + val appAttemptId = containerId.getApplicationAttemptId() + appAttemptId + } + + override def getProxyHostAndPort(conf: YarnConfiguration) = WebAppUtils.getProxyHostAndPort(conf) + +} From 99a52d574bbf1b0d3eedbd135ae1ff86463e25f9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Aug 2014 15:28:22 -0700 Subject: [PATCH 02/19] Changes to the yarn-alpha project to use common AM code. Made some tweaks to the YarnAllocator interface to cover both APIs more easily. There's still a lot of cleanup possible on that front, but I'll leave that as a separate task. --- .../spark/deploy/yarn/ApplicationMaster.scala | 449 ------------------ .../spark/deploy/yarn/ExecutorLauncher.scala | 326 ------------- .../deploy/yarn/YarnAllocationHandler.scala | 180 ++----- .../spark/deploy/yarn/YarnRMClientImpl.scala | 103 ++++ .../spark/deploy/yarn/ApplicationMaster.scala | 18 +- .../spark/deploy/yarn/YarnAllocator.scala | 2 - .../spark/deploy/yarn/YarnRMClient.scala | 3 + .../deploy/yarn/YarnAllocationHandler.scala | 12 +- .../spark/deploy/yarn/YarnRMClientImpl.scala | 3 + 9 files changed, 150 insertions(+), 946 deletions(-) delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala create mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala deleted file mode 100644 index 709d1da35b127..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ /dev/null @@ -1,449 +0,0 @@ -/* - * 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.yarn - -import java.io.IOException -import java.net.Socket -import java.util.concurrent.CopyOnWriteArrayList -import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} - -import scala.collection.JavaConversions._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.util.ShutdownHookManager -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{ConverterUtils, Records} - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.{SignalLogger, Utils} - -/** - * An application master that runs the users driver program and allocates executors. - */ -class ApplicationMaster(args: ApplicationMasterArguments) extends Logging { - - private val yarnConf: YarnConfiguration = new YarnConfiguration(new Configuration()) - private val sparkConf = new SparkConf() - private val rpc: YarnRPC = YarnRPC.create(yarnConf) - private var resourceManager: AMRMProtocol = _ - private var appAttemptId: ApplicationAttemptId = _ - private var userThread: Thread = _ - private val fs = FileSystem.get(yarnConf) - - private var yarnAllocator: YarnAllocationHandler = _ - private var isFinished: Boolean = false - private var uiAddress: String = _ - private var uiHistoryAddress: String = _ - private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, - YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) - private var isLastAMRetry: Boolean = true - - // Default to numExecutors * 2, with minimum of 3 - private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", - sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) - - private var registered = false - - def run() { - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - - // set the web ui port to be ephemeral for yarn so we don't conflict with - // other spark processes running on the same box - System.setProperty("spark.ui.port", "0") - - // when running the AM, the Spark master is always "yarn-cluster" - System.setProperty("spark.master", "yarn-cluster") - - // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. - ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) - - appAttemptId = getApplicationAttemptId() - isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts - resourceManager = registerWithResourceManager() - - // setup AmIpFilter for the SparkUI - do this before we start the UI - addAmIpFilter() - - ApplicationMaster.register(this) - - // Call this to force generation of secret so it gets populated into the - // hadoop UGI. This has to happen before the startUserClass which does a - // doAs in order for the credentials to be passed on to the executor containers. - val securityMgr = new SecurityManager(sparkConf) - - // Start the user's JAR - userThread = startUserClass() - - // This a bit hacky, but we need to wait until the spark.driver.port property has - // been set by the Thread executing the user class. - waitForSparkContextInitialized() - - // Do this after spark master is up and SparkContext is created so that we can register UI Url - synchronized { - if (!isFinished) { - registerApplicationMaster() - registered = true - } - } - - // Allocate all containers - allocateExecutors() - - // Wait for the user class to Finish - userThread.join() - - System.exit(0) - } - - // add the yarn amIpFilter that Yarn requires for properly securing the UI - private def addAmIpFilter() { - val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" - System.setProperty("spark.ui.filters", amFilter) - val proxy = YarnConfiguration.getProxyHostAndPort(conf) - val parts : Array[String] = proxy.split(":") - val uriBase = "http://" + proxy + - System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) - - val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase - System.setProperty("spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", - params) - } - - /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn Local dirs can't be empty") - case Some(l) => l - } - } - - private def getApplicationAttemptId(): ApplicationAttemptId = { - val envs = System.getenv() - val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) - val containerId = ConverterUtils.toContainerId(containerIdString) - val appAttemptId = containerId.getApplicationAttemptId() - logInfo("ApplicationAttemptId: " + appAttemptId) - appAttemptId - } - - private def registerWithResourceManager(): AMRMProtocol = { - val rmAddress = NetUtils.createSocketAddr(yarnConf.get( - YarnConfiguration.RM_SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) - logInfo("Connecting to ResourceManager at " + rmAddress) - rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] - } - - private def registerApplicationMaster(): RegisterApplicationMasterResponse = { - logInfo("Registering the ApplicationMaster") - val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) - .asInstanceOf[RegisterApplicationMasterRequest] - appMasterRequest.setApplicationAttemptId(appAttemptId) - // Setting this to master host,port - so that the ApplicationReport at client has some - // sensible info. - // Users can then monitor stderr/stdout on that node if required. - appMasterRequest.setHost(Utils.localHostName()) - appMasterRequest.setRpcPort(0) - appMasterRequest.setTrackingUrl(uiAddress) - resourceManager.registerApplicationMaster(appMasterRequest) - } - - private def startUserClass(): Thread = { - logInfo("Starting the user JAR in a separate Thread") - System.setProperty("spark.executor.instances", args.numExecutors.toString) - val mainMethod = Class.forName( - args.userClass, - false /* initialize */ , - Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) - val t = new Thread { - override def run() { - - var successed = false - try { - // Copy - var mainArgs: Array[String] = new Array[String](args.userArgs.size) - args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size) - mainMethod.invoke(null, mainArgs) - // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR - // userThread will stop here unless it has uncaught exception thrown out - // It need shutdown hook to set SUCCEEDED - successed = true - } finally { - logDebug("finishing main") - isLastAMRetry = true - if (successed) { - ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - } else { - ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) - } - } - } - } - t.start() - t - } - - // this need to happen before allocateExecutors - private def waitForSparkContextInitialized() { - logInfo("Waiting for spark context initialization") - try { - var sparkContext: SparkContext = null - ApplicationMaster.sparkContextRef.synchronized { - var count = 0 - val waitTime = 10000L - val numTries = sparkConf.getInt("spark.yarn.ApplicationMaster.waitTries", 10) - while (ApplicationMaster.sparkContextRef.get() == null && count < numTries - && !isFinished) { - logInfo("Waiting for spark context initialization ... " + count) - count = count + 1 - ApplicationMaster.sparkContextRef.wait(waitTime) - } - sparkContext = ApplicationMaster.sparkContextRef.get() - assert(sparkContext != null || count >= numTries) - - if (null != sparkContext) { - uiAddress = sparkContext.ui.appUIHostPort - uiHistoryAddress = YarnSparkHadoopUtil.getUIHistoryAddress(sparkContext, sparkConf) - this.yarnAllocator = YarnAllocationHandler.newAllocator( - yarnConf, - resourceManager, - appAttemptId, - args, - sparkContext.preferredNodeLocationData, - sparkContext.getConf) - } else { - logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d". - format(count * waitTime, numTries)) - this.yarnAllocator = YarnAllocationHandler.newAllocator( - yarnConf, - resourceManager, - appAttemptId, - args, - sparkContext.getConf) - } - } - } - } - - private def allocateExecutors() { - try { - logInfo("Allocating " + args.numExecutors + " executors.") - // Wait until all containers have finished - // TODO: This is a bit ugly. Can we make it nicer? - // TODO: Handle container failure - - // Exists the loop if the user thread exits. - while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { - if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of executor failures reached") - } - yarnAllocator.allocateContainers( - math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) - Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) - } - } - logInfo("All executors have launched.") - - // Launch a progress reporter thread, else the app will get killed after expiration - // (def: 10mins) timeout. - // TODO(harvey): Verify the timeout - if (userThread.isAlive) { - // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses. - val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) - - // we want to be reasonably responsive without causing too many requests to RM. - val schedulerInterval = - sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) - - // must be <= timeoutInterval / 2. - val interval = math.min(timeoutInterval / 2, schedulerInterval) - - launchReporterThread(interval) - } - } - - private def launchReporterThread(_sleepTime: Long): Thread = { - val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime - - val t = new Thread { - override def run() { - while (userThread.isAlive) { - if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of executor failures reached") - } - val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning - if (missingExecutorCount > 0) { - logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingExecutorCount)) - yarnAllocator.allocateContainers(missingExecutorCount) - } else { - sendProgress() - } - Thread.sleep(sleepTime) - } - } - } - // Setting to daemon status, though this is usually not a good idea. - t.setDaemon(true) - t.start() - logInfo("Started progress reporter thread - sleep time : " + sleepTime) - t - } - - private def sendProgress() { - logDebug("Sending progress") - // Simulated with an allocate request with no nodes requested ... - yarnAllocator.allocateContainers(0) - } - - /* - def printContainers(containers: List[Container]) = { - for (container <- containers) { - logInfo("Launching shell command on a new container." - + ", containerId=" + container.getId() - + ", containerNode=" + container.getNodeId().getHost() - + ":" + container.getNodeId().getPort() - + ", containerNodeURI=" + container.getNodeHttpAddress() - + ", containerState" + container.getState() - + ", containerResourceMemory" - + container.getResource().getMemory()) - } - } - */ - - override protected def finish() = { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - finishReq.setDiagnostics(diagnostics) - finishReq.setTrackingUrl(uiHistoryAddress) - resourceManager.finishApplicationMaster(finishReq) - } - - /** - * Clean up the staging directory. - */ - private def cleanupStagingDir() { - var stagingDirPath: Path = null - try { - val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean - if (!preserveFiles) { - stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) - if (stagingDirPath == null) { - logError("Staging directory is null") - return - } - logInfo("Deleting staging directory " + stagingDirPath) - fs.delete(stagingDirPath, true) - } - } catch { - case ioe: IOException => - logError("Failed to cleanup staging dir " + stagingDirPath, ioe) - } - } - - // The shutdown hook that runs when a signal is received AND during normal close of the JVM. - class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable { - - def run() { - logInfo("AppMaster received a signal.") - // we need to clean up staging dir before HDFS is shut down - // make sure we don't delete it until this is the last AM - if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir() - } - } - -} - -object ApplicationMaster extends Logging { - - private var master: ApplicationMaster = _ - - // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be - // optimal as more containers are available. Might need to handle this better. - private val ALLOCATE_HEARTBEAT_INTERVAL = 100 - - private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() - - def register(master: ApplicationMaster) { - applicationMasters.add(master) - } - - val sparkContextRef: AtomicReference[SparkContext] = - new AtomicReference[SparkContext](null /* initialValue */) - - def sparkContextInitialized(sc: SparkContext): Boolean = { - var modified = false - sparkContextRef.synchronized { - modified = sparkContextRef.compareAndSet(null, sc) - sparkContextRef.notifyAll() - } - - // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do - // System.exit. - // Should not really have to do this, but it helps YARN to evict resources earlier. - // Not to mention, prevent the Client from declaring failure even though we exited properly. - // Note that this will unfortunately not properly clean up the staging files because it gets - // called too late, after the filesystem is already shutdown. - if (modified) { - Runtime.getRuntime().addShutdownHook(new Thread with Logging { - // This is not only logs, but also ensures that log system is initialized for this instance - // when we are actually 'run'-ing. - logInfo("Adding shutdown hook for context " + sc) - - override def run() { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() - // Best case ... - for (master <- applicationMasters) { - master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - } - } - }) - } - - modified - } - - def main(argStrings: Array[String]) { - SignalLogger.register(log) - val args = new ApplicationMasterArguments(argStrings) - SparkHadoopUtil.get.runAsSparkUser { () => - master = new ApplicationMaster(args) - master.run() - } - } -} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala deleted file mode 100644 index c28be22a81aad..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ /dev/null @@ -1,326 +0,0 @@ -/* - * 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.yarn - -import java.net.Socket -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import akka.actor._ -import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter -import org.apache.spark.scheduler.SplitInfo -import org.apache.spark.deploy.SparkHadoopUtil - -/** - * An application master that allocates executors on behalf of a driver that is running outside - * the cluster. - * - * This is used only in yarn-client mode. - */ -class ExecutorLauncher(args: ApplicationMasterArguments) extends Logging { - - private val sparkConf = new SparkConf() - private val yarnConf: YarnConfiguration = new YarnConfiguration(new Configuration()) - private val rpc: YarnRPC = YarnRPC.create(yarnConf) - private var resourceManager: AMRMProtocol = _ - private var appAttemptId: ApplicationAttemptId = _ - private var reporterThread: Thread = _ - private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - - private var yarnAllocator: YarnAllocationHandler = _ - - private var driverClosed: Boolean = false - private var isFinished: Boolean = false - private var registered: Boolean = false - - // Default to numExecutors * 2, with minimum of 3 - private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", - sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) - - val securityManager = new SecurityManager(sparkConf) - val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = sparkConf, securityManager = securityManager)._1 - var actor: ActorRef = _ - - // This actor just working as a monitor to watch on Driver Actor. - class MonitorActor(driverUrl: String) extends Actor { - - var driver: ActorSelection = _ - - override def preStart() { - logInfo("Listen to driver: " + driverUrl) - driver = context.actorSelection(driverUrl) - // Send a hello message thus the connection is actually established, thus we can - // monitor Lifecycle Events. - driver ! "Hello" - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - } - - override def receive = { - case x: DisassociatedEvent => - logInfo(s"Driver terminated or disconnected! Shutting down. $x") - driverClosed = true - case x: AddWebUIFilter => - logInfo(s"Add WebUI Filter. $x") - driver ! x - } - } - - def run() { - - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - - appAttemptId = getApplicationAttemptId() - resourceManager = registerWithResourceManager() - - synchronized { - if (!isFinished) { - val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() - // Compute number of threads for akka - val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() - - if (minimumMemory > 0) { - val mem = args.executorMemory + sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnAllocationHandler.MEMORY_OVERHEAD) - val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) - - if (numCore > 0) { - // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 - // TODO: Uncomment when hadoop is on a version which has this fixed. - // args.workerCores = numCore - } - } - registered = true - } - } - waitForSparkMaster() - addAmIpFilter() - // Allocate all containers - allocateExecutors() - - // Launch a progress reporter thread, else app will get killed after expiration - // (def: 10mins) timeout ensure that progress is sent before - // YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. - - val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) - // we want to be reasonably responsive without causing too many requests to RM. - val schedulerInterval = - System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong - - // must be <= timeoutInterval / 2. - val interval = math.min(timeoutInterval / 2, schedulerInterval) - - reporterThread = launchReporterThread(interval) - - // Wait for the reporter thread to Finish. - reporterThread.join() - - finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - actorSystem.shutdown() - - logInfo("Exited") - System.exit(0) - } - - /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn Local dirs can't be empty") - case Some(l) => l - } - } - - private def getApplicationAttemptId(): ApplicationAttemptId = { - val envs = System.getenv() - val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) - val containerId = ConverterUtils.toContainerId(containerIdString) - val appAttemptId = containerId.getApplicationAttemptId() - logInfo("ApplicationAttemptId: " + appAttemptId) - appAttemptId - } - - private def registerWithResourceManager(): AMRMProtocol = { - val rmAddress = NetUtils.createSocketAddr(yarnConf.get( - YarnConfiguration.RM_SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) - logInfo("Connecting to ResourceManager at " + rmAddress) - rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] - } - - private def registerApplicationMaster(): RegisterApplicationMasterResponse = { - val appUIAddress = sparkConf.get("spark.driver.appUIAddress", "") - logInfo(s"Registering the ApplicationMaster with appUIAddress: $appUIAddress") - val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) - .asInstanceOf[RegisterApplicationMasterRequest] - appMasterRequest.setApplicationAttemptId(appAttemptId) - // Setting this to master host,port - so that the ApplicationReport at client has - // some sensible info. Users can then monitor stderr/stdout on that node if required. - appMasterRequest.setHost(Utils.localHostName()) - appMasterRequest.setRpcPort(0) - // What do we provide here ? Might make sense to expose something sensible later ? - appMasterRequest.setTrackingUrl(appUIAddress) - resourceManager.registerApplicationMaster(appMasterRequest) - } - - // add the yarn amIpFilter that Yarn requires for properly securing the UI - private def addAmIpFilter() { - val proxy = YarnConfiguration.getProxyHostAndPort(conf) - val parts = proxy.split(":") - val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) - val uriBase = "http://" + proxy + proxyBase - val amFilter = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase - val amFilterName = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" - actor ! AddWebUIFilter(amFilterName, amFilter, proxyBase) - } - - private def waitForSparkMaster() { - logInfo("Waiting for spark driver to be reachable.") - var driverUp = false - val hostport = args.userArgs(0) - val (driverHost, driverPort) = Utils.parseHostPort(hostport) - while(!driverUp) { - try { - val socket = new Socket(driverHost, driverPort) - socket.close() - logInfo("Master now available: " + driverHost + ":" + driverPort) - driverUp = true - } catch { - case e: Exception => - logError("Failed to connect to driver at " + driverHost + ":" + driverPort) - Thread.sleep(100) - } - } - sparkConf.set("spark.driver.host", driverHost) - sparkConf.set("spark.driver.port", driverPort.toString) - - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) - - actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") - } - - - private def allocateExecutors() { - - // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. - val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = - scala.collection.immutable.Map() - - yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, - args, preferredNodeLocationData, sparkConf) - - logInfo("Allocating " + args.numExecutors + " executors.") - // Wait until all containers have finished - // TODO: This is a bit ugly. Can we make it nicer? - // TODO: Handle container failure - while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { - yarnAllocator.allocateContainers( - math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) - checkNumExecutorsFailed() - Thread.sleep(100) - } - - logInfo("All executors have launched.") - } - private def checkNumExecutorsFailed() { - if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of executor failures reached") - } - } - - // TODO: We might want to extend this to allocate more containers in case they die ! - private def launchReporterThread(_sleepTime: Long): Thread = { - val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime - - val t = new Thread { - override def run() { - while (!driverClosed) { - checkNumExecutorsFailed() - val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning - if (missingExecutorCount > 0) { - logInfo("Allocating " + missingExecutorCount + - " containers to make up for (potentially ?) lost containers") - yarnAllocator.allocateContainers(missingExecutorCount) - } else { - sendProgress() - } - Thread.sleep(sleepTime) - } - } - } - // setting to daemon status, though this is usually not a good idea. - t.setDaemon(true) - t.start() - logInfo("Started progress reporter thread - sleep time : " + sleepTime) - t - } - - private def sendProgress() { - logDebug("Sending progress") - // simulated with an allocate request with no nodes requested ... - yarnAllocator.allocateContainers(0) - } - - def finishApplicationMaster(status: FinalApplicationStatus, appMessage: String = "") { - synchronized { - if (isFinished) { - return - } - logInfo("Unregistering ApplicationMaster with " + status) - if (registered) { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - finishReq.setTrackingUrl(sparkConf.get("spark.yarn.historyServer.address", "")) - finishReq.setDiagnostics(appMessage) - resourceManager.finishApplicationMaster(finishReq) - } - isFinished = true - } - } - -} - - -object ExecutorLauncher { - def main(argStrings: Array[String]) { - val args = new ApplicationMasterArguments(argStrings) - SparkHadoopUtil.get.runAsSparkUser { () => - new ExecutorLauncher(args).run() - } - } -} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index f5eb17e82189d..5fb3f323ed1a6 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -27,7 +27,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} +import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -59,16 +59,19 @@ object AllocationType extends Enumeration { * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ private[yarn] class YarnAllocationHandler( - val conf: Configuration, - val resourceManager: AMRMProtocol, - val appAttemptId: ApplicationAttemptId, - val maxExecutors: Int, - val executorMemory: Int, - val executorCores: Int, - val preferredHostToCount: Map[String, Int], - val preferredRackToCount: Map[String, Int], - val sparkConf: SparkConf) + conf: Configuration, + sparkConf: SparkConf, + resourceManager: AMRMProtocol, + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments, + map: collection.Map[String, collection.Set[SplitInfo]]) extends YarnAllocator with Logging { + + private val ANY_HOST = "*" + // All requests are issued with same priority : we do not (yet) have any distinction between + // request types (like map/reduce in hadoop for example) + private val PRIORITY = 1 + // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. @@ -90,7 +93,7 @@ private[yarn] class YarnAllocationHandler( // Additional memory overhead - in mb. private def memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnAllocationHandler.MEMORY_OVERHEAD) + YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) private val numExecutorsRunning = new AtomicInteger() // Used to generate a unique id per executor @@ -98,6 +101,11 @@ private[yarn] class YarnAllocationHandler( private val lastResponseId = new AtomicInteger() private val numExecutorsFailed = new AtomicInteger() + private val maxExecutors = args.numExecutors + private val executorMemory = args.executorMemory + private val executorCores = args.executorCores + private val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, map) + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue def getNumExecutorsFailed: Int = numExecutorsFailed.intValue @@ -106,9 +114,10 @@ private[yarn] class YarnAllocationHandler( container.getResource.getMemory >= (executorMemory + memoryOverhead) } - def allocateContainers(executorsToRequest: Int) { + override def allocateResources() = { // We need to send the request only once from what I understand ... but for now, not modifying // this much. + val executorsToRequest = Math.max(maxExecutors - numExecutorsRunning.get(), 0) // Keep polling the Resource Manager for containers val amResp = allocateExecutorResources(executorsToRequest).getAMResponse @@ -182,7 +191,7 @@ private[yarn] class YarnAllocationHandler( // Now rack local if (remainingContainers != null){ - val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) if (rack != null){ val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) @@ -254,7 +263,7 @@ private[yarn] class YarnAllocationHandler( // Should not be there, but .. pendingReleaseContainers.remove(containerId) - val rack = YarnAllocationHandler.lookupRack(conf, executorHostname) + val rack = YarnSparkHadoopUtil.lookupRack(conf, executorHostname) allocatedHostToContainersMap.synchronized { val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]()) @@ -329,7 +338,7 @@ private[yarn] class YarnAllocationHandler( allocatedContainerToHostMap -= containerId // Doing this within locked context, sigh ... move to outside ? - val rack = YarnAllocationHandler.lookupRack(conf, host) + val rack = YarnSparkHadoopUtil.lookupRack(conf, host) if (rack != null) { val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 if (rackCount > 0) { @@ -362,9 +371,9 @@ private[yarn] class YarnAllocationHandler( for (container <- hostContainers) { val candidateHost = container.getHostName val candidateNumContainers = container.getNumContainers - assert(YarnAllocationHandler.ANY_HOST != candidateHost) + assert(ANY_HOST != candidateHost) - val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) if (rack != null) { var count = rackToCounts.getOrElse(rack, 0) count += candidateNumContainers @@ -376,7 +385,7 @@ private[yarn] class YarnAllocationHandler( new ArrayBuffer[ResourceRequest](rackToCounts.size) for ((rack, count) <- rackToCounts){ requestedContainers += - createResourceRequest(AllocationType.RACK, rack, count, YarnAllocationHandler.PRIORITY) + createResourceRequest(AllocationType.RACK, rack, count, PRIORITY) } requestedContainers.toList @@ -407,7 +416,7 @@ private[yarn] class YarnAllocationHandler( logDebug("numExecutors: " + numExecutors + ", host preferences: " + preferredHostToCount.isEmpty) resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, numExecutors, YarnAllocationHandler.PRIORITY)) + AllocationType.ANY, null, numExecutors, PRIORITY)) } else { // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. @@ -421,7 +430,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.HOST, candidateHost, requiredCount, - YarnAllocationHandler.PRIORITY) + PRIORITY) } } val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests( @@ -431,7 +440,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.ANY, resource = null, numExecutors, - YarnAllocationHandler.PRIORITY) + PRIORITY) val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( hostContainerRequests.size + rackContainerRequests.size + 1) @@ -481,12 +490,12 @@ private[yarn] class YarnAllocationHandler( // There must be a third request - which is ANY : that will be specially handled. requestType match { case AllocationType.HOST => { - assert(YarnAllocationHandler.ANY_HOST != resource) + assert(ANY_HOST != resource) val hostname = resource val nodeLocal = createResourceRequestImpl(hostname, numExecutors, priority) // Add to host->rack mapping - YarnAllocationHandler.populateRackInfo(conf, hostname) + YarnSparkHadoopUtil.populateRackInfo(conf, hostname) nodeLocal } @@ -495,7 +504,7 @@ private[yarn] class YarnAllocationHandler( createResourceRequestImpl(rack, numExecutors, priority) } case AllocationType.ANY => createResourceRequestImpl( - YarnAllocationHandler.ANY_HOST, numExecutors, priority) + ANY_HOST, numExecutors, priority) case _ => throw new IllegalArgumentException( "Unexpected/unsupported request type: " + requestType) } @@ -539,90 +548,6 @@ private[yarn] class YarnAllocationHandler( retval } -} - -object YarnAllocationHandler { - - val ANY_HOST = "*" - // All requests are issued with same priority : we do not (yet) have any distinction between - // request types (like map/reduce in hadoop for example) - val PRIORITY = 1 - - // Additional memory overhead - in mb - val MEMORY_OVERHEAD = 384 - - // Host to rack map - saved from allocation requests - // We are expecting this not to change. - // Note that it is possible for this to change : and RM will indicate that to us via update - // response to allocate. But we are punting on handling that for now. - private val hostToRack = new ConcurrentHashMap[String, String]() - private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() - - - def newAllocator( - conf: Configuration, - resourceManager: AMRMProtocol, - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - sparkConf: SparkConf): YarnAllocationHandler = { - - new YarnAllocationHandler( - conf, - resourceManager, - appAttemptId, - args.numExecutors, - args.executorMemory, - args.executorCores, - Map[String, Int](), - Map[String, Int](), - sparkConf) - } - - def newAllocator( - conf: Configuration, - resourceManager: AMRMProtocol, - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - map: collection.Map[String, - collection.Set[SplitInfo]], - sparkConf: SparkConf): YarnAllocationHandler = { - - val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) - new YarnAllocationHandler( - conf, - resourceManager, - appAttemptId, - args.numExecutors, - args.executorMemory, - args.executorCores, - hostToCount, - rackToCount, - sparkConf) - } - - def newAllocator( - conf: Configuration, - resourceManager: AMRMProtocol, - appAttemptId: ApplicationAttemptId, - maxExecutors: Int, - executorMemory: Int, - executorCores: Int, - map: collection.Map[String, collection.Set[SplitInfo]], - sparkConf: SparkConf): YarnAllocationHandler = { - - val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) - - new YarnAllocationHandler( - conf, - resourceManager, - appAttemptId, - maxExecutors, - executorMemory, - executorCores, - hostToCount, - rackToCount, - sparkConf) - } // A simple method to copy the split info map. private def generateNodeToWeight( @@ -640,7 +565,7 @@ object YarnAllocationHandler { val hostCount = hostToCount.getOrElse(host, 0) hostToCount.put(host, hostCount + splits.size) - val rack = lookupRack(conf, host) + val rack = YarnSparkHadoopUtil.lookupRack(conf, host) if (rack != null){ val rackCount = rackToCount.getOrElse(host, 0) rackToCount.put(host, rackCount + splits.size) @@ -650,41 +575,4 @@ object YarnAllocationHandler { (hostToCount.toMap, rackToCount.toMap) } - def lookupRack(conf: Configuration, host: String): String = { - if (!hostToRack.contains(host)) populateRackInfo(conf, host) - hostToRack.get(host) - } - - def fetchCachedHostsForRack(rack: String): Option[Set[String]] = { - val set = rackToHostSet.get(rack) - if (set == null) return None - - // No better way to get a Set[String] from JSet ? - val convertedSet: collection.mutable.Set[String] = set - Some(convertedSet.toSet) - } - - def populateRackInfo(conf: Configuration, hostname: String) { - Utils.checkHost(hostname) - - if (!hostToRack.containsKey(hostname)) { - // If there are repeated failures to resolve, all to an ignore list ? - val rackInfo = RackResolver.resolve(conf, hostname) - if (rackInfo != null && rackInfo.getNetworkLocation != null) { - val rack = rackInfo.getNetworkLocation - hostToRack.put(hostname, rack) - if (! rackToHostSet.containsKey(rack)) { - rackToHostSet.putIfAbsent(rack, - Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]())) - } - rackToHostSet.get(rack).add(hostname) - - // TODO(harvey): Figure out this comment... - // Since RackResolver caches, we are disabling this for now ... - } /* else { - // right ? Else we will keep calling rack resolver in case we cant resolve rack info ... - hostToRack.put(hostname, null) - } */ - } - } } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala new file mode 100644 index 0000000000000..cc5392192ec51 --- /dev/null +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -0,0 +1,103 @@ +/* + * 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.yarn + +import scala.collection.{Map, Set} + +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.scheduler.SplitInfo +import org.apache.spark.util.Utils + +/** + * YarnRMClient implementation for the Yarn alpha API. + */ +private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMClient with Logging { + + private var rpc: YarnRPC = null + private var resourceManager: AMRMProtocol = _ + private var uiHistoryAddress: String = _ + + override def register( + conf: YarnConfiguration, + sparkConf: SparkConf, + preferredNodeLocations: Map[String, Set[SplitInfo]], + uiAddress: String, + uiHistoryAddress: String) = { + this.rpc = YarnRPC.create(conf) + this.uiHistoryAddress = uiHistoryAddress + + resourceManager = registerWithResourceManager(conf) + registerApplicationMaster(uiAddress) + + new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args, + preferredNodeLocations) + } + + override def getAttemptId() = { + val envs = System.getenv() + val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) + val containerId = ConverterUtils.toContainerId(containerIdString) + val appAttemptId = containerId.getApplicationAttemptId() + appAttemptId + } + + override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") = { + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(getAttemptId()) + finishReq.setFinishApplicationStatus(status) + finishReq.setDiagnostics(diagnostics) + finishReq.setTrackingUrl(uiHistoryAddress) + resourceManager.finishApplicationMaster(finishReq) + } + + override def getProxyHostAndPort(conf: YarnConfiguration) = + YarnConfiguration.getProxyHostAndPort(conf) + + override def getMaxRegAttempts(conf: YarnConfiguration) = + conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) + + private def registerWithResourceManager(conf: YarnConfiguration): AMRMProtocol = { + val rmAddress = NetUtils.createSocketAddr(conf.get(YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) + logInfo("Connecting to ResourceManager at " + rmAddress) + rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] + } + + private def registerApplicationMaster(uiAddress: String): RegisterApplicationMasterResponse = { + val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) + .asInstanceOf[RegisterApplicationMasterRequest] + appMasterRequest.setApplicationAttemptId(getAttemptId()) + // Setting this to master host,port - so that the ApplicationReport at client has some + // sensible info. + // Users can then monitor stderr/stdout on that node if required. + appMasterRequest.setHost(Utils.localHostName()) + appMasterRequest.setRpcPort(0) + appMasterRequest.setTrackingUrl(uiAddress) + resourceManager.registerApplicationMaster(appMasterRequest) + } + +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 64051c39d4f3a..0fab49eaccbf7 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -148,8 +148,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (success) { true } else { - val maxAppAttempts: Int = yarnConf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, - YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) + val maxAppAttempts = client.getMaxRegAttempts(yarnConf) attemptId.getAttemptId() >= maxAppAttempts } @@ -225,7 +224,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, override def run() { while (!finished) { checkNumExecutorsFailed() - allocateMissingExecutor() logDebug("Sending progress") allocator.allocateResources() Try(Thread.sleep(interval)) @@ -319,15 +317,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def allocateExecutors() { try { logInfo("Requesting" + args.numExecutors + " executors.") - // Wait until all containers have launched - allocator.addResourceRequests(args.numExecutors) allocator.allocateResources() - // Exits the loop if the user thread exits. var iters = 0 while (allocator.getNumExecutorsRunning < args.numExecutors && !finished) { checkNumExecutorsFailed() - allocateMissingExecutor() allocator.allocateResources() Thread.sleep(ALLOCATE_HEARTBEAT_INTERVAL) iters += 1 @@ -336,16 +330,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, logInfo("All executors have launched.") } - private def allocateMissingExecutor() { - val missingExecutorCount = args.numExecutors - allocator.getNumExecutorsRunning - - allocator.getNumPendingAllocate - if (missingExecutorCount > 0) { - logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingExecutorCount)) - allocator.addResourceRequests(missingExecutorCount) - } - } - private def checkNumExecutorsFailed() { if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finish(FinalApplicationStatus.FAILED, "Max number of executor failures reached.") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 9d6a3f61d1bb9..8e3e0c7c6af9f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -22,10 +22,8 @@ package org.apache.spark.deploy.yarn */ trait YarnAllocator { - def addResourceRequests(numExecutors: Int): Unit def allocateResources(): Unit def getNumExecutorsFailed: Int def getNumExecutorsRunning: Int - def getNumPendingAllocate: Int } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index db11e43768f9c..370177df25f98 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -60,4 +60,7 @@ trait YarnRMClient { /** Returns the RM's proxy host and port. */ def getProxyHostAndPort(conf: YarnConfiguration): String + /** Returns the maximum number of attempts to register the AM. */ + def getMaxRegAttempts(conf: YarnConfiguration): Int + } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index a58456bb39da6..6e5b912b63cb6 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -109,11 +109,9 @@ private[yarn] class YarnAllocationHandler( private val executorCores = args.executorCores private val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, map) - def getNumPendingAllocate: Int = numPendingAllocate.intValue + override def getNumExecutorsRunning: Int = numExecutorsRunning.intValue - def getNumExecutorsRunning: Int = numExecutorsRunning.intValue - - def getNumExecutorsFailed: Int = numExecutorsFailed.intValue + override def getNumExecutorsFailed: Int = numExecutorsFailed.intValue def isResourceConstraintSatisfied(container: Container): Boolean = { container.getResource.getMemory >= (executorMemory + memoryOverhead) @@ -125,7 +123,9 @@ private[yarn] class YarnAllocationHandler( amClient.releaseAssignedContainer(containerId) } - def allocateResources() { + override def allocateResources() = { + addResourceRequests(maxExecutors - numPendingAllocate.get()) + // We have already set the container request. Poll the ResourceManager for a response. // This doubles as a heartbeat if there are no pending container requests. val progressIndicator = 0.1f @@ -436,7 +436,7 @@ private[yarn] class YarnAllocationHandler( retval } - def addResourceRequests(numExecutors: Int) { + private def addResourceRequests(numExecutors: Int) { val containerRequests: List[ContainerRequest] = if (numExecutors <= 0 || preferredHostToCount.isEmpty) { logDebug("numExecutors: " + numExecutors + ", host preferences: " + diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index 7a3fd3e1a2520..8c5f6da81f76d 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -79,4 +79,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC override def getProxyHostAndPort(conf: YarnConfiguration) = WebAppUtils.getProxyHostAndPort(conf) + override def getMaxRegAttempts(conf: YarnConfiguration) = + conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) + } From 8c72239abbf479a681fba1f4c93e4b50593bc47e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Aug 2014 16:15:46 -0700 Subject: [PATCH 03/19] Trivial cleanups. --- .../deploy/yarn/YarnAllocationHandler.scala | 13 ++----------- .../spark/deploy/yarn/ApplicationMaster.scala | 16 ++++++++-------- .../apache/spark/deploy/yarn/YarnAllocator.scala | 5 +++++ .../deploy/yarn/YarnAllocationHandler.scala | 12 ++---------- .../spark/deploy/yarn/YarnRMClientImpl.scala | 13 ++----------- 5 files changed, 19 insertions(+), 40 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 5fb3f323ed1a6..88f41752d35ef 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -17,12 +17,9 @@ package org.apache.spark.deploy.yarn -import java.lang.{Boolean => JBoolean} -import java.util.{Collections, Set => JSet} import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} import java.util.concurrent.atomic.AtomicInteger -import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -34,16 +31,10 @@ import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.AMRMProtocol import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId} -import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus} +import org.apache.hadoop.yarn.api.records.{Container, ContainerId} import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest} import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} -import org.apache.hadoop.yarn.util.{RackResolver, Records} - - -object AllocationType extends Enumeration { - type AllocationType = Value - val HOST, RACK, ANY = Value -} +import org.apache.hadoop.yarn.util.Records // TODO: // Too many params. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 0fab49eaccbf7..9fe5bfd1d1134 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -68,7 +68,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private val sparkContextRef = new AtomicReference[SparkContext](null) private val userResult = new AtomicBoolean(false) - final def run() { + final def run() = { // Setup the directories so things go to YARN approved directories rather // than user specified and /tmp. System.setProperty("spark.local.dir", getLocalDirs()) @@ -314,7 +314,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } - private def allocateExecutors() { + private def allocateExecutors() = { try { logInfo("Requesting" + args.numExecutors + " executors.") allocator.allocateResources() @@ -330,14 +330,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, logInfo("All executors have launched.") } - private def checkNumExecutorsFailed() { + private def checkNumExecutorsFailed() = { if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finish(FinalApplicationStatus.FAILED, "Max number of executor failures reached.") } } // add the yarn amIpFilter that Yarn requires for properly securing the UI - private def addAmIpFilter() { + private def addAmIpFilter() = { val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" System.setProperty("spark.ui.filters", amFilter) val proxy = client.getProxyHostAndPort(yarnConf) @@ -346,8 +346,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase - System.setProperty( - "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", params) + System.setProperty("spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", + params) } private def startUserClass(): Thread = { @@ -383,7 +383,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, var driver: ActorSelection = _ - override def preStart() { + override def preStart() = { logInfo("Listen to driver: " + driverUrl) driver = context.actorSelection(driverUrl) // Send a hello message to establish the connection, after which @@ -409,7 +409,7 @@ object ApplicationMaster extends Logging { private var master: ApplicationMaster = _ - def main(argStrings: Array[String]) { + def main(argStrings: Array[String]) = { SignalLogger.register(log) val args = new ApplicationMasterArguments(argStrings) SparkHadoopUtil.get.runAsSparkUser { () => diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 8e3e0c7c6af9f..cad94e5e19e1f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -17,6 +17,11 @@ package org.apache.spark.deploy.yarn +object AllocationType extends Enumeration { + type AllocationType = Value + val HOST, RACK, ANY = Value +} + /** * Interface that defines a Yarn allocator. */ diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 6e5b912b63cb6..34c3a3849856f 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -20,7 +20,6 @@ package org.apache.spark.deploy.yarn import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} import java.util.concurrent.atomic.AtomicInteger -import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -30,20 +29,13 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.ApplicationMasterProtocol import org.apache.hadoop.yarn.api.records.ApplicationAttemptId -import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus} +import org.apache.hadoop.yarn.api.records.{Container, ContainerId} import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest} import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.util.{RackResolver, Records} - - -object AllocationType extends Enumeration { - type AllocationType = Value - val HOST, RACK, ANY = Value -} +import org.apache.hadoop.yarn.util.Records // TODO: // Too many params. diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index 8c5f6da81f76d..e8b8d9bc722bd 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -17,16 +17,8 @@ package org.apache.spark.deploy.yarn -import java.io.IOException -import java.util.concurrent.CopyOnWriteArrayList -import java.util.concurrent.atomic.AtomicReference - -import scala.collection.JavaConversions._ import scala.collection.{Map, Set} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ @@ -36,10 +28,9 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.hadoop.yarn.webapp.util.WebAppUtils -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.scheduler.SplitInfo -import org.apache.spark.util.{SignalLogger, Utils} +import org.apache.spark.util.Utils /** From 91beabb7749d31843a9f6422ff4bd865eac29195 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Aug 2014 17:43:33 -0700 Subject: [PATCH 04/19] Fix UI filter registration. --- .../spark/deploy/yarn/ApplicationMaster.scala | 26 +++++++++++-------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 9fe5bfd1d1134..74544ce33021e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -62,6 +62,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // Fields used in client mode. private var actorSystem: ActorSystem = null + private var actor: ActorRef = _ // Fields used in cluster mode. private var userThread: Thread = _ @@ -91,6 +92,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, val securityMgr = new SecurityManager(sparkConf) val (uiAddress, uiHistoryAddress) = if (isDriver) { + addAmIpFilter() + // Start the user's JAR userThread = startUserClass() @@ -104,6 +107,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf, securityManager = securityMgr)._1 waitForSparkMaster() + addAmIpFilter() (sparkConf.get("spark.driver.appUIAddress", ""), "") } @@ -117,8 +121,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, registered = true } - addAmIpFilter() - if (registered) { // Launch thread that will heartbeat to the RM so it won't think the app has died. reporterThread = launchReporterThread() @@ -310,8 +312,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) - - actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") + actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } private def allocateExecutors() = { @@ -339,15 +340,18 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // add the yarn amIpFilter that Yarn requires for properly securing the UI private def addAmIpFilter() = { val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" - System.setProperty("spark.ui.filters", amFilter) val proxy = client.getProxyHostAndPort(yarnConf) - val parts : Array[String] = proxy.split(":") - val uriBase = "http://" + proxy + - System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) - + val parts = proxy.split(":") + val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) + val uriBase = "http://" + proxy + proxyBase val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase - System.setProperty("spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", - params) + + if (isDriver) { + System.setProperty("spark.ui.filters", amFilter) + System.setProperty(s"spark.$amFilter.params", params) + } else { + actor ! AddWebUIFilter(amFilter, params, proxyBase) + } } private def startUserClass(): Thread = { From 0e4be3d9913d62d4c87248af26a845fdccbe2108 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Aug 2014 18:45:44 -0700 Subject: [PATCH 05/19] Keep "ExecutorLauncher" as the main class for client-mode AM. --- .../spark/deploy/yarn/ApplicationMaster.scala | 18 +++++++++++++++--- .../apache/spark/deploy/yarn/ClientBase.scala | 12 ++++++++++-- 2 files changed, 25 insertions(+), 5 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 74544ce33021e..40f337bcc4429 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -413,11 +413,11 @@ object ApplicationMaster extends Logging { private var master: ApplicationMaster = _ - def main(argStrings: Array[String]) = { + def main(args: Array[String]) = { SignalLogger.register(log) - val args = new ApplicationMasterArguments(argStrings) + val amArgs = new ApplicationMasterArguments(args) SparkHadoopUtil.get.runAsSparkUser { () => - master = new ApplicationMaster(args, new YarnRMClientImpl(args)) + master = new ApplicationMaster(amArgs, new YarnRMClientImpl(amArgs)) master.run() } } @@ -427,3 +427,15 @@ object ApplicationMaster extends Logging { } } + +/** + * This object does not provide any special functionality. It exists so that it's easy to tell + * apart the client-mode AM from the cluster-mode AM when using tools such as ps or jps. + */ +object ExecutorLauncher { + + def main(args: Array[String]) = { + ApplicationMaster.main(args) + } + +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 9e6507cc7187a..8a789f3eaac67 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -312,6 +312,8 @@ trait ClientBase extends Logging { val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) amContainer.setLocalResources(localResources) + val isLaunchingDriver = args.userClass != null + // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's // SparkContext will not let that set spark* system properties, which is expected behavior for @@ -320,7 +322,7 @@ trait ClientBase extends Logging { // Note that to warn the user about the deprecation in cluster mode, some code from // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition // described above). - if (args.userClass != null) { + if (isLaunchingDriver) { sys.env.get("SPARK_JAVA_OPTS").foreach { value => val warning = s""" @@ -380,7 +382,7 @@ trait ClientBase extends Logging { javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } - if (args.userClass != null) { + if (isLaunchingDriver) { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) .foreach(opts => javaOpts += opts) @@ -394,6 +396,12 @@ trait ClientBase extends Logging { } else { Nil } + val amClass = + if (isLaunchingDriver) { + classOf[ApplicationMaster].getName() + } else { + classOf[ApplicationMaster].getName().replace("ApplicationMaster", "ExecutorLauncher") + } val amArgs = Seq(classOf[ApplicationMaster].getName()) ++ userClass ++ (if (args.userJar != null) Seq("--jar", args.userJar) else Nil) ++ From 5657c7dff3db78a8cb6dd2bbabcaf2bd12ba4947 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Aug 2014 19:09:37 -0700 Subject: [PATCH 06/19] Finish app if SparkContext initialization times out. This avoids the NPEs that would happen if code just kept going. --- .../spark/deploy/yarn/ApplicationMaster.scala | 35 ++++++++++--------- 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 40f337bcc4429..0a042d230e508 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -69,7 +69,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private val sparkContextRef = new AtomicReference[SparkContext](null) private val userResult = new AtomicBoolean(false) - final def run() = { + final def run(): Unit = { // Setup the directories so things go to YARN approved directories rather // than user specified and /tmp. System.setProperty("spark.local.dir", getLocalDirs()) @@ -83,8 +83,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, System.setProperty("spark.master", "yarn-cluster") } - val attemptId = client.getAttemptId() - logInfo("ApplicationAttemptId: " + attemptId) + logInfo("ApplicationAttemptId: " + client.getAttemptId()) // Call this to force generation of secret so it gets populated into the // Hadoop UGI. This has to happen before the startUserClass which does a @@ -102,6 +101,16 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, waitForSparkContextInitialized() val sc = sparkContextRef.get() + + // If there is no SparkContext at this point, just fail the app. + if (sc == null) { + finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") + if (isLastAttempt()) { + cleanupStagingDir() + } + return + } + (sc.ui.appUIHostPort, YarnSparkHadoopUtil.getUIHistoryAddress(sc, sparkConf)) } else { actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, @@ -146,14 +155,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, finish(if (success) FinalApplicationStatus.SUCCEEDED else FinalApplicationStatus.FAILED) - val shouldCleanup = - if (success) { - true - } else { - val maxAppAttempts = client.getMaxRegAttempts(yarnConf) - attemptId.getAttemptId() >= maxAppAttempts - } - + val shouldCleanup = success || isLastAttempt() if (shouldCleanup) { cleanupStagingDir() } @@ -197,6 +199,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } + private def isLastAttempt() = { + val maxAppAttempts = client.getMaxRegAttempts(yarnConf) + client.getAttemptId().getAttemptId() >= maxAppAttempts + } + /** Get the Yarn approved local directories. */ private def getLocalDirs(): String = { // Hadoop 0.23 and 2.x have different Environment variable names for the @@ -316,16 +323,12 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } private def allocateExecutors() = { + logInfo("Requesting" + args.numExecutors + " executors.") try { - logInfo("Requesting" + args.numExecutors + " executors.") - allocator.allocateResources() - - var iters = 0 while (allocator.getNumExecutorsRunning < args.numExecutors && !finished) { checkNumExecutorsFailed() allocator.allocateResources() Thread.sleep(ALLOCATE_HEARTBEAT_INTERVAL) - iters += 1 } } logInfo("All executors have launched.") From 34f1e63e8a63e3631bcd45ce63dd38ef3a067c51 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Aug 2014 19:20:19 -0700 Subject: [PATCH 07/19] Fix some questionable error handling. --- .../spark/deploy/yarn/ApplicationMaster.scala | 104 +++++++++--------- 1 file changed, 50 insertions(+), 54 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 0a042d230e508..510c3a09cc29e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -90,71 +90,67 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) - val (uiAddress, uiHistoryAddress) = if (isDriver) { - addAmIpFilter() + val (sc, uiAddress, uiHistoryAddress) = + if (isDriver) { + addAmIpFilter() + userThread = startUserClass() - // Start the user's JAR - userThread = startUserClass() + // This a bit hacky, but we need to wait until the spark.driver.port property has + // been set by the Thread executing the user class. + waitForSparkContextInitialized() - // This a bit hacky, but we need to wait until the spark.driver.port property has - // been set by the Thread executing the user class. - waitForSparkContextInitialized() + val sc = sparkContextRef.get() - val sc = sparkContextRef.get() + // If there is no SparkContext at this point, just fail the app. + if (sc == null) { - // If there is no SparkContext at this point, just fail the app. - if (sc == null) { - finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") - if (isLastAttempt()) { - cleanupStagingDir() + finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") + if (isLastAttempt()) { + cleanupStagingDir() + } + return } - return - } - - (sc.ui.appUIHostPort, YarnSparkHadoopUtil.getUIHistoryAddress(sc, sparkConf)) - } else { - actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = sparkConf, securityManager = securityMgr)._1 - waitForSparkMaster() - addAmIpFilter() - (sparkConf.get("spark.driver.appUIAddress", ""), "") - } - - Utils.logUncaughtExceptions { - val sc = sparkContextRef.get() - allocator = client.register(yarnConf, - if (sc != null) sc.getConf else sparkConf, - if (sc != null) sc.preferredNodeLocationData else Map(), - uiAddress, - uiHistoryAddress) - registered = true - } - if (registered) { - // Launch thread that will heartbeat to the RM so it won't think the app has died. - reporterThread = launchReporterThread() - - // Allocate all containers - allocateExecutors() - } + (sc, sc.ui.appUIHostPort, YarnSparkHadoopUtil.getUIHistoryAddress(sc, sparkConf)) + } else { + actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, + conf = sparkConf, securityManager = securityMgr)._1 + waitForSparkMaster() + addAmIpFilter() + (null, sparkConf.get("spark.driver.appUIAddress", ""), "") + } val success = - if (isDriver) { - try { - userThread.join() - userResult.get() - } finally { - // In cluster mode, ask the reporter thread to stop since the user app is finished. - reporterThread.interrupt() + try { + allocator = client.register(yarnConf, + if (sc != null) sc.getConf else sparkConf, + if (sc != null) sc.preferredNodeLocationData else Map(), + uiAddress, + uiHistoryAddress) + + reporterThread = launchReporterThread() + allocateExecutors() + + if (isDriver) { + try { + userThread.join() + userResult.get() + } finally { + // In cluster mode, ask the reporter thread to stop since the user app is finished. + reporterThread.interrupt() + } + } else { + // In client mode the actor will stop the reporter thread. + reporterThread.join() + true } - } else { - // In client mode the actor will stop the reporter thread. - reporterThread.join() - true + } catch { + case e: Exception => + logError("Exception while running AM main loop.", e) + false } finish(if (success) FinalApplicationStatus.SUCCEEDED else FinalApplicationStatus.FAILED) - val shouldCleanup = success || isLastAttempt() if (shouldCleanup) { cleanupStagingDir() @@ -270,7 +266,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } - // Note: this need to happen before allocateExecutors. + // Note: this needs to happen before allocateExecutors. private def waitForSparkContextInitialized() { logInfo("Waiting for spark context initialization") try { From ecb23cd844630962d9a5810c08bf47b840c7af24 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Aug 2014 19:30:17 -0700 Subject: [PATCH 08/19] More trivial cleanup. --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 510c3a09cc29e..951d1ca593f75 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -115,7 +115,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } else { actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf, securityManager = securityMgr)._1 - waitForSparkMaster() + waitForSparkDriver() addAmIpFilter() (null, sparkConf.get("spark.driver.appUIAddress", ""), "") } @@ -292,7 +292,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } - private def waitForSparkMaster() { + private def waitForSparkDriver() { logInfo("Waiting for Spark driver to be reachable.") var driverUp = false val hostport = args.userArgs(0) From b6289ab84680604a1b0cabe15d3a0a742aead71f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Aug 2014 19:39:01 -0700 Subject: [PATCH 09/19] Move cluster/client code to separate methods. Makes code a little cleaner and easier to follow. --- .../spark/deploy/yarn/ApplicationMaster.scala | 106 +++++++++--------- 1 file changed, 52 insertions(+), 54 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 951d1ca593f75..446696f91ce88 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -90,60 +90,9 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) - val (sc, uiAddress, uiHistoryAddress) = - if (isDriver) { - addAmIpFilter() - userThread = startUserClass() - - // This a bit hacky, but we need to wait until the spark.driver.port property has - // been set by the Thread executing the user class. - waitForSparkContextInitialized() - - val sc = sparkContextRef.get() - - // If there is no SparkContext at this point, just fail the app. - if (sc == null) { - - finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") - if (isLastAttempt()) { - cleanupStagingDir() - } - return - } - - (sc, sc.ui.appUIHostPort, YarnSparkHadoopUtil.getUIHistoryAddress(sc, sparkConf)) - } else { - actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = sparkConf, securityManager = securityMgr)._1 - waitForSparkDriver() - addAmIpFilter() - (null, sparkConf.get("spark.driver.appUIAddress", ""), "") - } - val success = try { - allocator = client.register(yarnConf, - if (sc != null) sc.getConf else sparkConf, - if (sc != null) sc.preferredNodeLocationData else Map(), - uiAddress, - uiHistoryAddress) - - reporterThread = launchReporterThread() - allocateExecutors() - - if (isDriver) { - try { - userThread.join() - userResult.get() - } finally { - // In cluster mode, ask the reporter thread to stop since the user app is finished. - reporterThread.interrupt() - } - } else { - // In client mode the actor will stop the reporter thread. - reporterThread.join() - true - } + if (isDriver) runDriver() else runExecutorLauncher(securityMgr) } catch { case e: Exception => logError("Exception while running AM main loop.", e) @@ -195,6 +144,55 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } + private def registerAM(uiAddress: String, uiHistoryAddress: String) = { + val sc = sparkContextRef.get() + allocator = client.register(yarnConf, + if (sc != null) sc.getConf else sparkConf, + if (sc != null) sc.preferredNodeLocationData else Map(), + uiAddress, + uiHistoryAddress) + + reporterThread = launchReporterThread() + } + + private def runDriver(): Boolean = { + addAmIpFilter() + userThread = startUserClass() + + // This a bit hacky, but we need to wait until the spark.driver.port property has + // been set by the Thread executing the user class. + waitForSparkContextInitialized() + + val sc = sparkContextRef.get() + + // If there is no SparkContext at this point, just fail the app. + if (sc == null) { + finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") + false + } else { + registerAM(sc.ui.appUIHostPort, YarnSparkHadoopUtil.getUIHistoryAddress(sc, sparkConf)) + try { + userThread.join() + userResult.get() + } finally { + // In cluster mode, ask the reporter thread to stop since the user app is finished. + reporterThread.interrupt() + } + } + } + + private def runExecutorLauncher(securityMgr: SecurityManager): Boolean = { + actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, + conf = sparkConf, securityManager = securityMgr)._1 + waitForSparkDriver() + addAmIpFilter() + registerAM(sparkConf.get("spark.driver.appUIAddress", ""), "") + + // In client mode the actor will stop the reporter thread. + reporterThread.join() + true + } + private def isLastAttempt() = { val maxAppAttempts = client.getMaxRegAttempts(yarnConf) client.getAttemptId().getAttemptId() >= maxAppAttempts @@ -297,7 +295,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, var driverUp = false val hostport = args.userArgs(0) val (driverHost, driverPort) = Utils.parseHostPort(hostport) - while(!driverUp) { + while (!driverUp) { try { val socket = new Socket(driverHost, driverPort) socket.close() @@ -307,7 +305,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, case e: Exception => logError("Failed to connect to driver at %s:%s, retrying ...". format(driverHost, driverPort)) - Thread.sleep(100) + Thread.sleep(100) } } sparkConf.set("spark.driver.host", driverHost) From 5150993f0510ec283c727098e9721397707fce48 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Aug 2014 11:02:04 -0700 Subject: [PATCH 10/19] Some more cleanup. --- .../spark/deploy/yarn/ApplicationMaster.scala | 47 +++++++++---------- 1 file changed, 21 insertions(+), 26 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 446696f91ce88..68b42f309dce3 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -65,7 +65,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private var actor: ActorRef = _ // Fields used in cluster mode. - private var userThread: Thread = _ private val sparkContextRef = new AtomicReference[SparkContext](null) private val userResult = new AtomicBoolean(false) @@ -79,7 +78,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // other spark processes running on the same box System.setProperty("spark.ui.port", "0") - // When running the AM, the Spark master is always "yarn-cluster" + // Set the master property to match the requested mode. System.setProperty("spark.master", "yarn-cluster") } @@ -130,11 +129,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // Note that this will unfortunately not properly clean up the staging files because it gets // called too late, after the filesystem is already shutdown. if (modified) { - Runtime.getRuntime().addShutdownHook(new Thread with Logging { - // This is not only logs, but also ensures that log system is initialized for this instance - // when we are actually 'run'-ing. - logInfo("Adding shutdown hook for context " + sc) - + Runtime.getRuntime().addShutdownHook(new Thread { override def run() { logInfo("Invoking sc stop from shutdown hook") sc.stop() @@ -157,13 +152,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def runDriver(): Boolean = { addAmIpFilter() - userThread = startUserClass() + val userThread = startUserClass() // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. - waitForSparkContextInitialized() - - val sc = sparkContextRef.get() + val sc = waitForSparkContextInitialized() // If there is no SparkContext at this point, just fail the app. if (sc == null) { @@ -184,7 +177,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def runExecutorLauncher(securityMgr: SecurityManager): Boolean = { actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf, securityManager = securityMgr)._1 - waitForSparkDriver() + actor = waitForSparkDriver() addAmIpFilter() registerAM(sparkConf.get("spark.driver.appUIAddress", ""), "") @@ -198,7 +191,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, client.getAttemptId().getAttemptId() >= maxAppAttempts } - /** Get the Yarn approved local directories. */ + /** Get the Yarn-approved local directories. */ private def getLocalDirs(): String = { // Hadoop 0.23 and 2.x have different Environment variable names for the // local dirs, so lets check both. We assume one of the 2 is set. @@ -220,7 +213,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, val schedulerInterval = sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) - // must be <= timeoutInterval / 2. + // must be <= expiryInterval / 2. val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) val t = new Thread { @@ -229,7 +222,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, checkNumExecutorsFailed() logDebug("Sending progress") allocator.allocateResources() - Try(Thread.sleep(interval)) + try { + Thread.sleep(interval) + } catch { + case e: InterruptedException => + } } } } @@ -265,10 +262,9 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } // Note: this needs to happen before allocateExecutors. - private def waitForSparkContextInitialized() { + private def waitForSparkContextInitialized(): SparkContext = { logInfo("Waiting for spark context initialization") try { - var sparkContext: SparkContext = null sparkContextRef.synchronized { var count = 0 val waitTime = 10000L @@ -278,19 +274,20 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, count = count + 1 sparkContextRef.wait(waitTime) } - sparkContext = sparkContextRef.get() - assert(sparkContext != null || count >= numTries) + val sparkContext = sparkContextRef.get() + assert(sparkContext != null || count >= numTries) if (sparkContext == null) { - throw new IllegalStateException( + logError( "Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d".format( count * waitTime, numTries)) } + sparkContext } } } - private def waitForSparkDriver() { + private def waitForSparkDriver(): ActorRef = { logInfo("Waiting for Spark driver to be reachable.") var driverUp = false val hostport = args.userArgs(0) @@ -313,7 +310,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) - actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") + actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } private def allocateExecutors() = { @@ -334,7 +331,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } - // add the yarn amIpFilter that Yarn requires for properly securing the UI + /** Add the Yarn IP filter that is required for properly securing the UI. */ private def addAmIpFilter() = { val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" val proxy = client.getProxyHostAndPort(yarnConf) @@ -354,9 +351,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") System.setProperty("spark.executor.instances", args.numExecutors.toString) - val mainMethod = Class.forName( - args.userClass, - false, + val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { From be6068de32abcdb9e0a7f4371bfd542e72b42e3c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Aug 2014 11:17:30 -0700 Subject: [PATCH 11/19] Restore shutdown hook to clean up staging dir. --- .../spark/deploy/yarn/ApplicationMaster.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 68b42f309dce3..aa6b02c8ba11b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -28,6 +28,7 @@ import akka.actor._ import akka.remote._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration @@ -84,6 +85,21 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, logInfo("ApplicationAttemptId: " + client.getAttemptId()) + // If this is the last attempt, register a shutdown hook to cleanup the staging dir + // after the app is finished, in case it does not exit through the expected means. + // Use priority 30 as it's higher than HDFS. It's the same priority MapReduce is using. + if (isLastAttempt()) { + val cleanupHook = new Runnable { + override def run() { + logInfo("AppMaster received a signal.") + if (!finished) { + cleanupStagingDir() + } + } + } + ShutdownHookManager.get().addShutdownHook(cleanupHook, 30) + } + // Call this to force generation of secret so it gets populated into the // Hadoop UGI. This has to happen before the startUserClass which does a // doAs in order for the credentials to be passed on to the executor containers. From 557fdeba535d4517bb79502240cfddab561a35c3 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Aug 2014 11:26:09 -0700 Subject: [PATCH 12/19] Cleanup a couple more constants. --- .../deploy/yarn/YarnAllocationHandler.scala | 20 ++++++++----------- .../deploy/yarn/YarnSparkHadoopUtil.scala | 6 ++++++ .../deploy/yarn/YarnAllocationHandler.scala | 17 ++++++---------- 3 files changed, 20 insertions(+), 23 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 88f41752d35ef..97b1565755977 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -58,11 +58,6 @@ private[yarn] class YarnAllocationHandler( map: collection.Map[String, collection.Set[SplitInfo]]) extends YarnAllocator with Logging { - private val ANY_HOST = "*" - // All requests are issued with same priority : we do not (yet) have any distinction between - // request types (like map/reduce in hadoop for example) - private val PRIORITY = 1 - // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. @@ -362,7 +357,7 @@ private[yarn] class YarnAllocationHandler( for (container <- hostContainers) { val candidateHost = container.getHostName val candidateNumContainers = container.getNumContainers - assert(ANY_HOST != candidateHost) + assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) if (rack != null) { @@ -376,7 +371,8 @@ private[yarn] class YarnAllocationHandler( new ArrayBuffer[ResourceRequest](rackToCounts.size) for ((rack, count) <- rackToCounts){ requestedContainers += - createResourceRequest(AllocationType.RACK, rack, count, PRIORITY) + createResourceRequest(AllocationType.RACK, rack, count, + YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) } requestedContainers.toList @@ -407,7 +403,7 @@ private[yarn] class YarnAllocationHandler( logDebug("numExecutors: " + numExecutors + ", host preferences: " + preferredHostToCount.isEmpty) resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, numExecutors, PRIORITY)) + AllocationType.ANY, null, numExecutors, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) } else { // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. @@ -421,7 +417,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.HOST, candidateHost, requiredCount, - PRIORITY) + YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) } } val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests( @@ -431,7 +427,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.ANY, resource = null, numExecutors, - PRIORITY) + YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( hostContainerRequests.size + rackContainerRequests.size + 1) @@ -481,7 +477,7 @@ private[yarn] class YarnAllocationHandler( // There must be a third request - which is ANY : that will be specially handled. requestType match { case AllocationType.HOST => { - assert(ANY_HOST != resource) + assert(YarnSparkHadoopUtil.ANY_HOST != resource) val hostname = resource val nodeLocal = createResourceRequestImpl(hostname, numExecutors, priority) @@ -495,7 +491,7 @@ private[yarn] class YarnAllocationHandler( createResourceRequestImpl(rack, numExecutors, priority) } case AllocationType.ANY => createResourceRequestImpl( - ANY_HOST, numExecutors, priority) + YarnSparkHadoopUtil.ANY_HOST, numExecutors, priority) case _ => throw new IllegalArgumentException( "Unexpected/unsupported request type: " + requestType) } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 0738164f74283..2d5a0588a1aa2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -87,6 +87,12 @@ object YarnSparkHadoopUtil { // Additional memory overhead - in mb. val DEFAULT_MEMORY_OVERHEAD = 384 + val ANY_HOST = "*" + + // All RM requests are issued with same priority : we do not (yet) have any distinction between + // request types (like map/reduce in hadoop for example) + val RM_REQUEST_PRIORITY = 1 + // Host to rack map - saved from allocation requests. We are expecting this not to change. // Note that it is possible for this to change : and ResurceManager will indicate that to us via // update response to allocate. But we are punting on handling that for now. diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 34c3a3849856f..84818495fb267 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -59,11 +59,6 @@ private[yarn] class YarnAllocationHandler( map: collection.Map[String, collection.Set[SplitInfo]]) extends YarnAllocator with Logging { - private val ANY_HOST = "*" - // All requests are issued with same priority : we do not (yet) have any distinction between - // request types (like map/reduce in hadoop for example) - private val PRIORITY = 1 - // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. @@ -390,7 +385,7 @@ private[yarn] class YarnAllocationHandler( for (container <- hostContainers) { val candidateHost = container.getNodes.last - assert(ANY_HOST != candidateHost) + assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) if (rack != null) { @@ -406,7 +401,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.RACK, rack, count, - PRIORITY) + YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) } requestedContainers @@ -437,7 +432,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.ANY, resource = null, numExecutors, - PRIORITY).toList + YarnSparkHadoopUtil.RM_REQUEST_PRIORITY).toList } else { // Request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. @@ -450,7 +445,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.HOST, candidateHost, requiredCount, - PRIORITY) + YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) } } val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( @@ -460,7 +455,7 @@ private[yarn] class YarnAllocationHandler( AllocationType.ANY, resource = null, numExecutors, - PRIORITY) + YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size) @@ -509,7 +504,7 @@ private[yarn] class YarnAllocationHandler( // There must be a third request, which is ANY. That will be specially handled. requestType match { case AllocationType.HOST => { - assert(ANY_HOST != resource) + assert(YarnSparkHadoopUtil.ANY_HOST != resource) val hostname = resource val nodeLocal = constructContainerRequests( Array(hostname), From f5811226365ba62a1924d08e95405a656146e7a6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 19 Aug 2014 10:09:37 -0700 Subject: [PATCH 13/19] Review feedback. --- .../deploy/yarn/YarnAllocationHandler.scala | 5 +++-- .../spark/deploy/yarn/ApplicationMaster.scala | 19 ++++--------------- .../apache/spark/deploy/yarn/ClientBase.scala | 2 +- .../spark/deploy/yarn/YarnRMClient.scala | 3 ++- .../deploy/yarn/YarnSparkHadoopUtil.scala | 2 +- .../deploy/yarn/YarnAllocationHandler.scala | 5 +++-- 6 files changed, 14 insertions(+), 22 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 97b1565755977..b4a6a9b786072 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -55,7 +55,7 @@ private[yarn] class YarnAllocationHandler( resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, - map: collection.Map[String, collection.Set[SplitInfo]]) + preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) extends YarnAllocator with Logging { // These three are locked on allocatedHostToContainersMap. Complementary data structures @@ -90,7 +90,8 @@ private[yarn] class YarnAllocationHandler( private val maxExecutors = args.numExecutors private val executorMemory = args.executorMemory private val executorCores = args.executorCores - private val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, map) + private val (preferredHostToCount, preferredRackToCount) = + generateNodeToWeight(conf, preferredNodes) def getNumExecutorsRunning: Int = numExecutorsRunning.intValue diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index aa6b02c8ba11b..5c2fea40e185e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -56,8 +56,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) - private var finished = false - private var registered = false + @volatile private var finished = false private var reporterThread: Thread = _ private var allocator: YarnAllocator = _ @@ -123,7 +122,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, final def finish(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { if (!finished) { - logInfo(s"Finishing ApplicationMaster with $status") + logInfo(s"Finishing ApplicationMaster with $status" + + Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) finished = true reporterThread.interrupt() reporterThread.join() @@ -163,6 +163,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, uiAddress, uiHistoryAddress) + allocator.allocateResources() reporterThread = launchReporterThread() } @@ -329,18 +330,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } - private def allocateExecutors() = { - logInfo("Requesting" + args.numExecutors + " executors.") - try { - while (allocator.getNumExecutorsRunning < args.numExecutors && !finished) { - checkNumExecutorsFailed() - allocator.allocateResources() - Thread.sleep(ALLOCATE_HEARTBEAT_INTERVAL) - } - } - logInfo("All executors have launched.") - } - private def checkNumExecutorsFailed() = { if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finish(FinalApplicationStatus.FAILED, "Max number of executor failures reached.") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8a789f3eaac67..6cf300c3986ad 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -403,7 +403,7 @@ trait ClientBase extends Logging { classOf[ApplicationMaster].getName().replace("ApplicationMaster", "ExecutorLauncher") } val amArgs = - Seq(classOf[ApplicationMaster].getName()) ++ userClass ++ + Seq(amClass) ++ userClass ++ (if (args.userJar != null) Seq("--jar", args.userJar) else Nil) ++ Seq("--executor-memory", args.executorMemory.toString, "--executor-cores", args.executorCores.toString, diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 370177df25f98..3d6a7a012c75b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -26,7 +26,8 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.scheduler.SplitInfo /** - * Interface that defines a Yarn RM client. + * Interface that defines a Yarn RM client. Abstracts away Yarn version-specific functionality that + * is used by Spark's AM. */ trait YarnRMClient { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 2d5a0588a1aa2..719ca9f7583b2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -94,7 +94,7 @@ object YarnSparkHadoopUtil { val RM_REQUEST_PRIORITY = 1 // Host to rack map - saved from allocation requests. We are expecting this not to change. - // Note that it is possible for this to change : and ResurceManager will indicate that to us via + // Note that it is possible for this to change : and ResourceManager will indicate that to us via // update response to allocate. But we are punting on handling that for now. private val hostToRack = new ConcurrentHashMap[String, String]() private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 84818495fb267..cfb64be937efd 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -56,7 +56,7 @@ private[yarn] class YarnAllocationHandler( amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, - map: collection.Map[String, collection.Set[SplitInfo]]) + preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) extends YarnAllocator with Logging { // These three are locked on allocatedHostToContainersMap. Complementary data structures @@ -94,7 +94,8 @@ private[yarn] class YarnAllocationHandler( private val maxExecutors = args.numExecutors private val executorMemory = args.executorMemory private val executorCores = args.executorCores - private val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, map) + private val (preferredHostToCount, preferredRackToCount) = + generateNodeToWeight(conf, preferredNodes) override def getNumExecutorsRunning: Int = numExecutorsRunning.intValue From ecaf332ba2ad02c2fc49ffefb7f17eb756a4bc23 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 19 Aug 2014 10:30:31 -0700 Subject: [PATCH 14/19] Small fix to shutdown code. Avoid an interrupted exception when shutting down from the reporter thread. Also, make extra effort to finish the RM connection. --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 5fb87bd4b62c8..3aa0e581e076d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -126,8 +126,13 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) finished = true reporterThread.interrupt() - reporterThread.join() - client.shutdown(status, diagnostics) + try { + if (Thread.currentThread() != reporterThread) { + reporterThread.join() + } + } finally { + client.shutdown(status, diagnostics) + } } } From c0794befcd6ae6e802fe432f46f163ebdd0f549a Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 20 Aug 2014 15:15:05 -0700 Subject: [PATCH 15/19] Correctly clean up staging directory. This change also avoids overriding the app's status with "SUCCEEDED" in cluster mode when the shutdown hook runs, by signaling the AM that the SparkContext was shut down (see YarnClusterScheduler.scala). That way the AM can correctly expose its final status to the RM. --- .../spark/deploy/yarn/ApplicationMaster.scala | 107 ++++++++---------- .../cluster/YarnClusterScheduler.scala | 13 ++- .../deploy/yarn/YarnAllocationHandler.scala | 2 +- 3 files changed, 60 insertions(+), 62 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 75813b854e83f..9c52a9f653d79 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.yarn import java.io.IOException import java.net.Socket -import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} +import java.util.concurrent.atomic.AtomicReference import scala.collection.JavaConversions._ import scala.util.Try @@ -57,6 +57,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) @volatile private var finished = false + @volatile private var finalStatus = FinalApplicationStatus.UNDEFINED + private var reporterThread: Thread = _ private var allocator: YarnAllocator = _ @@ -66,9 +68,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // Fields used in cluster mode. private val sparkContextRef = new AtomicReference[SparkContext](null) - private val userResult = new AtomicBoolean(false) - final def run(): Unit = { + final def run(): Int = { if (isDriver) { // Set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box @@ -80,43 +81,49 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, logInfo("ApplicationAttemptId: " + client.getAttemptId()) - // If this is the last attempt, register a shutdown hook to cleanup the staging dir - // after the app is finished, in case it does not exit through the expected means. - // Use priority 30 as it's higher than HDFS. It's the same priority MapReduce is using. - if (isLastAttempt()) { - val cleanupHook = new Runnable { - override def run() { - logInfo("AppMaster received a signal.") - if (!finished) { - cleanupStagingDir() - } + val cleanupHook = new Runnable { + override def run() { + // If the SparkContext is still registered, shut it down as a best case effort in case + // users do not call sc.stop or do System.exit(). + val sc = sparkContextRef.get() + if (sc != null) { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() + finish(FinalApplicationStatus.SUCCEEDED) + } + + // Cleanup the staging dir after the app is finished, or if it's the last attempt at + // running the AM. + val maxAppAttempts = client.getMaxRegAttempts(yarnConf) + val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts + if (finished || isLastAttempt) { + cleanupStagingDir() } } - ShutdownHookManager.get().addShutdownHook(cleanupHook, 30) } + // Use priority 30 as it's higher than HDFS. It's the same priority MapReduce is using. + ShutdownHookManager.get().addShutdownHook(cleanupHook, 30) // Call this to force generation of secret so it gets populated into the // Hadoop UGI. This has to happen before the startUserClass which does a // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) - val success = - try { - if (isDriver) runDriver() else runExecutorLauncher(securityMgr) - } catch { - case e: Exception => - logError("Exception while running AM main loop.", e) - false - } + if (isDriver) { + runDriver() + } else { + runExecutorLauncher(securityMgr) + } - finish(if (success) FinalApplicationStatus.SUCCEEDED else FinalApplicationStatus.FAILED) - val shouldCleanup = success || isLastAttempt() - if (shouldCleanup) { - cleanupStagingDir() + if (finalStatus != FinalApplicationStatus.UNDEFINED) { + finish(finalStatus) + 0 + } else { + 1 } } - final def finish(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { + final def finish(status: FinalApplicationStatus, diagnostics: String = null) = synchronized { if (!finished) { logInfo(s"Finishing ApplicationMaster with $status" + Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) @@ -127,33 +134,20 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, reporterThread.join() } } finally { - client.shutdown(status, diagnostics) + client.shutdown(status, Option(diagnostics).getOrElse("")) } } } - private[spark] def sparkContextInitialized(sc: SparkContext) = { - var modified = false + private def sparkContextInitialized(sc: SparkContext) = { sparkContextRef.synchronized { - modified = sparkContextRef.compareAndSet(null, sc) + sparkContextRef.compareAndSet(null, sc) sparkContextRef.notifyAll() } + } - // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do - // System.exit. - // Should not really have to do this, but it helps YARN to evict resources earlier. - // Not to mention, prevent the Client from declaring failure even though we exited properly. - // Note that this will unfortunately not properly clean up the staging files because it gets - // called too late, after the filesystem is already shutdown. - if (modified) { - Runtime.getRuntime().addShutdownHook(new Thread { - override def run() { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() - finish(FinalApplicationStatus.SUCCEEDED) - } - }) - } + private def sparkContextStopped(sc: SparkContext) = { + sparkContextRef.compareAndSet(sc, null) } private def registerAM(uiAddress: String, uiHistoryAddress: String) = { @@ -168,7 +162,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, reporterThread = launchReporterThread() } - private def runDriver(): Boolean = { + private def runDriver(): Unit = { addAmIpFilter() val userThread = startUserClass() @@ -179,12 +173,10 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // If there is no SparkContext at this point, just fail the app. if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") - false } else { registerAM(sc.ui.appUIHostPort, YarnSparkHadoopUtil.getUIHistoryAddress(sc, sparkConf)) try { userThread.join() - userResult.get() } finally { // In cluster mode, ask the reporter thread to stop since the user app is finished. reporterThread.interrupt() @@ -192,7 +184,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } - private def runExecutorLauncher(securityMgr: SecurityManager): Boolean = { + private def runExecutorLauncher(securityMgr: SecurityManager): Unit = { actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf, securityManager = securityMgr)._1 actor = waitForSparkDriver() @@ -201,12 +193,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // In client mode the actor will stop the reporter thread. reporterThread.join() - true - } - - private def isLastAttempt() = { - val maxAppAttempts = client.getMaxRegAttempts(yarnConf) - client.getAttemptId().getAttemptId() >= maxAppAttempts + finalStatus = FinalApplicationStatus.SUCCEEDED } private def launchReporterThread(): Thread = { @@ -361,7 +348,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, mainMethod.invoke(null, mainArgs) // Some apps have "System.exit(0)" at the end. The user thread will stop here unless // it has an uncaught exception thrown out. It needs a shutdown hook to set SUCCEEDED. - userResult.set(true) + finalStatus = FinalApplicationStatus.SUCCEEDED } finally { logDebug("Finishing main") } @@ -408,7 +395,7 @@ object ApplicationMaster extends Logging { val amArgs = new ApplicationMasterArguments(args) SparkHadoopUtil.get.runAsSparkUser { () => master = new ApplicationMaster(amArgs, new YarnRMClientImpl(amArgs)) - master.run() + System.exit(master.run()) } } @@ -416,6 +403,10 @@ object ApplicationMaster extends Logging { master.sparkContextInitialized(sc) } + private[spark] def sparkContextStopped(sc: SparkContext) = { + master.sparkContextStopped(sc) + } + } /** diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala index d4b4097527780..69f40225a21f5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -24,10 +24,11 @@ import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration /** - * - * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of ApplicationMaster, etc is done + * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of + * ApplicationMaster, etc is done */ -private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) { +private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) + extends TaskSchedulerImpl(sc) { logInfo("Created YarnClusterScheduler") @@ -51,4 +52,10 @@ private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) super.postStartHook() logInfo("YarnClusterScheduler.postStartHook done") } + + override def stop() { + super.stop() + ApplicationMaster.sparkContextStopped(sc) + } + } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index cfb64be937efd..fa1f466f8b0ae 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -112,7 +112,7 @@ private[yarn] class YarnAllocationHandler( } override def allocateResources() = { - addResourceRequests(maxExecutors - numPendingAllocate.get()) + addResourceRequests(maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get()) // We have already set the container request. Poll the ResourceManager for a response. // This doubles as a heartbeat if there are no pending container requests. From 41f8c8a2ef588c1ee90c58189d0c8892c88251e9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 21 Aug 2014 09:32:03 -0700 Subject: [PATCH 16/19] Fix app status reporting. --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 9c52a9f653d79..5d80dd6c9c481 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -128,6 +128,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, logInfo(s"Finishing ApplicationMaster with $status" + Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) finished = true + finalStatus = status reporterThread.interrupt() try { if (Thread.currentThread() != reporterThread) { @@ -341,6 +342,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, val t = new Thread { override def run() { + var status = FinalApplicationStatus.FAILED try { // Copy val mainArgs = new Array[String](args.userArgs.size) @@ -348,10 +350,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, mainMethod.invoke(null, mainArgs) // Some apps have "System.exit(0)" at the end. The user thread will stop here unless // it has an uncaught exception thrown out. It needs a shutdown hook to set SUCCEEDED. - finalStatus = FinalApplicationStatus.SUCCEEDED + status = FinalApplicationStatus.SUCCEEDED } finally { logDebug("Finishing main") } + finalStatus = status } } t.setName("Driver") From 0f5142cc8ae00cde0ac96c09ae28c528d316823f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 25 Aug 2014 14:04:17 -0700 Subject: [PATCH 17/19] Review feedback. --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 ++- .../scala/org/apache/spark/deploy/yarn/YarnRMClient.scala | 2 +- .../org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala | 4 ++-- .../spark/scheduler/cluster/YarnClientSchedulerBackend.scala | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 5d80dd6c9c481..e8352bef4647e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -190,7 +190,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, conf = sparkConf, securityManager = securityMgr)._1 actor = waitForSparkDriver() addAmIpFilter() - registerAM(sparkConf.get("spark.driver.appUIAddress", ""), "") + registerAM(sparkConf.get("spark.driver.appUIAddress", ""), + sparkConf.get("spark.driver.appUIHistoryAddress", "")) // In client mode the actor will stop the reporter thread. reporterThread.join() diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 3d6a7a012c75b..922d7d1a854a5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -50,8 +50,8 @@ trait YarnRMClient { /** * Shuts down the AM. Guaranteed to only be called once. * - * @param registered Whether the AM was successfully registered with the RM. * @param status The final status of the AM. + * @param diagnostics Diagnostics message to include in the final status. */ def shutdown(status: FinalApplicationStatus, diagnostics: String = ""): Unit diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 719ca9f7583b2..2aa27a1908582 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -193,14 +193,14 @@ object YarnSparkHadoopUtil { } } - def lookupRack(conf: Configuration, host: String): String = { + private[spark] def lookupRack(conf: Configuration, host: String): String = { if (!hostToRack.contains(host)) { populateRackInfo(conf, host) } hostToRack.get(host) } - def populateRackInfo(conf: Configuration, hostname: String) { + private[spark] def populateRackInfo(conf: Configuration, hostname: String) { Utils.checkHost(hostname) if (!hostToRack.containsKey(hostname)) { diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 4eba6c42658b0..a5f537dd9de30 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{ApplicationMaster, Client, ClientArguments, YarnSparkHadoopUtil} +import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnSparkHadoopUtil} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer From 3a8ed3701c30cb6d1033337306a66760aca109fe Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 25 Aug 2014 15:30:28 -0700 Subject: [PATCH 18/19] Remote stale comment. --- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index e8352bef4647e..844fa9d1c1286 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -254,7 +254,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } - // Note: this needs to happen before allocateExecutors. private def waitForSparkContextInitialized(): SparkContext = { logInfo("Waiting for spark context initialization") try { From ff389ed2629309e8b97f22dd74571e32a0f482d4 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 26 Aug 2014 12:43:57 -0700 Subject: [PATCH 19/19] Do not interrupt reporter thread from within itself. --- .../spark/deploy/yarn/ApplicationMaster.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 844fa9d1c1286..f853ad5f38a7d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -129,9 +129,9 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) finished = true finalStatus = status - reporterThread.interrupt() try { if (Thread.currentThread() != reporterThread) { + reporterThread.interrupt() reporterThread.join() } } finally { @@ -213,12 +213,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, override def run() { while (!finished) { checkNumExecutorsFailed() - logDebug("Sending progress") - allocator.allocateResources() - try { - Thread.sleep(interval) - } catch { - case e: InterruptedException => + if (!finished) { + logDebug("Sending progress") + allocator.allocateResources() + try { + Thread.sleep(interval) + } catch { + case e: InterruptedException => + } } } }