diff --git a/core/pom.xml b/core/pom.xml index 6af229c71d3ad4742dab73dfe9cfab5fac03fe16..38f4be128019c13d177ff3fe6cc4d11f7b5dff5f 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -95,15 +95,15 @@ <version>0.3.1</version> </dependency> <dependency> - <groupId>com.typesafe.akka</groupId> + <groupId>${akka.group}</groupId> <artifactId>akka-actor</artifactId> </dependency> <dependency> - <groupId>com.typesafe.akka</groupId> + <groupId>${akka.group}</groupId> <artifactId>akka-remote</artifactId> </dependency> <dependency> - <groupId>com.typesafe.akka</groupId> + <groupId>${akka.group}</groupId> <artifactId>akka-slf4j</artifactId> </dependency> <dependency> @@ -126,10 +126,6 @@ <groupId>colt</groupId> <artifactId>colt</artifactId> </dependency> - <dependency> - <groupId>com.github.scala-incubator.io</groupId> - <artifactId>scala-io-file_2.9.2</artifactId> - </dependency> <dependency> <groupId>org.apache.mesos</groupId> <artifactId>mesos</artifactId> diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3da13133dacde4e342118ab5b28cc1c4593e1d02..66006bf21238d7676ee3827db7cc6b9d695a1cdf 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -81,7 +81,7 @@ class SparkContext( val sparkHome: String = null, val jars: Seq[String] = Nil, val environment: Map[String, String] = Map(), - // This is used only by yarn for now, but should be relevant to other cluster types (mesos, etc) + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set // of data-local splits on host val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml new file mode 100644 index 0000000000000000000000000000000000000000..8a065c6d7d1d7f458789f3ab50f0054c8a947dfb --- /dev/null +++ b/new-yarn/pom.xml @@ -0,0 +1,161 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ 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. + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <parent> + <groupId>org.apache.spark</groupId> + <artifactId>spark-parent</artifactId> + <version>0.9.0-incubating-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <groupId>org.apache.spark</groupId> + <artifactId>spark-yarn_2.9.3</artifactId> + <packaging>jar</packaging> + <name>Spark Project YARN Support</name> + <url>http://spark.incubator.apache.org/</url> + + <dependencies> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-core_2.9.3</artifactId> + <version>${project.version}</version> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-yarn-api</artifactId> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-yarn-common</artifactId> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-yarn-client</artifactId> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-client</artifactId> + <version>${yarn.version}</version> + </dependency> + <dependency> + <groupId>org.apache.avro</groupId> + <artifactId>avro</artifactId> + </dependency> + <dependency> + <groupId>org.apache.avro</groupId> + <artifactId>avro-ipc</artifactId> + </dependency> + <dependency> + <groupId>org.scalatest</groupId> + <artifactId>scalatest_2.9.3</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.mockito</groupId> + <artifactId>mockito-all</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + + <build> + <outputDirectory>target/scala-${scala.version}/classes</outputDirectory> + <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-shade-plugin</artifactId> + <configuration> + <shadedArtifactAttached>false</shadedArtifactAttached> + <outputFile>${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar</outputFile> + <artifactSet> + <includes> + <include>*:*</include> + </includes> + </artifactSet> + <filters> + <filter> + <artifact>*:*</artifact> + <excludes> + <exclude>META-INF/*.SF</exclude> + <exclude>META-INF/*.DSA</exclude> + <exclude>META-INF/*.RSA</exclude> + </excludes> + </filter> + </filters> + </configuration> + <executions> + <execution> + <phase>package</phase> + <goals> + <goal>shade</goal> + </goals> + <configuration> + <transformers> + <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" /> + <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer"> + <resource>reference.conf</resource> + </transformer> + </transformers> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-antrun-plugin</artifactId> + <executions> + <execution> + <phase>test</phase> + <goals> + <goal>run</goal> + </goals> + <configuration> + <exportAntProperties>true</exportAntProperties> + <tasks> + <property name="spark.classpath" refid="maven.test.classpath" /> + <property environment="env" /> + <fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry."> + <condition> + <not> + <or> + <isset property="env.SCALA_HOME" /> + <isset property="env.SCALA_LIBRARY_PATH" /> + </or> + </not> + </condition> + </fail> + </tasks> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.scalatest</groupId> + <artifactId>scalatest-maven-plugin</artifactId> + <configuration> + <environmentVariables> + <SPARK_HOME>${basedir}/..</SPARK_HOME> + <SPARK_TESTING>1</SPARK_TESTING> + <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH> + </environmentVariables> + </configuration> + </plugin> + </plugins> + </build> +</project> diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala new file mode 100644 index 0000000000000000000000000000000000000000..eeeca3ea8a33e4f77562383dfef50a7507628894 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -0,0 +1,446 @@ +/* + * 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.security.UserGroupInformation +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.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} + +import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.util.Utils + + +class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { + + def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + + private var rpc: YarnRPC = YarnRPC.create(conf) + 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 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 numWorkers * 2, with minimum of 3 + private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", + math.max(args.numWorkers * 2, 3).toString()).toInt + + def run() { + // Setup the directories so things go to YARN approved directories rather + // than user specified and /tmp. + System.setProperty("spark.local.dir", getLocalDirs()) + + // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using. + ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) + + appAttemptId = getApplicationAttemptId() + isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts + amClient = AMRMClient.createAMRMClient() + amClient.init(yarnConf) + amClient.start() + + // Workaround until hadoop moves to something which has + // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line) + // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) + + ApplicationMaster.register(this) + + // 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. + waitForSparkMaster() + + waitForSparkContextInitialized() + + // Do this after Spark master is up and SparkContext is created so that we can register UI Url. + val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() + + // Allocate all containers + allocateWorkers() + + // Wait for the user class to Finish + userThread.join() + + 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")) + .getOrElse(Option(System.getenv("LOCAL_DIRS")) + .getOrElse("")) + + if (localDirs.isEmpty()) { + throw new Exception("Yarn Local dirs can't be empty") + } + localDirs + } + + private def getApplicationAttemptId(): ApplicationAttemptId = { + val envs = System.getenv() + val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name()) + val containerId = ConverterUtils.toContainerId(containerIdString) + val appAttemptId = containerId.getApplicationAttemptId() + logInfo("ApplicationAttemptId: " + appAttemptId) + appAttemptId + } + + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { + logInfo("Registering the ApplicationMaster") + amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + } + + private def waitForSparkMaster() { + logInfo("Waiting for Spark driver to be reachable.") + var driverUp = false + var tries = 0 + val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt + while (!driverUp && tries < numTries) { + val driverHost = System.getProperty("spark.driver.host") + val driverPort = System.getProperty("spark.driver.port") + try { + val socket = new Socket(driverHost, driverPort.toInt) + socket.close() + logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) + driverUp = true + } catch { + case e: Exception => { + logWarning("Failed to connect to driver at %s:%s, retrying ...". + format(driverHost, driverPort)) + Thread.sleep(100) + tries = tries + 1 + } + } + } + } + + private def startUserClass(): Thread = { + logInfo("Starting the user JAR in a separate Thread") + 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 allocateWorkers() + 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 = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt + while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { + 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.appUIAddress + this.yarnAllocator = YarnAllocationHandler.newAllocator( + yarnConf, + amClient, + appAttemptId, + args, + sparkContext.preferredNodeLocationData) + } else { + logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d". + format(numTries * waitTime, maxNumTries)) + this.yarnAllocator = YarnAllocationHandler.newAllocator( + yarnConf, + amClient, + appAttemptId, + args) + } + } + } finally { + // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT : + // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks. + ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) + } + } + + private def allocateWorkers() { + try { + logInfo("Allocating " + args.numWorkers + " workers.") + // Wait until all containers have finished + // TODO: This is a bit ugly. Can we make it nicer? + // TODO: Handle container failure + yarnAllocator.addResourceRequests(args.numWorkers) + // Exits the loop if the user thread exits. + while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) { + if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of worker failures reached") + } + yarnAllocator.allocateResources() + ApplicationMaster.incrementAllocatorLoop(1) + Thread.sleep(100) + } + } finally { + // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT, + // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. + ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) + } + logInfo("All workers have launched.") + + // Launch a progress reporter thread, else the app will get killed after expiration + // (def: 10mins) 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 = + System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + + // 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.getNumWorkersFailed >= maxNumWorkerFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of worker failures reached") + } + val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - + yarnAllocator.getNumPendingAllocate + if (missingWorkerCount > 0) { + logInfo("Allocating %d containers to make up for (potentially) lost containers". + format(missingWorkerCount)) + yarnAllocator.addResourceRequests(missingWorkerCount) + } + 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.allocateResources() + } + + /* + 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()) + } + } + */ + + def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") { + synchronized { + if (isFinished) { + return + } + isFinished = true + } + + logInfo("finishApplicationMaster with " + status) + // Set tracking URL to empty since we don't have a history server. + amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */) + } + + /** + * Clean up the staging directory. + */ + private def cleanupStagingDir() { + var stagingDirPath: Path = null + try { + val preserveFiles = System.getProperty("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 { + // Number of times to wait for the allocator loop to complete. + // Each loop iteration waits for 100ms, so maximum of 3 seconds. + // This is to ensure that we have reasonable number of containers before we start + // 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 ALLOCATOR_LOOP_WAIT_COUNT = 30 + + private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() + + val sparkContextRef: AtomicReference[SparkContext] = + new AtomicReference[SparkContext](null /* initialValue */) + + val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0) + + def incrementAllocatorLoop(by: Int) { + val count = yarnAllocatorLoop.getAndAdd(by) + if (count >= ALLOCATOR_LOOP_WAIT_COUNT) { + yarnAllocatorLoop.synchronized { + // to wake threads off wait ... + yarnAllocatorLoop.notifyAll() + } + } + } + + def register(master: ApplicationMaster) { + applicationMasters.add(master) + } + + // TODO(harvey): See whether this should be discarded - it isn't used anywhere atm... + 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) + } + } + } ) + } + + // Wait for initialization to complete and atleast 'some' nodes can get allocated. + yarnAllocatorLoop.synchronized { + while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) { + yarnAllocatorLoop.wait(1000L) + } + } + modified + } + + def main(argStrings: Array[String]) { + val args = new ApplicationMasterArguments(argStrings) + new ApplicationMaster(args).run() + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala new file mode 100644 index 0000000000000000000000000000000000000000..f76a5ddd39e90d4998d7712113dd0949823d5e18 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -0,0 +1,94 @@ +/* + * 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 org.apache.spark.util.IntParam +import collection.mutable.ArrayBuffer + +class ApplicationMasterArguments(val args: Array[String]) { + var userJar: String = null + var userClass: String = null + var userArgs: Seq[String] = Seq[String]() + var workerMemory = 1024 + var workerCores = 1 + var numWorkers = 2 + + parseArgs(args.toList) + + private def parseArgs(inputArgs: List[String]): Unit = { + val userArgsBuffer = new ArrayBuffer[String]() + + var args = inputArgs + + while (! args.isEmpty) { + + args match { + case ("--jar") :: value :: tail => + userJar = value + args = tail + + case ("--class") :: value :: tail => + userClass = value + args = tail + + case ("--args") :: value :: tail => + userArgsBuffer += value + args = tail + + case ("--num-workers") :: IntParam(value) :: tail => + numWorkers = value + args = tail + + case ("--worker-memory") :: IntParam(value) :: tail => + workerMemory = value + args = tail + + case ("--worker-cores") :: IntParam(value) :: tail => + workerCores = value + args = tail + + case Nil => + if (userJar == null || userClass == null) { + printUsageAndExit(1) + } + + case _ => + printUsageAndExit(1, args) + } + } + + userArgs = userArgsBuffer.readOnly + } + + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + 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-workers NUM Number of workers to start (Default: 2)\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n") + System.exit(exitCode) + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala new file mode 100644 index 0000000000000000000000000000000000000000..94678815e806ad66680880e59ac4f7c01d1fd67c --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -0,0 +1,519 @@ +/* + * 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.{InetAddress, UnknownHostException, URI} +import java.nio.ByteBuffer + +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap +import scala.collection.mutable.Map + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil} +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.mapred.Master +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{Apps, Records} + +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil + + +/** + * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The + * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, + * which will launch a Spark master process and negotiate resources throughout its duration. + */ +class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { + + var rpc: YarnRPC = YarnRPC.create(conf) + val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + val credentials = UserGroupInformation.getCurrentUser().getCredentials() + private val SPARK_STAGING: String = ".sparkStaging" + private val distCacheMgr = new ClientDistributedCacheManager() + + // Staging directory is private! -> rwx-------- + val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short) + // App files are world-wide readable and owner writable -> rw-r--r-- + val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short) + + def this(args: ClientArguments) = this(new Configuration(), args) + + def runApp(): ApplicationId = { + validateArgs() + // Initialize and start the client service. + init(yarnConf) + start() + + // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers). + logClusterResourceDetails() + + // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM) + // interface). + + // Get a new client application. + val newApp = super.createApplication() + val newAppResponse = newApp.getNewApplicationResponse() + val appId = newAppResponse.getApplicationId() + + verifyClusterResources(newAppResponse) + + // Set up resource and environment variables. + val appStagingDir = getAppStagingDir(appId) + val localResources = prepareLocalResources(appStagingDir) + val launchEnv = setupLaunchEnv(localResources, appStagingDir) + val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv) + + // Set up an application submission context. + val appContext = newApp.getApplicationSubmissionContext() + appContext.setApplicationName(args.appName) + appContext.setQueue(args.amQueue) + appContext.setAMContainerSpec(amContainer) + + // Memory for the ApplicationMaster. + val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] + memoryResource.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + appContext.setResource(memoryResource) + + // Finally, submit and monitor the application. + submitApp(appContext) + appId + } + + def run() { + val appId = runApp() + monitorApplication(appId) + System.exit(0) + } + + // TODO(harvey): This could just go in ClientArguments. + def validateArgs() = { + Map( + (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", + (args.userJar == null) -> "Error: You must specify a user jar!", + (args.userClass == null) -> "Error: You must specify a user class!", + (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!", + (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" + + "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), + (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" + + "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString) + ).foreach { case(cond, errStr) => + if (cond) { + logError(errStr) + args.printUsageAndExit(1) + } + } + } + + def getAppStagingDir(appId: ApplicationId): String = { + SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR + } + + def logClusterResourceDetails() { + val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics + logInfo("Got Cluster metric info from ApplicationsManager (ASM), number of NodeManagers: " + + clusterMetrics.getNumNodeManagers) + + val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) + logInfo("""Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s, + queueApplicationCount = %s, queueChildQueueCount = %s""".format( + queueInfo.getQueueName, + queueInfo.getCurrentCapacity, + queueInfo.getMaximumCapacity, + queueInfo.getApplications.size, + queueInfo.getChildQueues.size)) + } + + def verifyClusterResources(app: GetNewApplicationResponse) = { + val maxMem = app.getMaximumResourceCapability().getMemory() + logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) + + // If we have requested more then the clusters max for a single resource then exit. + if (args.workerMemory > maxMem) { + logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.". + format(args.workerMemory, maxMem)) + System.exit(1) + } + val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD + if (amMem > maxMem) { + logError("Required AM memory (%d) is above the max threshold (%d) of this cluster". + format(args.amMemory, maxMem)) + System.exit(1) + } + + // We could add checks to make sure the entire cluster has enough resources but that involves + // getting all the node reports and computing ourselves. + } + + /** See if two file systems are the same or not. */ + private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { + val srcUri = srcFs.getUri() + val dstUri = destFs.getUri() + if (srcUri.getScheme() == null) { + return false + } + if (!srcUri.getScheme().equals(dstUri.getScheme())) { + return false + } + var srcHost = srcUri.getHost() + var dstHost = dstUri.getHost() + if ((srcHost != null) && (dstHost != null)) { + try { + srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() + dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() + } catch { + case e: UnknownHostException => + return false + } + if (!srcHost.equals(dstHost)) { + return false + } + } else if (srcHost == null && dstHost != null) { + return false + } else if (srcHost != null && dstHost == null) { + return false + } + //check for ports + if (srcUri.getPort() != dstUri.getPort()) { + return false + } + return true + } + + /** Copy the file into HDFS if needed. */ + private def copyRemoteFile( + dstDir: Path, + originalPath: Path, + replication: Short, + setPerms: Boolean = false): Path = { + val fs = FileSystem.get(conf) + val remoteFs = originalPath.getFileSystem(conf) + var newPath = originalPath + if (! compareFs(remoteFs, fs)) { + newPath = new Path(dstDir, originalPath.getName()) + logInfo("Uploading " + originalPath + " to " + newPath) + FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) + fs.setReplication(newPath, replication) + if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) + } + // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific + // version shows the specific version in the distributed cache configuration + val qualPath = fs.makeQualified(newPath) + val fc = FileContext.getFileContext(qualPath.toUri(), conf) + val destPath = fc.resolvePath(qualPath) + destPath + } + + def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { + logInfo("Preparing Local resources") + // Upload Spark and the application JAR to the remote file system if necessary. Add them as + // local resources to the application master. + val fs = FileSystem.get(conf) + + val delegTokenRenewer = Master.getMasterPrincipal(conf) + if (UserGroupInformation.isSecurityEnabled()) { + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + logError("Can't get Master Kerberos principal for use as renewer") + System.exit(1) + } + } + val dst = new Path(fs.getHomeDirectory(), appStagingDir) + val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort + + if (UserGroupInformation.isSecurityEnabled()) { + val dstFs = dst.getFileSystem(conf) + dstFs.addDelegationTokens(delegTokenRenewer, credentials) + } + + val localResources = HashMap[String, LocalResource]() + FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) + + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + + Map( + Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, + Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF") + ).foreach { case(destName, _localPath) => + val localPath: String = if (_localPath != null) _localPath.trim() else "" + if (! localPath.isEmpty()) { + var localURI = new URI(localPath) + // If not specified assume these are in the local filesystem to keep behavior like Hadoop + if (localURI.getScheme() == null) { + localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString) + } + val setPermissions = if (destName.equals(Client.APP_JAR)) true else false + val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions) + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + destName, statCache) + } + } + + // Handle jars local to the ApplicationMaster. + if ((args.addJars != null) && (!args.addJars.isEmpty())){ + args.addJars.split(',').foreach { case file: String => + val localURI = new URI(file.trim()) + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyRemoteFile(dst, localPath, replication) + // Only add the resource to the Spark ApplicationMaster. + val appMasterOnly = true + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + linkname, statCache, appMasterOnly) + } + } + + // Handle any distributed cache files + if ((args.files != null) && (!args.files.isEmpty())){ + args.files.split(',').foreach { case file: String => + val localURI = new URI(file.trim()) + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyRemoteFile(dst, localPath, replication) + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + linkname, statCache) + } + } + + // Handle any distributed cache archives + if ((args.archives != null) && (!args.archives.isEmpty())) { + args.archives.split(',').foreach { case file:String => + val localURI = new URI(file.trim()) + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyRemoteFile(dst, localPath, replication) + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, + linkname, statCache) + } + } + + UserGroupInformation.getCurrentUser().addCredentials(credentials) + localResources + } + + def setupLaunchEnv( + localResources: HashMap[String, LocalResource], + stagingDir: String): HashMap[String, String] = { + logInfo("Setting up the launch environment") + val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null) + + val env = new HashMap[String, String]() + + Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env) + env("SPARK_YARN_MODE") = "true" + env("SPARK_YARN_STAGING_DIR") = stagingDir + + // Set the environment variables to be passed on to the Workers. + distCacheMgr.setDistFilesEnv(env) + distCacheMgr.setDistArchivesEnv(env) + + // Allow users to specify some environment variables. + Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) + + // Add each SPARK_* key to the environment. + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + + env + } + + def userArgsToString(clientArgs: ClientArguments): String = { + val prefix = " --args " + val args = clientArgs.userArgs + val retval = new StringBuilder() + for (arg <- args){ + retval.append(prefix).append(" '").append(arg).append("' ") + } + retval.toString + } + + def createContainerLaunchContext( + newApp: GetNewApplicationResponse, + localResources: HashMap[String, LocalResource], + env: HashMap[String, String]): ContainerLaunchContext = { + logInfo("Setting up container launch context") + val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) + amContainer.setLocalResources(localResources) + amContainer.setEnvironment(env) + + // TODO: Need a replacement for the following code to fix -Xmx? + // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() + // var amMemory = ((args.amMemory / minResMemory) * minResMemory) + + // ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - + // YarnAllocationHandler.MEMORY_OVERHEAD) + + // Extra options for the JVM + var JAVA_OPTS = "" + + // Add Xmx for AM memory + JAVA_OPTS += "-Xmx" + args.amMemory + "m" + + val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir + + // TODO: Remove once cpuset version is pushed out. + // The context is, default gc for server class machines ends up using all cores to do gc - + // hence if there are multiple containers in same node, Spark GC affects all other containers' + // performance (which can be that of other Spark containers) + // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in + // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset + // of cores on a node. + val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && + java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC")) + if (useConcurrentAndIncrementalGC) { + // In our expts, using (default) throughput collector has severe perf ramifications in + // multi-tenant machines + JAVA_OPTS += " -XX:+UseConcMarkSweepGC " + JAVA_OPTS += " -XX:+CMSIncrementalMode " + JAVA_OPTS += " -XX:+CMSIncrementalPacing " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + } + + if (env.isDefinedAt("SPARK_JAVA_OPTS")) { + JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") + } + + // Command for the ApplicationMaster + var javaCommand = "java" + val javaHome = System.getenv("JAVA_HOME") + if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { + javaCommand = Environment.JAVA_HOME.$() + "/bin/java" + } + + val commands = List[String]( + javaCommand + + " -server " + + JAVA_OPTS + + " " + args.amClass + + " --class " + args.userClass + + " --jar " + args.userJar + + userArgsToString(args) + + " --worker-memory " + args.workerMemory + + " --worker-cores " + args.workerCores + + " --num-workers " + args.numWorkers + + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + logInfo("Command for starting the Spark ApplicationMaster: " + commands(0)) + amContainer.setCommands(commands) + + // Setup security tokens. + val dob = new DataOutputBuffer() + credentials.writeTokenStorageToStream(dob) + amContainer.setTokens(ByteBuffer.wrap(dob.getData())) + + amContainer + } + + def submitApp(appContext: ApplicationSubmissionContext) = { + // Submit the application to the applications manager. + logInfo("Submitting application to ASM") + super.submitApplication(appContext) + } + + def monitorApplication(appId: ApplicationId): Boolean = { + while (true) { + Thread.sleep(1000) + val report = super.getApplicationReport(appId) + + logInfo("Application report from ASM: \n" + + "\t application identifier: " + appId.toString() + "\n" + + "\t appId: " + appId.getId() + "\n" + + "\t clientToAMToken: " + report.getClientToAMToken() + "\n" + + "\t appDiagnostics: " + report.getDiagnostics() + "\n" + + "\t appMasterHost: " + report.getHost() + "\n" + + "\t appQueue: " + report.getQueue() + "\n" + + "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + + "\t appStartTime: " + report.getStartTime() + "\n" + + "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + + "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" + + "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + + "\t appUser: " + report.getUser() + ) + + val state = report.getYarnApplicationState() + val dsStatus = report.getFinalApplicationStatus() + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + return true + } + } + true + } +} + +object Client { + val SPARK_JAR: String = "spark.jar" + val APP_JAR: String = "app.jar" + val LOG4J_PROP: String = "log4j.properties" + + def main(argStrings: Array[String]) { + // Set an env variable indicating we are running in YARN mode. + // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - + // see Client#setupLaunchEnv(). + System.setProperty("SPARK_YARN_MODE", "true") + + val args = new ClientArguments(argStrings) + + (new Client(args)).run() + } + + // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps + def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { + for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim) + } + } + + def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$()) + // If log4j present, ensure ours overrides all others + if (addLog4j) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + LOG4J_PROP) + } + // Normally the users app.jar is last in case conflicts with spark jars + val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") + .toBoolean + if (userClasspathFirst) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + APP_JAR) + } + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + SPARK_JAR) + Client.populateHadoopClasspath(conf, env) + + if (!userClasspathFirst) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + APP_JAR) + } + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + "*") + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala new file mode 100644 index 0000000000000000000000000000000000000000..9efb28a9426726c4e41e4287291b8ab939ba942e --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -0,0 +1,148 @@ +/* + * 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.mutable.{ArrayBuffer, HashMap} + +import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} +import org.apache.spark.util.IntParam +import org.apache.spark.util.MemoryParam + + +// TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! +class ClientArguments(val args: Array[String]) { + var addJars: String = null + var files: String = null + var archives: String = null + var userJar: String = null + var userClass: String = null + var userArgs: Seq[String] = Seq[String]() + var workerMemory = 1024 // MB + var workerCores = 1 + var numWorkers = 2 + var amQueue = System.getProperty("QUEUE", "default") + var amMemory: Int = 512 // MB + var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" + var appName: String = "Spark" + // TODO + var inputFormatInfo: List[InputFormatInfo] = null + // TODO(harvey) + var priority = 0 + + parseArgs(args.toList) + + private def parseArgs(inputArgs: List[String]): Unit = { + val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() + val inputFormatMap: HashMap[String, InputFormatInfo] = new HashMap[String, InputFormatInfo]() + + var args = inputArgs + + while (!args.isEmpty) { + args match { + case ("--jar") :: value :: tail => + userJar = value + args = tail + + case ("--class") :: value :: tail => + userClass = value + args = tail + + case ("--args") :: value :: tail => + userArgsBuffer += value + args = tail + + case ("--master-class") :: value :: tail => + amClass = value + args = tail + + case ("--master-memory") :: MemoryParam(value) :: tail => + amMemory = value + args = tail + + case ("--num-workers") :: IntParam(value) :: tail => + numWorkers = value + args = tail + + case ("--worker-memory") :: MemoryParam(value) :: tail => + workerMemory = value + args = tail + + case ("--worker-cores") :: IntParam(value) :: tail => + workerCores = value + args = tail + + case ("--queue") :: value :: tail => + amQueue = value + args = tail + + case ("--name") :: value :: tail => + appName = value + + case ("--addJars") :: value :: tail => + addJars = value + args = tail + + case ("--files") :: value :: tail => + files = value + args = tail + + case ("--archives") :: value :: tail => + archives = value + args = tail + + case Nil => + if (userJar == null || userClass == null) { + printUsageAndExit(1) + } + + case _ => + printUsageAndExit(1, args) + } + } + + userArgs = userArgsBuffer.readOnly + inputFormatInfo = inputFormatMap.values.toList + } + + + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + if (unknownParam != null) { + System.err.println("Unknown/unsupported param " + unknownParam) + } + System.err.println( + "Usage: org.apache.spark.deploy.yarn.Client [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-workers NUM Number of workers to start (Default: 2)\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + + " --master-class CLASS_NAME Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" + + " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + + " --name NAME The name of your application (Default: Spark)\n" + + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + + " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + + " --files files Comma separated list of files to be distributed with the job.\n" + + " --archives archives Comma separated list of archives to be distributed with the job." + ) + System.exit(exitCode) + } + +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala new file mode 100644 index 0000000000000000000000000000000000000000..5f159b073f5372dc9e1f73347f88ae8aa48e0cbb --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -0,0 +1,228 @@ +/* + * 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.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.permission.FsAction +import org.apache.hadoop.yarn.api.records.LocalResource +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility +import org.apache.hadoop.yarn.api.records.LocalResourceType +import org.apache.hadoop.yarn.util.{Records, ConverterUtils} + +import org.apache.spark.Logging + +import scala.collection.mutable.HashMap +import scala.collection.mutable.LinkedHashMap +import scala.collection.mutable.Map + + +/** Client side methods to setup the Hadoop distributed cache */ +class ClientDistributedCacheManager() extends Logging { + private val distCacheFiles: Map[String, Tuple3[String, String, String]] = + LinkedHashMap[String, Tuple3[String, String, String]]() + private val distCacheArchives: Map[String, Tuple3[String, String, String]] = + LinkedHashMap[String, Tuple3[String, String, String]]() + + + /** + * Add a resource to the list of distributed cache resources. This list can + * be sent to the ApplicationMaster and possibly the workers so that it can + * be downloaded into the Hadoop distributed cache for use by this application. + * Adds the LocalResource to the localResources HashMap passed in and saves + * the stats of the resources to they can be sent to the workers and verified. + * + * @param fs FileSystem + * @param conf Configuration + * @param destPath path to the resource + * @param localResources localResource hashMap to insert the resource into + * @param resourceType LocalResourceType + * @param link link presented in the distributed cache to the destination + * @param statCache cache to store the file/directory stats + * @param appMasterOnly Whether to only add the resource to the app master + */ + def addResource( + fs: FileSystem, + conf: Configuration, + destPath: Path, + localResources: HashMap[String, LocalResource], + resourceType: LocalResourceType, + link: String, + statCache: Map[URI, FileStatus], + appMasterOnly: Boolean = false) = { + val destStatus = fs.getFileStatus(destPath) + val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + amJarRsrc.setType(resourceType) + val visibility = getVisibility(conf, destPath.toUri(), statCache) + amJarRsrc.setVisibility(visibility) + amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath)) + amJarRsrc.setTimestamp(destStatus.getModificationTime()) + amJarRsrc.setSize(destStatus.getLen()) + if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name") + localResources(link) = amJarRsrc + + if (appMasterOnly == false) { + val uri = destPath.toUri() + val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link) + if (resourceType == LocalResourceType.FILE) { + distCacheFiles(pathURI.toString()) = (destStatus.getLen().toString(), + destStatus.getModificationTime().toString(), visibility.name()) + } else { + distCacheArchives(pathURI.toString()) = (destStatus.getLen().toString(), + destStatus.getModificationTime().toString(), visibility.name()) + } + } + } + + /** + * Adds the necessary cache file env variables to the env passed in + * @param env + */ + def setDistFilesEnv(env: Map[String, String]) = { + val (keys, tupleValues) = distCacheFiles.unzip + val (sizes, timeStamps, visibilities) = tupleValues.unzip3 + + if (keys.size > 0) { + env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = + timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = + sizes.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_FILES_VISIBILITIES") = + visibilities.reduceLeft[String] { (acc,n) => acc + "," + n } + } + } + + /** + * Adds the necessary cache archive env variables to the env passed in + * @param env + */ + def setDistArchivesEnv(env: Map[String, String]) = { + val (keys, tupleValues) = distCacheArchives.unzip + val (sizes, timeStamps, visibilities) = tupleValues.unzip3 + + if (keys.size > 0) { + env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = + timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") = + sizes.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") = + visibilities.reduceLeft[String] { (acc,n) => acc + "," + n } + } + } + + /** + * Returns the local resource visibility depending on the cache file permissions + * @param conf + * @param uri + * @param statCache + * @return LocalResourceVisibility + */ + def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): + LocalResourceVisibility = { + if (isPublic(conf, uri, statCache)) { + return LocalResourceVisibility.PUBLIC + } + return LocalResourceVisibility.PRIVATE + } + + /** + * Returns a boolean to denote whether a cache file is visible to all(public) + * or not + * @param conf + * @param uri + * @param statCache + * @return true if the path in the uri is visible to all, false otherwise + */ + def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { + val fs = FileSystem.get(uri, conf) + val current = new Path(uri.getPath()) + //the leaf level file should be readable by others + if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) { + return false + } + return ancestorsHaveExecutePermissions(fs, current.getParent(), statCache) + } + + /** + * Returns true if all ancestors of the specified path have the 'execute' + * permission set for all users (i.e. that other users can traverse + * the directory heirarchy to the given path) + * @param fs + * @param path + * @param statCache + * @return true if all ancestors have the 'execute' permission set for all users + */ + def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, + statCache: Map[URI, FileStatus]): Boolean = { + var current = path + while (current != null) { + //the subdirs in the path should have execute permissions for others + if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) { + return false + } + current = current.getParent() + } + return true + } + + /** + * Checks for a given path whether the Other permissions on it + * imply the permission in the passed FsAction + * @param fs + * @param path + * @param action + * @param statCache + * @return true if the path in the uri is visible to all, false otherwise + */ + def checkPermissionOfOther(fs: FileSystem, path: Path, + action: FsAction, statCache: Map[URI, FileStatus]): Boolean = { + val status = getFileStatus(fs, path.toUri(), statCache) + val perms = status.getPermission() + val otherAction = perms.getOtherAction() + if (otherAction.implies(action)) { + return true + } + return false + } + + /** + * Checks to see if the given uri exists in the cache, if it does it + * returns the existing FileStatus, otherwise it stats the uri, stores + * it in the cache, and returns the FileStatus. + * @param fs + * @param uri + * @param statCache + * @return FileStatus + */ + def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = { + val stat = statCache.get(uri) match { + case Some(existstat) => existstat + case None => + val newStat = fs.getFileStatus(new Path(uri)) + statCache.put(uri, newStat) + newStat + } + return stat + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala new file mode 100644 index 0000000000000000000000000000000000000000..c38f33e212fbf2e2bf60fa1dd4793012ea288f81 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -0,0 +1,223 @@ +/* + * 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.util.{ConverterUtils, Records} +import akka.actor._ +import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import akka.actor.Terminated +import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.SplitInfo +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest + +class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging { + + def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + + 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 amClient: AMRMClient[ContainerRequest] = _ + + val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1 + var actor: ActorRef = _ + + // This actor just working as a monitor to watch on Driver Actor. + class MonitorActor(driverUrl: String) extends Actor { + + var driver: ActorRef = null + + override def preStart() { + logInfo("Listen to driver: " + driverUrl) + driver = context.actorFor(driverUrl) + context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.watch(driver) // Doesn't work with remote actors, but useful for testing + } + + override def receive = { + case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + logInfo("Driver terminated or disconnected! Shutting down.") + driverClosed = true + } + } + + def run() { + + amClient = AMRMClient.createAMRMClient() + amClient.init(yarnConf) + amClient.start() + + appAttemptId = getApplicationAttemptId() + registerApplicationMaster() + + waitForSparkMaster() + + // Allocate all containers + allocateWorkers() + + // 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) + // must be <= timeoutInterval/ 2. + // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. + // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. + val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L)) + reporterThread = launchReporterThread(interval) + + // Wait for the reporter thread to Finish. + reporterThread.join() + + finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + actorSystem.shutdown() + + logInfo("Exited") + System.exit(0) + } + + private def getApplicationAttemptId(): ApplicationAttemptId = { + val envs = System.getenv() + val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name()) + val containerId = ConverterUtils.toContainerId(containerIdString) + val appAttemptId = containerId.getApplicationAttemptId() + logInfo("ApplicationAttemptId: " + appAttemptId) + appAttemptId + } + + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { + logInfo("Registering the ApplicationMaster") + // TODO:(Raymond) Find out Spark UI address and fill in here? + amClient.registerApplicationMaster(Utils.localHostName(), 0, "") + } + + 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) + } + } + System.setProperty("spark.driver.host", driverHost) + System.setProperty("spark.driver.port", driverPort.toString) + + val driverUrl = "akka://spark@%s:%s/user/%s".format( + driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + + actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") + } + + + private def allocateWorkers() { + + // 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, + amClient, + appAttemptId, + args, + preferredNodeLocationData) + + logInfo("Allocating " + args.numWorkers + " workers.") + // Wait until all containers have finished + // TODO: This is a bit ugly. Can we make it nicer? + // TODO: Handle container failure + + yarnAllocator.addResourceRequests(args.numWorkers) + while(yarnAllocator.getNumWorkersRunning < args.numWorkers) { + yarnAllocator.allocateResources() + Thread.sleep(100) + } + + logInfo("All workers have launched.") + + } + + // 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) { + val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - + yarnAllocator.getNumPendingAllocate + if (missingWorkerCount > 0) { + logInfo("Allocating %d containers to make up for (potentially) lost containers". + format(missingWorkerCount)) + yarnAllocator.addResourceRequests(missingWorkerCount) + } + 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.allocateResources() + } + + def finishApplicationMaster(status: FinalApplicationStatus) { + logInfo("finish ApplicationMaster with " + status) + amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */) + } + +} + + +object WorkerLauncher { + def main(argStrings: Array[String]) { + val args = new ApplicationMasterArguments(argStrings) + new WorkerLauncher(args).run() + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala new file mode 100644 index 0000000000000000000000000000000000000000..9f5523c4b97a8811c14a19f9e34c6cd1e32028e7 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -0,0 +1,209 @@ +/* + * 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.URI +import java.nio.ByteBuffer +import java.security.PrivilegedExceptionAction + +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.client.api.NMClient +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} + +import org.apache.spark.Logging + + +class WorkerRunnable( + container: Container, + conf: Configuration, + masterAddress: String, + slaveId: String, + hostname: String, + workerMemory: Int, + workerCores: Int) + extends Runnable with Logging { + + var rpc: YarnRPC = YarnRPC.create(conf) + var nmClient: NMClient = _ + val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + + def run = { + logInfo("Starting Worker Container") + nmClient = NMClient.createNMClient() + nmClient.init(yarnConf) + nmClient.start() + startContainer + } + + def startContainer = { + logInfo("Setting up ContainerLaunchContext") + + val ctx = Records.newRecord(classOf[ContainerLaunchContext]) + .asInstanceOf[ContainerLaunchContext] + + val localResources = prepareLocalResources + ctx.setLocalResources(localResources) + + val env = prepareEnvironment + ctx.setEnvironment(env) + + // Extra options for the JVM + var JAVA_OPTS = "" + // Set the JVM memory + val workerMemoryString = workerMemory + "m" + JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " " + if (env.isDefinedAt("SPARK_JAVA_OPTS")) { + JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " + } + + JAVA_OPTS += " -Djava.io.tmpdir=" + + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " + + // Commenting it out for now - so that people can refer to the properties if required. Remove + // it once cpuset version is pushed out. + // The context is, default gc for server class machines end up using all cores to do gc - hence + // if there are multiple containers in same node, spark gc effects all other containers + // performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in + // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset + // of cores on a node. +/* + else { + // If no java_opts specified, default to using -XX:+CMSIncrementalMode + // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont + // want to mess with it. + // In our expts, using (default) throughput collector has severe perf ramnifications in + // multi-tennent machines + // The options are based on + // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline + JAVA_OPTS += " -XX:+UseConcMarkSweepGC " + JAVA_OPTS += " -XX:+CMSIncrementalMode " + JAVA_OPTS += " -XX:+CMSIncrementalPacing " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + } +*/ + + val credentials = UserGroupInformation.getCurrentUser().getCredentials() + val dob = new DataOutputBuffer() + credentials.writeTokenStorageToStream(dob) + ctx.setTokens(ByteBuffer.wrap(dob.getData())) + + var javaCommand = "java" + val javaHome = System.getenv("JAVA_HOME") + if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { + javaCommand = Environment.JAVA_HOME.$() + "/bin/java" + } + + val commands = List[String](javaCommand + + " -server " + + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. + // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in + // an inconsistent state. + // TODO: If the OOM is not recoverable by rescheduling it on different node, then do + // 'something' to fail job ... akin to blacklisting trackers in mapred ? + " -XX:OnOutOfMemoryError='kill %p' " + + JAVA_OPTS + + " org.apache.spark.executor.CoarseGrainedExecutorBackend " + + masterAddress + " " + + slaveId + " " + + hostname + " " + + workerCores + + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + logInfo("Setting up worker with commands: " + commands) + ctx.setCommands(commands) + + // Send the start request to the ContainerManager + nmClient.startContainer(container, ctx) + } + + private def setupDistributedCache( + file: String, + rtype: LocalResourceType, + localResources: HashMap[String, LocalResource], + timestamp: String, + size: String, + vis: String) = { + val uri = new URI(file) + val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + amJarRsrc.setType(rtype) + amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) + amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) + amJarRsrc.setTimestamp(timestamp.toLong) + amJarRsrc.setSize(size.toLong) + localResources(uri.getFragment()) = amJarRsrc + } + + def prepareLocalResources: HashMap[String, LocalResource] = { + logInfo("Preparing Local resources") + val localResources = HashMap[String, LocalResource]() + + if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { + val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') + val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') + val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',') + val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',') + for( i <- 0 to distFiles.length - 1) { + setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i), + fileSizes(i), visibilities(i)) + } + } + + if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) { + val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',') + val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',') + val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',') + val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',') + for( i <- 0 to distArchives.length - 1) { + setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, + timeStamps(i), fileSizes(i), visibilities(i)) + } + } + + logInfo("Prepared Local resources " + localResources) + localResources + } + + def prepareEnvironment: HashMap[String, String] = { + val env = new HashMap[String, String]() + + Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env) + + // Allow users to specify some environment variables + Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) + + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + env + } + +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala new file mode 100644 index 0000000000000000000000000000000000000000..dba0f7640e67cc88bbe432ec12e218c6103244ff --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -0,0 +1,687 @@ +/* + * 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.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} + +import org.apache.spark.Logging +import org.apache.spark.scheduler.SplitInfo +import org.apache.spark.scheduler.cluster.{ClusterScheduler, 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.{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 ("HOST", "RACK", "ANY") { + type AllocationType = Value + val HOST, RACK, ANY = Value +} + +// TODO: +// Too many params. +// Needs to be mt-safe +// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should +// make it more proactive and decoupled. + +// Note that right now, we assume all node asks as uniform in terms of capabilities and priority +// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for +// more info on how we are requesting for containers. +private[yarn] class YarnAllocationHandler( + val conf: Configuration, + val amClient: AMRMClient[ContainerRequest], + val appAttemptId: ApplicationAttemptId, + val maxWorkers: Int, + val workerMemory: Int, + val workerCores: Int, + val preferredHostToCount: Map[String, Int], + val preferredRackToCount: Map[String, Int]) + extends Logging { + // These three are locked on allocatedHostToContainersMap. Complementary data structures + // allocatedHostToContainersMap : containers which are running : host, Set<containerid> + // allocatedContainerToHostMap: container to host mapping. + private val allocatedHostToContainersMap = + new HashMap[String, collection.mutable.Set[ContainerId]]() + + private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() + + // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an + // allocated node) + // As with the two data structures above, tightly coupled with them, and to be locked on + // allocatedHostToContainersMap + private val allocatedRackCount = new HashMap[String, Int]() + + // Containers which have been released. + private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]() + // Containers to be released in next request to RM + private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] + + // Number of container requests that have been sent to, but not yet allocated by the + // ApplicationMaster. + private val numPendingAllocate = new AtomicInteger() + private val numWorkersRunning = new AtomicInteger() + // Used to generate a unique id per worker + private val workerIdCounter = new AtomicInteger() + private val lastResponseId = new AtomicInteger() + private val numWorkersFailed = new AtomicInteger() + + def getNumPendingAllocate: Int = numPendingAllocate.intValue + + def getNumWorkersRunning: Int = numWorkersRunning.intValue + + def getNumWorkersFailed: Int = numWorkersFailed.intValue + + def isResourceConstraintSatisfied(container: Container): Boolean = { + container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + } + + def releaseContainer(container: Container) { + val containerId = container.getId + pendingReleaseContainers.put(containerId, true) + amClient.releaseAssignedContainer(containerId) + } + + def allocateResources() { + // 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 + val allocateResponse = amClient.allocate(progressIndicator) + + val allocatedContainers = allocateResponse.getAllocatedContainers() + if (allocatedContainers.size > 0) { + var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size) + + if (numPendingAllocateNow < 0) { + numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow) + } + + logDebug(""" + Allocated containers: %d + Current worker count: %d + Containers released: %s + Containers to-be-released: %s + Cluster resources: %s + """.format( + allocatedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers, + allocateResponse.getAvailableResources)) + + val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() + + for (container <- allocatedContainers) { + if (isResourceConstraintSatisfied(container)) { + // Add the accepted `container` to the host's list of already accepted, + // allocated containers + val host = container.getNodeId.getHost + val containersForHost = hostToContainers.getOrElseUpdate(host, + new ArrayBuffer[Container]()) + containersForHost += container + } else { + // Release container, since it doesn't satisfy resource constraints. + releaseContainer(container) + } + } + + // Find the appropriate containers to use. + // TODO: Cleanup this group-by... + val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() + + for (candidateHost <- hostToContainers.keySet) { + val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) + val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) + + val remainingContainersOpt = hostToContainers.get(candidateHost) + assert(remainingContainersOpt.isDefined) + var remainingContainers = remainingContainersOpt.get + + if (requiredHostCount >= remainingContainers.size) { + // Since we have <= required containers, add all remaining containers to + // `dataLocalContainers`. + dataLocalContainers.put(candidateHost, remainingContainers) + // There are no more free containers remaining. + remainingContainers = null + } else if (requiredHostCount > 0) { + // Container list has more containers than we need for data locality. + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. + val (dataLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredHostCount) + dataLocalContainers.put(candidateHost, dataLocal) + + // Invariant: remainingContainers == remaining + + // YARN has a nasty habit of allocating a ton of containers on a host - discourage this. + // Add each container in `remaining` to list of containers to release. If we have an + // insufficient number of containers, then the next allocation cycle will reallocate + // (but won't treat it as data local). + // TODO(harvey): Rephrase this comment some more. + for (container <- remaining) releaseContainer(container) + remainingContainers = null + } + + // For rack local containers + if (remainingContainers != null) { + val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + if (rack != null) { + val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) + val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - + rackLocalContainers.getOrElse(rack, List()).size + + if (requiredRackCount >= remainingContainers.size) { + // Add all remaining containers to to `dataLocalContainers`. + dataLocalContainers.put(rack, remainingContainers) + remainingContainers = null + } else if (requiredRackCount > 0) { + // Container list has more containers that we need for data locality. + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. + val (rackLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredRackCount) + val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, + new ArrayBuffer[Container]()) + + existingRackLocal ++= rackLocal + + remainingContainers = remaining + } + } + } + + if (remainingContainers != null) { + // Not all containers have been consumed - add them to the list of off-rack containers. + offRackContainers.put(candidateHost, remainingContainers) + } + } + + // Now that we have split the containers into various groups, go through them in order: + // first host-local, then rack-local, and finally off-rack. + // Note that the list we create below tries to ensure that not all containers end up within + // a host if there is a sufficiently large number of hosts/containers. + val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size) + allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(dataLocalContainers) + allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(rackLocalContainers) + allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(offRackContainers) + + // Run each of the allocated containers. + for (container <- allocatedContainersToProcess) { + val numWorkersRunningNow = numWorkersRunning.incrementAndGet() + val workerHostname = container.getNodeId.getHost + val containerId = container.getId + + val workerMemoryOverhead = (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + assert(container.getResource.getMemory >= workerMemoryOverhead) + + if (numWorkersRunningNow > maxWorkers) { + logInfo("""Ignoring container %s at host %s, since we already have the required number of + containers for it.""".format(containerId, workerHostname)) + releaseContainer(container) + numWorkersRunning.decrementAndGet() + } else { + val workerId = workerIdCounter.incrementAndGet().toString + val driverUrl = "akka://spark@%s:%s/user/%s".format( + System.getProperty("spark.driver.host"), + System.getProperty("spark.driver.port"), + CoarseGrainedSchedulerBackend.ACTOR_NAME) + + logInfo("Launching container %s for on host %s".format(containerId, workerHostname)) + + // To be safe, remove the container from `pendingReleaseContainers`. + pendingReleaseContainers.remove(containerId) + + val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) + allocatedHostToContainersMap.synchronized { + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, + new HashSet[ContainerId]()) + + containerSet += containerId + allocatedContainerToHostMap.put(containerId, workerHostname) + + if (rack != null) { + allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) + } + } + logInfo("Launching WorkerRunnable. driverUrl: %s, workerHostname: %s".format(driverUrl, workerHostname)) + val workerRunnable = new WorkerRunnable( + container, + conf, + driverUrl, + workerId, + workerHostname, + workerMemory, + workerCores) + new Thread(workerRunnable).start() + } + } + logDebug(""" + Finished allocating %s containers (from %s originally). + Current number of workers running: %d, + releasedContainerList: %s, + pendingReleaseContainers: %s + """.format( + allocatedContainersToProcess, + allocatedContainers, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers)) + } + + val completedContainers = allocateResponse.getCompletedContainersStatuses() + if (completedContainers.size > 0) { + logDebug("Completed %d containers".format(completedContainers.size)) + + for (completedContainer <- completedContainers) { + val containerId = completedContainer.getContainerId + + if (pendingReleaseContainers.containsKey(containerId)) { + // YarnAllocationHandler already marked the container for release, so remove it from + // `pendingReleaseContainers`. + pendingReleaseContainers.remove(containerId) + } else { + // Decrement the number of workers running. The next iteration of the ApplicationMaster's + // reporting thread will take care of allocating. + numWorkersRunning.decrementAndGet() + logInfo("Completed container %s (state: %s, exit status: %s)".format( + containerId, + completedContainer.getState, + completedContainer.getExitStatus())) + // Hadoop 2.2.X added a ContainerExitStatus we should switch to use + // there are some exit status' we shouldn't necessarily count against us, but for + // now I think its ok as none of the containers are expected to exit + if (completedContainer.getExitStatus() != 0) { + logInfo("Container marked as failed: " + containerId) + numWorkersFailed.incrementAndGet() + } + } + + allocatedHostToContainersMap.synchronized { + if (allocatedContainerToHostMap.containsKey(containerId)) { + val hostOpt = allocatedContainerToHostMap.get(containerId) + assert(hostOpt.isDefined) + val host = hostOpt.get + + val containerSetOpt = allocatedHostToContainersMap.get(host) + assert(containerSetOpt.isDefined) + val containerSet = containerSetOpt.get + + containerSet.remove(containerId) + if (containerSet.isEmpty) { + allocatedHostToContainersMap.remove(host) + } else { + allocatedHostToContainersMap.update(host, containerSet) + } + + allocatedContainerToHostMap.remove(containerId) + + // TODO: Move this part outside the synchronized block? + val rack = YarnAllocationHandler.lookupRack(conf, host) + if (rack != null) { + val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 + if (rackCount > 0) { + allocatedRackCount.put(rack, rackCount) + } else { + allocatedRackCount.remove(rack) + } + } + } + } + } + logDebug(""" + Finished processing %d completed containers. + Current number of workers running: %d, + releasedContainerList: %s, + pendingReleaseContainers: %s + """.format( + completedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers)) + } + } + + def createRackResourceRequests( + hostContainers: ArrayBuffer[ContainerRequest] + ): ArrayBuffer[ContainerRequest] = { + // Generate modified racks and new set of hosts under it before issuing requests. + val rackToCounts = new HashMap[String, Int]() + + for (container <- hostContainers) { + val candidateHost = container.getNodes.last + assert(YarnAllocationHandler.ANY_HOST != candidateHost) + + val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + if (rack != null) { + var count = rackToCounts.getOrElse(rack, 0) + count += 1 + rackToCounts.put(rack, count) + } + } + + val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size) + for ((rack, count) <- rackToCounts) { + requestedContainers ++= createResourceRequests( + AllocationType.RACK, + rack, + count, + YarnAllocationHandler.PRIORITY) + } + + requestedContainers + } + + def allocatedContainersOnHost(host: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedHostToContainersMap.getOrElse(host, Set()).size + } + retval + } + + def allocatedContainersOnRack(rack: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedRackCount.getOrElse(rack, 0) + } + retval + } + + def addResourceRequests(numWorkers: Int) { + val containerRequests: List[ContainerRequest] = + if (numWorkers <= 0 || preferredHostToCount.isEmpty) { + logDebug("numWorkers: " + numWorkers + ", host preferences: " + + preferredHostToCount.isEmpty) + createResourceRequests( + AllocationType.ANY, + resource = null, + numWorkers, + YarnAllocationHandler.PRIORITY).toList + } else { + // Request for all hosts in preferred nodes and for numWorkers - + // candidates.size, request by default allocation policy. + val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) + for ((candidateHost, candidateCount) <- preferredHostToCount) { + val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) + + if (requiredCount > 0) { + hostContainerRequests ++= createResourceRequests( + AllocationType.HOST, + candidateHost, + requiredCount, + YarnAllocationHandler.PRIORITY) + } + } + val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( + hostContainerRequests).toList + + val anyContainerRequests = createResourceRequests( + AllocationType.ANY, + resource = null, + numWorkers, + YarnAllocationHandler.PRIORITY) + + val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( + hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size) + + containerRequestBuffer ++= hostContainerRequests + containerRequestBuffer ++= rackContainerRequests + containerRequestBuffer ++= anyContainerRequests + containerRequestBuffer.toList + } + + for (request <- containerRequests) { + amClient.addContainerRequest(request) + } + + if (numWorkers > 0) { + numPendingAllocate.addAndGet(numWorkers) + logInfo("Will Allocate %d worker containers, each with %d memory".format( + numWorkers, + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))) + } else { + logDebug("Empty allocation request ...") + } + + for (request <- containerRequests) { + val nodes = request.getNodes + var hostStr = if (nodes == null || nodes.isEmpty) { + "Any" + } else { + nodes.last + } + logInfo("Container request (host: %s, priority: %s, capability: %s".format( + hostStr, + request.getPriority().getPriority, + request.getCapability)) + } + } + + private def createResourceRequests( + requestType: AllocationType.AllocationType, + resource: String, + numWorkers: Int, + priority: Int + ): ArrayBuffer[ContainerRequest] = { + + // If hostname is specified, then we need at least two requests - node local and rack local. + // There must be a third request, which is ANY. That will be specially handled. + requestType match { + case AllocationType.HOST => { + assert(YarnAllocationHandler.ANY_HOST != resource) + val hostname = resource + val nodeLocal = constructContainerRequests( + Array(hostname), + racks = null, + numWorkers, + priority) + + // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. + YarnAllocationHandler.populateRackInfo(conf, hostname) + nodeLocal + } + case AllocationType.RACK => { + val rack = resource + constructContainerRequests(hosts = null, Array(rack), numWorkers, priority) + } + case AllocationType.ANY => constructContainerRequests( + hosts = null, racks = null, numWorkers, priority) + case _ => throw new IllegalArgumentException( + "Unexpected/unsupported request type: " + requestType) + } + } + + private def constructContainerRequests( + hosts: Array[String], + racks: Array[String], + numWorkers: Int, + priority: Int + ): ArrayBuffer[ContainerRequest] = { + + val memoryResource = Records.newRecord(classOf[Resource]) + memoryResource.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + + val prioritySetting = Records.newRecord(classOf[Priority]) + prioritySetting.setPriority(priority) + + val requests = new ArrayBuffer[ContainerRequest]() + for (i <- 0 until numWorkers) { + requests += new ContainerRequest(memoryResource, hosts, racks, prioritySetting) + } + 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 + ): YarnAllocationHandler = { + new YarnAllocationHandler( + conf, + amClient, + appAttemptId, + args.numWorkers, + args.workerMemory, + args.workerCores, + Map[String, Int](), + Map[String, Int]()) + } + + def newAllocator( + conf: Configuration, + amClient: AMRMClient[ContainerRequest], + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments, + map: collection.Map[String, + collection.Set[SplitInfo]] + ): YarnAllocationHandler = { + val (hostToSplitCount, rackToSplitCount) = generateNodeToWeight(conf, map) + new YarnAllocationHandler( + conf, + amClient, + appAttemptId, + args.numWorkers, + args.workerMemory, + args.workerCores, + hostToSplitCount, + rackToSplitCount) + } + + def newAllocator( + conf: Configuration, + amClient: AMRMClient[ContainerRequest], + appAttemptId: ApplicationAttemptId, + maxWorkers: Int, + workerMemory: Int, + workerCores: Int, + map: collection.Map[String, collection.Set[SplitInfo]] + ): YarnAllocationHandler = { + val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) + new YarnAllocationHandler( + conf, + amClient, + appAttemptId, + maxWorkers, + workerMemory, + workerCores, + hostToCount, + rackToCount) + } + + // A simple method to copy the split info map. + private def generateNodeToWeight( + conf: Configuration, + input: collection.Map[String, collection.Set[SplitInfo]] + ): (Map[String, Int], Map[String, Int]) = { + + if (input == null) { + return (Map[String, Int](), Map[String, Int]()) + } + + val hostToCount = new HashMap[String, Int] + val rackToCount = new HashMap[String, Int] + + for ((host, splits) <- input) { + val hostCount = hostToCount.getOrElse(host, 0) + hostToCount.put(host, hostCount + splits.size) + + val rack = lookupRack(conf, host) + if (rack != null){ + val rackCount = rackToCount.getOrElse(host, 0) + rackToCount.put(host, rackCount + splits.size) + } + } + + (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/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala new file mode 100644 index 0000000000000000000000000000000000000000..2ba2366ead17113c764663ed934a5221f0eeb0ee --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -0,0 +1,43 @@ +/* + * 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 org.apache.spark.deploy.SparkHadoopUtil +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.conf.Configuration + +/** + * Contains util methods to interact with Hadoop from spark. + */ +class YarnSparkHadoopUtil extends SparkHadoopUtil { + + // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true. + override def isYarnMode(): Boolean = { true } + + // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems + // Always create a new config, dont reuse yarnConf. + override def newConfiguration(): Configuration = new YarnConfiguration(new Configuration()) + + // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + override def addCredentials(conf: JobConf) { + val jobCreds = conf.getCredentials() + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala new file mode 100644 index 0000000000000000000000000000000000000000..63a0449e5a0730085554d2b8ae86067135fa8dba --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -0,0 +1,47 @@ +/* + * 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.scheduler.cluster + +import org.apache.spark._ +import org.apache.hadoop.conf.Configuration +import org.apache.spark.deploy.yarn.YarnAllocationHandler +import org.apache.spark.util.Utils + +/** + * + * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM. + */ +private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(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 + } + + override def postStartHook() { + + // The yarn application is running, but the worker might not yet ready + // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt + Thread.sleep(2000L) + logInfo("YarnClientClusterScheduler.postStartHook done") + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala new file mode 100644 index 0000000000000000000000000000000000000000..b206780c7806e15c84944db05876f89c8f848040 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -0,0 +1,109 @@ +/* + * 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.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} + +private[spark] class YarnClientSchedulerBackend( + scheduler: ClusterScheduler, + sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + with Logging { + + var client: Client = null + var appId: ApplicationId = null + + override def start() { + super.start() + + val defalutWorkerCores = "2" + val defalutWorkerMemory = "512m" + val defaultWorkerNumber = "1" + + val userJar = System.getenv("SPARK_YARN_APP_JAR") + var workerCores = System.getenv("SPARK_WORKER_CORES") + var workerMemory = System.getenv("SPARK_WORKER_MEMORY") + var workerNumber = System.getenv("SPARK_WORKER_INSTANCES") + + if (userJar == null) + throw new SparkException("env SPARK_YARN_APP_JAR is not set") + + if (workerCores == null) + workerCores = defalutWorkerCores + if (workerMemory == null) + workerMemory = defalutWorkerMemory + if (workerNumber == null) + workerNumber = defaultWorkerNumber + + val driverHost = System.getProperty("spark.driver.host") + val driverPort = System.getProperty("spark.driver.port") + val hostport = driverHost + ":" + driverPort + + val argsArray = Array[String]( + "--class", "notused", + "--jar", userJar, + "--args", hostport, + "--worker-memory", workerMemory, + "--worker-cores", workerCores, + "--num-workers", workerNumber, + "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" + ) + + val args = new ClientArguments(argsArray) + client = new Client(args) + appId = client.runApp() + waitForApp() + } + + def waitForApp() { + + // TODO : need a better way to find out whether the workers are ready or not + // maybe by resource usage report? + while(true) { + val report = client.getApplicationReport(appId) + + logInfo("Application report from ASM: \n" + + "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + + "\t appStartTime: " + report.getStartTime() + "\n" + + "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + ) + + // Ready to go, or already gone. + val state = report.getYarnApplicationState() + if (state == YarnApplicationState.RUNNING) { + return + } else if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + throw new SparkException("Yarn application already ended," + + "might be killed or not able to launch application master.") + } + + Thread.sleep(1000) + } + } + + override def stop() { + super.stop() + client.stop() + logInfo("Stoped") + } + +} diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala new file mode 100644 index 0000000000000000000000000000000000000000..29b3f22e13697b38bc501e2f914d8fc0a202d722 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -0,0 +1,55 @@ +/* + * 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.scheduler.cluster + +import org.apache.spark._ +import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler} +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 + */ +private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) { + + logInfo("Created YarnClusterScheduler") + + def this(sc: SparkContext) = this(sc, new Configuration()) + + // Nothing else for now ... initialize application master : which needs sparkContext to determine how to allocate + // Note that only the first creation of SparkContext influences (and ideally, there must be only one SparkContext, right ?) + // Subsequent creations are ignored - since nodes are already allocated by then. + + + // 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 + } + + override def postStartHook() { + val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc) + if (sparkContextInitialized){ + // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt + Thread.sleep(3000L) + } + logInfo("YarnClusterScheduler.postStartHook done") + } +} diff --git a/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala new file mode 100644 index 0000000000000000000000000000000000000000..2941356bc55f9f85ca176c3ef0d23a1a08c6a8e5 --- /dev/null +++ b/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala @@ -0,0 +1,220 @@ +/* + * 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.URI + +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar +import org.mockito.Mockito.when + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.permission.FsAction +import org.apache.hadoop.yarn.api.records.LocalResource +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility +import org.apache.hadoop.yarn.api.records.LocalResourceType +import org.apache.hadoop.yarn.util.{Records, ConverterUtils} + +import scala.collection.mutable.HashMap +import scala.collection.mutable.Map + + +class ClientDistributedCacheManagerSuite extends FunSuite with MockitoSugar { + + class MockClientDistributedCacheManager extends ClientDistributedCacheManager { + override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): + LocalResourceVisibility = { + return LocalResourceVisibility.PRIVATE + } + } + + test("test getFileStatus empty") { + val distMgr = new ClientDistributedCacheManager() + val fs = mock[FileSystem] + val uri = new URI("/tmp/testing") + when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus()) + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + val stat = distMgr.getFileStatus(fs, uri, statCache) + assert(stat.getPath() === null) + } + + test("test getFileStatus cached") { + val distMgr = new ClientDistributedCacheManager() + val fs = mock[FileSystem] + val uri = new URI("/tmp/testing") + val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner", + null, new Path("/tmp/testing")) + when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus()) + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus) + val stat = distMgr.getFileStatus(fs, uri, statCache) + assert(stat.getPath().toString() === "/tmp/testing") + } + + test("test addResource") { + val distMgr = new MockClientDistributedCacheManager() + val fs = mock[FileSystem] + val conf = new Configuration() + val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") + val localResources = HashMap[String, LocalResource]() + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + when(fs.getFileStatus(destPath)).thenReturn(new FileStatus()) + + distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link", + statCache, false) + val resource = localResources("link") + assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) + assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) + assert(resource.getTimestamp() === 0) + assert(resource.getSize() === 0) + assert(resource.getType() === LocalResourceType.FILE) + + val env = new HashMap[String, String]() + distMgr.setDistFilesEnv(env) + assert(env("SPARK_YARN_CACHE_FILES") === "file:/foo.invalid.com:8080/tmp/testing#link") + assert(env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === "0") + assert(env("SPARK_YARN_CACHE_FILES_FILE_SIZES") === "0") + assert(env("SPARK_YARN_CACHE_FILES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) + + distMgr.setDistArchivesEnv(env) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) + + //add another one and verify both there and order correct + val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", + null, new Path("/tmp/testing2")) + val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2") + when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus) + distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2", + statCache, false) + val resource2 = localResources("link2") + assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE) + assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2) + assert(resource2.getTimestamp() === 10) + assert(resource2.getSize() === 20) + assert(resource2.getType() === LocalResourceType.FILE) + + val env2 = new HashMap[String, String]() + distMgr.setDistFilesEnv(env2) + val timestamps = env2("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') + val files = env2("SPARK_YARN_CACHE_FILES").split(',') + val sizes = env2("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') + val visibilities = env2("SPARK_YARN_CACHE_FILES_VISIBILITIES") .split(',') + assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link") + assert(timestamps(0) === "0") + assert(sizes(0) === "0") + assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name()) + + assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2") + assert(timestamps(1) === "10") + assert(sizes(1) === "20") + assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name()) + } + + test("test addResource link null") { + val distMgr = new MockClientDistributedCacheManager() + val fs = mock[FileSystem] + val conf = new Configuration() + val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") + val localResources = HashMap[String, LocalResource]() + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + when(fs.getFileStatus(destPath)).thenReturn(new FileStatus()) + + intercept[Exception] { + distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null, + statCache, false) + } + assert(localResources.get("link") === None) + assert(localResources.size === 0) + } + + test("test addResource appmaster only") { + val distMgr = new MockClientDistributedCacheManager() + val fs = mock[FileSystem] + val conf = new Configuration() + val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") + val localResources = HashMap[String, LocalResource]() + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", + null, new Path("/tmp/testing")) + when(fs.getFileStatus(destPath)).thenReturn(realFileStatus) + + distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", + statCache, true) + val resource = localResources("link") + assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) + assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) + assert(resource.getTimestamp() === 10) + assert(resource.getSize() === 20) + assert(resource.getType() === LocalResourceType.ARCHIVE) + + val env = new HashMap[String, String]() + distMgr.setDistFilesEnv(env) + assert(env.get("SPARK_YARN_CACHE_FILES") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) + + distMgr.setDistArchivesEnv(env) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) + } + + test("test addResource archive") { + val distMgr = new MockClientDistributedCacheManager() + val fs = mock[FileSystem] + val conf = new Configuration() + val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") + val localResources = HashMap[String, LocalResource]() + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", + null, new Path("/tmp/testing")) + when(fs.getFileStatus(destPath)).thenReturn(realFileStatus) + + distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", + statCache, false) + val resource = localResources("link") + assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) + assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) + assert(resource.getTimestamp() === 10) + assert(resource.getSize() === 20) + assert(resource.getType() === LocalResourceType.ARCHIVE) + + val env = new HashMap[String, String]() + + distMgr.setDistArchivesEnv(env) + assert(env("SPARK_YARN_CACHE_ARCHIVES") === "file:/foo.invalid.com:8080/tmp/testing#link") + assert(env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === "10") + assert(env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === "20") + assert(env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) + + distMgr.setDistFilesEnv(env) + assert(env.get("SPARK_YARN_CACHE_FILES") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) + } + + +} diff --git a/pom.xml b/pom.xml index 42c1e00e9d11da954ea90e26f5a48a5ff1d09db7..9348c770fb523877db19fc4dca81c63e71962840 100644 --- a/pom.xml +++ b/pom.xml @@ -102,7 +102,9 @@ <java.version>1.5</java.version> <scala.version>2.9.3</scala.version> <mesos.version>0.13.0</mesos.version> + <akka.group>com.typesafe.akka</akka.group> <akka.version>2.0.5</akka.version> + <protobuf.version>2.4.1</protobuf.version> <slf4j.version>1.7.2</slf4j.version> <log4j.version>1.2.17</log4j.version> <hadoop.version>1.0.4</hadoop.version> @@ -245,7 +247,7 @@ <dependency> <groupId>com.google.protobuf</groupId> <artifactId>protobuf-java</artifactId> - <version>2.4.1</version> + <version>${protobuf.version}</version> </dependency> <dependency> <groupId>com.twitter</groupId> @@ -258,7 +260,7 @@ <version>0.3.1</version> </dependency> <dependency> - <groupId>com.typesafe.akka</groupId> + <groupId>${akka.group}</groupId> <artifactId>akka-actor</artifactId> <version>${akka.version}</version> <exclusions> @@ -269,7 +271,7 @@ </exclusions> </dependency> <dependency> - <groupId>com.typesafe.akka</groupId> + <groupId>${akka.group}</groupId> <artifactId>akka-remote</artifactId> <version>${akka.version}</version> <exclusions> @@ -280,7 +282,7 @@ </exclusions> </dependency> <dependency> - <groupId>com.typesafe.akka</groupId> + <groupId>${akka.group}</groupId> <artifactId>akka-slf4j</artifactId> <version>${akka.version}</version> <exclusions> @@ -290,6 +292,17 @@ </exclusion> </exclusions> </dependency> + <dependency> + <groupId>${akka.group}</groupId> + <artifactId>akka-zeromq</artifactId> + <version>${akka.version}</version> + <exclusions> + <exclusion> + <groupId>org.jboss.netty</groupId> + <artifactId>netty</artifactId> + </exclusion> + </exclusions> + </dependency> <dependency> <groupId>it.unimi.dsi</groupId> <artifactId>fastutil</artifactId> @@ -300,11 +313,6 @@ <artifactId>colt</artifactId> <version>1.2.0</version> </dependency> - <dependency> - <groupId>com.github.scala-incubator.io</groupId> - <artifactId>scala-io-file_2.9.2</artifactId> - <version>0.4.1</version> - </dependency> <dependency> <groupId>org.apache.mesos</groupId> <artifactId>mesos</artifactId> @@ -770,6 +778,41 @@ </dependencies> </dependencyManagement> </profile> + + <profile> + <id>new-yarn</id> + <properties> + <akka.group>org.spark-project</akka.group> + <akka.version>2.0.5-protobuf-2.5-java-1.5</akka.version> + <hadoop.major.version>2</hadoop.major.version> + <hadoop.version>2.2.0</hadoop.version> + <protobuf.version>2.5.0</protobuf.version> + </properties> + + <modules> + <module>new-yarn</module> + </modules> + + <repositories> + <repository> + <id>maven-root</id> + <name>Maven root repository</name> + <url>http://repo1.maven.org/maven2/</url> + <releases> + <enabled>true</enabled> + </releases> + <snapshots> + <enabled>false</enabled> + </snapshots> + </repository> + </repositories> + + <dependencyManagement> + <dependencies> + </dependencies> + </dependencyManagement> + </profile> + <profile> <id>repl-bin</id> <activation> diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 819dfa29a9e02c714a5260f8a9f4560d7e27f8fe..ac87cffd9fbd434875fad4d205b5c8ecb72310dd 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -28,6 +28,11 @@ object SparkBuild extends Build { // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set // through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN. val DEFAULT_HADOOP_VERSION = "1.0.4" + + // Whether the Hadoop version to build against is 2.2.x, or a variant of it. This can be set + // through the SPARK_IS_NEW_HADOOP environment variable. + val DEFAULT_IS_NEW_HADOOP = false + val DEFAULT_YARN = false // HBase version; set as appropriate. @@ -55,8 +60,6 @@ object SparkBuild extends Build { lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) - lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core) - lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) @@ -68,14 +71,27 @@ object SparkBuild extends Build { // Allows build configuration to be set through environment variables lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) + lazy val isNewHadoop = scala.util.Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { + case None => { + val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined + (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP) + } + case Some(v) => v.toBoolean + } lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match { case None => DEFAULT_YARN case Some(v) => v.toBoolean } + // Build against a protobuf-2.5 compatible Akka if Hadoop 2 is used. + lazy val protobufVersion = if (isNewHadoop) "2.5.0" else "2.4.1" + lazy val akkaVersion = if (isNewHadoop) "2.0.5-protobuf-2.5-java-1.5" else "2.0.5" + lazy val akkaGroup = if (isNewHadoop) "org.spark-project" else "com.typesafe.akka" + // Conditionally include the yarn sub-project - lazy val maybeYarn = if(isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() - lazy val maybeYarnRef = if(isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]() + lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "yarn"), settings = yarnSettings) dependsOn(core) + lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() + lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]() // Everything except assembly, tools and examples belong to packageProjects lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef @@ -211,10 +227,10 @@ object SparkBuild extends Build { "com.ning" % "compress-lzf" % "0.8.4", "org.xerial.snappy" % "snappy-java" % "1.0.5", "org.ow2.asm" % "asm" % "4.0", - "com.google.protobuf" % "protobuf-java" % "2.4.1", - "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty), + "com.google.protobuf" % "protobuf-java" % protobufVersion, + akkaGroup % "akka-actor" % akkaVersion excludeAll(excludeNetty), + akkaGroup % "akka-remote" % akkaVersion excludeAll(excludeNetty), + akkaGroup % "akka-slf4j" % akkaVersion excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", @@ -290,7 +306,7 @@ object SparkBuild extends Build { "org.eclipse.paho" % "mqtt-client" % "0.4.0", "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty), + akkaGroup % "akka-zeromq" % akkaVersion excludeAll(excludeNetty), "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1" exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") diff --git a/streaming/pom.xml b/streaming/pom.xml index 7a9ae6a97ba7ff615e3e24c76923dc941d76b8f3..40892937b8ca5f031a6844938739bc75324f29ff 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -110,15 +110,8 @@ <artifactId>scala-library</artifactId> </dependency> <dependency> - <groupId>com.typesafe.akka</groupId> + <groupId>${akka.group}</groupId> <artifactId>akka-zeromq</artifactId> - <version>2.0.3</version> - <exclusions> - <exclusion> - <groupId>org.jboss.netty</groupId> - <artifactId>netty</artifactId> - </exclusion> - </exclusions> </dependency> <dependency> <groupId>org.scalatest</groupId> diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a7baf0c36cfd4db3ab2831c073b0af0f830a834e..240ed8b32ae93b7099b7d83adb2fcbf600a444b8 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -22,9 +22,12 @@ 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.security.UserGroupInformation import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ @@ -32,38 +35,38 @@ 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.{SparkContext, Logging} import org.apache.spark.util.Utils -import scala.collection.JavaConversions._ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { def this(args: ApplicationMasterArguments) = this(args, new Configuration()) private var rpc: YarnRPC = YarnRPC.create(conf) - private var resourceManager: AMRMProtocol = null - private var appAttemptId: ApplicationAttemptId = null - private var userThread: Thread = null + 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 = null - private var isFinished:Boolean = false - private var uiAddress: String = "" + private var yarnAllocator: YarnAllocationHandler = _ + private var isFinished: Boolean = false + private var uiAddress: 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 numWorkers * 2, with minimum of 3 + // default to numWorkers * 2, with minimum of 3 private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3).toString()).toInt def run() { - // setup the directories so things go to yarn approved directories rather - // then user specified and /tmp + // Setup the directories so things go to yarn approved directories rather + // then user specified and /tmp. System.setProperty("spark.local.dir", getLocalDirs()) - // use priority 30 as its higher then HDFS. Its same priority as MapReduce is using + // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) appAttemptId = getApplicationAttemptId() @@ -72,9 +75,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Workaround until hadoop moves to something which has // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line) - // ignore result + // ignore result. // This does not, unfortunately, always work reliably ... but alleviates the bug a lot of times - // Hence args.workerCores = numCore disabled above. Any better option ? + // Hence args.workerCores = numCore disabled above. Any better option? // Compute number of threads for akka //val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() @@ -100,7 +103,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e waitForSparkContextInitialized() - // do this after spark master is up and SparkContext is created so that we can register UI Url + // Do this after spark master is up and SparkContext is created so that we can register UI Url val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() // Allocate all containers @@ -119,12 +122,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) + .getOrElse("")) if (localDirs.isEmpty()) { throw new Exception("Yarn Local dirs can't be empty") } - return localDirs + localDirs } private def getApplicationAttemptId(): ApplicationAttemptId = { @@ -133,7 +136,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val containerId = ConverterUtils.toContainerId(containerIdString) val appAttemptId = containerId.getApplicationAttemptId() logInfo("ApplicationAttemptId: " + appAttemptId) - return appAttemptId + appAttemptId } private def registerWithResourceManager(): AMRMProtocol = { @@ -141,7 +144,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e YarnConfiguration.RM_SCHEDULER_ADDRESS, YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) logInfo("Connecting to ResourceManager at " + rmAddress) - return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] + rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] } private def registerApplicationMaster(): RegisterApplicationMasterResponse = { @@ -149,12 +152,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e 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. + // 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) - return resourceManager.registerApplicationMaster(appMasterRequest) + resourceManager.registerApplicationMaster(appMasterRequest) } private def waitForSparkMaster() { @@ -168,21 +172,25 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e try { val socket = new Socket(driverHost, driverPort.toInt) socket.close() - logInfo("Driver now available: " + driverHost + ":" + driverPort) + logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) driverUp = true } catch { - case e: Exception => - logWarning("Failed to connect to driver at " + driverHost + ":" + driverPort + ", retrying") - Thread.sleep(100) - tries = tries + 1 + case e: Exception => { + logWarning("Failed to connect to driver at %s:%s, retrying ...". + format(driverHost, driverPort)) + Thread.sleep(100) + tries = tries + 1 + } } } } private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") - val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader) - .getMethod("main", classOf[Array[String]]) + 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 @@ -207,7 +215,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } t.start() - return t + t } // this need to happen before allocateWorkers @@ -229,13 +237,20 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e if (null != sparkContext) { uiAddress = sparkContext.ui.appUIAddress - this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, - appAttemptId, args, sparkContext.preferredNodeLocationData) + this.yarnAllocator = YarnAllocationHandler.newAllocator( + yarnConf, + resourceManager, + appAttemptId, + args, + sparkContext.preferredNodeLocationData) } else { - logWarning("Unable to retrieve sparkContext inspite of waiting for " + count * waitTime + - ", numTries = " + numTries) - this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, - appAttemptId, args) + logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d". + format(count * waitTime, numTries)) + this.yarnAllocator = YarnAllocationHandler.newAllocator( + yarnConf, + resourceManager, + appAttemptId, + args) } } } finally { @@ -251,36 +266,39 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - while(yarnAllocator.getNumWorkersRunning < args.numWorkers && - // If user thread exists, then quit ! - userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") - } - yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) - ApplicationMaster.incrementAllocatorLoop(1) - Thread.sleep(100) + + // Exists the loop if the user thread exits. + while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) { + if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of worker failures reached") + } + yarnAllocator.allocateContainers( + math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + ApplicationMaster.incrementAllocatorLoop(1) + Thread.sleep(100) } } finally { - // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT : - // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks + // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT, + // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } logInfo("All workers have launched.") - // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout + // 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 elapse. - + // 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 = + val schedulerInterval = System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) + launchReporterThread(interval) } } @@ -292,12 +310,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e override def run() { while (userThread.isAlive) { if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, + finishApplicationMaster(FinalApplicationStatus.FAILED, "max number of worker failures reached") } val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning if (missingWorkerCount > 0) { - logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") + logInfo("Allocating %d containers to make up for (potentially) lost containers". + format(missingWorkerCount)) yarnAllocator.allocateContainers(missingWorkerCount) } else sendProgress() @@ -305,16 +324,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } } - // setting to daemon status, though this is usually not a good idea. + // 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) - return t + t } private def sendProgress() { logDebug("Sending progress") - // simulated with an allocate request with no nodes requested ... + // Simulated with an allocate request with no nodes requested ... yarnAllocator.allocateContainers(0) } @@ -334,7 +353,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e */ def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") { - synchronized { if (isFinished) { return @@ -348,14 +366,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e finishReq.setAppAttemptId(appAttemptId) finishReq.setFinishApplicationStatus(status) finishReq.setDiagnostics(diagnostics) - // set tracking url to empty since we don't have a history server + // Set tracking url to empty since we don't have a history server. finishReq.setTrackingUrl("") resourceManager.finishApplicationMaster(finishReq) - } /** - * clean up the staging directory. + * Clean up the staging directory. */ private def cleanupStagingDir() { var stagingDirPath: Path = null @@ -371,13 +388,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e fs.delete(stagingDirPath, true) } } catch { - case e: IOException => - logError("Failed to cleanup staging dir " + stagingDirPath, e) + 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. + // 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() { @@ -387,15 +403,14 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir() } } - } object ApplicationMaster { - // number of times to wait for the allocator loop to complete. - // each loop iteration waits for 100ms, so maximum of 3 seconds. + // Number of times to wait for the allocator loop to complete. + // Each loop iteration waits for 100ms, so maximum of 3 seconds. // This is to ensure that we have reasonable number of containers before we start - // 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. + // 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 ALLOCATOR_LOOP_WAIT_COUNT = 30 def incrementAllocatorLoop(by: Int) { val count = yarnAllocatorLoop.getAndAdd(by) @@ -413,7 +428,8 @@ object ApplicationMaster { applicationMasters.add(master) } - val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) + val sparkContextRef: AtomicReference[SparkContext] = + new AtomicReference[SparkContext](null /* initialValue */) val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0) def sparkContextInitialized(sc: SparkContext): Boolean = { @@ -423,19 +439,21 @@ object ApplicationMaster { 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 Client declaring failure even though we exit'ed properly. - // Note that this will unfortunately not properly clean up the staging files because it gets called to - // late and the filesystem is already shutdown. + // 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 just to log, but also to ensure that log system is initialized for this instance when we actually are 'run' + // 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 ... + // Best case ... for (master <- applicationMasters) { master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } @@ -443,7 +461,7 @@ object ApplicationMaster { } ) } - // Wait for initialization to complete and atleast 'some' nodes can get allocated + // Wait for initialization to complete and atleast 'some' nodes can get allocated. yarnAllocatorLoop.synchronized { while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) { yarnAllocatorLoop.wait(1000L) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index bb73f6d337ba04f47bd407c24b6311f0a50e10b6..79dd03806523bcea066fcafd05160fe732032d22 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -20,41 +20,46 @@ package org.apache.spark.deploy.yarn import java.net.{InetAddress, UnknownHostException, URI} import java.nio.ByteBuffer +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap +import scala.collection.mutable.Map + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil} -import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.mapred.Master +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.mapred.Master +import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.YarnClientImpl import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map -import scala.collection.JavaConversions._ +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.Logging class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { - + def this(args: ClientArguments) = this(new Configuration(), args) - + var rpc: YarnRPC = YarnRPC.create(conf) val yarnConf: YarnConfiguration = new YarnConfiguration(conf) val credentials = UserGroupInformation.getCurrentUser().getCredentials() private val SPARK_STAGING: String = ".sparkStaging" private val distCacheMgr = new ClientDistributedCacheManager() - // staging directory is private! -> rwx-------- + // Staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short) - // app files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) + + // App files are world-wide readable and owner writable -> rw-r--r-- + val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) // for client user who want to monitor app status by itself. def runApp() = { @@ -89,15 +94,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } def validateArgs() = { - Map((System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", + Map( + (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", (args.userJar == null) -> "Error: You must specify a user jar!", (args.userClass == null) -> "Error: You must specify a user class!", (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!", - (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> - ("Error: AM memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD), - (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> - ("Error: Worker memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString())) - .foreach { case(cond, errStr) => + (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be " + + "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), + (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " + + "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD) + ).foreach { case(cond, errStr) => if (cond) { logError(errStr) args.printUsageAndExit(1) @@ -111,19 +117,24 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl def logClusterResourceDetails() { val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics - logInfo("Got Cluster metric info from ASM, numNodeManagers=" + clusterMetrics.getNumNodeManagers) + logInfo("Got Cluster metric info from ASM, numNodeManagers = " + + clusterMetrics.getNumNodeManagers) val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) - logInfo("Queue info .. queueName=" + queueInfo.getQueueName + ", queueCurrentCapacity=" + queueInfo.getCurrentCapacity + - ", queueMaxCapacity=" + queueInfo.getMaximumCapacity + ", queueApplicationCount=" + queueInfo.getApplications.size + - ", queueChildQueueCount=" + queueInfo.getChildQueues.size) + logInfo("""Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s, + queueApplicationCount = %s, queueChildQueueCount = %s""".format( + queueInfo.getQueueName, + queueInfo.getCurrentCapacity, + queueInfo.getMaximumCapacity, + queueInfo.getApplications.size, + queueInfo.getChildQueues.size)) } - + def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) - - // if we have requested more then the clusters max for a single resource then exit. + + // If we have requested more then the clusters max for a single resource then exit. if (args.workerMemory > maxMem) { logError("the worker size is to large to run on this cluster " + args.workerMemory) System.exit(1) @@ -134,10 +145,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl System.exit(1) } - // We could add checks to make sure the entire cluster has enough resources but that involves getting - // all the node reports and computing ourselves + // We could add checks to make sure the entire cluster has enough resources but that involves + // getting all the node reports and computing ourselves } - + def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { logInfo("Setting up application submission context for ASM") val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) @@ -146,9 +157,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl return appContext } - /* - * see if two file systems are the same or not. - */ + /** See if two file systems are the same or not. */ private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { val srcUri = srcFs.getUri() val dstUri = destFs.getUri() @@ -183,9 +192,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl return true } - /** - * Copy the file into HDFS if needed. - */ + /** Copy the file into HDFS if needed. */ private def copyRemoteFile( dstDir: Path, originalPath: Path, @@ -201,9 +208,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl fs.setReplication(newPath, replication) if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) } - // resolve any symlinks in the URI path so using a "current" symlink - // to point to a specific version shows the specific version - // in the distributed cache configuration + // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific + // version shows the specific version in the distributed cache configuration val qualPath = fs.makeQualified(newPath) val fc = FileContext.getFileContext(qualPath.toUri(), conf) val destPath = fc.resolvePath(qualPath) @@ -212,8 +218,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { logInfo("Preparing Local resources") - // Upload Spark and the application JAR to the remote file system if necessary - // Add them as local resources to the AM + // Upload Spark and the application JAR to the remote file system if necessary. Add them as + // local resources to the AM. val fs = FileSystem.get(conf) val delegTokenRenewer = Master.getMasterPrincipal(conf) @@ -243,7 +249,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl var localURI = new URI(localPath) // if not specified assume these are in the local filesystem to keep behavior like Hadoop if (localURI.getScheme() == null) { - localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString()) + localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString) } val setPermissions = if (destName.equals(Client.APP_JAR)) true else false val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions) @@ -291,7 +297,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl UserGroupInformation.getCurrentUser().addCredentials(credentials) return localResources } - + def setupLaunchEnv( localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { @@ -304,16 +310,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir - // set the environment variables to be passed on to the Workers + // Set the environment variables to be passed on to the Workers. distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) - // allow users to specify some environment variables + // Allow users to specify some environment variables. Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) - // Add each SPARK-* key to the environment + // Add each SPARK-* key to the environment. System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } - return env + env } def userArgsToString(clientArgs: ClientArguments): String = { @@ -323,13 +329,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl for (arg <- args){ retval.append(prefix).append(" '").append(arg).append("' ") } - retval.toString } - def createContainerLaunchContext(newApp: GetNewApplicationResponse, - localResources: HashMap[String, LocalResource], - env: HashMap[String, String]): ContainerLaunchContext = { + def createContainerLaunchContext( + newApp: GetNewApplicationResponse, + localResources: HashMap[String, LocalResource], + env: HashMap[String, String]): ContainerLaunchContext = { logInfo("Setting up container launch context") val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) amContainer.setLocalResources(localResources) @@ -337,8 +343,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() + // TODO(harvey): This can probably be a val. var amMemory = ((args.amMemory / minResMemory) * minResMemory) + - (if (0 != (args.amMemory % minResMemory)) minResMemory else 0) - YarnAllocationHandler.MEMORY_OVERHEAD + ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - + YarnAllocationHandler.MEMORY_OVERHEAD) // Extra options for the JVM var JAVA_OPTS = "" @@ -349,13 +357,18 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " - // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out. - // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same - // node, spark gc effects all other containers performance (which can also be other spark containers) - // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is - // limited to subset of cores on a node. - if (env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))) { - // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tenant machines + // Commenting it out for now - so that people can refer to the properties if required. Remove + // it once cpuset version is pushed out. The context is, default gc for server class machines + // end up using all cores to do gc - hence if there are multiple containers in same node, + // spark gc effects all other containers performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in + // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset + // of cores on a node. + val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && + java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC")) + if (useConcurrentAndIncrementalGC) { + // In our expts, using (default) throughput collector has severe perf ramnifications in + // multi-tenant machines JAVA_OPTS += " -XX:+UseConcMarkSweepGC " JAVA_OPTS += " -XX:+CMSIncrementalMode " JAVA_OPTS += " -XX:+CMSIncrementalPacing " @@ -388,28 +401,28 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") logInfo("Command for the ApplicationMaster: " + commands(0)) amContainer.setCommands(commands) - + val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] - // Memory for the ApplicationMaster + // Memory for the ApplicationMaster. capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) amContainer.setResource(capability) - // Setup security tokens + // Setup security tokens. val dob = new DataOutputBuffer() credentials.writeTokenStorageToStream(dob) amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) - return amContainer + amContainer } - + def submitApp(appContext: ApplicationSubmissionContext) = { - // Submit the application to the applications manager + // Submit the application to the applications manager. logInfo("Submitting application to ASM") super.submitApplication(appContext) } - + def monitorApplication(appId: ApplicationId): Boolean = { - while(true) { + while (true) { Thread.sleep(1000) val report = super.getApplicationReport(appId) @@ -427,16 +440,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + "\t appUser: " + report.getUser() ) - + val state = report.getYarnApplicationState() val dsStatus = report.getFinalApplicationStatus() if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { - return true + return true } } - return true + true } } @@ -469,7 +482,7 @@ object Client { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + LOG4J_PROP) } - // normally the users app.jar is last in case conflicts with spark jars + // Normally the users app.jar is last in case conflicts with spark jars val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala index a4d6e1d87d1270fd48e8f0f6273f8cfd076e7e64..6a90cc51cfbaf71ef5683703dd84fa6d5b872ccb 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -21,52 +21,59 @@ import java.net.URI import java.nio.ByteBuffer import java.security.PrivilegedExceptionAction +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment 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.{Apps, ConverterUtils, Records, ProtoUtils} -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment - -import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap import org.apache.spark.Logging -class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String, - slaveId: String, hostname: String, workerMemory: Int, workerCores: Int) - extends Runnable with Logging { - + +class WorkerRunnable( + container: Container, + conf: Configuration, + masterAddress: String, + slaveId: String, + hostname: String, + workerMemory: Int, + workerCores: Int) + extends Runnable with Logging { + var rpc: YarnRPC = YarnRPC.create(conf) var cm: ContainerManager = null val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - + def run = { logInfo("Starting Worker Container") cm = connectToCM startContainer } - + def startContainer = { logInfo("Setting up ContainerLaunchContext") - + val ctx = Records.newRecord(classOf[ContainerLaunchContext]) .asInstanceOf[ContainerLaunchContext] - + ctx.setContainerId(container.getId()) ctx.setResource(container.getResource()) val localResources = prepareLocalResources ctx.setLocalResources(localResources) - + val env = prepareEnvironment ctx.setEnvironment(env) - + // Extra options for the JVM var JAVA_OPTS = "" // Set the JVM memory @@ -79,17 +86,21 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " - - // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out. - // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same - // node, spark gc effects all other containers performance (which can also be other spark containers) - // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is - // limited to subset of cores on a node. + // Commenting it out for now - so that people can refer to the properties if required. Remove + // it once cpuset version is pushed out. + // The context is, default gc for server class machines end up using all cores to do gc - hence + // if there are multiple containers in same node, spark gc effects all other containers + // performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in + // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset + // of cores on a node. /* else { // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont want to mess with it. - // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tennent machines + // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont + // want to mess with it. + // In our expts, using (default) throughput collector has severe perf ramnifications in + // multi-tennent machines // The options are based on // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline JAVA_OPTS += " -XX:+UseConcMarkSweepGC " @@ -116,8 +127,10 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S val commands = List[String](javaCommand + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. - // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state. - // TODO: If the OOM is not recoverable by rescheduling it on different node, then do 'something' to fail job ... akin to blacklisting trackers in mapred ? + // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in + // an inconsistent state. + // TODO: If the OOM is not recoverable by rescheduling it on different node, then do + // 'something' to fail job ... akin to blacklisting trackers in mapred ? " -XX:OnOutOfMemoryError='kill %p' " + JAVA_OPTS + " org.apache.spark.executor.CoarseGrainedExecutorBackend " + @@ -129,7 +142,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") logInfo("Setting up worker with commands: " + commands) ctx.setCommands(commands) - + // Send the start request to the ContainerManager val startReq = Records.newRecord(classOf[StartContainerRequest]) .asInstanceOf[StartContainerRequest] @@ -137,7 +150,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S cm.startContainer(startReq) } - private def setupDistributedCache(file: String, + private def setupDistributedCache( + file: String, rtype: LocalResourceType, localResources: HashMap[String, LocalResource], timestamp: String, @@ -152,12 +166,11 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S amJarRsrc.setSize(size.toLong) localResources(uri.getFragment()) = amJarRsrc } - - + def prepareLocalResources: HashMap[String, LocalResource] = { logInfo("Preparing Local resources") val localResources = HashMap[String, LocalResource]() - + if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') @@ -179,30 +192,30 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S timeStamps(i), fileSizes(i), visibilities(i)) } } - + logInfo("Prepared Local resources " + localResources) return localResources } - + def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env) - // allow users to specify some environment variables + // Allow users to specify some environment variables Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } return env } - + def connectToCM: ContainerManager = { val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort() val cmAddress = NetUtils.createSocketAddr(cmHostPortStr) logInfo("Connecting to ContainerManager at " + cmHostPortStr) - // use doAs and remoteUser here so we can add the container token and not - // pollute the current users credentials with all of the individual container tokens + // Use doAs and remoteUser here so we can add the container token and not pollute the current + // users credentials with all of the individual container tokens val user = UserGroupInformation.createRemoteUser(container.getId().toString()) val containerToken = container.getContainerToken() if (containerToken != null) { @@ -218,5 +231,5 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S }) proxy } - + } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 507a0743fd77abf8a7feed9aa953f1d52b4e9b35..f15f3c7c1195c61cf8ad31987e8879d0885c8233 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -17,55 +17,70 @@ 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} + import org.apache.spark.Logging -import org.apache.spark.util.Utils import org.apache.spark.scheduler.SplitInfo -import scala.collection -import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId, ContainerId, Priority, Resource, ResourceRequest, ContainerStatus, Container} import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend} +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.{Priority, Resource, ResourceRequest} import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} import org.apache.hadoop.yarn.util.{RackResolver, Records} -import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} -import java.util.concurrent.atomic.AtomicInteger -import org.apache.hadoop.yarn.api.AMRMProtocol -import collection.JavaConversions._ -import collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.hadoop.conf.Configuration -import java.util.{Collections, Set => JSet} -import java.lang.{Boolean => JBoolean} + object AllocationType extends Enumeration ("HOST", "RACK", "ANY") { type AllocationType = Value val HOST, RACK, ANY = Value } -// too many params ? refactor it 'somehow' ? -// needs to be mt-safe -// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive : should make it -// more proactive and decoupled. +// TODO: +// Too many params. +// Needs to be mt-safe +// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should +// make it more proactive and decoupled. + // Note that right now, we assume all node asks as uniform in terms of capabilities and priority -// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for more info -// on how we are requesting for containers. -private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceManager: AMRMProtocol, - val appAttemptId: ApplicationAttemptId, - val maxWorkers: Int, val workerMemory: Int, val workerCores: Int, - val preferredHostToCount: Map[String, Int], - val preferredRackToCount: Map[String, Int]) +// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for +// more info on how we are requesting for containers. +private[yarn] class YarnAllocationHandler( + val conf: Configuration, + val resourceManager: AMRMProtocol, + val appAttemptId: ApplicationAttemptId, + val maxWorkers: Int, + val workerMemory: Int, + val workerCores: Int, + val preferredHostToCount: Map[String, Int], + val preferredRackToCount: Map[String, Int]) extends Logging { - - // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set<containerid> - // allocatedContainerToHostMap: container to host mapping - private val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]() + // allocatedContainerToHostMap: container to host mapping. + private val allocatedHostToContainersMap = + new HashMap[String, collection.mutable.Set[ContainerId]]() + private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() - // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an allocated node) - // As with the two data structures above, tightly coupled with them, and to be locked on allocatedHostToContainersMap + + // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an + // allocated node) + // As with the two data structures above, tightly coupled with them, and to be locked on + // allocatedHostToContainersMap private val allocatedRackCount = new HashMap[String, Int]() - // containers which have been released. + // Containers which have been released. private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]() - // containers to be released in next request to RM + // Containers to be released in next request to RM private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] private val numWorkersRunning = new AtomicInteger() @@ -83,23 +98,31 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM } def allocateContainers(workersToRequest: Int) { - // We need to send the request only once from what I understand ... but for now, not modifying this much. + // We need to send the request only once from what I understand ... but for now, not modifying + // this much. // Keep polling the Resource Manager for containers val amResp = allocateWorkerResources(workersToRequest).getAMResponse val _allocatedContainers = amResp.getAllocatedContainers() - if (_allocatedContainers.size > 0) { - - logDebug("Allocated " + _allocatedContainers.size + " containers, current count " + - numWorkersRunning.get() + ", to-be-released " + releasedContainerList + - ", pendingReleaseContainers : " + pendingReleaseContainers) - logDebug("Cluster Resources: " + amResp.getAvailableResources) + if (_allocatedContainers.size > 0) { + logDebug(""" + Allocated containers: %d + Current worker count: %d + Containers released: %s + Containers to be released: %s + Cluster resources: %s + """.format( + _allocatedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers, + amResp.getAvailableResources)) val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() - // ignore if not satisfying constraints { + // Ignore if not satisfying constraints { for (container <- _allocatedContainers) { if (isResourceConstraintSatisfied(container)) { // allocatedContainers += container @@ -113,8 +136,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM else releasedContainerList.add(container.getId()) } - // Find the appropriate containers to use - // Slightly non trivial groupBy I guess ... + // Find the appropriate containers to use. Slightly non trivial groupBy ... val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() @@ -134,21 +156,22 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM remainingContainers = null } else if (requiredHostCount > 0) { - // container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - requiredHostCount) - // and rest as remainingContainer - val (dataLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredHostCount) + // Container list has more containers than we need for data locality. + // Split into two : data local container count of (remainingContainers.size - + // requiredHostCount) and rest as remainingContainer + val (dataLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredHostCount) dataLocalContainers.put(candidateHost, dataLocal) // remainingContainers = remaining // yarn has nasty habit of allocating a tonne of containers on a host - discourage this : - // add remaining to release list. If we have insufficient containers, next allocation cycle - // will reallocate (but wont treat it as data local) + // add remaining to release list. If we have insufficient containers, next allocation + // cycle will reallocate (but wont treat it as data local) for (container <- remaining) releasedContainerList.add(container.getId()) remainingContainers = null } - // now rack local + // Now rack local if (remainingContainers != null){ val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) @@ -161,15 +184,17 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM if (requiredRackCount >= remainingContainers.size){ // Add all to dataLocalContainers dataLocalContainers.put(rack, remainingContainers) - // all consumed + // All consumed remainingContainers = null } else if (requiredRackCount > 0) { // container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - requiredRackCount) - // and rest as remainingContainer - val (rackLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredRackCount) - val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, new ArrayBuffer[Container]()) + // Split into two : data local container count of (remainingContainers.size - + // requiredRackCount) and rest as remainingContainer + val (rackLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredRackCount) + val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, + new ArrayBuffer[Container]()) existingRackLocal ++= rackLocal remainingContainers = remaining @@ -185,8 +210,8 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM // Now that we have split the containers into various groups, go through them in order : // first host local, then rack local and then off rack (everything else). - // Note that the list we create below tries to ensure that not all containers end up within a host - // if there are sufficiently large number of hosts/containers. + // Note that the list we create below tries to ensure that not all containers end up within a + // host if there are sufficiently large number of hosts/containers. val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size) allocatedContainers ++= ClusterScheduler.prioritizeContainers(dataLocalContainers) @@ -199,33 +224,39 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM val workerHostname = container.getNodeId.getHost val containerId = container.getId - assert (container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) + assert( + container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) if (numWorkersRunningNow > maxWorkers) { - logInfo("Ignoring container " + containerId + " at host " + workerHostname + - " .. we already have required number of containers") + logInfo("""Ignoring container %s at host %s, since we already have the required number of + containers for it.""".format(containerId, workerHostname)) releasedContainerList.add(containerId) // reset counter back to old value. numWorkersRunning.decrementAndGet() } else { - // deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter) + // Deallocate + allocate can result in reusing id's wrongly - so use a different counter + // (workerIdCounter) val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + workerHostname) - // just to be safe, simply remove it from pendingReleaseContainers. Should not be there, but .. + // Just to be safe, simply remove it from pendingReleaseContainers. + // Should not be there, but .. pendingReleaseContainers.remove(containerId) val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, new HashSet[ContainerId]()) + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, + new HashSet[ContainerId]()) containerSet += containerId allocatedContainerToHostMap.put(containerId, workerHostname) - if (rack != null) allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) + if (rack != null) { + allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) + } } new Thread( @@ -234,17 +265,23 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM ).start() } } - logDebug("After allocated " + allocatedContainers.size + " containers (orig : " + - _allocatedContainers.size + "), current count " + numWorkersRunning.get() + - ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers) + logDebug(""" + Finished processing %d containers. + Current number of workers running: %d, + releasedContainerList: %s, + pendingReleaseContainers: %s + """.format( + allocatedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers)) } val completedContainers = amResp.getCompletedContainersStatuses() if (completedContainers.size > 0){ - logDebug("Completed " + completedContainers.size + " containers, current count " + numWorkersRunning.get() + - ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers) - + logDebug("Completed %d containers, to-be-released: %s".format( + completedContainers.size, releasedContainerList)) for (completedContainer <- completedContainers){ val containerId = completedContainer.getContainerId @@ -253,16 +290,17 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM pendingReleaseContainers.remove(containerId) } else { - // simply decrement count - next iteration of ReporterThread will take care of allocating ! + // Simply decrement count - next iteration of ReporterThread will take care of allocating. numWorkersRunning.decrementAndGet() - logInfo("Container completed not by us ? nodeId: " + containerId + ", state " + completedContainer.getState + - " httpaddress: " + completedContainer.getDiagnostics + " exit status: " + completedContainer.getExitStatus()) - + logInfo("Completed container %s (state: %s, exit status: %s)".format( + containerId, + completedContainer.getState, + completedContainer.getExitStatus())) // Hadoop 2.2.X added a ContainerExitStatus we should switch to use // there are some exit status' we shouldn't necessarily count against us, but for // now I think its ok as none of the containers are expected to exit if (completedContainer.getExitStatus() != 0) { - logInfo("Container marked as failed: " + containerId) + logInfo("Container marked as failed: " + containerId) numWorkersFailed.incrementAndGet() } } @@ -281,7 +319,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM allocatedContainerToHostMap -= containerId - // doing this within locked context, sigh ... move to outside ? + // Doing this within locked context, sigh ... move to outside ? val rack = YarnAllocationHandler.lookupRack(conf, host) if (rack != null) { val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 @@ -291,9 +329,16 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM } } } - logDebug("After completed " + completedContainers.size + " containers, current count " + - numWorkersRunning.get() + ", to-be-released " + releasedContainerList + - ", pendingReleaseContainers : " + pendingReleaseContainers) + logDebug(""" + Finished processing %d completed containers. + Current number of workers running: %d, + releasedContainerList: %s, + pendingReleaseContainers: %s + """.format( + completedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers)) } } @@ -347,7 +392,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM // default. if (numWorkers <= 0 || preferredHostToCount.isEmpty) { - logDebug("numWorkers: " + numWorkers + ", host preferences ? " + preferredHostToCount.isEmpty) + logDebug("numWorkers: " + numWorkers + ", host preferences: " + preferredHostToCount.isEmpty) resourceRequests = List( createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)) } @@ -360,17 +405,24 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) if (requiredCount > 0) { - hostContainerRequests += - createResourceRequest(AllocationType.HOST, candidateHost, requiredCount, YarnAllocationHandler.PRIORITY) + hostContainerRequests += createResourceRequest( + AllocationType.HOST, + candidateHost, + requiredCount, + YarnAllocationHandler.PRIORITY) } } - val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(hostContainerRequests.toList) + val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests( + hostContainerRequests.toList) - val anyContainerRequests: ResourceRequest = - createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY) + val anyContainerRequests: ResourceRequest = createResourceRequest( + AllocationType.ANY, + resource = null, + numWorkers, + YarnAllocationHandler.PRIORITY) - val containerRequests: ArrayBuffer[ResourceRequest] = - new ArrayBuffer[ResourceRequest](hostContainerRequests.size() + rackContainerRequests.size() + 1) + val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( + hostContainerRequests.size + rackContainerRequests.size + 1) containerRequests ++= hostContainerRequests containerRequests ++= rackContainerRequests @@ -389,52 +441,59 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM req.addAllReleases(releasedContainerList) if (numWorkers > 0) { - logInfo("Allocating " + numWorkers + " worker containers with " + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + " of memory each.") + logInfo("Allocating %d worker containers with %d of memory each.".format(numWorkers, + workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) } else { logDebug("Empty allocation req .. release : " + releasedContainerList) } - for (req <- resourceRequests) { - logInfo("rsrcRequest ... host : " + req.getHostName + ", numContainers : " + req.getNumContainers + - ", p = " + req.getPriority().getPriority + ", capability: " + req.getCapability) + for (request <- resourceRequests) { + logInfo("ResourceRequest (host : %s, num containers: %d, priority = %s , capability : %s)". + format( + request.getHostName, + request.getNumContainers, + request.getPriority, + request.getCapability)) } resourceManager.allocate(req) } - private def createResourceRequest(requestType: AllocationType.AllocationType, - resource:String, numWorkers: Int, priority: Int): ResourceRequest = { + private def createResourceRequest( + requestType: AllocationType.AllocationType, + resource:String, + numWorkers: Int, + priority: Int): ResourceRequest = { // If hostname specified, we need atleast two requests - node local and rack local. // 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(YarnAllocationHandler.ANY_HOST != resource) val hostname = resource val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority) - // add to host->rack mapping + // Add to host->rack mapping YarnAllocationHandler.populateRackInfo(conf, hostname) nodeLocal } - case AllocationType.RACK => { val rack = resource createResourceRequestImpl(rack, numWorkers, priority) } - - case AllocationType.ANY => { - createResourceRequestImpl(YarnAllocationHandler.ANY_HOST, numWorkers, priority) - } - - case _ => throw new IllegalArgumentException("Unexpected/unsupported request type .. " + requestType) + case AllocationType.ANY => createResourceRequestImpl( + YarnAllocationHandler.ANY_HOST, numWorkers, priority) + case _ => throw new IllegalArgumentException( + "Unexpected/unsupported request type: " + requestType) } } - private def createResourceRequestImpl(hostname:String, numWorkers: Int, priority: Int): ResourceRequest = { + private def createResourceRequestImpl( + hostname:String, + numWorkers: Int, + priority: Int): ResourceRequest = { val rsrcRequest = Records.newRecord(classOf[ResourceRequest]) val memCapability = Records.newRecord(classOf[Resource]) @@ -455,11 +514,11 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM def createReleasedContainerList(): ArrayBuffer[ContainerId] = { val retval = new ArrayBuffer[ContainerId](1) - // iterator on COW list ... + // Iterator on COW list ... for (container <- releasedContainerList.iterator()){ retval += container } - // remove from the original list. + // Remove from the original list. if (! retval.isEmpty) { releasedContainerList.removeAll(retval) for (v <- retval) pendingReleaseContainers.put(v, true) @@ -474,14 +533,14 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM object YarnAllocationHandler { val ANY_HOST = "*" - // all requests are issued with same priority : we do not (yet) have any distinction between + // 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 + // 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. @@ -489,38 +548,69 @@ object YarnAllocationHandler { private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() - def newAllocator(conf: Configuration, - resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments): YarnAllocationHandler = { - - new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers, - args.workerMemory, args.workerCores, Map[String, Int](), Map[String, Int]()) + def newAllocator( + conf: Configuration, + resourceManager: AMRMProtocol, + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments): YarnAllocationHandler = { + + new YarnAllocationHandler( + conf, + resourceManager, + appAttemptId, + args.numWorkers, + args.workerMemory, + args.workerCores, + Map[String, Int](), + Map[String, Int]()) } - def newAllocator(conf: Configuration, - resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { + def newAllocator( + conf: Configuration, + resourceManager: AMRMProtocol, + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments, + map: collection.Map[String, + collection.Set[SplitInfo]]): YarnAllocationHandler = { val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) - - new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers, - args.workerMemory, args.workerCores, hostToCount, rackToCount) + new YarnAllocationHandler( + conf, + resourceManager, + appAttemptId, + args.numWorkers, + args.workerMemory, + args.workerCores, + hostToCount, + rackToCount) } - def newAllocator(conf: Configuration, - resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, - maxWorkers: Int, workerMemory: Int, workerCores: Int, - map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { + def newAllocator( + conf: Configuration, + resourceManager: AMRMProtocol, + appAttemptId: ApplicationAttemptId, + maxWorkers: Int, + workerMemory: Int, + workerCores: Int, + map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) - new YarnAllocationHandler(conf, resourceManager, appAttemptId, maxWorkers, - workerMemory, workerCores, hostToCount, rackToCount) + new YarnAllocationHandler( + conf, + resourceManager, + appAttemptId, + maxWorkers, + workerMemory, + workerCores, + hostToCount, + rackToCount) } // A simple method to copy the split info map. - private def generateNodeToWeight(conf: Configuration, input: collection.Map[String, collection.Set[SplitInfo]]) : + private def generateNodeToWeight( + conf: Configuration, + input: collection.Map[String, collection.Set[SplitInfo]]) : // host to count, rack to count (Map[String, Int], Map[String, Int]) = { @@ -544,7 +634,7 @@ object YarnAllocationHandler { } def lookupRack(conf: Configuration, host: String): String = { - if (! hostToRack.contains(host)) populateRackInfo(conf, host) + if (!hostToRack.contains(host)) populateRackInfo(conf, host) hostToRack.get(host) } @@ -567,10 +657,12 @@ object YarnAllocationHandler { val rack = rackInfo.getNetworkLocation hostToRack.put(hostname, rack) if (! rackToHostSet.containsKey(rack)) { - rackToHostSet.putIfAbsent(rack, Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]())) + 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 ...