aboutsummaryrefslogtreecommitdiff
path: root/resource-managers/yarn
diff options
context:
space:
mode:
Diffstat (limited to 'resource-managers/yarn')
-rw-r--r--resource-managers/yarn/pom.xml215
-rw-r--r--resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider3
-rw-r--r--resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager1
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala791
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala105
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala1541
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala86
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala186
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala266
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala224
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala727
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala135
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala317
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala347
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala235
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala105
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala130
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala74
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala110
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala129
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala57
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala53
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala143
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala157
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala56
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala37
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala67
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala39
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala315
-rw-r--r--resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider1
-rw-r--r--resource-managers/yarn/src/test/resources/log4j.properties31
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala241
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala204
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala462
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala153
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala344
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala493
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala112
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala213
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala150
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala71
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala36
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala70
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala372
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala37
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala72
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala34
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala48
-rw-r--r--resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala42
49 files changed, 9837 insertions, 0 deletions
diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml
new file mode 100644
index 0000000000..04b51dc92a
--- /dev/null
+++ b/resource-managers/yarn/pom.xml
@@ -0,0 +1,215 @@
+<?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_2.11</artifactId>
+ <version>2.2.0-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>spark-yarn_2.11</artifactId>
+ <packaging>jar</packaging>
+ <name>Spark Project YARN</name>
+ <properties>
+ <sbt.project.name>yarn</sbt.project.name>
+ <jersey-1.version>1.9</jersey-1.version>
+ </properties>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-network-yarn_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-tags_${scala.binary.version}</artifactId>
+ </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-server-web-proxy</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-client</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-client</artifactId>
+ </dependency>
+
+ <!-- Explicit listing of transitive deps that are shaded. Otherwise, odd compiler crashes. -->
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-server</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-plus</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-util</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-http</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-servlet</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty</groupId>
+ <artifactId>jetty-servlets</artifactId>
+ </dependency>
+ <!-- End of shaded deps. -->
+
+ <!--
+ SPARK-10059: Explicitly add JSP dependencies for tests since the MiniYARN cluster needs them.
+ -->
+ <dependency>
+ <groupId>org.eclipse.jetty.orbit</groupId>
+ <artifactId>javax.servlet.jsp</artifactId>
+ <version>2.2.0.v201112011158</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.jetty.orbit</groupId>
+ <artifactId>javax.servlet.jsp.jstl</artifactId>
+ <version>1.2.0.v201105211821</version>
+ <scope>test</scope>
+ </dependency>
+
+ <!--
+ See SPARK-3710. hadoop-yarn-server-tests in Hadoop 2.2 fails to pull some needed
+ dependencies, so they need to be added manually for the tests to work.
+ -->
+
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-server-tests</artifactId>
+ <classifier>tests</classifier>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-core</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mortbay.jetty</groupId>
+ <artifactId>jetty</artifactId>
+ <version>6.1.26</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.mortbay.jetty</groupId>
+ <artifactId>servlet-api</artifactId>
+ </exclusion>
+ </exclusions>
+ <scope>test</scope>
+ </dependency>
+
+ <!--
+ Jersey 1 dependencies only required for YARN integration testing. Creating a YARN cluster
+ in the JVM requires starting a Jersey 1-based web application.
+ -->
+ <dependency>
+ <groupId>com.sun.jersey</groupId>
+ <artifactId>jersey-core</artifactId>
+ <scope>test</scope>
+ <version>${jersey-1.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.sun.jersey</groupId>
+ <artifactId>jersey-json</artifactId>
+ <scope>test</scope>
+ <version>${jersey-1.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.sun.jersey</groupId>
+ <artifactId>jersey-server</artifactId>
+ <scope>test</scope>
+ <version>${jersey-1.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.sun.jersey.contribs</groupId>
+ <artifactId>jersey-guice</artifactId>
+ <scope>test</scope>
+ <version>${jersey-1.version}</version>
+ </dependency>
+
+ <!--
+ Testing Hive reflection needs hive on the test classpath only.
+ It doesn't need the spark hive modules, so the -Phive flag is not checked.
+ -->
+ <dependency>
+ <groupId>${hive.group}</groupId>
+ <artifactId>hive-exec</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>${hive.group}</groupId>
+ <artifactId>hive-metastore</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.thrift</groupId>
+ <artifactId>libthrift</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.thrift</groupId>
+ <artifactId>libfb303</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+ </build>
+
+</project>
diff --git a/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
new file mode 100644
index 0000000000..22ead56d23
--- /dev/null
+++ b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
@@ -0,0 +1,3 @@
+org.apache.spark.deploy.yarn.security.HDFSCredentialProvider
+org.apache.spark.deploy.yarn.security.HBaseCredentialProvider
+org.apache.spark.deploy.yarn.security.HiveCredentialProvider
diff --git a/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
new file mode 100644
index 0000000000..6e8a1ebfc6
--- /dev/null
+++ b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager
@@ -0,0 +1 @@
+org.apache.spark.scheduler.cluster.YarnClusterManager
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
new file mode 100644
index 0000000000..0378ef4fac
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -0,0 +1,791 @@
+/*
+ * 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.{File, IOException}
+import java.lang.reflect.InvocationTargetException
+import java.net.{Socket, URI, URL}
+import java.util.concurrent.{TimeoutException, TimeUnit}
+
+import scala.collection.mutable.HashMap
+import scala.concurrent.Promise
+import scala.concurrent.duration.Duration
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+
+import org.apache.spark._
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.history.HistoryServer
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.deploy.yarn.security.{AMCredentialRenewer, ConfigurableCredentialManager}
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc._
+import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend}
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
+import org.apache.spark.util._
+
+/**
+ * Common application master functionality for Spark on Yarn.
+ */
+private[spark] class ApplicationMaster(
+ args: ApplicationMasterArguments,
+ client: YarnRMClient)
+ extends Logging {
+
+ // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
+ // optimal as more containers are available. Might need to handle this better.
+
+ private val sparkConf = new SparkConf()
+ private val yarnConf: YarnConfiguration = SparkHadoopUtil.get.newConfiguration(sparkConf)
+ .asInstanceOf[YarnConfiguration]
+ private val isClusterMode = args.userClass != null
+
+ // Default to twice the number of executors (twice the maximum number of executors if dynamic
+ // allocation is enabled), with a minimum of 3.
+
+ private val maxNumExecutorFailures = {
+ val effectiveNumExecutors =
+ if (Utils.isDynamicAllocationEnabled(sparkConf)) {
+ sparkConf.get(DYN_ALLOCATION_MAX_EXECUTORS)
+ } else {
+ sparkConf.get(EXECUTOR_INSTANCES).getOrElse(0)
+ }
+ // By default, effectiveNumExecutors is Int.MaxValue if dynamic allocation is enabled. We need
+ // avoid the integer overflow here.
+ val defaultMaxNumExecutorFailures = math.max(3,
+ if (effectiveNumExecutors > Int.MaxValue / 2) Int.MaxValue else (2 * effectiveNumExecutors))
+
+ sparkConf.get(MAX_EXECUTOR_FAILURES).getOrElse(defaultMaxNumExecutorFailures)
+ }
+
+ @volatile private var exitCode = 0
+ @volatile private var unregistered = false
+ @volatile private var finished = false
+ @volatile private var finalStatus = getDefaultFinalStatus
+ @volatile private var finalMsg: String = ""
+ @volatile private var userClassThread: Thread = _
+
+ @volatile private var reporterThread: Thread = _
+ @volatile private var allocator: YarnAllocator = _
+
+ // Lock for controlling the allocator (heartbeat) thread.
+ private val allocatorLock = new Object()
+
+ // Steady state heartbeat interval. We want to be reasonably responsive without causing too many
+ // requests to RM.
+ private val heartbeatInterval = {
+ // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
+ val expiryInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+ math.max(0, math.min(expiryInterval / 2, sparkConf.get(RM_HEARTBEAT_INTERVAL)))
+ }
+
+ // Initial wait interval before allocator poll, to allow for quicker ramp up when executors are
+ // being requested.
+ private val initialAllocationInterval = math.min(heartbeatInterval,
+ sparkConf.get(INITIAL_HEARTBEAT_INTERVAL))
+
+ // Next wait interval before allocator poll.
+ private var nextAllocationInterval = initialAllocationInterval
+
+ private var rpcEnv: RpcEnv = null
+ private var amEndpoint: RpcEndpointRef = _
+
+ // In cluster mode, used to tell the AM when the user's SparkContext has been initialized.
+ private val sparkContextPromise = Promise[SparkContext]()
+
+ private var credentialRenewer: AMCredentialRenewer = _
+
+ // Load the list of localized files set by the client. This is used when launching executors,
+ // and is loaded here so that these configs don't pollute the Web UI's environment page in
+ // cluster mode.
+ private val localResources = {
+ logInfo("Preparing Local resources")
+ val resources = HashMap[String, LocalResource]()
+
+ def setupDistributedCache(
+ file: String,
+ rtype: LocalResourceType,
+ timestamp: String,
+ size: String,
+ vis: String): Unit = {
+ val uri = new URI(file)
+ val amJarRsrc = Records.newRecord(classOf[LocalResource])
+ amJarRsrc.setType(rtype)
+ amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
+ amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
+ amJarRsrc.setTimestamp(timestamp.toLong)
+ amJarRsrc.setSize(size.toLong)
+
+ val fileName = Option(uri.getFragment()).getOrElse(new Path(uri).getName())
+ resources(fileName) = amJarRsrc
+ }
+
+ val distFiles = sparkConf.get(CACHED_FILES)
+ val fileSizes = sparkConf.get(CACHED_FILES_SIZES)
+ val timeStamps = sparkConf.get(CACHED_FILES_TIMESTAMPS)
+ val visibilities = sparkConf.get(CACHED_FILES_VISIBILITIES)
+ val resTypes = sparkConf.get(CACHED_FILES_TYPES)
+
+ for (i <- 0 to distFiles.size - 1) {
+ val resType = LocalResourceType.valueOf(resTypes(i))
+ setupDistributedCache(distFiles(i), resType, timeStamps(i).toString, fileSizes(i).toString,
+ visibilities(i))
+ }
+
+ // Distribute the conf archive to executors.
+ sparkConf.get(CACHED_CONF_ARCHIVE).foreach { path =>
+ val uri = new URI(path)
+ val fs = FileSystem.get(uri, yarnConf)
+ val status = fs.getFileStatus(new Path(uri))
+ // SPARK-16080: Make sure to use the correct name for the destination when distributing the
+ // conf archive to executors.
+ val destUri = new URI(uri.getScheme(), uri.getRawSchemeSpecificPart(),
+ Client.LOCALIZED_CONF_DIR)
+ setupDistributedCache(destUri.toString(), LocalResourceType.ARCHIVE,
+ status.getModificationTime().toString, status.getLen.toString,
+ LocalResourceVisibility.PRIVATE.name())
+ }
+
+ // Clean up the configuration so it doesn't show up in the Web UI (since it's really noisy).
+ CACHE_CONFIGS.foreach { e =>
+ sparkConf.remove(e)
+ sys.props.remove(e.key)
+ }
+
+ resources.toMap
+ }
+
+ def getAttemptId(): ApplicationAttemptId = {
+ client.getAttemptId()
+ }
+
+ final def run(): Int = {
+ try {
+ val appAttemptId = client.getAttemptId()
+
+ var attemptID: Option[String] = None
+
+ if (isClusterMode) {
+ // Set the web ui port to be ephemeral for yarn so we don't conflict with
+ // other spark processes running on the same box
+ System.setProperty("spark.ui.port", "0")
+
+ // Set the master and deploy mode property to match the requested mode.
+ System.setProperty("spark.master", "yarn")
+ System.setProperty("spark.submit.deployMode", "cluster")
+
+ // Set this internal configuration if it is running on cluster mode, this
+ // configuration will be checked in SparkContext to avoid misuse of yarn cluster mode.
+ System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString())
+
+ attemptID = Option(appAttemptId.getAttemptId.toString)
+ }
+
+ new CallerContext(
+ "APPMASTER", sparkConf.get(APP_CALLER_CONTEXT),
+ Option(appAttemptId.getApplicationId.toString), attemptID).setCurrentContext()
+
+ logInfo("ApplicationAttemptId: " + appAttemptId)
+
+ val fs = FileSystem.get(yarnConf)
+
+ // This shutdown hook should run *after* the SparkContext is shut down.
+ val priority = ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY - 1
+ ShutdownHookManager.addShutdownHook(priority) { () =>
+ val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf)
+ val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts
+
+ if (!finished) {
+ // The default state of ApplicationMaster is failed if it is invoked by shut down hook.
+ // This behavior is different compared to 1.x version.
+ // If user application is exited ahead of time by calling System.exit(N), here mark
+ // this application as failed with EXIT_EARLY. For a good shutdown, user shouldn't call
+ // System.exit(0) to terminate the application.
+ finish(finalStatus,
+ ApplicationMaster.EXIT_EARLY,
+ "Shutdown hook called before final status was reported.")
+ }
+
+ if (!unregistered) {
+ // we only want to unregister if we don't want the RM to retry
+ if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) {
+ unregister(finalStatus, finalMsg)
+ cleanupStagingDir(fs)
+ }
+ }
+ }
+
+ // Call this to force generation of secret so it gets populated into the
+ // Hadoop UGI. This has to happen before the startUserApplication which does a
+ // doAs in order for the credentials to be passed on to the executor containers.
+ val securityMgr = new SecurityManager(sparkConf)
+
+ // If the credentials file config is present, we must periodically renew tokens. So create
+ // a new AMDelegationTokenRenewer
+ if (sparkConf.contains(CREDENTIALS_FILE_PATH.key)) {
+ // If a principal and keytab have been set, use that to create new credentials for executors
+ // periodically
+ credentialRenewer =
+ new ConfigurableCredentialManager(sparkConf, yarnConf).credentialRenewer()
+ credentialRenewer.scheduleLoginFromKeytab()
+ }
+
+ if (isClusterMode) {
+ runDriver(securityMgr)
+ } else {
+ runExecutorLauncher(securityMgr)
+ }
+ } catch {
+ case e: Exception =>
+ // catch everything else if not specifically handled
+ logError("Uncaught exception: ", e)
+ finish(FinalApplicationStatus.FAILED,
+ ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION,
+ "Uncaught exception: " + e)
+ }
+ exitCode
+ }
+
+ /**
+ * Set the default final application status for client mode to UNDEFINED to handle
+ * if YARN HA restarts the application so that it properly retries. Set the final
+ * status to SUCCEEDED in cluster mode to handle if the user calls System.exit
+ * from the application code.
+ */
+ final def getDefaultFinalStatus(): FinalApplicationStatus = {
+ if (isClusterMode) {
+ FinalApplicationStatus.FAILED
+ } else {
+ FinalApplicationStatus.UNDEFINED
+ }
+ }
+
+ /**
+ * unregister is used to completely unregister the application from the ResourceManager.
+ * This means the ResourceManager will not retry the application attempt on your behalf if
+ * a failure occurred.
+ */
+ final def unregister(status: FinalApplicationStatus, diagnostics: String = null): Unit = {
+ synchronized {
+ if (!unregistered) {
+ logInfo(s"Unregistering ApplicationMaster with $status" +
+ Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse(""))
+ unregistered = true
+ client.unregister(status, Option(diagnostics).getOrElse(""))
+ }
+ }
+ }
+
+ final def finish(status: FinalApplicationStatus, code: Int, msg: String = null): Unit = {
+ synchronized {
+ if (!finished) {
+ val inShutdown = ShutdownHookManager.inShutdown()
+ logInfo(s"Final app status: $status, exitCode: $code" +
+ Option(msg).map(msg => s", (reason: $msg)").getOrElse(""))
+ exitCode = code
+ finalStatus = status
+ finalMsg = msg
+ finished = true
+ if (!inShutdown && Thread.currentThread() != reporterThread && reporterThread != null) {
+ logDebug("shutting down reporter thread")
+ reporterThread.interrupt()
+ }
+ if (!inShutdown && Thread.currentThread() != userClassThread && userClassThread != null) {
+ logDebug("shutting down user thread")
+ userClassThread.interrupt()
+ }
+ if (!inShutdown && credentialRenewer != null) {
+ credentialRenewer.stop()
+ credentialRenewer = null
+ }
+ }
+ }
+ }
+
+ private def sparkContextInitialized(sc: SparkContext) = {
+ sparkContextPromise.success(sc)
+ }
+
+ private def registerAM(
+ _sparkConf: SparkConf,
+ _rpcEnv: RpcEnv,
+ driverRef: RpcEndpointRef,
+ uiAddress: String,
+ securityMgr: SecurityManager) = {
+ val appId = client.getAttemptId().getApplicationId().toString()
+ val attemptId = client.getAttemptId().getAttemptId().toString()
+ val historyAddress =
+ _sparkConf.get(HISTORY_SERVER_ADDRESS)
+ .map { text => SparkHadoopUtil.get.substituteHadoopVariables(text, yarnConf) }
+ .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}/${attemptId}" }
+ .getOrElse("")
+
+ val driverUrl = RpcEndpointAddress(
+ _sparkConf.get("spark.driver.host"),
+ _sparkConf.get("spark.driver.port").toInt,
+ CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+ // Before we initialize the allocator, let's log the information about how executors will
+ // be run up front, to avoid printing this out for every single executor being launched.
+ // Use placeholders for information that changes such as executor IDs.
+ logInfo {
+ val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt
+ val executorCores = sparkConf.get(EXECUTOR_CORES)
+ val dummyRunner = new ExecutorRunnable(None, yarnConf, sparkConf, driverUrl, "<executorId>",
+ "<hostname>", executorMemory, executorCores, appId, securityMgr, localResources)
+ dummyRunner.launchContextDebugInfo()
+ }
+
+ allocator = client.register(driverUrl,
+ driverRef,
+ yarnConf,
+ _sparkConf,
+ uiAddress,
+ historyAddress,
+ securityMgr,
+ localResources)
+
+ allocator.allocateResources()
+ reporterThread = launchReporterThread()
+ }
+
+ /**
+ * Create an [[RpcEndpoint]] that communicates with the driver.
+ *
+ * In cluster mode, the AM and the driver belong to same process
+ * so the AMEndpoint need not monitor lifecycle of the driver.
+ *
+ * @return A reference to the driver's RPC endpoint.
+ */
+ private def runAMEndpoint(
+ host: String,
+ port: String,
+ isClusterMode: Boolean): RpcEndpointRef = {
+ val driverEndpoint = rpcEnv.setupEndpointRef(
+ RpcAddress(host, port.toInt),
+ YarnSchedulerBackend.ENDPOINT_NAME)
+ amEndpoint =
+ rpcEnv.setupEndpoint("YarnAM", new AMEndpoint(rpcEnv, driverEndpoint, isClusterMode))
+ driverEndpoint
+ }
+
+ private def runDriver(securityMgr: SecurityManager): Unit = {
+ addAmIpFilter()
+ userClassThread = startUserApplication()
+
+ // 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.
+ logInfo("Waiting for spark context initialization...")
+ val totalWaitTime = sparkConf.get(AM_MAX_WAIT_TIME)
+ try {
+ val sc = ThreadUtils.awaitResult(sparkContextPromise.future,
+ Duration(totalWaitTime, TimeUnit.MILLISECONDS))
+ if (sc != null) {
+ rpcEnv = sc.env.rpcEnv
+ val driverRef = runAMEndpoint(
+ sc.getConf.get("spark.driver.host"),
+ sc.getConf.get("spark.driver.port"),
+ isClusterMode = true)
+ registerAM(sc.getConf, rpcEnv, driverRef, sc.ui.map(_.webUrl).getOrElse(""),
+ securityMgr)
+ } else {
+ // Sanity check; should never happen in normal operation, since sc should only be null
+ // if the user app did not create a SparkContext.
+ if (!finished) {
+ throw new IllegalStateException("SparkContext is null but app is still running!")
+ }
+ }
+ userClassThread.join()
+ } catch {
+ case e: SparkException if e.getCause().isInstanceOf[TimeoutException] =>
+ logError(
+ s"SparkContext did not initialize after waiting for $totalWaitTime ms. " +
+ "Please check earlier log output for errors. Failing the application.")
+ finish(FinalApplicationStatus.FAILED,
+ ApplicationMaster.EXIT_SC_NOT_INITED,
+ "Timed out waiting for SparkContext.")
+ }
+ }
+
+ private def runExecutorLauncher(securityMgr: SecurityManager): Unit = {
+ val port = sparkConf.get(AM_PORT)
+ rpcEnv = RpcEnv.create("sparkYarnAM", Utils.localHostName, port, sparkConf, securityMgr,
+ clientMode = true)
+ val driverRef = waitForSparkDriver()
+ addAmIpFilter()
+ registerAM(sparkConf, rpcEnv, driverRef, sparkConf.get("spark.driver.appUIAddress", ""),
+ securityMgr)
+
+ // In client mode the actor will stop the reporter thread.
+ reporterThread.join()
+ }
+
+ private def launchReporterThread(): Thread = {
+ // The number of failures in a row until Reporter thread give up
+ val reporterMaxFailures = sparkConf.get(MAX_REPORTER_THREAD_FAILURES)
+
+ val t = new Thread {
+ override def run() {
+ var failureCount = 0
+ while (!finished) {
+ try {
+ if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) {
+ finish(FinalApplicationStatus.FAILED,
+ ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES,
+ s"Max number of executor failures ($maxNumExecutorFailures) reached")
+ } else {
+ logDebug("Sending progress")
+ allocator.allocateResources()
+ }
+ failureCount = 0
+ } catch {
+ case i: InterruptedException =>
+ case e: Throwable =>
+ failureCount += 1
+ // this exception was introduced in hadoop 2.4 and this code would not compile
+ // with earlier versions if we refer it directly.
+ if ("org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException" ==
+ e.getClass().getName()) {
+ logError("Exception from Reporter thread.", e)
+ finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_REPORTER_FAILURE,
+ e.getMessage)
+ } else if (!NonFatal(e) || failureCount >= reporterMaxFailures) {
+ finish(FinalApplicationStatus.FAILED,
+ ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " +
+ s"$failureCount time(s) from Reporter thread.")
+ } else {
+ logWarning(s"Reporter thread fails $failureCount time(s) in a row.", e)
+ }
+ }
+ try {
+ val numPendingAllocate = allocator.getPendingAllocate.size
+ var sleepStart = 0L
+ var sleepInterval = 200L // ms
+ allocatorLock.synchronized {
+ sleepInterval =
+ if (numPendingAllocate > 0 || allocator.getNumPendingLossReasonRequests > 0) {
+ val currentAllocationInterval =
+ math.min(heartbeatInterval, nextAllocationInterval)
+ nextAllocationInterval = currentAllocationInterval * 2 // avoid overflow
+ currentAllocationInterval
+ } else {
+ nextAllocationInterval = initialAllocationInterval
+ heartbeatInterval
+ }
+ sleepStart = System.currentTimeMillis()
+ allocatorLock.wait(sleepInterval)
+ }
+ val sleepDuration = System.currentTimeMillis() - sleepStart
+ if (sleepDuration < sleepInterval) {
+ // log when sleep is interrupted
+ logDebug(s"Number of pending allocations is $numPendingAllocate. " +
+ s"Slept for $sleepDuration/$sleepInterval ms.")
+ // if sleep was less than the minimum interval, sleep for the rest of it
+ val toSleep = math.max(0, initialAllocationInterval - sleepDuration)
+ if (toSleep > 0) {
+ logDebug(s"Going back to sleep for $toSleep ms")
+ // use Thread.sleep instead of allocatorLock.wait. there is no need to be woken up
+ // by the methods that signal allocatorLock because this is just finishing the min
+ // sleep interval, which should happen even if this is signalled again.
+ Thread.sleep(toSleep)
+ }
+ } else {
+ logDebug(s"Number of pending allocations is $numPendingAllocate. " +
+ s"Slept for $sleepDuration/$sleepInterval.")
+ }
+ } catch {
+ case e: InterruptedException =>
+ }
+ }
+ }
+ }
+ // setting to daemon status, though this is usually not a good idea.
+ t.setDaemon(true)
+ t.setName("Reporter")
+ t.start()
+ logInfo(s"Started progress reporter thread with (heartbeat : $heartbeatInterval, " +
+ s"initial allocation : $initialAllocationInterval) intervals")
+ t
+ }
+
+ /**
+ * Clean up the staging directory.
+ */
+ private def cleanupStagingDir(fs: FileSystem) {
+ var stagingDirPath: Path = null
+ try {
+ val preserveFiles = sparkConf.get(PRESERVE_STAGING_FILES)
+ 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)
+ }
+ }
+
+ private def waitForSparkDriver(): RpcEndpointRef = {
+ logInfo("Waiting for Spark driver to be reachable.")
+ var driverUp = false
+ val hostport = args.userArgs(0)
+ val (driverHost, driverPort) = Utils.parseHostPort(hostport)
+
+ // Spark driver should already be up since it launched us, but we don't want to
+ // wait forever, so wait 100 seconds max to match the cluster mode setting.
+ val totalWaitTimeMs = sparkConf.get(AM_MAX_WAIT_TIME)
+ val deadline = System.currentTimeMillis + totalWaitTimeMs
+
+ while (!driverUp && !finished && System.currentTimeMillis < deadline) {
+ 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(100L)
+ }
+ }
+
+ if (!driverUp) {
+ throw new SparkException("Failed to connect to driver!")
+ }
+
+ sparkConf.set("spark.driver.host", driverHost)
+ sparkConf.set("spark.driver.port", driverPort.toString)
+
+ runAMEndpoint(driverHost, driverPort.toString, isClusterMode = false)
+ }
+
+ /** Add the Yarn IP filter that is required for properly securing the UI. */
+ private def addAmIpFilter() = {
+ val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV)
+ val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter"
+ val params = client.getAmIpFilterParams(yarnConf, proxyBase)
+ if (isClusterMode) {
+ System.setProperty("spark.ui.filters", amFilter)
+ params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) }
+ } else {
+ amEndpoint.send(AddWebUIFilter(amFilter, params.toMap, proxyBase))
+ }
+ }
+
+ /**
+ * Start the user class, which contains the spark driver, in a separate Thread.
+ * If the main routine exits cleanly or exits with System.exit(N) for any N
+ * we assume it was successful, for all other cases we assume failure.
+ *
+ * Returns the user thread that was started.
+ */
+ private def startUserApplication(): Thread = {
+ logInfo("Starting the user application in a separate Thread")
+
+ val classpath = Client.getUserClasspath(sparkConf)
+ val urls = classpath.map { entry =>
+ new URL("file:" + new File(entry.getPath()).getAbsolutePath())
+ }
+ val userClassLoader =
+ if (Client.isUserClassPathFirst(sparkConf, isDriver = true)) {
+ new ChildFirstURLClassLoader(urls, Utils.getContextOrSparkClassLoader)
+ } else {
+ new MutableURLClassLoader(urls, Utils.getContextOrSparkClassLoader)
+ }
+
+ var userArgs = args.userArgs
+ if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) {
+ // When running pyspark, the app is run using PythonRunner. The second argument is the list
+ // of files to add to PYTHONPATH, which Client.scala already handles, so it's empty.
+ userArgs = Seq(args.primaryPyFile, "") ++ userArgs
+ }
+ if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
+ // TODO(davies): add R dependencies here
+ }
+ val mainMethod = userClassLoader.loadClass(args.userClass)
+ .getMethod("main", classOf[Array[String]])
+
+ val userThread = new Thread {
+ override def run() {
+ try {
+ mainMethod.invoke(null, userArgs.toArray)
+ finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
+ logDebug("Done running users class")
+ } catch {
+ case e: InvocationTargetException =>
+ e.getCause match {
+ case _: InterruptedException =>
+ // Reporter thread can interrupt to stop user class
+ case SparkUserAppException(exitCode) =>
+ val msg = s"User application exited with status $exitCode"
+ logError(msg)
+ finish(FinalApplicationStatus.FAILED, exitCode, msg)
+ case cause: Throwable =>
+ logError("User class threw exception: " + cause, cause)
+ finish(FinalApplicationStatus.FAILED,
+ ApplicationMaster.EXIT_EXCEPTION_USER_CLASS,
+ "User class threw exception: " + cause)
+ }
+ sparkContextPromise.tryFailure(e.getCause())
+ } finally {
+ // Notify the thread waiting for the SparkContext, in case the application did not
+ // instantiate one. This will do nothing when the user code instantiates a SparkContext
+ // (with the correct master), or when the user code throws an exception (due to the
+ // tryFailure above).
+ sparkContextPromise.trySuccess(null)
+ }
+ }
+ }
+ userThread.setContextClassLoader(userClassLoader)
+ userThread.setName("Driver")
+ userThread.start()
+ userThread
+ }
+
+ private def resetAllocatorInterval(): Unit = allocatorLock.synchronized {
+ nextAllocationInterval = initialAllocationInterval
+ allocatorLock.notifyAll()
+ }
+
+ /**
+ * An [[RpcEndpoint]] that communicates with the driver's scheduler backend.
+ */
+ private class AMEndpoint(
+ override val rpcEnv: RpcEnv, driver: RpcEndpointRef, isClusterMode: Boolean)
+ extends RpcEndpoint with Logging {
+
+ override def onStart(): Unit = {
+ driver.send(RegisterClusterManager(self))
+ }
+
+ override def receive: PartialFunction[Any, Unit] = {
+ case x: AddWebUIFilter =>
+ logInfo(s"Add WebUI Filter. $x")
+ driver.send(x)
+ }
+
+ override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
+ case RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount) =>
+ Option(allocator) match {
+ case Some(a) =>
+ if (a.requestTotalExecutorsWithPreferredLocalities(requestedTotal,
+ localityAwareTasks, hostToLocalTaskCount)) {
+ resetAllocatorInterval()
+ }
+ context.reply(true)
+
+ case None =>
+ logWarning("Container allocator is not ready to request executors yet.")
+ context.reply(false)
+ }
+
+ case KillExecutors(executorIds) =>
+ logInfo(s"Driver requested to kill executor(s) ${executorIds.mkString(", ")}.")
+ Option(allocator) match {
+ case Some(a) => executorIds.foreach(a.killExecutor)
+ case None => logWarning("Container allocator is not ready to kill executors yet.")
+ }
+ context.reply(true)
+
+ case GetExecutorLossReason(eid) =>
+ Option(allocator) match {
+ case Some(a) =>
+ a.enqueueGetLossReasonRequest(eid, context)
+ resetAllocatorInterval()
+ case None =>
+ logWarning("Container allocator is not ready to find executor loss reasons yet.")
+ }
+ }
+
+ override def onDisconnected(remoteAddress: RpcAddress): Unit = {
+ // In cluster mode, do not rely on the disassociated event to exit
+ // This avoids potentially reporting incorrect exit codes if the driver fails
+ if (!isClusterMode) {
+ logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress")
+ finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS)
+ }
+ }
+ }
+
+}
+
+object ApplicationMaster extends Logging {
+
+ // exit codes for different causes, no reason behind the values
+ private val EXIT_SUCCESS = 0
+ private val EXIT_UNCAUGHT_EXCEPTION = 10
+ private val EXIT_MAX_EXECUTOR_FAILURES = 11
+ private val EXIT_REPORTER_FAILURE = 12
+ private val EXIT_SC_NOT_INITED = 13
+ private val EXIT_SECURITY = 14
+ private val EXIT_EXCEPTION_USER_CLASS = 15
+ private val EXIT_EARLY = 16
+
+ private var master: ApplicationMaster = _
+
+ def main(args: Array[String]): Unit = {
+ SignalUtils.registerLogger(log)
+ val amArgs = new ApplicationMasterArguments(args)
+
+ // Load the properties file with the Spark configuration and set entries as system properties,
+ // so that user code run inside the AM also has access to them.
+ // Note: we must do this before SparkHadoopUtil instantiated
+ if (amArgs.propertiesFile != null) {
+ Utils.getPropertiesFromFile(amArgs.propertiesFile).foreach { case (k, v) =>
+ sys.props(k) = v
+ }
+ }
+ SparkHadoopUtil.get.runAsSparkUser { () =>
+ master = new ApplicationMaster(amArgs, new YarnRMClient)
+ System.exit(master.run())
+ }
+ }
+
+ private[spark] def sparkContextInitialized(sc: SparkContext): Unit = {
+ master.sparkContextInitialized(sc)
+ }
+
+ private[spark] def getAttemptId(): ApplicationAttemptId = {
+ master.getAttemptId
+ }
+
+}
+
+/**
+ * This object does not provide any special functionality. It exists so that it's easy to tell
+ * apart the client-mode AM from the cluster-mode AM when using tools such as ps or jps.
+ */
+object ExecutorLauncher {
+
+ def main(args: Array[String]): Unit = {
+ ApplicationMaster.main(args)
+ }
+
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
new file mode 100644
index 0000000000..5cdec87667
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import org.apache.spark.util.{IntParam, MemoryParam}
+
+class ApplicationMasterArguments(val args: Array[String]) {
+ var userJar: String = null
+ var userClass: String = null
+ var primaryPyFile: String = null
+ var primaryRFile: String = null
+ var userArgs: Seq[String] = Nil
+ var propertiesFile: String = null
+
+ parseArgs(args.toList)
+
+ private def parseArgs(inputArgs: List[String]): Unit = {
+ val userArgsBuffer = new ArrayBuffer[String]()
+
+ var args = inputArgs
+
+ while (!args.isEmpty) {
+ // --num-workers, --worker-memory, and --worker-cores are deprecated since 1.0,
+ // the properties with executor in their names are preferred.
+ args match {
+ case ("--jar") :: value :: tail =>
+ userJar = value
+ args = tail
+
+ case ("--class") :: value :: tail =>
+ userClass = value
+ args = tail
+
+ case ("--primary-py-file") :: value :: tail =>
+ primaryPyFile = value
+ args = tail
+
+ case ("--primary-r-file") :: value :: tail =>
+ primaryRFile = value
+ args = tail
+
+ case ("--arg") :: value :: tail =>
+ userArgsBuffer += value
+ args = tail
+
+ case ("--properties-file") :: value :: tail =>
+ propertiesFile = value
+ args = tail
+
+ case _ =>
+ printUsageAndExit(1, args)
+ }
+ }
+
+ if (primaryPyFile != null && primaryRFile != null) {
+ // scalastyle:off println
+ System.err.println("Cannot have primary-py-file and primary-r-file at the same time")
+ // scalastyle:on println
+ System.exit(-1)
+ }
+
+ userArgs = userArgsBuffer.toList
+ }
+
+ def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
+ // scalastyle:off println
+ if (unknownParam != null) {
+ System.err.println("Unknown/unsupported param " + unknownParam)
+ }
+ System.err.println("""
+ |Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options]
+ |Options:
+ | --jar JAR_PATH Path to your application's JAR file
+ | --class CLASS_NAME Name of your application's main class
+ | --primary-py-file A main Python file
+ | --primary-r-file A main R file
+ | --arg ARG Argument to be passed to your application's main class.
+ | Multiple invocations are possible, each will be passed in order.
+ | --properties-file FILE Path to a custom Spark properties file.
+ """.stripMargin)
+ // scalastyle:on println
+ System.exit(exitCode)
+ }
+}
+
+object ApplicationMasterArguments {
+ val DEFAULT_NUMBER_EXECUTORS = 2
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
new file mode 100644
index 0000000000..be419cee77
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -0,0 +1,1541 @@
+/*
+ * 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.{File, FileOutputStream, IOException, OutputStreamWriter}
+import java.net.{InetAddress, UnknownHostException, URI}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import java.util.{Properties, UUID}
+import java.util.zip.{ZipEntry, ZipOutputStream}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map}
+import scala.util.{Failure, Success, Try}
+import scala.util.control.NonFatal
+
+import com.google.common.base.Objects
+import com.google.common.io.Files
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs._
+import org.apache.hadoop.fs.permission.FsPermission
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.mapreduce.MRJobConfig
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+import org.apache.hadoop.util.StringUtils
+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.{YarnClient, YarnClientApplication}
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException
+import org.apache.hadoop.yarn.util.Records
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils}
+import org.apache.spark.util.{CallerContext, Utils}
+
+private[spark] class Client(
+ val args: ClientArguments,
+ val hadoopConf: Configuration,
+ val sparkConf: SparkConf)
+ extends Logging {
+
+ import Client._
+ import YarnSparkHadoopUtil._
+
+ def this(clientArgs: ClientArguments, spConf: SparkConf) =
+ this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf)
+
+ private val yarnClient = YarnClient.createYarnClient
+ private val yarnConf = new YarnConfiguration(hadoopConf)
+
+ private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster"
+
+ // AM related configurations
+ private val amMemory = if (isClusterMode) {
+ sparkConf.get(DRIVER_MEMORY).toInt
+ } else {
+ sparkConf.get(AM_MEMORY).toInt
+ }
+ private val amMemoryOverhead = {
+ val amMemoryOverheadEntry = if (isClusterMode) DRIVER_MEMORY_OVERHEAD else AM_MEMORY_OVERHEAD
+ sparkConf.get(amMemoryOverheadEntry).getOrElse(
+ math.max((MEMORY_OVERHEAD_FACTOR * amMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt
+ }
+ private val amCores = if (isClusterMode) {
+ sparkConf.get(DRIVER_CORES)
+ } else {
+ sparkConf.get(AM_CORES)
+ }
+
+ // Executor related configurations
+ private val executorMemory = sparkConf.get(EXECUTOR_MEMORY)
+ private val executorMemoryOverhead = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse(
+ math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt
+
+ private val distCacheMgr = new ClientDistributedCacheManager()
+
+ private var loginFromKeytab = false
+ private var principal: String = null
+ private var keytab: String = null
+ private var credentials: Credentials = null
+
+ private val launcherBackend = new LauncherBackend() {
+ override def onStopRequest(): Unit = {
+ if (isClusterMode && appId != null) {
+ yarnClient.killApplication(appId)
+ } else {
+ setState(SparkAppHandle.State.KILLED)
+ stop()
+ }
+ }
+ }
+ private val fireAndForget = isClusterMode && !sparkConf.get(WAIT_FOR_APP_COMPLETION)
+
+ private var appId: ApplicationId = null
+
+ // The app staging dir based on the STAGING_DIR configuration if configured
+ // otherwise based on the users home directory.
+ private val appStagingBaseDir = sparkConf.get(STAGING_DIR).map { new Path(_) }
+ .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory())
+
+ private val credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+
+ def reportLauncherState(state: SparkAppHandle.State): Unit = {
+ launcherBackend.setState(state)
+ }
+
+ def stop(): Unit = {
+ launcherBackend.close()
+ yarnClient.stop()
+ // Unset YARN mode system env variable, to allow switching between cluster types.
+ System.clearProperty("SPARK_YARN_MODE")
+ }
+
+ /**
+ * Submit an application running our ApplicationMaster to the ResourceManager.
+ *
+ * The stable Yarn API provides a convenience method (YarnClient#createApplication) for
+ * creating applications and setting up the application submission context. This was not
+ * available in the alpha API.
+ */
+ def submitApplication(): ApplicationId = {
+ var appId: ApplicationId = null
+ try {
+ launcherBackend.connect()
+ // Setup the credentials before doing anything else,
+ // so we have don't have issues at any point.
+ setupCredentials()
+ yarnClient.init(yarnConf)
+ yarnClient.start()
+
+ logInfo("Requesting a new application from cluster with %d NodeManagers"
+ .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers))
+
+ // Get a new application from our RM
+ val newApp = yarnClient.createApplication()
+ val newAppResponse = newApp.getNewApplicationResponse()
+ appId = newAppResponse.getApplicationId()
+ reportLauncherState(SparkAppHandle.State.SUBMITTED)
+ launcherBackend.setAppId(appId.toString)
+
+ new CallerContext("CLIENT", sparkConf.get(APP_CALLER_CONTEXT),
+ Option(appId.toString)).setCurrentContext()
+
+ // Verify whether the cluster has enough resources for our AM
+ verifyClusterResources(newAppResponse)
+
+ // Set up the appropriate contexts to launch our AM
+ val containerContext = createContainerLaunchContext(newAppResponse)
+ val appContext = createApplicationSubmissionContext(newApp, containerContext)
+
+ // Finally, submit and monitor the application
+ logInfo(s"Submitting application $appId to ResourceManager")
+ yarnClient.submitApplication(appContext)
+ appId
+ } catch {
+ case e: Throwable =>
+ if (appId != null) {
+ cleanupStagingDir(appId)
+ }
+ throw e
+ }
+ }
+
+ /**
+ * Cleanup application staging directory.
+ */
+ private def cleanupStagingDir(appId: ApplicationId): Unit = {
+ val stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId))
+ try {
+ val preserveFiles = sparkConf.get(PRESERVE_STAGING_FILES)
+ val fs = stagingDirPath.getFileSystem(hadoopConf)
+ if (!preserveFiles && fs.delete(stagingDirPath, true)) {
+ logInfo(s"Deleted staging directory $stagingDirPath")
+ }
+ } catch {
+ case ioe: IOException =>
+ logWarning("Failed to cleanup staging dir " + stagingDirPath, ioe)
+ }
+ }
+
+ /**
+ * Set up the context for submitting our ApplicationMaster.
+ * This uses the YarnClientApplication not available in the Yarn alpha API.
+ */
+ def createApplicationSubmissionContext(
+ newApp: YarnClientApplication,
+ containerContext: ContainerLaunchContext): ApplicationSubmissionContext = {
+ val appContext = newApp.getApplicationSubmissionContext
+ appContext.setApplicationName(sparkConf.get("spark.app.name", "Spark"))
+ appContext.setQueue(sparkConf.get(QUEUE_NAME))
+ appContext.setAMContainerSpec(containerContext)
+ appContext.setApplicationType("SPARK")
+
+ sparkConf.get(APPLICATION_TAGS).foreach { tags =>
+ try {
+ // The setApplicationTags method was only introduced in Hadoop 2.4+, so we need to use
+ // reflection to set it, printing a warning if a tag was specified but the YARN version
+ // doesn't support it.
+ val method = appContext.getClass().getMethod(
+ "setApplicationTags", classOf[java.util.Set[String]])
+ method.invoke(appContext, new java.util.HashSet[String](tags.asJava))
+ } catch {
+ case e: NoSuchMethodException =>
+ logWarning(s"Ignoring ${APPLICATION_TAGS.key} because this version of " +
+ "YARN does not support it")
+ }
+ }
+ sparkConf.get(MAX_APP_ATTEMPTS) match {
+ case Some(v) => appContext.setMaxAppAttempts(v)
+ case None => logDebug(s"${MAX_APP_ATTEMPTS.key} is not set. " +
+ "Cluster's default value will be used.")
+ }
+
+ sparkConf.get(AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS).foreach { interval =>
+ try {
+ val method = appContext.getClass().getMethod(
+ "setAttemptFailuresValidityInterval", classOf[Long])
+ method.invoke(appContext, interval: java.lang.Long)
+ } catch {
+ case e: NoSuchMethodException =>
+ logWarning(s"Ignoring ${AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS.key} because " +
+ "the version of YARN does not support it")
+ }
+ }
+
+ val capability = Records.newRecord(classOf[Resource])
+ capability.setMemory(amMemory + amMemoryOverhead)
+ capability.setVirtualCores(amCores)
+
+ sparkConf.get(AM_NODE_LABEL_EXPRESSION) match {
+ case Some(expr) =>
+ try {
+ val amRequest = Records.newRecord(classOf[ResourceRequest])
+ amRequest.setResourceName(ResourceRequest.ANY)
+ amRequest.setPriority(Priority.newInstance(0))
+ amRequest.setCapability(capability)
+ amRequest.setNumContainers(1)
+ val method = amRequest.getClass.getMethod("setNodeLabelExpression", classOf[String])
+ method.invoke(amRequest, expr)
+
+ val setResourceRequestMethod =
+ appContext.getClass.getMethod("setAMContainerResourceRequest", classOf[ResourceRequest])
+ setResourceRequestMethod.invoke(appContext, amRequest)
+ } catch {
+ case e: NoSuchMethodException =>
+ logWarning(s"Ignoring ${AM_NODE_LABEL_EXPRESSION.key} because the version " +
+ "of YARN does not support it")
+ appContext.setResource(capability)
+ }
+ case None =>
+ appContext.setResource(capability)
+ }
+
+ sparkConf.get(ROLLED_LOG_INCLUDE_PATTERN).foreach { includePattern =>
+ try {
+ val logAggregationContext = Records.newRecord(
+ Utils.classForName("org.apache.hadoop.yarn.api.records.LogAggregationContext"))
+ .asInstanceOf[Object]
+
+ val setRolledLogsIncludePatternMethod =
+ logAggregationContext.getClass.getMethod("setRolledLogsIncludePattern", classOf[String])
+ setRolledLogsIncludePatternMethod.invoke(logAggregationContext, includePattern)
+
+ sparkConf.get(ROLLED_LOG_EXCLUDE_PATTERN).foreach { excludePattern =>
+ val setRolledLogsExcludePatternMethod =
+ logAggregationContext.getClass.getMethod("setRolledLogsExcludePattern", classOf[String])
+ setRolledLogsExcludePatternMethod.invoke(logAggregationContext, excludePattern)
+ }
+
+ val setLogAggregationContextMethod =
+ appContext.getClass.getMethod("setLogAggregationContext",
+ Utils.classForName("org.apache.hadoop.yarn.api.records.LogAggregationContext"))
+ setLogAggregationContextMethod.invoke(appContext, logAggregationContext)
+ } catch {
+ case NonFatal(e) =>
+ logWarning(s"Ignoring ${ROLLED_LOG_INCLUDE_PATTERN.key} because the version of YARN " +
+ s"does not support it", e)
+ }
+ }
+
+ appContext
+ }
+
+ /** Set up security tokens for launching our ApplicationMaster container. */
+ private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = {
+ val dob = new DataOutputBuffer
+ credentials.writeTokenStorageToStream(dob)
+ amContainer.setTokens(ByteBuffer.wrap(dob.getData))
+ }
+
+ /** Get the application report from the ResourceManager for an application we have submitted. */
+ def getApplicationReport(appId: ApplicationId): ApplicationReport =
+ yarnClient.getApplicationReport(appId)
+
+ /**
+ * Return the security token used by this client to communicate with the ApplicationMaster.
+ * If no security is enabled, the token returned by the report is null.
+ */
+ private def getClientToken(report: ApplicationReport): String =
+ Option(report.getClientToAMToken).map(_.toString).getOrElse("")
+
+ /**
+ * Fail fast if we have requested more resources per container than is available in the cluster.
+ */
+ private def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = {
+ val maxMem = newAppResponse.getMaximumResourceCapability().getMemory()
+ logInfo("Verifying our application has not requested more than the maximum " +
+ s"memory capability of the cluster ($maxMem MB per container)")
+ val executorMem = executorMemory + executorMemoryOverhead
+ if (executorMem > maxMem) {
+ throw new IllegalArgumentException(s"Required executor memory ($executorMemory" +
+ s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " +
+ "Please check the values of 'yarn.scheduler.maximum-allocation-mb' and/or " +
+ "'yarn.nodemanager.resource.memory-mb'.")
+ }
+ val amMem = amMemory + amMemoryOverhead
+ if (amMem > maxMem) {
+ throw new IllegalArgumentException(s"Required AM memory ($amMemory" +
+ s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " +
+ "Please increase the value of 'yarn.scheduler.maximum-allocation-mb'.")
+ }
+ logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format(
+ amMem,
+ amMemoryOverhead))
+
+ // We could add checks to make sure the entire cluster has enough resources but that involves
+ // getting all the node reports and computing ourselves.
+ }
+
+ /**
+ * Copy the given file to a remote file system (e.g. HDFS) if needed.
+ * The file is only copied if the source and destination file systems are different. This is used
+ * for preparing resources for launching the ApplicationMaster container. Exposed for testing.
+ */
+ private[yarn] def copyFileToRemote(
+ destDir: Path,
+ srcPath: Path,
+ replication: Short,
+ force: Boolean = false,
+ destName: Option[String] = None): Path = {
+ val destFs = destDir.getFileSystem(hadoopConf)
+ val srcFs = srcPath.getFileSystem(hadoopConf)
+ var destPath = srcPath
+ if (force || !compareFs(srcFs, destFs)) {
+ destPath = new Path(destDir, destName.getOrElse(srcPath.getName()))
+ logInfo(s"Uploading resource $srcPath -> $destPath")
+ FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf)
+ destFs.setReplication(destPath, replication)
+ destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION))
+ } else {
+ logInfo(s"Source and destination file systems are the same. Not copying $srcPath")
+ }
+ // 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 qualifiedDestPath = destFs.makeQualified(destPath)
+ val fc = FileContext.getFileContext(qualifiedDestPath.toUri(), hadoopConf)
+ fc.resolvePath(qualifiedDestPath)
+ }
+
+ /**
+ * Upload any resources to the distributed cache if needed. If a resource is intended to be
+ * consumed locally, set up the appropriate config for downstream code to handle it properly.
+ * This is used for setting up a container launch context for our ApplicationMaster.
+ * Exposed for testing.
+ */
+ def prepareLocalResources(
+ destDir: Path,
+ pySparkArchives: Seq[String]): HashMap[String, LocalResource] = {
+ logInfo("Preparing resources for our AM container")
+ // Upload Spark and the application JAR to the remote file system if necessary,
+ // and add them as local resources to the application master.
+ val fs = destDir.getFileSystem(hadoopConf)
+
+ // Merge credentials obtained from registered providers
+ val nearestTimeOfNextRenewal = credentialManager.obtainCredentials(hadoopConf, credentials)
+
+ if (credentials != null) {
+ logDebug(YarnSparkHadoopUtil.get.dumpTokens(credentials).mkString("\n"))
+ }
+
+ // If we use principal and keytab to login, also credentials can be renewed some time
+ // after current time, we should pass the next renewal and updating time to credential
+ // renewer and updater.
+ if (loginFromKeytab && nearestTimeOfNextRenewal > System.currentTimeMillis() &&
+ nearestTimeOfNextRenewal != Long.MaxValue) {
+
+ // Valid renewal time is 75% of next renewal time, and the valid update time will be
+ // slightly later then renewal time (80% of next renewal time). This is to make sure
+ // credentials are renewed and updated before expired.
+ val currTime = System.currentTimeMillis()
+ val renewalTime = (nearestTimeOfNextRenewal - currTime) * 0.75 + currTime
+ val updateTime = (nearestTimeOfNextRenewal - currTime) * 0.8 + currTime
+
+ sparkConf.set(CREDENTIALS_RENEWAL_TIME, renewalTime.toLong)
+ sparkConf.set(CREDENTIALS_UPDATE_TIME, updateTime.toLong)
+ }
+
+ // Used to keep track of URIs added to the distributed cache. If the same URI is added
+ // multiple times, YARN will fail to launch containers for the app with an internal
+ // error.
+ val distributedUris = new HashSet[String]
+ // Used to keep track of URIs(files) added to the distribute cache have the same name. If
+ // same name but different path files are added multiple time, YARN will fail to launch
+ // containers for the app with an internal error.
+ val distributedNames = new HashSet[String]
+
+ val replication = sparkConf.get(STAGING_FILE_REPLICATION).map(_.toShort)
+ .getOrElse(fs.getDefaultReplication(destDir))
+ val localResources = HashMap[String, LocalResource]()
+ FileSystem.mkdirs(fs, destDir, new FsPermission(STAGING_DIR_PERMISSION))
+
+ val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+
+ def addDistributedUri(uri: URI): Boolean = {
+ val uriStr = uri.toString()
+ val fileName = new File(uri.getPath).getName
+ if (distributedUris.contains(uriStr)) {
+ logWarning(s"Same path resource $uri added multiple times to distributed cache.")
+ false
+ } else if (distributedNames.contains(fileName)) {
+ logWarning(s"Same name resource $uri added multiple times to distributed cache")
+ false
+ } else {
+ distributedUris += uriStr
+ distributedNames += fileName
+ true
+ }
+ }
+
+ /**
+ * Distribute a file to the cluster.
+ *
+ * If the file's path is a "local:" URI, it's actually not distributed. Other files are copied
+ * to HDFS (if not already there) and added to the application's distributed cache.
+ *
+ * @param path URI of the file to distribute.
+ * @param resType Type of resource being distributed.
+ * @param destName Name of the file in the distributed cache.
+ * @param targetDir Subdirectory where to place the file.
+ * @param appMasterOnly Whether to distribute only to the AM.
+ * @return A 2-tuple. First item is whether the file is a "local:" URI. Second item is the
+ * localized path for non-local paths, or the input `path` for local paths.
+ * The localized path will be null if the URI has already been added to the cache.
+ */
+ def distribute(
+ path: String,
+ resType: LocalResourceType = LocalResourceType.FILE,
+ destName: Option[String] = None,
+ targetDir: Option[String] = None,
+ appMasterOnly: Boolean = false): (Boolean, String) = {
+ val trimmedPath = path.trim()
+ val localURI = Utils.resolveURI(trimmedPath)
+ if (localURI.getScheme != LOCAL_SCHEME) {
+ if (addDistributedUri(localURI)) {
+ val localPath = getQualifiedLocalPath(localURI, hadoopConf)
+ val linkname = targetDir.map(_ + "/").getOrElse("") +
+ destName.orElse(Option(localURI.getFragment())).getOrElse(localPath.getName())
+ val destPath = copyFileToRemote(destDir, localPath, replication)
+ val destFs = FileSystem.get(destPath.toUri(), hadoopConf)
+ distCacheMgr.addResource(
+ destFs, hadoopConf, destPath, localResources, resType, linkname, statCache,
+ appMasterOnly = appMasterOnly)
+ (false, linkname)
+ } else {
+ (false, null)
+ }
+ } else {
+ (true, trimmedPath)
+ }
+ }
+
+ // If we passed in a keytab, make sure we copy the keytab to the staging directory on
+ // HDFS, and setup the relevant environment vars, so the AM can login again.
+ if (loginFromKeytab) {
+ logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" +
+ " via the YARN Secure Distributed Cache.")
+ val (_, localizedPath) = distribute(keytab,
+ destName = sparkConf.get(KEYTAB),
+ appMasterOnly = true)
+ require(localizedPath != null, "Keytab file already distributed.")
+ }
+
+ /**
+ * Add Spark to the cache. There are two settings that control what files to add to the cache:
+ * - if a Spark archive is defined, use the archive. The archive is expected to contain
+ * jar files at its root directory.
+ * - if a list of jars is provided, filter the non-local ones, resolve globs, and
+ * add the found files to the cache.
+ *
+ * Note that the archive cannot be a "local" URI. If none of the above settings are found,
+ * then upload all files found in $SPARK_HOME/jars.
+ */
+ val sparkArchive = sparkConf.get(SPARK_ARCHIVE)
+ if (sparkArchive.isDefined) {
+ val archive = sparkArchive.get
+ require(!isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.")
+ distribute(Utils.resolveURI(archive).toString,
+ resType = LocalResourceType.ARCHIVE,
+ destName = Some(LOCALIZED_LIB_DIR))
+ } else {
+ sparkConf.get(SPARK_JARS) match {
+ case Some(jars) =>
+ // Break the list of jars to upload, and resolve globs.
+ val localJars = new ArrayBuffer[String]()
+ jars.foreach { jar =>
+ if (!isLocalUri(jar)) {
+ val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf)
+ val pathFs = FileSystem.get(path.toUri(), hadoopConf)
+ pathFs.globStatus(path).filter(_.isFile()).foreach { entry =>
+ distribute(entry.getPath().toUri().toString(),
+ targetDir = Some(LOCALIZED_LIB_DIR))
+ }
+ } else {
+ localJars += jar
+ }
+ }
+
+ // Propagate the local URIs to the containers using the configuration.
+ sparkConf.set(SPARK_JARS, localJars)
+
+ case None =>
+ // No configuration, so fall back to uploading local jar files.
+ logWarning(s"Neither ${SPARK_JARS.key} nor ${SPARK_ARCHIVE.key} is set, falling back " +
+ "to uploading libraries under SPARK_HOME.")
+ val jarsDir = new File(YarnCommandBuilderUtils.findJarsDir(
+ sparkConf.getenv("SPARK_HOME")))
+ val jarsArchive = File.createTempFile(LOCALIZED_LIB_DIR, ".zip",
+ new File(Utils.getLocalDir(sparkConf)))
+ val jarsStream = new ZipOutputStream(new FileOutputStream(jarsArchive))
+
+ try {
+ jarsStream.setLevel(0)
+ jarsDir.listFiles().foreach { f =>
+ if (f.isFile && f.getName.toLowerCase().endsWith(".jar") && f.canRead) {
+ jarsStream.putNextEntry(new ZipEntry(f.getName))
+ Files.copy(f, jarsStream)
+ jarsStream.closeEntry()
+ }
+ }
+ } finally {
+ jarsStream.close()
+ }
+
+ distribute(jarsArchive.toURI.getPath,
+ resType = LocalResourceType.ARCHIVE,
+ destName = Some(LOCALIZED_LIB_DIR))
+ }
+ }
+
+ /**
+ * Copy user jar to the distributed cache if their scheme is not "local".
+ * Otherwise, set the corresponding key in our SparkConf to handle it downstream.
+ */
+ Option(args.userJar).filter(_.trim.nonEmpty).foreach { jar =>
+ val (isLocal, localizedPath) = distribute(jar, destName = Some(APP_JAR_NAME))
+ if (isLocal) {
+ require(localizedPath != null, s"Path $jar already distributed")
+ // If the resource is intended for local use only, handle this downstream
+ // by setting the appropriate property
+ sparkConf.set(APP_JAR, localizedPath)
+ }
+ }
+
+ /**
+ * Do the same for any additional resources passed in through ClientArguments.
+ * Each resource category is represented by a 3-tuple of:
+ * (1) comma separated list of resources in this category,
+ * (2) resource type, and
+ * (3) whether to add these resources to the classpath
+ */
+ val cachedSecondaryJarLinks = ListBuffer.empty[String]
+ List(
+ (sparkConf.get(JARS_TO_DISTRIBUTE), LocalResourceType.FILE, true),
+ (sparkConf.get(FILES_TO_DISTRIBUTE), LocalResourceType.FILE, false),
+ (sparkConf.get(ARCHIVES_TO_DISTRIBUTE), LocalResourceType.ARCHIVE, false)
+ ).foreach { case (flist, resType, addToClasspath) =>
+ flist.foreach { file =>
+ val (_, localizedPath) = distribute(file, resType = resType)
+ // If addToClassPath, we ignore adding jar multiple times to distitrbuted cache.
+ if (addToClasspath) {
+ if (localizedPath != null) {
+ cachedSecondaryJarLinks += localizedPath
+ }
+ } else {
+ if (localizedPath == null) {
+ throw new IllegalArgumentException(s"Attempt to add ($file) multiple times" +
+ " to the distributed cache.")
+ }
+ }
+ }
+ }
+ if (cachedSecondaryJarLinks.nonEmpty) {
+ sparkConf.set(SECONDARY_JARS, cachedSecondaryJarLinks)
+ }
+
+ if (isClusterMode && args.primaryPyFile != null) {
+ distribute(args.primaryPyFile, appMasterOnly = true)
+ }
+
+ pySparkArchives.foreach { f => distribute(f) }
+
+ // The python files list needs to be treated especially. All files that are not an
+ // archive need to be placed in a subdirectory that will be added to PYTHONPATH.
+ sparkConf.get(PY_FILES).foreach { f =>
+ val targetDir = if (f.endsWith(".py")) Some(LOCALIZED_PYTHON_DIR) else None
+ distribute(f, targetDir = targetDir)
+ }
+
+ // Update the configuration with all the distributed files, minus the conf archive. The
+ // conf archive will be handled by the AM differently so that we avoid having to send
+ // this configuration by other means. See SPARK-14602 for one reason of why this is needed.
+ distCacheMgr.updateConfiguration(sparkConf)
+
+ // Upload the conf archive to HDFS manually, and record its location in the configuration.
+ // This will allow the AM to know where the conf archive is in HDFS, so that it can be
+ // distributed to the containers.
+ //
+ // This code forces the archive to be copied, so that unit tests pass (since in that case both
+ // file systems are the same and the archive wouldn't normally be copied). In most (all?)
+ // deployments, the archive would be copied anyway, since it's a temp file in the local file
+ // system.
+ val remoteConfArchivePath = new Path(destDir, LOCALIZED_CONF_ARCHIVE)
+ val remoteFs = FileSystem.get(remoteConfArchivePath.toUri(), hadoopConf)
+ sparkConf.set(CACHED_CONF_ARCHIVE, remoteConfArchivePath.toString())
+
+ val localConfArchive = new Path(createConfArchive().toURI())
+ copyFileToRemote(destDir, localConfArchive, replication, force = true,
+ destName = Some(LOCALIZED_CONF_ARCHIVE))
+
+ // Manually add the config archive to the cache manager so that the AM is launched with
+ // the proper files set up.
+ distCacheMgr.addResource(
+ remoteFs, hadoopConf, remoteConfArchivePath, localResources, LocalResourceType.ARCHIVE,
+ LOCALIZED_CONF_DIR, statCache, appMasterOnly = false)
+
+ // Clear the cache-related entries from the configuration to avoid them polluting the
+ // UI's environment page. This works for client mode; for cluster mode, this is handled
+ // by the AM.
+ CACHE_CONFIGS.foreach(sparkConf.remove)
+
+ localResources
+ }
+
+ /**
+ * Create an archive with the config files for distribution.
+ *
+ * These will be used by AM and executors. The files are zipped and added to the job as an
+ * archive, so that YARN will explode it when distributing to AM and executors. This directory
+ * is then added to the classpath of AM and executor process, just to make sure that everybody
+ * is using the same default config.
+ *
+ * This follows the order of precedence set by the startup scripts, in which HADOOP_CONF_DIR
+ * shows up in the classpath before YARN_CONF_DIR.
+ *
+ * Currently this makes a shallow copy of the conf directory. If there are cases where a
+ * Hadoop config directory contains subdirectories, this code will have to be fixed.
+ *
+ * The archive also contains some Spark configuration. Namely, it saves the contents of
+ * SparkConf in a file to be loaded by the AM process.
+ */
+ private def createConfArchive(): File = {
+ val hadoopConfFiles = new HashMap[String, File]()
+
+ // Uploading $SPARK_CONF_DIR/log4j.properties file to the distributed cache to make sure that
+ // the executors will use the latest configurations instead of the default values. This is
+ // required when user changes log4j.properties directly to set the log configurations. If
+ // configuration file is provided through --files then executors will be taking configurations
+ // from --files instead of $SPARK_CONF_DIR/log4j.properties.
+
+ // Also uploading metrics.properties to distributed cache if exists in classpath.
+ // If user specify this file using --files then executors will use the one
+ // from --files instead.
+ for { prop <- Seq("log4j.properties", "metrics.properties")
+ url <- Option(Utils.getContextOrSparkClassLoader.getResource(prop))
+ if url.getProtocol == "file" } {
+ hadoopConfFiles(prop) = new File(url.getPath)
+ }
+
+ Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey =>
+ sys.env.get(envKey).foreach { path =>
+ val dir = new File(path)
+ if (dir.isDirectory()) {
+ val files = dir.listFiles()
+ if (files == null) {
+ logWarning("Failed to list files under directory " + dir)
+ } else {
+ files.foreach { file =>
+ if (file.isFile && !hadoopConfFiles.contains(file.getName())) {
+ hadoopConfFiles(file.getName()) = file
+ }
+ }
+ }
+ }
+ }
+ }
+
+ val confArchive = File.createTempFile(LOCALIZED_CONF_DIR, ".zip",
+ new File(Utils.getLocalDir(sparkConf)))
+ val confStream = new ZipOutputStream(new FileOutputStream(confArchive))
+
+ try {
+ confStream.setLevel(0)
+ hadoopConfFiles.foreach { case (name, file) =>
+ if (file.canRead()) {
+ confStream.putNextEntry(new ZipEntry(name))
+ Files.copy(file, confStream)
+ confStream.closeEntry()
+ }
+ }
+
+ // Save Spark configuration to a file in the archive.
+ val props = new Properties()
+ sparkConf.getAll.foreach { case (k, v) => props.setProperty(k, v) }
+ confStream.putNextEntry(new ZipEntry(SPARK_CONF_FILE))
+ val writer = new OutputStreamWriter(confStream, StandardCharsets.UTF_8)
+ props.store(writer, "Spark configuration.")
+ writer.flush()
+ confStream.closeEntry()
+ } finally {
+ confStream.close()
+ }
+ confArchive
+ }
+
+ /**
+ * Set up the environment for launching our ApplicationMaster container.
+ */
+ private def setupLaunchEnv(
+ stagingDirPath: Path,
+ pySparkArchives: Seq[String]): HashMap[String, String] = {
+ logInfo("Setting up the launch environment for our AM container")
+ val env = new HashMap[String, String]()
+ populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH))
+ env("SPARK_YARN_MODE") = "true"
+ env("SPARK_YARN_STAGING_DIR") = stagingDirPath.toString
+ env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName()
+ if (loginFromKeytab) {
+ val credentialsFile = "credentials-" + UUID.randomUUID().toString
+ sparkConf.set(CREDENTIALS_FILE_PATH, new Path(stagingDirPath, credentialsFile).toString)
+ logInfo(s"Credentials file set to: $credentialsFile")
+ }
+
+ // Pick up any environment variables for the AM provided through spark.yarn.appMasterEnv.*
+ val amEnvPrefix = "spark.yarn.appMasterEnv."
+ sparkConf.getAll
+ .filter { case (k, v) => k.startsWith(amEnvPrefix) }
+ .map { case (k, v) => (k.substring(amEnvPrefix.length), v) }
+ .foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) }
+
+ // Keep this for backwards compatibility but users should move to the config
+ sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs =>
+ // Allow users to specify some environment variables.
+ YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs)
+ // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments.
+ env("SPARK_YARN_USER_ENV") = userEnvs
+ }
+
+ // If pyFiles contains any .py files, we need to add LOCALIZED_PYTHON_DIR to the PYTHONPATH
+ // of the container processes too. Add all non-.py files directly to PYTHONPATH.
+ //
+ // NOTE: the code currently does not handle .py files defined with a "local:" scheme.
+ val pythonPath = new ListBuffer[String]()
+ val (pyFiles, pyArchives) = sparkConf.get(PY_FILES).partition(_.endsWith(".py"))
+ if (pyFiles.nonEmpty) {
+ pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+ LOCALIZED_PYTHON_DIR)
+ }
+ (pySparkArchives ++ pyArchives).foreach { path =>
+ val uri = Utils.resolveURI(path)
+ if (uri.getScheme != LOCAL_SCHEME) {
+ pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+ new Path(uri).getName())
+ } else {
+ pythonPath += uri.getPath()
+ }
+ }
+
+ // Finally, update the Spark config to propagate PYTHONPATH to the AM and executors.
+ if (pythonPath.nonEmpty) {
+ val pythonPathStr = (sys.env.get("PYTHONPATH") ++ pythonPath)
+ .mkString(YarnSparkHadoopUtil.getClassPathSeparator)
+ env("PYTHONPATH") = pythonPathStr
+ sparkConf.setExecutorEnv("PYTHONPATH", pythonPathStr)
+ }
+
+ // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to
+ // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's
+ // SparkContext will not let that set spark* system properties, which is expected behavior for
+ // Yarn clients. So propagate it through the environment.
+ //
+ // Note that to warn the user about the deprecation in cluster mode, some code from
+ // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition
+ // described above).
+ if (isClusterMode) {
+ sys.env.get("SPARK_JAVA_OPTS").foreach { value =>
+ val warning =
+ s"""
+ |SPARK_JAVA_OPTS was detected (set to '$value').
+ |This is deprecated in Spark 1.0+.
+ |
+ |Please instead use:
+ | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application
+ | - ./spark-submit with --driver-java-options to set -X options for a driver
+ | - spark.executor.extraJavaOptions to set -X options for executors
+ """.stripMargin
+ logWarning(warning)
+ for (proc <- Seq("driver", "executor")) {
+ val key = s"spark.$proc.extraJavaOptions"
+ if (sparkConf.contains(key)) {
+ throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.")
+ }
+ }
+ env("SPARK_JAVA_OPTS") = value
+ }
+ // propagate PYSPARK_DRIVER_PYTHON and PYSPARK_PYTHON to driver in cluster mode
+ Seq("PYSPARK_DRIVER_PYTHON", "PYSPARK_PYTHON").foreach { envname =>
+ if (!env.contains(envname)) {
+ sys.env.get(envname).foreach(env(envname) = _)
+ }
+ }
+ }
+
+ sys.env.get(ENV_DIST_CLASSPATH).foreach { dcp =>
+ env(ENV_DIST_CLASSPATH) = dcp
+ }
+
+ env
+ }
+
+ /**
+ * Set up a ContainerLaunchContext to launch our ApplicationMaster container.
+ * This sets up the launch environment, java options, and the command for launching the AM.
+ */
+ private def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse)
+ : ContainerLaunchContext = {
+ logInfo("Setting up container launch context for our AM")
+ val appId = newAppResponse.getApplicationId
+ val appStagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId))
+ val pySparkArchives =
+ if (sparkConf.get(IS_PYTHON_APP)) {
+ findPySparkArchives()
+ } else {
+ Nil
+ }
+ val launchEnv = setupLaunchEnv(appStagingDirPath, pySparkArchives)
+ val localResources = prepareLocalResources(appStagingDirPath, pySparkArchives)
+
+ val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
+ amContainer.setLocalResources(localResources.asJava)
+ amContainer.setEnvironment(launchEnv.asJava)
+
+ val javaOpts = ListBuffer[String]()
+
+ // Set the environment variable through a command prefix
+ // to append to the existing value of the variable
+ var prefixEnv: Option[String] = None
+
+ // Add Xmx for AM memory
+ javaOpts += "-Xmx" + amMemory + "m"
+
+ val tmpDir = new Path(
+ YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+ YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR
+ )
+ javaOpts += "-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 = launchEnv.get("SPARK_USE_CONC_INCR_GC").exists(_.toBoolean)
+ if (useConcurrentAndIncrementalGC) {
+ // In our expts, using (default) throughput collector has severe perf ramifications in
+ // multi-tenant machines
+ javaOpts += "-XX:+UseConcMarkSweepGC"
+ javaOpts += "-XX:MaxTenuringThreshold=31"
+ javaOpts += "-XX:SurvivorRatio=8"
+ javaOpts += "-XX:+CMSIncrementalMode"
+ javaOpts += "-XX:+CMSIncrementalPacing"
+ javaOpts += "-XX:CMSIncrementalDutyCycleMin=0"
+ javaOpts += "-XX:CMSIncrementalDutyCycle=10"
+ }
+
+ // Include driver-specific java options if we are launching a driver
+ if (isClusterMode) {
+ val driverOpts = sparkConf.get(DRIVER_JAVA_OPTIONS).orElse(sys.env.get("SPARK_JAVA_OPTS"))
+ driverOpts.foreach { opts =>
+ javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
+ }
+ val libraryPaths = Seq(sparkConf.get(DRIVER_LIBRARY_PATH),
+ sys.props.get("spark.driver.libraryPath")).flatten
+ if (libraryPaths.nonEmpty) {
+ prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(libraryPaths)))
+ }
+ if (sparkConf.get(AM_JAVA_OPTIONS).isDefined) {
+ logWarning(s"${AM_JAVA_OPTIONS.key} will not take effect in cluster mode")
+ }
+ } else {
+ // Validate and include yarn am specific java options in yarn-client mode.
+ sparkConf.get(AM_JAVA_OPTIONS).foreach { opts =>
+ if (opts.contains("-Dspark")) {
+ val msg = s"${AM_JAVA_OPTIONS.key} is not allowed to set Spark options (was '$opts')."
+ throw new SparkException(msg)
+ }
+ if (opts.contains("-Xmx")) {
+ val msg = s"${AM_JAVA_OPTIONS.key} is not allowed to specify max heap memory settings " +
+ s"(was '$opts'). Use spark.yarn.am.memory instead."
+ throw new SparkException(msg)
+ }
+ javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
+ }
+ sparkConf.get(AM_LIBRARY_PATH).foreach { paths =>
+ prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(Seq(paths))))
+ }
+ }
+
+ // For log4j configuration to reference
+ javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR)
+ YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts)
+
+ val userClass =
+ if (isClusterMode) {
+ Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass))
+ } else {
+ Nil
+ }
+ val userJar =
+ if (args.userJar != null) {
+ Seq("--jar", args.userJar)
+ } else {
+ Nil
+ }
+ val primaryPyFile =
+ if (isClusterMode && args.primaryPyFile != null) {
+ Seq("--primary-py-file", new Path(args.primaryPyFile).getName())
+ } else {
+ Nil
+ }
+ val primaryRFile =
+ if (args.primaryRFile != null) {
+ Seq("--primary-r-file", args.primaryRFile)
+ } else {
+ Nil
+ }
+ val amClass =
+ if (isClusterMode) {
+ Utils.classForName("org.apache.spark.deploy.yarn.ApplicationMaster").getName
+ } else {
+ Utils.classForName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName
+ }
+ if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) {
+ args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs
+ }
+ val userArgs = args.userArgs.flatMap { arg =>
+ Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg))
+ }
+ val amArgs =
+ Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ primaryRFile ++
+ userArgs ++ Seq(
+ "--properties-file", buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+ LOCALIZED_CONF_DIR, SPARK_CONF_FILE))
+
+ // Command for the ApplicationMaster
+ val commands = prefixEnv ++ Seq(
+ YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java", "-server"
+ ) ++
+ javaOpts ++ amArgs ++
+ Seq(
+ "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout",
+ "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+
+ // TODO: it would be nicer to just make sure there are no null commands here
+ val printableCommands = commands.map(s => if (s == null) "null" else s).toList
+ amContainer.setCommands(printableCommands.asJava)
+
+ logDebug("===============================================================================")
+ logDebug("YARN AM launch context:")
+ logDebug(s" user class: ${Option(args.userClass).getOrElse("N/A")}")
+ logDebug(" env:")
+ launchEnv.foreach { case (k, v) => logDebug(s" $k -> $v") }
+ logDebug(" resources:")
+ localResources.foreach { case (k, v) => logDebug(s" $k -> $v")}
+ logDebug(" command:")
+ logDebug(s" ${printableCommands.mkString(" ")}")
+ logDebug("===============================================================================")
+
+ // send the acl settings into YARN to control who has access via YARN interfaces
+ val securityManager = new SecurityManager(sparkConf)
+ amContainer.setApplicationACLs(
+ YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager).asJava)
+ setupSecurityToken(amContainer)
+ amContainer
+ }
+
+ def setupCredentials(): Unit = {
+ loginFromKeytab = sparkConf.contains(PRINCIPAL.key)
+ if (loginFromKeytab) {
+ principal = sparkConf.get(PRINCIPAL).get
+ keytab = sparkConf.get(KEYTAB).orNull
+
+ require(keytab != null, "Keytab must be specified when principal is specified.")
+ logInfo("Attempting to login to the Kerberos" +
+ s" using principal: $principal and keytab: $keytab")
+ val f = new File(keytab)
+ // Generate a file name that can be used for the keytab file, that does not conflict
+ // with any user file.
+ val keytabFileName = f.getName + "-" + UUID.randomUUID().toString
+ sparkConf.set(KEYTAB.key, keytabFileName)
+ sparkConf.set(PRINCIPAL.key, principal)
+ }
+ // Defensive copy of the credentials
+ credentials = new Credentials(UserGroupInformation.getCurrentUser.getCredentials)
+ }
+
+ /**
+ * Report the state of an application until it has exited, either successfully or
+ * due to some failure, then return a pair of the yarn application state (FINISHED, FAILED,
+ * KILLED, or RUNNING) and the final application state (UNDEFINED, SUCCEEDED, FAILED,
+ * or KILLED).
+ *
+ * @param appId ID of the application to monitor.
+ * @param returnOnRunning Whether to also return the application state when it is RUNNING.
+ * @param logApplicationReport Whether to log details of the application report every iteration.
+ * @return A pair of the yarn application state and the final application state.
+ */
+ def monitorApplication(
+ appId: ApplicationId,
+ returnOnRunning: Boolean = false,
+ logApplicationReport: Boolean = true): (YarnApplicationState, FinalApplicationStatus) = {
+ val interval = sparkConf.get(REPORT_INTERVAL)
+ var lastState: YarnApplicationState = null
+ while (true) {
+ Thread.sleep(interval)
+ val report: ApplicationReport =
+ try {
+ getApplicationReport(appId)
+ } catch {
+ case e: ApplicationNotFoundException =>
+ logError(s"Application $appId not found.")
+ cleanupStagingDir(appId)
+ return (YarnApplicationState.KILLED, FinalApplicationStatus.KILLED)
+ case NonFatal(e) =>
+ logError(s"Failed to contact YARN for application $appId.", e)
+ // Don't necessarily clean up staging dir because status is unknown
+ return (YarnApplicationState.FAILED, FinalApplicationStatus.FAILED)
+ }
+ val state = report.getYarnApplicationState
+
+ if (logApplicationReport) {
+ logInfo(s"Application report for $appId (state: $state)")
+
+ // If DEBUG is enabled, log report details every iteration
+ // Otherwise, log them every time the application changes state
+ if (log.isDebugEnabled) {
+ logDebug(formatReportDetails(report))
+ } else if (lastState != state) {
+ logInfo(formatReportDetails(report))
+ }
+ }
+
+ if (lastState != state) {
+ state match {
+ case YarnApplicationState.RUNNING =>
+ reportLauncherState(SparkAppHandle.State.RUNNING)
+ case YarnApplicationState.FINISHED =>
+ report.getFinalApplicationStatus match {
+ case FinalApplicationStatus.FAILED =>
+ reportLauncherState(SparkAppHandle.State.FAILED)
+ case FinalApplicationStatus.KILLED =>
+ reportLauncherState(SparkAppHandle.State.KILLED)
+ case _ =>
+ reportLauncherState(SparkAppHandle.State.FINISHED)
+ }
+ case YarnApplicationState.FAILED =>
+ reportLauncherState(SparkAppHandle.State.FAILED)
+ case YarnApplicationState.KILLED =>
+ reportLauncherState(SparkAppHandle.State.KILLED)
+ case _ =>
+ }
+ }
+
+ if (state == YarnApplicationState.FINISHED ||
+ state == YarnApplicationState.FAILED ||
+ state == YarnApplicationState.KILLED) {
+ cleanupStagingDir(appId)
+ return (state, report.getFinalApplicationStatus)
+ }
+
+ if (returnOnRunning && state == YarnApplicationState.RUNNING) {
+ return (state, report.getFinalApplicationStatus)
+ }
+
+ lastState = state
+ }
+
+ // Never reached, but keeps compiler happy
+ throw new SparkException("While loop is depleted! This should never happen...")
+ }
+
+ private def formatReportDetails(report: ApplicationReport): String = {
+ val details = Seq[(String, String)](
+ ("client token", getClientToken(report)),
+ ("diagnostics", report.getDiagnostics),
+ ("ApplicationMaster host", report.getHost),
+ ("ApplicationMaster RPC port", report.getRpcPort.toString),
+ ("queue", report.getQueue),
+ ("start time", report.getStartTime.toString),
+ ("final status", report.getFinalApplicationStatus.toString),
+ ("tracking URL", report.getTrackingUrl),
+ ("user", report.getUser)
+ )
+
+ // Use more loggable format if value is null or empty
+ details.map { case (k, v) =>
+ val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A")
+ s"\n\t $k: $newValue"
+ }.mkString("")
+ }
+
+ /**
+ * Submit an application to the ResourceManager.
+ * If set spark.yarn.submit.waitAppCompletion to true, it will stay alive
+ * reporting the application's status until the application has exited for any reason.
+ * Otherwise, the client process will exit after submission.
+ * If the application finishes with a failed, killed, or undefined status,
+ * throw an appropriate SparkException.
+ */
+ def run(): Unit = {
+ this.appId = submitApplication()
+ if (!launcherBackend.isConnected() && fireAndForget) {
+ val report = getApplicationReport(appId)
+ val state = report.getYarnApplicationState
+ logInfo(s"Application report for $appId (state: $state)")
+ logInfo(formatReportDetails(report))
+ if (state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) {
+ throw new SparkException(s"Application $appId finished with status: $state")
+ }
+ } else {
+ val (yarnApplicationState, finalApplicationStatus) = monitorApplication(appId)
+ if (yarnApplicationState == YarnApplicationState.FAILED ||
+ finalApplicationStatus == FinalApplicationStatus.FAILED) {
+ throw new SparkException(s"Application $appId finished with failed status")
+ }
+ if (yarnApplicationState == YarnApplicationState.KILLED ||
+ finalApplicationStatus == FinalApplicationStatus.KILLED) {
+ throw new SparkException(s"Application $appId is killed")
+ }
+ if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) {
+ throw new SparkException(s"The final status of application $appId is undefined")
+ }
+ }
+ }
+
+ private def findPySparkArchives(): Seq[String] = {
+ sys.env.get("PYSPARK_ARCHIVES_PATH")
+ .map(_.split(",").toSeq)
+ .getOrElse {
+ val pyLibPath = Seq(sys.env("SPARK_HOME"), "python", "lib").mkString(File.separator)
+ val pyArchivesFile = new File(pyLibPath, "pyspark.zip")
+ require(pyArchivesFile.exists(),
+ s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.")
+ val py4jFile = new File(pyLibPath, "py4j-0.10.4-src.zip")
+ require(py4jFile.exists(),
+ s"$py4jFile not found; cannot run pyspark application in YARN mode.")
+ Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath())
+ }
+ }
+
+}
+
+private object Client extends Logging {
+
+ def main(argStrings: Array[String]) {
+ if (!sys.props.contains("SPARK_SUBMIT")) {
+ logWarning("WARNING: This client is deprecated and will be removed in a " +
+ "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"")
+ }
+
+ // Set an env variable indicating we are running in YARN mode.
+ // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes
+ System.setProperty("SPARK_YARN_MODE", "true")
+ val sparkConf = new SparkConf
+ // SparkSubmit would use yarn cache to distribute files & jars in yarn mode,
+ // so remove them from sparkConf here for yarn mode.
+ sparkConf.remove("spark.jars")
+ sparkConf.remove("spark.files")
+ val args = new ClientArguments(argStrings)
+ new Client(args, sparkConf).run()
+ }
+
+ // Alias for the user jar
+ val APP_JAR_NAME: String = "__app__.jar"
+
+ // URI scheme that identifies local resources
+ val LOCAL_SCHEME = "local"
+
+ // Staging directory for any temporary jars or files
+ val SPARK_STAGING: String = ".sparkStaging"
+
+
+ // Staging directory is private! -> rwx--------
+ val STAGING_DIR_PERMISSION: FsPermission =
+ FsPermission.createImmutable(Integer.parseInt("700", 8).toShort)
+
+ // App files are world-wide readable and owner writable -> rw-r--r--
+ val APP_FILE_PERMISSION: FsPermission =
+ FsPermission.createImmutable(Integer.parseInt("644", 8).toShort)
+
+ // Distribution-defined classpath to add to processes
+ val ENV_DIST_CLASSPATH = "SPARK_DIST_CLASSPATH"
+
+ // Subdirectory where the user's Spark and Hadoop config files will be placed.
+ val LOCALIZED_CONF_DIR = "__spark_conf__"
+
+ // File containing the conf archive in the AM. See prepareLocalResources().
+ val LOCALIZED_CONF_ARCHIVE = LOCALIZED_CONF_DIR + ".zip"
+
+ // Name of the file in the conf archive containing Spark configuration.
+ val SPARK_CONF_FILE = "__spark_conf__.properties"
+
+ // Subdirectory where the user's python files (not archives) will be placed.
+ val LOCALIZED_PYTHON_DIR = "__pyfiles__"
+
+ // Subdirectory where Spark libraries will be placed.
+ val LOCALIZED_LIB_DIR = "__spark_libs__"
+
+ /**
+ * Return the path to the given application's staging directory.
+ */
+ private def getAppStagingDir(appId: ApplicationId): String = {
+ buildPath(SPARK_STAGING, appId.toString())
+ }
+
+ /**
+ * Populate the classpath entry in the given environment map with any application
+ * classpath specified through the Hadoop and Yarn configurations.
+ */
+ private[yarn] def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String])
+ : Unit = {
+ val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf)
+ for (c <- classPathElementsToAdd.flatten) {
+ YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, c.trim)
+ }
+ }
+
+ private def getYarnAppClasspath(conf: Configuration): Option[Seq[String]] =
+ Option(conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) match {
+ case Some(s) => Some(s.toSeq)
+ case None => getDefaultYarnApplicationClasspath
+ }
+
+ private def getMRAppClasspath(conf: Configuration): Option[Seq[String]] =
+ Option(conf.getStrings("mapreduce.application.classpath")) match {
+ case Some(s) => Some(s.toSeq)
+ case None => getDefaultMRApplicationClasspath
+ }
+
+ private[yarn] def getDefaultYarnApplicationClasspath: Option[Seq[String]] = {
+ val triedDefault = Try[Seq[String]] {
+ val field = classOf[YarnConfiguration].getField("DEFAULT_YARN_APPLICATION_CLASSPATH")
+ val value = field.get(null).asInstanceOf[Array[String]]
+ value.toSeq
+ } recoverWith {
+ case e: NoSuchFieldException => Success(Seq.empty[String])
+ }
+
+ triedDefault match {
+ case f: Failure[_] =>
+ logError("Unable to obtain the default YARN Application classpath.", f.exception)
+ case s: Success[Seq[String]] =>
+ logDebug(s"Using the default YARN application classpath: ${s.get.mkString(",")}")
+ }
+
+ triedDefault.toOption
+ }
+
+ private[yarn] def getDefaultMRApplicationClasspath: Option[Seq[String]] = {
+ val triedDefault = Try[Seq[String]] {
+ val field = classOf[MRJobConfig].getField("DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH")
+ StringUtils.getStrings(field.get(null).asInstanceOf[String]).toSeq
+ } recoverWith {
+ case e: NoSuchFieldException => Success(Seq.empty[String])
+ }
+
+ triedDefault match {
+ case f: Failure[_] =>
+ logError("Unable to obtain the default MR Application classpath.", f.exception)
+ case s: Success[Seq[String]] =>
+ logDebug(s"Using the default MR application classpath: ${s.get.mkString(",")}")
+ }
+
+ triedDefault.toOption
+ }
+
+ /**
+ * Populate the classpath entry in the given environment map.
+ *
+ * User jars are generally not added to the JVM's system classpath; those are handled by the AM
+ * and executor backend. When the deprecated `spark.yarn.user.classpath.first` is used, user jars
+ * are included in the system classpath, though. The extra class path and other uploaded files are
+ * always made available through the system class path.
+ *
+ * @param args Client arguments (when starting the AM) or null (when starting executors).
+ */
+ private[yarn] def populateClasspath(
+ args: ClientArguments,
+ conf: Configuration,
+ sparkConf: SparkConf,
+ env: HashMap[String, String],
+ extraClassPath: Option[String] = None): Unit = {
+ extraClassPath.foreach { cp =>
+ addClasspathEntry(getClusterPath(sparkConf, cp), env)
+ }
+
+ addClasspathEntry(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), env)
+
+ addClasspathEntry(
+ YarnSparkHadoopUtil.expandEnvironment(Environment.PWD) + Path.SEPARATOR +
+ LOCALIZED_CONF_DIR, env)
+
+ if (sparkConf.get(USER_CLASS_PATH_FIRST)) {
+ // in order to properly add the app jar when user classpath is first
+ // we have to do the mainJar separate in order to send the right thing
+ // into addFileToClasspath
+ val mainJar =
+ if (args != null) {
+ getMainJarUri(Option(args.userJar))
+ } else {
+ getMainJarUri(sparkConf.get(APP_JAR))
+ }
+ mainJar.foreach(addFileToClasspath(sparkConf, conf, _, APP_JAR_NAME, env))
+
+ val secondaryJars =
+ if (args != null) {
+ getSecondaryJarUris(Option(sparkConf.get(JARS_TO_DISTRIBUTE)))
+ } else {
+ getSecondaryJarUris(sparkConf.get(SECONDARY_JARS))
+ }
+ secondaryJars.foreach { x =>
+ addFileToClasspath(sparkConf, conf, x, null, env)
+ }
+ }
+
+ // Add the Spark jars to the classpath, depending on how they were distributed.
+ addClasspathEntry(buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+ LOCALIZED_LIB_DIR, "*"), env)
+ if (!sparkConf.get(SPARK_ARCHIVE).isDefined) {
+ sparkConf.get(SPARK_JARS).foreach { jars =>
+ jars.filter(isLocalUri).foreach { jar =>
+ addClasspathEntry(getClusterPath(sparkConf, jar), env)
+ }
+ }
+ }
+
+ populateHadoopClasspath(conf, env)
+ sys.env.get(ENV_DIST_CLASSPATH).foreach { cp =>
+ addClasspathEntry(getClusterPath(sparkConf, cp), env)
+ }
+ }
+
+ /**
+ * Returns a list of URIs representing the user classpath.
+ *
+ * @param conf Spark configuration.
+ */
+ def getUserClasspath(conf: SparkConf): Array[URI] = {
+ val mainUri = getMainJarUri(conf.get(APP_JAR))
+ val secondaryUris = getSecondaryJarUris(conf.get(SECONDARY_JARS))
+ (mainUri ++ secondaryUris).toArray
+ }
+
+ private def getMainJarUri(mainJar: Option[String]): Option[URI] = {
+ mainJar.flatMap { path =>
+ val uri = Utils.resolveURI(path)
+ if (uri.getScheme == LOCAL_SCHEME) Some(uri) else None
+ }.orElse(Some(new URI(APP_JAR_NAME)))
+ }
+
+ private def getSecondaryJarUris(secondaryJars: Option[Seq[String]]): Seq[URI] = {
+ secondaryJars.getOrElse(Nil).map(new URI(_))
+ }
+
+ /**
+ * Adds the given path to the classpath, handling "local:" URIs correctly.
+ *
+ * If an alternate name for the file is given, and it's not a "local:" file, the alternate
+ * name will be added to the classpath (relative to the job's work directory).
+ *
+ * If not a "local:" file and no alternate name, the linkName will be added to the classpath.
+ *
+ * @param conf Spark configuration.
+ * @param hadoopConf Hadoop configuration.
+ * @param uri URI to add to classpath (optional).
+ * @param fileName Alternate name for the file (optional).
+ * @param env Map holding the environment variables.
+ */
+ private def addFileToClasspath(
+ conf: SparkConf,
+ hadoopConf: Configuration,
+ uri: URI,
+ fileName: String,
+ env: HashMap[String, String]): Unit = {
+ if (uri != null && uri.getScheme == LOCAL_SCHEME) {
+ addClasspathEntry(getClusterPath(conf, uri.getPath), env)
+ } else if (fileName != null) {
+ addClasspathEntry(buildPath(
+ YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), fileName), env)
+ } else if (uri != null) {
+ val localPath = getQualifiedLocalPath(uri, hadoopConf)
+ val linkName = Option(uri.getFragment()).getOrElse(localPath.getName())
+ addClasspathEntry(buildPath(
+ YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), linkName), env)
+ }
+ }
+
+ /**
+ * Add the given path to the classpath entry of the given environment map.
+ * If the classpath is already set, this appends the new path to the existing classpath.
+ */
+ private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit =
+ YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path)
+
+ /**
+ * Returns the path to be sent to the NM for a path that is valid on the gateway.
+ *
+ * This method uses two configuration values:
+ *
+ * - spark.yarn.config.gatewayPath: a string that identifies a portion of the input path that may
+ * only be valid in the gateway node.
+ * - spark.yarn.config.replacementPath: a string with which to replace the gateway path. This may
+ * contain, for example, env variable references, which will be expanded by the NMs when
+ * starting containers.
+ *
+ * If either config is not available, the input path is returned.
+ */
+ def getClusterPath(conf: SparkConf, path: String): String = {
+ val localPath = conf.get(GATEWAY_ROOT_PATH)
+ val clusterPath = conf.get(REPLACEMENT_ROOT_PATH)
+ if (localPath != null && clusterPath != null) {
+ path.replace(localPath, clusterPath)
+ } else {
+ path
+ }
+ }
+
+ /**
+ * Return whether the two file systems are the same.
+ */
+ private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
+ val srcUri = srcFs.getUri()
+ val dstUri = destFs.getUri()
+ if (srcUri.getScheme() == null || srcUri.getScheme() != dstUri.getScheme()) {
+ return false
+ }
+
+ var srcHost = srcUri.getHost()
+ var dstHost = dstUri.getHost()
+
+ // In HA or when using viewfs, the host part of the URI may not actually be a host, but the
+ // name of the HDFS namespace. Those names won't resolve, so avoid even trying if they
+ // match.
+ if (srcHost != null && dstHost != null && srcHost != dstHost) {
+ try {
+ srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
+ dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
+ } catch {
+ case e: UnknownHostException =>
+ return false
+ }
+ }
+
+ Objects.equal(srcHost, dstHost) && srcUri.getPort() == dstUri.getPort()
+ }
+
+ /**
+ * Given a local URI, resolve it and return a qualified local path that corresponds to the URI.
+ * This is used for preparing local resources to be included in the container launch context.
+ */
+ private def getQualifiedLocalPath(localURI: URI, hadoopConf: Configuration): Path = {
+ val qualifiedURI =
+ if (localURI.getScheme == null) {
+ // If not specified, assume this is in the local filesystem to keep the behavior
+ // consistent with that of Hadoop
+ new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(localURI)).toString)
+ } else {
+ localURI
+ }
+ new Path(qualifiedURI)
+ }
+
+ /**
+ * Whether to consider jars provided by the user to have precedence over the Spark jars when
+ * loading user classes.
+ */
+ def isUserClassPathFirst(conf: SparkConf, isDriver: Boolean): Boolean = {
+ if (isDriver) {
+ conf.get(DRIVER_USER_CLASS_PATH_FIRST)
+ } else {
+ conf.get(EXECUTOR_USER_CLASS_PATH_FIRST)
+ }
+ }
+
+ /**
+ * Joins all the path components using Path.SEPARATOR.
+ */
+ def buildPath(components: String*): String = {
+ components.mkString(Path.SEPARATOR)
+ }
+
+ /** Returns whether the URI is a "local:" URI. */
+ def isLocalUri(uri: String): Boolean = {
+ uri.startsWith(s"$LOCAL_SCHEME:")
+ }
+
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
new file mode 100644
index 0000000000..61c027ec44
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -0,0 +1,86 @@
+/*
+ * 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
+
+// TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware !
+private[spark] class ClientArguments(args: Array[String]) {
+
+ var userJar: String = null
+ var userClass: String = null
+ var primaryPyFile: String = null
+ var primaryRFile: String = null
+ var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]()
+
+ parseArgs(args.toList)
+
+ private def parseArgs(inputArgs: List[String]): Unit = {
+ 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 ("--primary-py-file") :: value :: tail =>
+ primaryPyFile = value
+ args = tail
+
+ case ("--primary-r-file") :: value :: tail =>
+ primaryRFile = value
+ args = tail
+
+ case ("--arg") :: value :: tail =>
+ userArgs += value
+ args = tail
+
+ case Nil =>
+
+ case _ =>
+ throw new IllegalArgumentException(getUsageMessage(args))
+ }
+ }
+
+ if (primaryPyFile != null && primaryRFile != null) {
+ throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" +
+ " at the same time")
+ }
+ }
+
+ private def getUsageMessage(unknownParam: List[String] = null): String = {
+ val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else ""
+ message +
+ s"""
+ |Usage: org.apache.spark.deploy.yarn.Client [options]
+ |Options:
+ | --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster
+ | mode)
+ | --class CLASS_NAME Name of your application's main class (required)
+ | --primary-py-file A main Python file
+ | --primary-r-file A main R file
+ | --arg ARG Argument to be passed to your application's main class.
+ | Multiple invocations are possible, each will be passed in order.
+ """.stripMargin
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
new file mode 100644
index 0000000000..dcc2288dd1
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
@@ -0,0 +1,186 @@
+/*
+ * 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 scala.collection.mutable.{HashMap, ListBuffer, Map}
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
+import org.apache.hadoop.fs.permission.FsAction
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+
+private case class CacheEntry(
+ uri: URI,
+ size: Long,
+ modTime: Long,
+ visibility: LocalResourceVisibility,
+ resType: LocalResourceType)
+
+/** Client side methods to setup the Hadoop distributed cache */
+private[spark] class ClientDistributedCacheManager() extends Logging {
+
+ private val distCacheEntries = new ListBuffer[CacheEntry]()
+
+ /**
+ * Add a resource to the list of distributed cache resources. This list can
+ * be sent to the ApplicationMaster and possibly the executors 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 executors 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): Unit = {
+ val destStatus = fs.getFileStatus(destPath)
+ val amJarRsrc = Records.newRecord(classOf[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())
+ require(link != null && link.nonEmpty, "You must specify a valid link name.")
+ localResources(link) = amJarRsrc
+
+ if (!appMasterOnly) {
+ val uri = destPath.toUri()
+ val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link)
+ distCacheEntries += CacheEntry(pathURI, destStatus.getLen(), destStatus.getModificationTime(),
+ visibility, resourceType)
+ }
+ }
+
+ /**
+ * Writes down information about cached files needed in executors to the given configuration.
+ */
+ def updateConfiguration(conf: SparkConf): Unit = {
+ conf.set(CACHED_FILES, distCacheEntries.map(_.uri.toString))
+ conf.set(CACHED_FILES_SIZES, distCacheEntries.map(_.size))
+ conf.set(CACHED_FILES_TIMESTAMPS, distCacheEntries.map(_.modTime))
+ conf.set(CACHED_FILES_VISIBILITIES, distCacheEntries.map(_.visibility.name()))
+ conf.set(CACHED_FILES_TYPES, distCacheEntries.map(_.resType.name()))
+ }
+
+ /**
+ * Returns the local resource visibility depending on the cache file permissions
+ * @return LocalResourceVisibility
+ */
+ private[yarn] def getVisibility(
+ conf: Configuration,
+ uri: URI,
+ statCache: Map[URI, FileStatus]): LocalResourceVisibility = {
+ if (isPublic(conf, uri, statCache)) {
+ LocalResourceVisibility.PUBLIC
+ } else {
+ LocalResourceVisibility.PRIVATE
+ }
+ }
+
+ /**
+ * Returns a boolean to denote whether a cache file is visible to all (public)
+ * @return true if the path in the uri is visible to all, false otherwise
+ */
+ private 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
+ }
+ 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 hierarchy to the given path)
+ * @return true if all ancestors have the 'execute' permission set for all users
+ */
+ private 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()
+ }
+ true
+ }
+
+ /**
+ * Checks for a given path whether the Other permissions on it
+ * imply the permission in the passed FsAction
+ * @return true if the path in the uri is visible to all, false otherwise
+ */
+ private 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()
+ otherAction.implies(action)
+ }
+
+ /**
+ * 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.
+ * @return FileStatus
+ */
+ private[yarn] 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
+ }
+ stat
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
new file mode 100644
index 0000000000..868c2edc5a
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala
@@ -0,0 +1,266 @@
+/*
+ * 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.File
+import java.nio.ByteBuffer
+import java.util.Collections
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{HashMap, ListBuffer}
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.DataOutputBuffer
+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.client.api.NMClient
+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.{SecurityManager, SparkConf, SparkException}
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.launcher.YarnCommandBuilderUtils
+import org.apache.spark.network.util.JavaUtils
+import org.apache.spark.util.Utils
+
+private[yarn] class ExecutorRunnable(
+ container: Option[Container],
+ conf: YarnConfiguration,
+ sparkConf: SparkConf,
+ masterAddress: String,
+ executorId: String,
+ hostname: String,
+ executorMemory: Int,
+ executorCores: Int,
+ appId: String,
+ securityMgr: SecurityManager,
+ localResources: Map[String, LocalResource]) extends Logging {
+
+ var rpc: YarnRPC = YarnRPC.create(conf)
+ var nmClient: NMClient = _
+
+ def run(): Unit = {
+ logDebug("Starting Executor Container")
+ nmClient = NMClient.createNMClient()
+ nmClient.init(conf)
+ nmClient.start()
+ startContainer()
+ }
+
+ def launchContextDebugInfo(): String = {
+ val commands = prepareCommand()
+ val env = prepareEnvironment()
+
+ s"""
+ |===============================================================================
+ |YARN executor launch context:
+ | env:
+ |${Utils.redact(sparkConf, env.toSeq).map { case (k, v) => s" $k -> $v\n" }.mkString}
+ | command:
+ | ${commands.mkString(" \\ \n ")}
+ |
+ | resources:
+ |${localResources.map { case (k, v) => s" $k -> $v\n" }.mkString}
+ |===============================================================================""".stripMargin
+ }
+
+ def startContainer(): java.util.Map[String, ByteBuffer] = {
+ val ctx = Records.newRecord(classOf[ContainerLaunchContext])
+ .asInstanceOf[ContainerLaunchContext]
+ val env = prepareEnvironment().asJava
+
+ ctx.setLocalResources(localResources.asJava)
+ ctx.setEnvironment(env)
+
+ val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+ val dob = new DataOutputBuffer()
+ credentials.writeTokenStorageToStream(dob)
+ ctx.setTokens(ByteBuffer.wrap(dob.getData()))
+
+ val commands = prepareCommand()
+
+ ctx.setCommands(commands.asJava)
+ ctx.setApplicationACLs(
+ YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr).asJava)
+
+ // If external shuffle service is enabled, register with the Yarn shuffle service already
+ // started on the NodeManager and, if authentication is enabled, provide it with our secret
+ // key for fetching shuffle files later
+ if (sparkConf.get(SHUFFLE_SERVICE_ENABLED)) {
+ val secretString = securityMgr.getSecretKey()
+ val secretBytes =
+ if (secretString != null) {
+ // This conversion must match how the YarnShuffleService decodes our secret
+ JavaUtils.stringToBytes(secretString)
+ } else {
+ // Authentication is not enabled, so just provide dummy metadata
+ ByteBuffer.allocate(0)
+ }
+ ctx.setServiceData(Collections.singletonMap("spark_shuffle", secretBytes))
+ }
+
+ // Send the start request to the ContainerManager
+ try {
+ nmClient.startContainer(container.get, ctx)
+ } catch {
+ case ex: Exception =>
+ throw new SparkException(s"Exception while starting container ${container.get.getId}" +
+ s" on host $hostname", ex)
+ }
+ }
+
+ private def prepareCommand(): List[String] = {
+ // Extra options for the JVM
+ val javaOpts = ListBuffer[String]()
+
+ // Set the environment variable through a command prefix
+ // to append to the existing value of the variable
+ var prefixEnv: Option[String] = None
+
+ // Set the JVM memory
+ val executorMemoryString = executorMemory + "m"
+ javaOpts += "-Xmx" + executorMemoryString
+
+ // Set extra Java options for the executor, if defined
+ sparkConf.get(EXECUTOR_JAVA_OPTIONS).foreach { opts =>
+ javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
+ }
+ sys.env.get("SPARK_JAVA_OPTS").foreach { opts =>
+ javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell)
+ }
+ sparkConf.get(EXECUTOR_LIBRARY_PATH).foreach { p =>
+ prefixEnv = Some(Client.getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(Seq(p))))
+ }
+
+ javaOpts += "-Djava.io.tmpdir=" +
+ new Path(
+ YarnSparkHadoopUtil.expandEnvironment(Environment.PWD),
+ YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR
+ )
+
+ // Certain configs need to be passed here because they are needed before the Executor
+ // registers with the Scheduler and transfers the spark configs. Since the Executor backend
+ // uses RPC to connect to the scheduler, the RPC settings are needed as well as the
+ // authentication settings.
+ sparkConf.getAll
+ .filter { case (k, v) => SparkConf.isExecutorStartupConf(k) }
+ .foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") }
+
+ // 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.executor.extraJavaOptions, so we don't want to mess with it.
+ // In our expts, using (default) throughput collector has severe perf ramifications in
+ // multi-tenant 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
+ javaOpts += "-XX:+UseConcMarkSweepGC"
+ javaOpts += "-XX:+CMSIncrementalMode"
+ javaOpts += "-XX:+CMSIncrementalPacing"
+ javaOpts += "-XX:CMSIncrementalDutyCycleMin=0"
+ javaOpts += "-XX:CMSIncrementalDutyCycle=10"
+ }
+ */
+
+ // For log4j configuration to reference
+ javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR)
+ YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts)
+
+ val userClassPath = Client.getUserClasspath(sparkConf).flatMap { uri =>
+ val absPath =
+ if (new File(uri.getPath()).isAbsolute()) {
+ Client.getClusterPath(sparkConf, uri.getPath())
+ } else {
+ Client.buildPath(Environment.PWD.$(), uri.getPath())
+ }
+ Seq("--user-class-path", "file:" + absPath)
+ }.toSeq
+
+ YarnSparkHadoopUtil.addOutOfMemoryErrorArgument(javaOpts)
+ val commands = prefixEnv ++ Seq(
+ YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java",
+ "-server") ++
+ javaOpts ++
+ Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend",
+ "--driver-url", masterAddress,
+ "--executor-id", executorId,
+ "--hostname", hostname,
+ "--cores", executorCores.toString,
+ "--app-id", appId) ++
+ userClassPath ++
+ Seq(
+ s"1>${ApplicationConstants.LOG_DIR_EXPANSION_VAR}/stdout",
+ s"2>${ApplicationConstants.LOG_DIR_EXPANSION_VAR}/stderr")
+
+ // TODO: it would be nicer to just make sure there are no null commands here
+ commands.map(s => if (s == null) "null" else s).toList
+ }
+
+ private def prepareEnvironment(): HashMap[String, String] = {
+ val env = new HashMap[String, String]()
+ Client.populateClasspath(null, conf, sparkConf, env, sparkConf.get(EXECUTOR_CLASS_PATH))
+
+ sparkConf.getExecutorEnv.foreach { case (key, value) =>
+ // This assumes each executor environment variable set here is a path
+ // This is kept for backward compatibility and consistency with hadoop
+ YarnSparkHadoopUtil.addPathToEnvironment(env, key, value)
+ }
+
+ // Keep this for backwards compatibility but users should move to the config
+ sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs =>
+ YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs)
+ }
+
+ // lookup appropriate http scheme for container log urls
+ val yarnHttpPolicy = conf.get(
+ YarnConfiguration.YARN_HTTP_POLICY_KEY,
+ YarnConfiguration.YARN_HTTP_POLICY_DEFAULT
+ )
+ val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://"
+
+ // Add log urls
+ container.foreach { c =>
+ sys.env.get("SPARK_USER").foreach { user =>
+ val containerId = ConverterUtils.toString(c.getId)
+ val address = c.getNodeHttpAddress
+ val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user"
+
+ env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096"
+ env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096"
+ }
+ }
+
+ System.getenv().asScala.filterKeys(_.startsWith("SPARK"))
+ .foreach { case (k, v) => env(k) = v }
+ env
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
new file mode 100644
index 0000000000..8772e26f43
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
@@ -0,0 +1,224 @@
+/*
+ * 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, Set}
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.records.{ContainerId, Resource}
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.util.RackResolver
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.config._
+
+private[yarn] case class ContainerLocalityPreferences(nodes: Array[String], racks: Array[String])
+
+/**
+ * This strategy is calculating the optimal locality preferences of YARN containers by considering
+ * the node ratio of pending tasks, number of required cores/containers and and locality of current
+ * existing and pending allocated containers. The target of this algorithm is to maximize the number
+ * of tasks that would run locally.
+ *
+ * Consider a situation in which we have 20 tasks that require (host1, host2, host3)
+ * and 10 tasks that require (host1, host2, host4), besides each container has 2 cores
+ * and cpus per task is 1, so the required container number is 15,
+ * and host ratio is (host1: 30, host2: 30, host3: 20, host4: 10).
+ *
+ * 1. If requested container number (18) is more than the required container number (15):
+ *
+ * requests for 5 containers with nodes: (host1, host2, host3, host4)
+ * requests for 5 containers with nodes: (host1, host2, host3)
+ * requests for 5 containers with nodes: (host1, host2)
+ * requests for 3 containers with no locality preferences.
+ *
+ * The placement ratio is 3 : 3 : 2 : 1, and set the additional containers with no locality
+ * preferences.
+ *
+ * 2. If requested container number (10) is less than or equal to the required container number
+ * (15):
+ *
+ * requests for 4 containers with nodes: (host1, host2, host3, host4)
+ * requests for 3 containers with nodes: (host1, host2, host3)
+ * requests for 3 containers with nodes: (host1, host2)
+ *
+ * The placement ratio is 10 : 10 : 7 : 4, close to expected ratio (3 : 3 : 2 : 1)
+ *
+ * 3. If containers exist but none of them can match the requested localities,
+ * follow the method of 1 and 2.
+ *
+ * 4. If containers exist and some of them can match the requested localities.
+ * For example if we have 1 containers on each node (host1: 1, host2: 1: host3: 1, host4: 1),
+ * and the expected containers on each node would be (host1: 5, host2: 5, host3: 4, host4: 2),
+ * so the newly requested containers on each node would be updated to (host1: 4, host2: 4,
+ * host3: 3, host4: 1), 12 containers by total.
+ *
+ * 4.1 If requested container number (18) is more than newly required containers (12). Follow
+ * method 1 with updated ratio 4 : 4 : 3 : 1.
+ *
+ * 4.2 If request container number (10) is more than newly required containers (12). Follow
+ * method 2 with updated ratio 4 : 4 : 3 : 1.
+ *
+ * 5. If containers exist and existing localities can fully cover the requested localities.
+ * For example if we have 5 containers on each node (host1: 5, host2: 5, host3: 5, host4: 5),
+ * which could cover the current requested localities. This algorithm will allocate all the
+ * requested containers with no localities.
+ */
+private[yarn] class LocalityPreferredContainerPlacementStrategy(
+ val sparkConf: SparkConf,
+ val yarnConf: Configuration,
+ val resource: Resource) {
+
+ /**
+ * Calculate each container's node locality and rack locality
+ * @param numContainer number of containers to calculate
+ * @param numLocalityAwareTasks number of locality required tasks
+ * @param hostToLocalTaskCount a map to store the preferred hostname and possible task
+ * numbers running on it, used as hints for container allocation
+ * @param allocatedHostToContainersMap host to allocated containers map, used to calculate the
+ * expected locality preference by considering the existing
+ * containers
+ * @param localityMatchedPendingAllocations A sequence of pending container request which
+ * matches the localities of current required tasks.
+ * @return node localities and rack localities, each locality is an array of string,
+ * the length of localities is the same as number of containers
+ */
+ def localityOfRequestedContainers(
+ numContainer: Int,
+ numLocalityAwareTasks: Int,
+ hostToLocalTaskCount: Map[String, Int],
+ allocatedHostToContainersMap: HashMap[String, Set[ContainerId]],
+ localityMatchedPendingAllocations: Seq[ContainerRequest]
+ ): Array[ContainerLocalityPreferences] = {
+ val updatedHostToContainerCount = expectedHostToContainerCount(
+ numLocalityAwareTasks, hostToLocalTaskCount, allocatedHostToContainersMap,
+ localityMatchedPendingAllocations)
+ val updatedLocalityAwareContainerNum = updatedHostToContainerCount.values.sum
+
+ // The number of containers to allocate, divided into two groups, one with preferred locality,
+ // and the other without locality preference.
+ val requiredLocalityFreeContainerNum =
+ math.max(0, numContainer - updatedLocalityAwareContainerNum)
+ val requiredLocalityAwareContainerNum = numContainer - requiredLocalityFreeContainerNum
+
+ val containerLocalityPreferences = ArrayBuffer[ContainerLocalityPreferences]()
+ if (requiredLocalityFreeContainerNum > 0) {
+ for (i <- 0 until requiredLocalityFreeContainerNum) {
+ containerLocalityPreferences += ContainerLocalityPreferences(
+ null.asInstanceOf[Array[String]], null.asInstanceOf[Array[String]])
+ }
+ }
+
+ if (requiredLocalityAwareContainerNum > 0) {
+ val largestRatio = updatedHostToContainerCount.values.max
+ // Round the ratio of preferred locality to the number of locality required container
+ // number, which is used for locality preferred host calculating.
+ var preferredLocalityRatio = updatedHostToContainerCount.mapValues { ratio =>
+ val adjustedRatio = ratio.toDouble * requiredLocalityAwareContainerNum / largestRatio
+ adjustedRatio.ceil.toInt
+ }
+
+ for (i <- 0 until requiredLocalityAwareContainerNum) {
+ // Only filter out the ratio which is larger than 0, which means the current host can
+ // still be allocated with new container request.
+ val hosts = preferredLocalityRatio.filter(_._2 > 0).keys.toArray
+ val racks = hosts.map { h =>
+ RackResolver.resolve(yarnConf, h).getNetworkLocation
+ }.toSet
+ containerLocalityPreferences += ContainerLocalityPreferences(hosts, racks.toArray)
+
+ // Minus 1 each time when the host is used. When the current ratio is 0,
+ // which means all the required ratio is satisfied, this host will not be allocated again.
+ preferredLocalityRatio = preferredLocalityRatio.mapValues(_ - 1)
+ }
+ }
+
+ containerLocalityPreferences.toArray
+ }
+
+ /**
+ * Calculate the number of executors need to satisfy the given number of pending tasks.
+ */
+ private def numExecutorsPending(numTasksPending: Int): Int = {
+ val coresPerExecutor = resource.getVirtualCores
+ (numTasksPending * sparkConf.get(CPUS_PER_TASK) + coresPerExecutor - 1) / coresPerExecutor
+ }
+
+ /**
+ * Calculate the expected host to number of containers by considering with allocated containers.
+ * @param localityAwareTasks number of locality aware tasks
+ * @param hostToLocalTaskCount a map to store the preferred hostname and possible task
+ * numbers running on it, used as hints for container allocation
+ * @param allocatedHostToContainersMap host to allocated containers map, used to calculate the
+ * expected locality preference by considering the existing
+ * containers
+ * @param localityMatchedPendingAllocations A sequence of pending container request which
+ * matches the localities of current required tasks.
+ * @return a map with hostname as key and required number of containers on this host as value
+ */
+ private def expectedHostToContainerCount(
+ localityAwareTasks: Int,
+ hostToLocalTaskCount: Map[String, Int],
+ allocatedHostToContainersMap: HashMap[String, Set[ContainerId]],
+ localityMatchedPendingAllocations: Seq[ContainerRequest]
+ ): Map[String, Int] = {
+ val totalLocalTaskNum = hostToLocalTaskCount.values.sum
+ val pendingHostToContainersMap = pendingHostToContainerCount(localityMatchedPendingAllocations)
+
+ hostToLocalTaskCount.map { case (host, count) =>
+ val expectedCount =
+ count.toDouble * numExecutorsPending(localityAwareTasks) / totalLocalTaskNum
+ // Take the locality of pending containers into consideration
+ val existedCount = allocatedHostToContainersMap.get(host).map(_.size).getOrElse(0) +
+ pendingHostToContainersMap.getOrElse(host, 0.0)
+
+ // If existing container can not fully satisfy the expected number of container,
+ // the required container number is expected count minus existed count. Otherwise the
+ // required container number is 0.
+ (host, math.max(0, (expectedCount - existedCount).ceil.toInt))
+ }
+ }
+
+ /**
+ * According to the locality ratio and number of container requests, calculate the host to
+ * possible number of containers for pending allocated containers.
+ *
+ * If current locality ratio of hosts is: Host1 : Host2 : Host3 = 20 : 20 : 10,
+ * and pending container requests is 3, so the possible number of containers on
+ * Host1 : Host2 : Host3 will be 1.2 : 1.2 : 0.6.
+ * @param localityMatchedPendingAllocations A sequence of pending container request which
+ * matches the localities of current required tasks.
+ * @return a Map with hostname as key and possible number of containers on this host as value
+ */
+ private def pendingHostToContainerCount(
+ localityMatchedPendingAllocations: Seq[ContainerRequest]): Map[String, Double] = {
+ val pendingHostToContainerCount = new HashMap[String, Int]()
+ localityMatchedPendingAllocations.foreach { cr =>
+ cr.getNodes.asScala.foreach { n =>
+ val count = pendingHostToContainerCount.getOrElse(n, 0) + 1
+ pendingHostToContainerCount(n) = count
+ }
+ }
+
+ val possibleTotalContainerNum = pendingHostToContainerCount.values.sum
+ val localityMatchedPendingNum = localityMatchedPendingAllocations.size.toDouble
+ pendingHostToContainerCount.mapValues(_ * localityMatchedPendingNum / possibleTotalContainerNum)
+ .toMap
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
new file mode 100644
index 0000000000..0b66d1cf08
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
@@ -0,0 +1,727 @@
+/*
+ * 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.util.Collections
+import java.util.concurrent._
+import java.util.regex.Pattern
+
+import scala.collection.mutable
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue}
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.RackResolver
+import org.apache.log4j.{Level, Logger}
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkException}
+import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef}
+import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason}
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RetrieveLastAllocatedExecutorId
+import org.apache.spark.util.{Clock, SystemClock, ThreadUtils}
+
+/**
+ * YarnAllocator is charged with requesting containers from the YARN ResourceManager and deciding
+ * what to do with containers when YARN fulfills these requests.
+ *
+ * This class makes use of YARN's AMRMClient APIs. We interact with the AMRMClient in three ways:
+ * * Making our resource needs known, which updates local bookkeeping about containers requested.
+ * * Calling "allocate", which syncs our local container requests with the RM, and returns any
+ * containers that YARN has granted to us. This also functions as a heartbeat.
+ * * Processing the containers granted to us to possibly launch executors inside of them.
+ *
+ * The public methods of this class are thread-safe. All methods that mutate state are
+ * synchronized.
+ */
+private[yarn] class YarnAllocator(
+ driverUrl: String,
+ driverRef: RpcEndpointRef,
+ conf: YarnConfiguration,
+ sparkConf: SparkConf,
+ amClient: AMRMClient[ContainerRequest],
+ appAttemptId: ApplicationAttemptId,
+ securityMgr: SecurityManager,
+ localResources: Map[String, LocalResource])
+ extends Logging {
+
+ import YarnAllocator._
+
+ // RackResolver logs an INFO message whenever it resolves a rack, which is way too often.
+ if (Logger.getLogger(classOf[RackResolver]).getLevel == null) {
+ Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN)
+ }
+
+ // Visible for testing.
+ val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]
+ val allocatedContainerToHostMap = new HashMap[ContainerId, String]
+
+ // Containers that we no longer care about. We've either already told the RM to release them or
+ // will on the next heartbeat. Containers get removed from this map after the RM tells us they've
+ // completed.
+ private val releasedContainers = Collections.newSetFromMap[ContainerId](
+ new ConcurrentHashMap[ContainerId, java.lang.Boolean])
+
+ @volatile private var numExecutorsRunning = 0
+
+ /**
+ * Used to generate a unique ID per executor
+ *
+ * Init `executorIdCounter`. when AM restart, `executorIdCounter` will reset to 0. Then
+ * the id of new executor will start from 1, this will conflict with the executor has
+ * already created before. So, we should initialize the `executorIdCounter` by getting
+ * the max executorId from driver.
+ *
+ * And this situation of executorId conflict is just in yarn client mode, so this is an issue
+ * in yarn client mode. For more details, can check in jira.
+ *
+ * @see SPARK-12864
+ */
+ private var executorIdCounter: Int =
+ driverRef.askWithRetry[Int](RetrieveLastAllocatedExecutorId)
+
+ // Queue to store the timestamp of failed executors
+ private val failedExecutorsTimeStamps = new Queue[Long]()
+
+ private var clock: Clock = new SystemClock
+
+ private val executorFailuresValidityInterval =
+ sparkConf.get(EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS).getOrElse(-1L)
+
+ @volatile private var targetNumExecutors =
+ YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf)
+
+ // Executor loss reason requests that are pending - maps from executor ID for inquiry to a
+ // list of requesters that should be responded to once we find out why the given executor
+ // was lost.
+ private val pendingLossReasonRequests = new HashMap[String, mutable.Buffer[RpcCallContext]]
+
+ // Maintain loss reasons for already released executors, it will be added when executor loss
+ // reason is got from AM-RM call, and be removed after querying this loss reason.
+ private val releasedExecutorLossReasons = new HashMap[String, ExecutorLossReason]
+
+ // Keep track of which container is running which executor to remove the executors later
+ // Visible for testing.
+ private[yarn] val executorIdToContainer = new HashMap[String, Container]
+
+ private var numUnexpectedContainerRelease = 0L
+ private val containerIdToExecutorId = new HashMap[ContainerId, String]
+
+ // Executor memory in MB.
+ protected val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt
+ // Additional memory overhead.
+ protected val memoryOverhead: Int = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse(
+ math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)).toInt
+ // Number of cores per executor.
+ protected val executorCores = sparkConf.get(EXECUTOR_CORES)
+ // Resource capability requested for each executors
+ private[yarn] val resource = Resource.newInstance(executorMemory + memoryOverhead, executorCores)
+
+ private val launcherPool = ThreadUtils.newDaemonCachedThreadPool(
+ "ContainerLauncher", sparkConf.get(CONTAINER_LAUNCH_MAX_THREADS))
+
+ // For testing
+ private val launchContainers = sparkConf.getBoolean("spark.yarn.launchContainers", true)
+
+ private val labelExpression = sparkConf.get(EXECUTOR_NODE_LABEL_EXPRESSION)
+
+ // ContainerRequest constructor that can take a node label expression. We grab it through
+ // reflection because it's only available in later versions of YARN.
+ private val nodeLabelConstructor = labelExpression.flatMap { expr =>
+ try {
+ Some(classOf[ContainerRequest].getConstructor(classOf[Resource],
+ classOf[Array[String]], classOf[Array[String]], classOf[Priority], classOf[Boolean],
+ classOf[String]))
+ } catch {
+ case e: NoSuchMethodException =>
+ logWarning(s"Node label expression $expr will be ignored because YARN version on" +
+ " classpath does not support it.")
+ None
+ }
+ }
+
+ // A map to store preferred hostname and possible task numbers running on it.
+ private var hostToLocalTaskCounts: Map[String, Int] = Map.empty
+
+ // Number of tasks that have locality preferences in active stages
+ private var numLocalityAwareTasks: Int = 0
+
+ // A container placement strategy based on pending tasks' locality preference
+ private[yarn] val containerPlacementStrategy =
+ new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resource)
+
+ /**
+ * Use a different clock for YarnAllocator. This is mainly used for testing.
+ */
+ def setClock(newClock: Clock): Unit = {
+ clock = newClock
+ }
+
+ def getNumExecutorsRunning: Int = numExecutorsRunning
+
+ def getNumExecutorsFailed: Int = synchronized {
+ val endTime = clock.getTimeMillis()
+
+ while (executorFailuresValidityInterval > 0
+ && failedExecutorsTimeStamps.nonEmpty
+ && failedExecutorsTimeStamps.head < endTime - executorFailuresValidityInterval) {
+ failedExecutorsTimeStamps.dequeue()
+ }
+
+ failedExecutorsTimeStamps.size
+ }
+
+ /**
+ * A sequence of pending container requests that have not yet been fulfilled.
+ */
+ def getPendingAllocate: Seq[ContainerRequest] = getPendingAtLocation(ANY_HOST)
+
+ /**
+ * A sequence of pending container requests at the given location that have not yet been
+ * fulfilled.
+ */
+ private def getPendingAtLocation(location: String): Seq[ContainerRequest] = {
+ amClient.getMatchingRequests(RM_REQUEST_PRIORITY, location, resource).asScala
+ .flatMap(_.asScala)
+ .toSeq
+ }
+
+ /**
+ * Request as many executors from the ResourceManager as needed to reach the desired total. If
+ * the requested total is smaller than the current number of running executors, no executors will
+ * be killed.
+ * @param requestedTotal total number of containers requested
+ * @param localityAwareTasks number of locality aware tasks to be used as container placement hint
+ * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as
+ * container placement hint.
+ * @return Whether the new requested total is different than the old value.
+ */
+ def requestTotalExecutorsWithPreferredLocalities(
+ requestedTotal: Int,
+ localityAwareTasks: Int,
+ hostToLocalTaskCount: Map[String, Int]): Boolean = synchronized {
+ this.numLocalityAwareTasks = localityAwareTasks
+ this.hostToLocalTaskCounts = hostToLocalTaskCount
+
+ if (requestedTotal != targetNumExecutors) {
+ logInfo(s"Driver requested a total number of $requestedTotal executor(s).")
+ targetNumExecutors = requestedTotal
+ true
+ } else {
+ false
+ }
+ }
+
+ /**
+ * Request that the ResourceManager release the container running the specified executor.
+ */
+ def killExecutor(executorId: String): Unit = synchronized {
+ if (executorIdToContainer.contains(executorId)) {
+ val container = executorIdToContainer.get(executorId).get
+ internalReleaseContainer(container)
+ numExecutorsRunning -= 1
+ } else {
+ logWarning(s"Attempted to kill unknown executor $executorId!")
+ }
+ }
+
+ /**
+ * Request resources such that, if YARN gives us all we ask for, we'll have a number of containers
+ * equal to maxExecutors.
+ *
+ * Deal with any containers YARN has granted to us by possibly launching executors in them.
+ *
+ * This must be synchronized because variables read in this method are mutated by other methods.
+ */
+ def allocateResources(): Unit = synchronized {
+ updateResourceRequests()
+
+ val progressIndicator = 0.1f
+ // Poll the ResourceManager. This doubles as a heartbeat if there are no pending container
+ // requests.
+ val allocateResponse = amClient.allocate(progressIndicator)
+
+ val allocatedContainers = allocateResponse.getAllocatedContainers()
+
+ if (allocatedContainers.size > 0) {
+ logDebug("Allocated containers: %d. Current executor count: %d. Cluster resources: %s."
+ .format(
+ allocatedContainers.size,
+ numExecutorsRunning,
+ allocateResponse.getAvailableResources))
+
+ handleAllocatedContainers(allocatedContainers.asScala)
+ }
+
+ val completedContainers = allocateResponse.getCompletedContainersStatuses()
+ if (completedContainers.size > 0) {
+ logDebug("Completed %d containers".format(completedContainers.size))
+ processCompletedContainers(completedContainers.asScala)
+ logDebug("Finished processing %d completed containers. Current running executor count: %d."
+ .format(completedContainers.size, numExecutorsRunning))
+ }
+ }
+
+ /**
+ * Update the set of container requests that we will sync with the RM based on the number of
+ * executors we have currently running and our target number of executors.
+ *
+ * Visible for testing.
+ */
+ def updateResourceRequests(): Unit = {
+ val pendingAllocate = getPendingAllocate
+ val numPendingAllocate = pendingAllocate.size
+ val missing = targetNumExecutors - numPendingAllocate - numExecutorsRunning
+
+ if (missing > 0) {
+ logInfo(s"Will request $missing executor container(s), each with " +
+ s"${resource.getVirtualCores} core(s) and " +
+ s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)")
+
+ // Split the pending container request into three groups: locality matched list, locality
+ // unmatched list and non-locality list. Take the locality matched container request into
+ // consideration of container placement, treat as allocated containers.
+ // For locality unmatched and locality free container requests, cancel these container
+ // requests, since required locality preference has been changed, recalculating using
+ // container placement strategy.
+ val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality(
+ hostToLocalTaskCounts, pendingAllocate)
+
+ // cancel "stale" requests for locations that are no longer needed
+ staleRequests.foreach { stale =>
+ amClient.removeContainerRequest(stale)
+ }
+ val cancelledContainers = staleRequests.size
+ if (cancelledContainers > 0) {
+ logInfo(s"Canceled $cancelledContainers container request(s) (locality no longer needed)")
+ }
+
+ // consider the number of new containers and cancelled stale containers available
+ val availableContainers = missing + cancelledContainers
+
+ // to maximize locality, include requests with no locality preference that can be cancelled
+ val potentialContainers = availableContainers + anyHostRequests.size
+
+ val containerLocalityPreferences = containerPlacementStrategy.localityOfRequestedContainers(
+ potentialContainers, numLocalityAwareTasks, hostToLocalTaskCounts,
+ allocatedHostToContainersMap, localRequests)
+
+ val newLocalityRequests = new mutable.ArrayBuffer[ContainerRequest]
+ containerLocalityPreferences.foreach {
+ case ContainerLocalityPreferences(nodes, racks) if nodes != null =>
+ newLocalityRequests += createContainerRequest(resource, nodes, racks)
+ case _ =>
+ }
+
+ if (availableContainers >= newLocalityRequests.size) {
+ // more containers are available than needed for locality, fill in requests for any host
+ for (i <- 0 until (availableContainers - newLocalityRequests.size)) {
+ newLocalityRequests += createContainerRequest(resource, null, null)
+ }
+ } else {
+ val numToCancel = newLocalityRequests.size - availableContainers
+ // cancel some requests without locality preferences to schedule more local containers
+ anyHostRequests.slice(0, numToCancel).foreach { nonLocal =>
+ amClient.removeContainerRequest(nonLocal)
+ }
+ if (numToCancel > 0) {
+ logInfo(s"Canceled $numToCancel unlocalized container requests to resubmit with locality")
+ }
+ }
+
+ newLocalityRequests.foreach { request =>
+ amClient.addContainerRequest(request)
+ }
+
+ if (log.isInfoEnabled()) {
+ val (localized, anyHost) = newLocalityRequests.partition(_.getNodes() != null)
+ if (anyHost.nonEmpty) {
+ logInfo(s"Submitted ${anyHost.size} unlocalized container requests.")
+ }
+ localized.foreach { request =>
+ logInfo(s"Submitted container request for host ${hostStr(request)}.")
+ }
+ }
+ } else if (numPendingAllocate > 0 && missing < 0) {
+ val numToCancel = math.min(numPendingAllocate, -missing)
+ logInfo(s"Canceling requests for $numToCancel executor container(s) to have a new desired " +
+ s"total $targetNumExecutors executors.")
+
+ val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource)
+ if (!matchingRequests.isEmpty) {
+ matchingRequests.iterator().next().asScala
+ .take(numToCancel).foreach(amClient.removeContainerRequest)
+ } else {
+ logWarning("Expected to find pending requests, but found none.")
+ }
+ }
+ }
+
+ private def hostStr(request: ContainerRequest): String = {
+ Option(request.getNodes) match {
+ case Some(nodes) => nodes.asScala.mkString(",")
+ case None => "Any"
+ }
+ }
+
+ /**
+ * Creates a container request, handling the reflection required to use YARN features that were
+ * added in recent versions.
+ */
+ private def createContainerRequest(
+ resource: Resource,
+ nodes: Array[String],
+ racks: Array[String]): ContainerRequest = {
+ nodeLabelConstructor.map { constructor =>
+ constructor.newInstance(resource, nodes, racks, RM_REQUEST_PRIORITY, true: java.lang.Boolean,
+ labelExpression.orNull)
+ }.getOrElse(new ContainerRequest(resource, nodes, racks, RM_REQUEST_PRIORITY))
+ }
+
+ /**
+ * Handle containers granted by the RM by launching executors on them.
+ *
+ * Due to the way the YARN allocation protocol works, certain healthy race conditions can result
+ * in YARN granting containers that we no longer need. In this case, we release them.
+ *
+ * Visible for testing.
+ */
+ def handleAllocatedContainers(allocatedContainers: Seq[Container]): Unit = {
+ val containersToUse = new ArrayBuffer[Container](allocatedContainers.size)
+
+ // Match incoming requests by host
+ val remainingAfterHostMatches = new ArrayBuffer[Container]
+ for (allocatedContainer <- allocatedContainers) {
+ matchContainerToRequest(allocatedContainer, allocatedContainer.getNodeId.getHost,
+ containersToUse, remainingAfterHostMatches)
+ }
+
+ // Match remaining by rack
+ val remainingAfterRackMatches = new ArrayBuffer[Container]
+ for (allocatedContainer <- remainingAfterHostMatches) {
+ val rack = RackResolver.resolve(conf, allocatedContainer.getNodeId.getHost).getNetworkLocation
+ matchContainerToRequest(allocatedContainer, rack, containersToUse,
+ remainingAfterRackMatches)
+ }
+
+ // Assign remaining that are neither node-local nor rack-local
+ val remainingAfterOffRackMatches = new ArrayBuffer[Container]
+ for (allocatedContainer <- remainingAfterRackMatches) {
+ matchContainerToRequest(allocatedContainer, ANY_HOST, containersToUse,
+ remainingAfterOffRackMatches)
+ }
+
+ if (!remainingAfterOffRackMatches.isEmpty) {
+ logDebug(s"Releasing ${remainingAfterOffRackMatches.size} unneeded containers that were " +
+ s"allocated to us")
+ for (container <- remainingAfterOffRackMatches) {
+ internalReleaseContainer(container)
+ }
+ }
+
+ runAllocatedContainers(containersToUse)
+
+ logInfo("Received %d containers from YARN, launching executors on %d of them."
+ .format(allocatedContainers.size, containersToUse.size))
+ }
+
+ /**
+ * Looks for requests for the given location that match the given container allocation. If it
+ * finds one, removes the request so that it won't be submitted again. Places the container into
+ * containersToUse or remaining.
+ *
+ * @param allocatedContainer container that was given to us by YARN
+ * @param location resource name, either a node, rack, or *
+ * @param containersToUse list of containers that will be used
+ * @param remaining list of containers that will not be used
+ */
+ private def matchContainerToRequest(
+ allocatedContainer: Container,
+ location: String,
+ containersToUse: ArrayBuffer[Container],
+ remaining: ArrayBuffer[Container]): Unit = {
+ // SPARK-6050: certain Yarn configurations return a virtual core count that doesn't match the
+ // request; for example, capacity scheduler + DefaultResourceCalculator. So match on requested
+ // memory, but use the asked vcore count for matching, effectively disabling matching on vcore
+ // count.
+ val matchingResource = Resource.newInstance(allocatedContainer.getResource.getMemory,
+ resource.getVirtualCores)
+ val matchingRequests = amClient.getMatchingRequests(allocatedContainer.getPriority, location,
+ matchingResource)
+
+ // Match the allocation to a request
+ if (!matchingRequests.isEmpty) {
+ val containerRequest = matchingRequests.get(0).iterator.next
+ amClient.removeContainerRequest(containerRequest)
+ containersToUse += allocatedContainer
+ } else {
+ remaining += allocatedContainer
+ }
+ }
+
+ /**
+ * Launches executors in the allocated containers.
+ */
+ private def runAllocatedContainers(containersToUse: ArrayBuffer[Container]): Unit = {
+ for (container <- containersToUse) {
+ executorIdCounter += 1
+ val executorHostname = container.getNodeId.getHost
+ val containerId = container.getId
+ val executorId = executorIdCounter.toString
+ assert(container.getResource.getMemory >= resource.getMemory)
+ logInfo(s"Launching container $containerId on host $executorHostname")
+
+ def updateInternalState(): Unit = synchronized {
+ numExecutorsRunning += 1
+ executorIdToContainer(executorId) = container
+ containerIdToExecutorId(container.getId) = executorId
+
+ val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname,
+ new HashSet[ContainerId])
+ containerSet += containerId
+ allocatedContainerToHostMap.put(containerId, executorHostname)
+ }
+
+ if (numExecutorsRunning < targetNumExecutors) {
+ if (launchContainers) {
+ launcherPool.execute(new Runnable {
+ override def run(): Unit = {
+ try {
+ new ExecutorRunnable(
+ Some(container),
+ conf,
+ sparkConf,
+ driverUrl,
+ executorId,
+ executorHostname,
+ executorMemory,
+ executorCores,
+ appAttemptId.getApplicationId.toString,
+ securityMgr,
+ localResources
+ ).run()
+ updateInternalState()
+ } catch {
+ case NonFatal(e) =>
+ logError(s"Failed to launch executor $executorId on container $containerId", e)
+ // Assigned container should be released immediately to avoid unnecessary resource
+ // occupation.
+ amClient.releaseAssignedContainer(containerId)
+ }
+ }
+ })
+ } else {
+ // For test only
+ updateInternalState()
+ }
+ } else {
+ logInfo(("Skip launching executorRunnable as runnning Excecutors count: %d " +
+ "reached target Executors count: %d.").format(numExecutorsRunning, targetNumExecutors))
+ }
+ }
+ }
+
+ // Visible for testing.
+ private[yarn] def processCompletedContainers(completedContainers: Seq[ContainerStatus]): Unit = {
+ for (completedContainer <- completedContainers) {
+ val containerId = completedContainer.getContainerId
+ val alreadyReleased = releasedContainers.remove(containerId)
+ val hostOpt = allocatedContainerToHostMap.get(containerId)
+ val onHostStr = hostOpt.map(host => s" on host: $host").getOrElse("")
+ val exitReason = if (!alreadyReleased) {
+ // Decrement the number of executors running. The next iteration of
+ // the ApplicationMaster's reporting thread will take care of allocating.
+ numExecutorsRunning -= 1
+ logInfo("Completed container %s%s (state: %s, exit status: %s)".format(
+ containerId,
+ onHostStr,
+ 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.
+ val exitStatus = completedContainer.getExitStatus
+ val (exitCausedByApp, containerExitReason) = exitStatus match {
+ case ContainerExitStatus.SUCCESS =>
+ (false, s"Executor for container $containerId exited because of a YARN event (e.g., " +
+ "pre-emption) and not because of an error in the running job.")
+ case ContainerExitStatus.PREEMPTED =>
+ // Preemption is not the fault of the running tasks, since YARN preempts containers
+ // merely to do resource sharing, and tasks that fail due to preempted executors could
+ // just as easily finish on any other executor. See SPARK-8167.
+ (false, s"Container ${containerId}${onHostStr} was preempted.")
+ // Should probably still count memory exceeded exit codes towards task failures
+ case VMEM_EXCEEDED_EXIT_CODE =>
+ (true, memLimitExceededLogMessage(
+ completedContainer.getDiagnostics,
+ VMEM_EXCEEDED_PATTERN))
+ case PMEM_EXCEEDED_EXIT_CODE =>
+ (true, memLimitExceededLogMessage(
+ completedContainer.getDiagnostics,
+ PMEM_EXCEEDED_PATTERN))
+ case _ =>
+ // Enqueue the timestamp of failed executor
+ failedExecutorsTimeStamps.enqueue(clock.getTimeMillis())
+ (true, "Container marked as failed: " + containerId + onHostStr +
+ ". Exit status: " + completedContainer.getExitStatus +
+ ". Diagnostics: " + completedContainer.getDiagnostics)
+
+ }
+ if (exitCausedByApp) {
+ logWarning(containerExitReason)
+ } else {
+ logInfo(containerExitReason)
+ }
+ ExecutorExited(exitStatus, exitCausedByApp, containerExitReason)
+ } else {
+ // If we have already released this container, then it must mean
+ // that the driver has explicitly requested it to be killed
+ ExecutorExited(completedContainer.getExitStatus, exitCausedByApp = false,
+ s"Container $containerId exited from explicit termination request.")
+ }
+
+ for {
+ host <- hostOpt
+ containerSet <- allocatedHostToContainersMap.get(host)
+ } {
+ containerSet.remove(containerId)
+ if (containerSet.isEmpty) {
+ allocatedHostToContainersMap.remove(host)
+ } else {
+ allocatedHostToContainersMap.update(host, containerSet)
+ }
+
+ allocatedContainerToHostMap.remove(containerId)
+ }
+
+ containerIdToExecutorId.remove(containerId).foreach { eid =>
+ executorIdToContainer.remove(eid)
+ pendingLossReasonRequests.remove(eid) match {
+ case Some(pendingRequests) =>
+ // Notify application of executor loss reasons so it can decide whether it should abort
+ pendingRequests.foreach(_.reply(exitReason))
+
+ case None =>
+ // We cannot find executor for pending reasons. This is because completed container
+ // is processed before querying pending result. We should store it for later query.
+ // This is usually happened when explicitly killing a container, the result will be
+ // returned in one AM-RM communication. So query RPC will be later than this completed
+ // container process.
+ releasedExecutorLossReasons.put(eid, exitReason)
+ }
+ if (!alreadyReleased) {
+ // The executor could have gone away (like no route to host, node failure, etc)
+ // Notify backend about the failure of the executor
+ numUnexpectedContainerRelease += 1
+ driverRef.send(RemoveExecutor(eid, exitReason))
+ }
+ }
+ }
+ }
+
+ /**
+ * Register that some RpcCallContext has asked the AM why the executor was lost. Note that
+ * we can only find the loss reason to send back in the next call to allocateResources().
+ */
+ private[yarn] def enqueueGetLossReasonRequest(
+ eid: String,
+ context: RpcCallContext): Unit = synchronized {
+ if (executorIdToContainer.contains(eid)) {
+ pendingLossReasonRequests
+ .getOrElseUpdate(eid, new ArrayBuffer[RpcCallContext]) += context
+ } else if (releasedExecutorLossReasons.contains(eid)) {
+ // Executor is already released explicitly before getting the loss reason, so directly send
+ // the pre-stored lost reason
+ context.reply(releasedExecutorLossReasons.remove(eid).get)
+ } else {
+ logWarning(s"Tried to get the loss reason for non-existent executor $eid")
+ context.sendFailure(
+ new SparkException(s"Fail to find loss reason for non-existent executor $eid"))
+ }
+ }
+
+ private def internalReleaseContainer(container: Container): Unit = {
+ releasedContainers.add(container.getId())
+ amClient.releaseAssignedContainer(container.getId())
+ }
+
+ private[yarn] def getNumUnexpectedContainerRelease = numUnexpectedContainerRelease
+
+ private[yarn] def getNumPendingLossReasonRequests: Int = synchronized {
+ pendingLossReasonRequests.size
+ }
+
+ /**
+ * Split the pending container requests into 3 groups based on current localities of pending
+ * tasks.
+ * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as
+ * container placement hint.
+ * @param pendingAllocations A sequence of pending allocation container request.
+ * @return A tuple of 3 sequences, first is a sequence of locality matched container
+ * requests, second is a sequence of locality unmatched container requests, and third is a
+ * sequence of locality free container requests.
+ */
+ private def splitPendingAllocationsByLocality(
+ hostToLocalTaskCount: Map[String, Int],
+ pendingAllocations: Seq[ContainerRequest]
+ ): (Seq[ContainerRequest], Seq[ContainerRequest], Seq[ContainerRequest]) = {
+ val localityMatched = ArrayBuffer[ContainerRequest]()
+ val localityUnMatched = ArrayBuffer[ContainerRequest]()
+ val localityFree = ArrayBuffer[ContainerRequest]()
+
+ val preferredHosts = hostToLocalTaskCount.keySet
+ pendingAllocations.foreach { cr =>
+ val nodes = cr.getNodes
+ if (nodes == null) {
+ localityFree += cr
+ } else if (nodes.asScala.toSet.intersect(preferredHosts).nonEmpty) {
+ localityMatched += cr
+ } else {
+ localityUnMatched += cr
+ }
+ }
+
+ (localityMatched.toSeq, localityUnMatched.toSeq, localityFree.toSeq)
+ }
+
+}
+
+private object YarnAllocator {
+ val MEM_REGEX = "[0-9.]+ [KMG]B"
+ val PMEM_EXCEEDED_PATTERN =
+ Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used")
+ val VMEM_EXCEEDED_PATTERN =
+ Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used")
+ val VMEM_EXCEEDED_EXIT_CODE = -103
+ val PMEM_EXCEEDED_EXIT_CODE = -104
+
+ def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = {
+ val matcher = pattern.matcher(diagnostics)
+ val diag = if (matcher.find()) " " + matcher.group() + "." else ""
+ ("Container killed by YARN for exceeding memory limits." + diag
+ + " Consider boosting spark.yarn.executor.memoryOverhead.")
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
new file mode 100644
index 0000000000..53df11eb66
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
@@ -0,0 +1,135 @@
+/*
+ * 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.util.{List => JList}
+
+import scala.collection.JavaConverters._
+import scala.util.Try
+
+import org.apache.hadoop.conf.Configuration
+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.webapp.util.WebAppUtils
+
+import org.apache.spark.{SecurityManager, SparkConf}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.rpc.RpcEndpointRef
+import org.apache.spark.util.Utils
+
+/**
+ * Handles registering and unregistering the application with the YARN ResourceManager.
+ */
+private[spark] class YarnRMClient extends Logging {
+
+ private var amClient: AMRMClient[ContainerRequest] = _
+ private var uiHistoryAddress: String = _
+ private var registered: Boolean = false
+
+ /**
+ * Registers the application master with the RM.
+ *
+ * @param conf The Yarn configuration.
+ * @param sparkConf The Spark configuration.
+ * @param uiAddress Address of the SparkUI.
+ * @param uiHistoryAddress Address of the application on the History Server.
+ * @param securityMgr The security manager.
+ * @param localResources Map with information about files distributed via YARN's cache.
+ */
+ def register(
+ driverUrl: String,
+ driverRef: RpcEndpointRef,
+ conf: YarnConfiguration,
+ sparkConf: SparkConf,
+ uiAddress: String,
+ uiHistoryAddress: String,
+ securityMgr: SecurityManager,
+ localResources: Map[String, LocalResource]
+ ): YarnAllocator = {
+ amClient = AMRMClient.createAMRMClient()
+ amClient.init(conf)
+ amClient.start()
+ this.uiHistoryAddress = uiHistoryAddress
+
+ logInfo("Registering the ApplicationMaster")
+ synchronized {
+ amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
+ registered = true
+ }
+ new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), securityMgr,
+ localResources)
+ }
+
+ /**
+ * Unregister the AM. Guaranteed to only be called once.
+ *
+ * @param status The final status of the AM.
+ * @param diagnostics Diagnostics message to include in the final status.
+ */
+ def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit = synchronized {
+ if (registered) {
+ amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress)
+ }
+ }
+
+ /** Returns the attempt ID. */
+ def getAttemptId(): ApplicationAttemptId = {
+ YarnSparkHadoopUtil.get.getContainerId.getApplicationAttemptId()
+ }
+
+ /** Returns the configuration for the AmIpFilter to add to the Spark UI. */
+ def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] = {
+ // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2,
+ // so not all stable releases have it.
+ val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration])
+ .invoke(null, conf).asInstanceOf[String]).getOrElse("http://")
+
+ // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses.
+ try {
+ val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter",
+ classOf[Configuration])
+ val proxies = method.invoke(null, conf).asInstanceOf[JList[String]]
+ val hosts = proxies.asScala.map { proxy => proxy.split(":")(0) }
+ val uriBases = proxies.asScala.map { proxy => prefix + proxy + proxyBase }
+ Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(","))
+ } catch {
+ case e: NoSuchMethodException =>
+ val proxy = WebAppUtils.getProxyHostAndPort(conf)
+ val parts = proxy.split(":")
+ val uriBase = prefix + proxy + proxyBase
+ Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase)
+ }
+ }
+
+ /** Returns the maximum number of attempts to register the AM. */
+ def getMaxRegAttempts(sparkConf: SparkConf, yarnConf: YarnConfiguration): Int = {
+ val sparkMaxAttempts = sparkConf.get(MAX_APP_ATTEMPTS).map(_.toInt)
+ val yarnMaxAttempts = yarnConf.getInt(
+ YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)
+ val retval: Int = sparkMaxAttempts match {
+ case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
+ case None => yarnMaxAttempts
+ }
+
+ retval
+ }
+
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
new file mode 100644
index 0000000000..cc53b1b06e
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -0,0 +1,317 @@
+/*
+ * 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.File
+import java.nio.charset.StandardCharsets.UTF_8
+import java.util.regex.Matcher
+import java.util.regex.Pattern
+
+import scala.collection.mutable.{HashMap, ListBuffer}
+import scala.util.Try
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api.ApplicationConstants
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.records.{ApplicationAccessType, ContainerId, Priority}
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.ConverterUtils
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkException}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.security.{ConfigurableCredentialManager, CredentialUpdater}
+import org.apache.spark.internal.config._
+import org.apache.spark.launcher.YarnCommandBuilderUtils
+import org.apache.spark.util.Utils
+
+/**
+ * Contains util methods to interact with Hadoop from spark.
+ */
+class YarnSparkHadoopUtil extends SparkHadoopUtil {
+
+ private var credentialUpdater: CredentialUpdater = _
+
+ override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) {
+ dest.addCredentials(source.getCredentials())
+ }
+
+ // 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 a config initializes some Hadoop
+ // subsystems. Always create a new config, don't reuse yarnConf.
+ override def newConfiguration(conf: SparkConf): Configuration =
+ new YarnConfiguration(super.newConfiguration(conf))
+
+ // 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())
+ }
+
+ override def getCurrentUserCredentials(): Credentials = {
+ UserGroupInformation.getCurrentUser().getCredentials()
+ }
+
+ override def addCurrentUserCredentials(creds: Credentials) {
+ UserGroupInformation.getCurrentUser().addCredentials(creds)
+ }
+
+ override def addSecretKeyToUserCredentials(key: String, secret: String) {
+ val creds = new Credentials()
+ creds.addSecretKey(new Text(key), secret.getBytes(UTF_8))
+ addCurrentUserCredentials(creds)
+ }
+
+ override def getSecretKeyFromUserCredentials(key: String): Array[Byte] = {
+ val credentials = getCurrentUserCredentials()
+ if (credentials != null) credentials.getSecretKey(new Text(key)) else null
+ }
+
+ private[spark] override def startCredentialUpdater(sparkConf: SparkConf): Unit = {
+ credentialUpdater =
+ new ConfigurableCredentialManager(sparkConf, newConfiguration(sparkConf)).credentialUpdater()
+ credentialUpdater.start()
+ }
+
+ private[spark] override def stopCredentialUpdater(): Unit = {
+ if (credentialUpdater != null) {
+ credentialUpdater.stop()
+ credentialUpdater = null
+ }
+ }
+
+ private[spark] def getContainerId: ContainerId = {
+ val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name())
+ ConverterUtils.toContainerId(containerIdString)
+ }
+}
+
+object YarnSparkHadoopUtil {
+ // Additional memory overhead
+ // 10% was arrived at experimentally. In the interest of minimizing memory waste while covering
+ // the common cases. Memory overhead tends to grow with container size.
+
+ val MEMORY_OVERHEAD_FACTOR = 0.10
+ val MEMORY_OVERHEAD_MIN = 384L
+
+ val ANY_HOST = "*"
+
+ val DEFAULT_NUMBER_EXECUTORS = 2
+
+ // All RM requests are issued with same priority : we do not (yet) have any distinction between
+ // request types (like map/reduce in hadoop for example)
+ val RM_REQUEST_PRIORITY = Priority.newInstance(1)
+
+ def get: YarnSparkHadoopUtil = {
+ val yarnMode = java.lang.Boolean.parseBoolean(
+ System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
+ if (!yarnMode) {
+ throw new SparkException("YarnSparkHadoopUtil is not available in non-YARN mode!")
+ }
+ SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil]
+ }
+ /**
+ * Add a path variable to the given environment map.
+ * If the map already contains this key, append the value to the existing value instead.
+ */
+ def addPathToEnvironment(env: HashMap[String, String], key: String, value: String): Unit = {
+ val newValue = if (env.contains(key)) { env(key) + getClassPathSeparator + value } else value
+ env.put(key, newValue)
+ }
+
+ /**
+ * Set zero or more environment variables specified by the given input string.
+ * The input string is expected to take the form "KEY1=VAL1,KEY2=VAL2,KEY3=VAL3".
+ */
+ def setEnvFromInputString(env: HashMap[String, String], inputString: String): Unit = {
+ if (inputString != null && inputString.length() > 0) {
+ val childEnvs = inputString.split(",")
+ val p = Pattern.compile(environmentVariableRegex)
+ for (cEnv <- childEnvs) {
+ val parts = cEnv.split("=") // split on '='
+ val m = p.matcher(parts(1))
+ val sb = new StringBuffer
+ while (m.find()) {
+ val variable = m.group(1)
+ var replace = ""
+ if (env.get(variable) != None) {
+ replace = env.get(variable).get
+ } else {
+ // if this key is not configured for the child .. get it from the env
+ replace = System.getenv(variable)
+ if (replace == null) {
+ // the env key is note present anywhere .. simply set it
+ replace = ""
+ }
+ }
+ m.appendReplacement(sb, Matcher.quoteReplacement(replace))
+ }
+ m.appendTail(sb)
+ // This treats the environment variable as path variable delimited by `File.pathSeparator`
+ // This is kept for backward compatibility and consistency with Hadoop's behavior
+ addPathToEnvironment(env, parts(0), sb.toString)
+ }
+ }
+ }
+
+ private val environmentVariableRegex: String = {
+ if (Utils.isWindows) {
+ "%([A-Za-z_][A-Za-z0-9_]*?)%"
+ } else {
+ "\\$([A-Za-z_][A-Za-z0-9_]*)"
+ }
+ }
+
+ /**
+ * Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
+ * Not killing the task leaves various aspects of the executor 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 ?
+ *
+ * The handler if an OOM Exception is thrown by the JVM must be configured on Windows
+ * differently: the 'taskkill' command should be used, whereas Unix-based systems use 'kill'.
+ *
+ * As the JVM interprets both %p and %%p as the same, we can use either of them. However,
+ * some tests on Windows computers suggest, that the JVM only accepts '%%p'.
+ *
+ * Furthermore, the behavior of the character '%' on the Windows command line differs from
+ * the behavior of '%' in a .cmd file: it gets interpreted as an incomplete environment
+ * variable. Windows .cmd files escape a '%' by '%%'. Thus, the correct way of writing
+ * '%%p' in an escaped way is '%%%%p'.
+ */
+ private[yarn] def addOutOfMemoryErrorArgument(javaOpts: ListBuffer[String]): Unit = {
+ if (!javaOpts.exists(_.contains("-XX:OnOutOfMemoryError"))) {
+ if (Utils.isWindows) {
+ javaOpts += escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p")
+ } else {
+ javaOpts += "-XX:OnOutOfMemoryError='kill %p'"
+ }
+ }
+ }
+
+ /**
+ * Escapes a string for inclusion in a command line executed by Yarn. Yarn executes commands
+ * using either
+ *
+ * (Unix-based) `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work.
+ * The argument is enclosed in single quotes and some key characters are escaped.
+ *
+ * (Windows-based) part of a .cmd file in which case windows escaping for each argument must be
+ * applied. Windows is quite lenient, however it is usually Java that causes trouble, needing to
+ * distinguish between arguments starting with '-' and class names. If arguments are surrounded
+ * by ' java takes the following string as is, hence an argument is mistakenly taken as a class
+ * name which happens to start with a '-'. The way to avoid this, is to surround nothing with
+ * a ', but instead with a ".
+ *
+ * @param arg A single argument.
+ * @return Argument quoted for execution via Yarn's generated shell script.
+ */
+ def escapeForShell(arg: String): String = {
+ if (arg != null) {
+ if (Utils.isWindows) {
+ YarnCommandBuilderUtils.quoteForBatchScript(arg)
+ } else {
+ val escaped = new StringBuilder("'")
+ for (i <- 0 to arg.length() - 1) {
+ arg.charAt(i) match {
+ case '$' => escaped.append("\\$")
+ case '"' => escaped.append("\\\"")
+ case '\'' => escaped.append("'\\''")
+ case c => escaped.append(c)
+ }
+ }
+ escaped.append("'").toString()
+ }
+ } else {
+ arg
+ }
+ }
+
+ // YARN/Hadoop acls are specified as user1,user2 group1,group2
+ // Users and groups are separated by a space and hence we need to pass the acls in same format
+ def getApplicationAclsForYarn(securityMgr: SecurityManager)
+ : Map[ApplicationAccessType, String] = {
+ Map[ApplicationAccessType, String] (
+ ApplicationAccessType.VIEW_APP -> (securityMgr.getViewAcls + " " +
+ securityMgr.getViewAclsGroups),
+ ApplicationAccessType.MODIFY_APP -> (securityMgr.getModifyAcls + " " +
+ securityMgr.getModifyAclsGroups)
+ )
+ }
+
+ /**
+ * Expand environment variable using Yarn API.
+ * If environment.$$() is implemented, return the result of it.
+ * Otherwise, return the result of environment.$()
+ * Note: $$() is added in Hadoop 2.4.
+ */
+ private lazy val expandMethod =
+ Try(classOf[Environment].getMethod("$$"))
+ .getOrElse(classOf[Environment].getMethod("$"))
+
+ def expandEnvironment(environment: Environment): String =
+ expandMethod.invoke(environment).asInstanceOf[String]
+
+ /**
+ * Get class path separator using Yarn API.
+ * If ApplicationConstants.CLASS_PATH_SEPARATOR is implemented, return it.
+ * Otherwise, return File.pathSeparator
+ * Note: CLASS_PATH_SEPARATOR is added in Hadoop 2.4.
+ */
+ private lazy val classPathSeparatorField =
+ Try(classOf[ApplicationConstants].getField("CLASS_PATH_SEPARATOR"))
+ .getOrElse(classOf[File].getField("pathSeparator"))
+
+ def getClassPathSeparator(): String = {
+ classPathSeparatorField.get(null).asInstanceOf[String]
+ }
+
+ /**
+ * Getting the initial target number of executors depends on whether dynamic allocation is
+ * enabled.
+ * If not using dynamic allocation it gets the number of executors requested by the user.
+ */
+ def getInitialTargetExecutorNumber(
+ conf: SparkConf,
+ numExecutors: Int = DEFAULT_NUMBER_EXECUTORS): Int = {
+ if (Utils.isDynamicAllocationEnabled(conf)) {
+ val minNumExecutors = conf.get(DYN_ALLOCATION_MIN_EXECUTORS)
+ val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf)
+ val maxNumExecutors = conf.get(DYN_ALLOCATION_MAX_EXECUTORS)
+ require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors,
+ s"initial executor number $initialNumExecutors must between min executor number " +
+ s"$minNumExecutors and max executor number $maxNumExecutors")
+
+ initialNumExecutors
+ } else {
+ val targetNumExecutors =
+ sys.env.get("SPARK_EXECUTOR_INSTANCES").map(_.toInt).getOrElse(numExecutors)
+ // System property can override environment variable.
+ conf.get(EXECUTOR_INSTANCES).getOrElse(targetNumExecutors)
+ }
+ }
+}
+
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
new file mode 100644
index 0000000000..666cb456a9
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
@@ -0,0 +1,347 @@
+/*
+ * 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.util.concurrent.TimeUnit
+
+import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.network.util.ByteUnit
+
+package object config {
+
+ /* Common app configuration. */
+
+ private[spark] val APPLICATION_TAGS = ConfigBuilder("spark.yarn.tags")
+ .doc("Comma-separated list of strings to pass through as YARN application tags appearing " +
+ "in YARN Application Reports, which can be used for filtering when querying YARN.")
+ .stringConf
+ .toSequence
+ .createOptional
+
+ private[spark] val AM_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS =
+ ConfigBuilder("spark.yarn.am.attemptFailuresValidityInterval")
+ .doc("Interval after which AM failures will be considered independent and " +
+ "not accumulate towards the attempt count.")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createOptional
+
+ private[spark] val AM_PORT =
+ ConfigBuilder("spark.yarn.am.port")
+ .intConf
+ .createWithDefault(0)
+
+ private[spark] val EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS =
+ ConfigBuilder("spark.yarn.executor.failuresValidityInterval")
+ .doc("Interval after which Executor failures will be considered independent and not " +
+ "accumulate towards the attempt count.")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createOptional
+
+ private[spark] val MAX_APP_ATTEMPTS = ConfigBuilder("spark.yarn.maxAppAttempts")
+ .doc("Maximum number of AM attempts before failing the app.")
+ .intConf
+ .createOptional
+
+ private[spark] val USER_CLASS_PATH_FIRST = ConfigBuilder("spark.yarn.user.classpath.first")
+ .doc("Whether to place user jars in front of Spark's classpath.")
+ .booleanConf
+ .createWithDefault(false)
+
+ private[spark] val GATEWAY_ROOT_PATH = ConfigBuilder("spark.yarn.config.gatewayPath")
+ .doc("Root of configuration paths that is present on gateway nodes, and will be replaced " +
+ "with the corresponding path in cluster machines.")
+ .stringConf
+ .createWithDefault(null)
+
+ private[spark] val REPLACEMENT_ROOT_PATH = ConfigBuilder("spark.yarn.config.replacementPath")
+ .doc(s"Path to use as a replacement for ${GATEWAY_ROOT_PATH.key} when launching processes " +
+ "in the YARN cluster.")
+ .stringConf
+ .createWithDefault(null)
+
+ private[spark] val QUEUE_NAME = ConfigBuilder("spark.yarn.queue")
+ .stringConf
+ .createWithDefault("default")
+
+ private[spark] val HISTORY_SERVER_ADDRESS = ConfigBuilder("spark.yarn.historyServer.address")
+ .stringConf
+ .createOptional
+
+ /* File distribution. */
+
+ private[spark] val SPARK_ARCHIVE = ConfigBuilder("spark.yarn.archive")
+ .doc("Location of archive containing jars files with Spark classes.")
+ .stringConf
+ .createOptional
+
+ private[spark] val SPARK_JARS = ConfigBuilder("spark.yarn.jars")
+ .doc("Location of jars containing Spark classes.")
+ .stringConf
+ .toSequence
+ .createOptional
+
+ private[spark] val ARCHIVES_TO_DISTRIBUTE = ConfigBuilder("spark.yarn.dist.archives")
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ private[spark] val FILES_TO_DISTRIBUTE = ConfigBuilder("spark.yarn.dist.files")
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ private[spark] val JARS_TO_DISTRIBUTE = ConfigBuilder("spark.yarn.dist.jars")
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ private[spark] val PRESERVE_STAGING_FILES = ConfigBuilder("spark.yarn.preserve.staging.files")
+ .doc("Whether to preserve temporary files created by the job in HDFS.")
+ .booleanConf
+ .createWithDefault(false)
+
+ private[spark] val STAGING_FILE_REPLICATION = ConfigBuilder("spark.yarn.submit.file.replication")
+ .doc("Replication factor for files uploaded by Spark to HDFS.")
+ .intConf
+ .createOptional
+
+ private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir")
+ .doc("Staging directory used while submitting applications.")
+ .stringConf
+ .createOptional
+
+ /* Cluster-mode launcher configuration. */
+
+ private[spark] val WAIT_FOR_APP_COMPLETION = ConfigBuilder("spark.yarn.submit.waitAppCompletion")
+ .doc("In cluster mode, whether to wait for the application to finish before exiting the " +
+ "launcher process.")
+ .booleanConf
+ .createWithDefault(true)
+
+ private[spark] val REPORT_INTERVAL = ConfigBuilder("spark.yarn.report.interval")
+ .doc("Interval between reports of the current app status in cluster mode.")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefaultString("1s")
+
+ /* Shared Client-mode AM / Driver configuration. */
+
+ private[spark] val AM_MAX_WAIT_TIME = ConfigBuilder("spark.yarn.am.waitTime")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefaultString("100s")
+
+ private[spark] val AM_NODE_LABEL_EXPRESSION = ConfigBuilder("spark.yarn.am.nodeLabelExpression")
+ .doc("Node label expression for the AM.")
+ .stringConf
+ .createOptional
+
+ private[spark] val CONTAINER_LAUNCH_MAX_THREADS =
+ ConfigBuilder("spark.yarn.containerLauncherMaxThreads")
+ .intConf
+ .createWithDefault(25)
+
+ private[spark] val MAX_EXECUTOR_FAILURES = ConfigBuilder("spark.yarn.max.executor.failures")
+ .intConf
+ .createOptional
+
+ private[spark] val MAX_REPORTER_THREAD_FAILURES =
+ ConfigBuilder("spark.yarn.scheduler.reporterThread.maxFailures")
+ .intConf
+ .createWithDefault(5)
+
+ private[spark] val RM_HEARTBEAT_INTERVAL =
+ ConfigBuilder("spark.yarn.scheduler.heartbeat.interval-ms")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefaultString("3s")
+
+ private[spark] val INITIAL_HEARTBEAT_INTERVAL =
+ ConfigBuilder("spark.yarn.scheduler.initial-allocation.interval")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefaultString("200ms")
+
+ private[spark] val SCHEDULER_SERVICES = ConfigBuilder("spark.yarn.services")
+ .doc("A comma-separated list of class names of services to add to the scheduler.")
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ /* Client-mode AM configuration. */
+
+ private[spark] val AM_CORES = ConfigBuilder("spark.yarn.am.cores")
+ .intConf
+ .createWithDefault(1)
+
+ private[spark] val AM_JAVA_OPTIONS = ConfigBuilder("spark.yarn.am.extraJavaOptions")
+ .doc("Extra Java options for the client-mode AM.")
+ .stringConf
+ .createOptional
+
+ private[spark] val AM_LIBRARY_PATH = ConfigBuilder("spark.yarn.am.extraLibraryPath")
+ .doc("Extra native library path for the client-mode AM.")
+ .stringConf
+ .createOptional
+
+ private[spark] val AM_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.am.memoryOverhead")
+ .bytesConf(ByteUnit.MiB)
+ .createOptional
+
+ private[spark] val AM_MEMORY = ConfigBuilder("spark.yarn.am.memory")
+ .bytesConf(ByteUnit.MiB)
+ .createWithDefaultString("512m")
+
+ /* Driver configuration. */
+
+ private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores")
+ .intConf
+ .createWithDefault(1)
+
+ private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.driver.memoryOverhead")
+ .bytesConf(ByteUnit.MiB)
+ .createOptional
+
+ /* Executor configuration. */
+
+ private[spark] val EXECUTOR_CORES = ConfigBuilder("spark.executor.cores")
+ .intConf
+ .createWithDefault(1)
+
+ private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.executor.memoryOverhead")
+ .bytesConf(ByteUnit.MiB)
+ .createOptional
+
+ private[spark] val EXECUTOR_NODE_LABEL_EXPRESSION =
+ ConfigBuilder("spark.yarn.executor.nodeLabelExpression")
+ .doc("Node label expression for executors.")
+ .stringConf
+ .createOptional
+
+ /* Security configuration. */
+
+ private[spark] val CREDENTIAL_FILE_MAX_COUNT =
+ ConfigBuilder("spark.yarn.credentials.file.retention.count")
+ .intConf
+ .createWithDefault(5)
+
+ private[spark] val CREDENTIALS_FILE_MAX_RETENTION =
+ ConfigBuilder("spark.yarn.credentials.file.retention.days")
+ .intConf
+ .createWithDefault(5)
+
+ private[spark] val NAMENODES_TO_ACCESS = ConfigBuilder("spark.yarn.access.namenodes")
+ .doc("Extra NameNode URLs for which to request delegation tokens. The NameNode that hosts " +
+ "fs.defaultFS does not need to be listed here.")
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ /* Rolled log aggregation configuration. */
+
+ private[spark] val ROLLED_LOG_INCLUDE_PATTERN =
+ ConfigBuilder("spark.yarn.rolledLog.includePattern")
+ .doc("Java Regex to filter the log files which match the defined include pattern and those " +
+ "log files will be aggregated in a rolling fashion.")
+ .stringConf
+ .createOptional
+
+ private[spark] val ROLLED_LOG_EXCLUDE_PATTERN =
+ ConfigBuilder("spark.yarn.rolledLog.excludePattern")
+ .doc("Java Regex to filter the log files which match the defined exclude pattern and those " +
+ "log files will not be aggregated in a rolling fashion.")
+ .stringConf
+ .createOptional
+
+ /* Private configs. */
+
+ private[spark] val CREDENTIALS_FILE_PATH = ConfigBuilder("spark.yarn.credentials.file")
+ .internal()
+ .stringConf
+ .createWithDefault(null)
+
+ // Internal config to propagate the location of the user's jar to the driver/executors
+ private[spark] val APP_JAR = ConfigBuilder("spark.yarn.user.jar")
+ .internal()
+ .stringConf
+ .createOptional
+
+ // Internal config to propagate the locations of any extra jars to add to the classpath
+ // of the executors
+ private[spark] val SECONDARY_JARS = ConfigBuilder("spark.yarn.secondary.jars")
+ .internal()
+ .stringConf
+ .toSequence
+ .createOptional
+
+ /* Configuration and cached file propagation. */
+
+ private[spark] val CACHED_FILES = ConfigBuilder("spark.yarn.cache.filenames")
+ .internal()
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ private[spark] val CACHED_FILES_SIZES = ConfigBuilder("spark.yarn.cache.sizes")
+ .internal()
+ .longConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ private[spark] val CACHED_FILES_TIMESTAMPS = ConfigBuilder("spark.yarn.cache.timestamps")
+ .internal()
+ .longConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ private[spark] val CACHED_FILES_VISIBILITIES = ConfigBuilder("spark.yarn.cache.visibilities")
+ .internal()
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ // Either "file" or "archive", for each file.
+ private[spark] val CACHED_FILES_TYPES = ConfigBuilder("spark.yarn.cache.types")
+ .internal()
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ // The location of the conf archive in HDFS.
+ private[spark] val CACHED_CONF_ARCHIVE = ConfigBuilder("spark.yarn.cache.confArchive")
+ .internal()
+ .stringConf
+ .createOptional
+
+ private[spark] val CREDENTIALS_RENEWAL_TIME = ConfigBuilder("spark.yarn.credentials.renewalTime")
+ .internal()
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefault(Long.MaxValue)
+
+ private[spark] val CREDENTIALS_UPDATE_TIME = ConfigBuilder("spark.yarn.credentials.updateTime")
+ .internal()
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefault(Long.MaxValue)
+
+ // The list of cache-related config entries. This is used by Client and the AM to clean
+ // up the environment so that these settings do not appear on the web UI.
+ private[yarn] val CACHE_CONFIGS = Seq(
+ CACHED_FILES,
+ CACHED_FILES_SIZES,
+ CACHED_FILES_TIMESTAMPS,
+ CACHED_FILES_VISIBILITIES,
+ CACHED_FILES_TYPES,
+ CACHED_CONF_ARCHIVE)
+
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala
new file mode 100644
index 0000000000..7e76f402db
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala
@@ -0,0 +1,235 @@
+/*
+ * 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.security
+
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.{Executors, TimeUnit}
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.security.UserGroupInformation
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * The following methods are primarily meant to make sure long-running apps like Spark
+ * Streaming apps can run without interruption while accessing secured services. The
+ * scheduleLoginFromKeytab method is called on the AM to get the new credentials.
+ * This method wakes up a thread that logs into the KDC
+ * once 75% of the renewal interval of the original credentials used for the container
+ * has elapsed. It then obtains new credentials and writes them to HDFS in a
+ * pre-specified location - the prefix of which is specified in the sparkConf by
+ * spark.yarn.credentials.file (so the file(s) would be named c-timestamp1-1, c-timestamp2-2 etc.
+ * - each update goes to a new file, with a monotonically increasing suffix), also the
+ * timestamp1, timestamp2 here indicates the time of next update for CredentialUpdater.
+ * After this, the credentials are renewed once 75% of the new tokens renewal interval has elapsed.
+ *
+ * On the executor and driver (yarn client mode) side, the updateCredentialsIfRequired method is
+ * called once 80% of the validity of the original credentials has elapsed. At that time the
+ * executor finds the credentials file with the latest timestamp and checks if it has read those
+ * credentials before (by keeping track of the suffix of the last file it read). If a new file has
+ * appeared, it will read the credentials and update the currently running UGI with it. This
+ * process happens again once 80% of the validity of this has expired.
+ */
+private[yarn] class AMCredentialRenewer(
+ sparkConf: SparkConf,
+ hadoopConf: Configuration,
+ credentialManager: ConfigurableCredentialManager) extends Logging {
+
+ private var lastCredentialsFileSuffix = 0
+
+ private val credentialRenewer =
+ Executors.newSingleThreadScheduledExecutor(
+ ThreadUtils.namedThreadFactory("Credential Refresh Thread"))
+
+ private val hadoopUtil = YarnSparkHadoopUtil.get
+
+ private val credentialsFile = sparkConf.get(CREDENTIALS_FILE_PATH)
+ private val daysToKeepFiles = sparkConf.get(CREDENTIALS_FILE_MAX_RETENTION)
+ private val numFilesToKeep = sparkConf.get(CREDENTIAL_FILE_MAX_COUNT)
+ private val freshHadoopConf =
+ hadoopUtil.getConfBypassingFSCache(hadoopConf, new Path(credentialsFile).toUri.getScheme)
+
+ @volatile private var timeOfNextRenewal = sparkConf.get(CREDENTIALS_RENEWAL_TIME)
+
+ /**
+ * Schedule a login from the keytab and principal set using the --principal and --keytab
+ * arguments to spark-submit. This login happens only when the credentials of the current user
+ * are about to expire. This method reads spark.yarn.principal and spark.yarn.keytab from
+ * SparkConf to do the login. This method is a no-op in non-YARN mode.
+ *
+ */
+ private[spark] def scheduleLoginFromKeytab(): Unit = {
+ val principal = sparkConf.get(PRINCIPAL).get
+ val keytab = sparkConf.get(KEYTAB).get
+
+ /**
+ * Schedule re-login and creation of new credentials. If credentials have already expired, this
+ * method will synchronously create new ones.
+ */
+ def scheduleRenewal(runnable: Runnable): Unit = {
+ // Run now!
+ val remainingTime = timeOfNextRenewal - System.currentTimeMillis()
+ if (remainingTime <= 0) {
+ logInfo("Credentials have expired, creating new ones now.")
+ runnable.run()
+ } else {
+ logInfo(s"Scheduling login from keytab in $remainingTime millis.")
+ credentialRenewer.schedule(runnable, remainingTime, TimeUnit.MILLISECONDS)
+ }
+ }
+
+ // This thread periodically runs on the AM to update the credentials on HDFS.
+ val credentialRenewerRunnable =
+ new Runnable {
+ override def run(): Unit = {
+ try {
+ writeNewCredentialsToHDFS(principal, keytab)
+ cleanupOldFiles()
+ } catch {
+ case e: Exception =>
+ // Log the error and try to write new tokens back in an hour
+ logWarning("Failed to write out new credentials to HDFS, will try again in an " +
+ "hour! If this happens too often tasks will fail.", e)
+ credentialRenewer.schedule(this, 1, TimeUnit.HOURS)
+ return
+ }
+ scheduleRenewal(this)
+ }
+ }
+ // Schedule update of credentials. This handles the case of updating the credentials right now
+ // as well, since the renewal interval will be 0, and the thread will get scheduled
+ // immediately.
+ scheduleRenewal(credentialRenewerRunnable)
+ }
+
+ // Keeps only files that are newer than daysToKeepFiles days, and deletes everything else. At
+ // least numFilesToKeep files are kept for safety
+ private def cleanupOldFiles(): Unit = {
+ import scala.concurrent.duration._
+ try {
+ val remoteFs = FileSystem.get(freshHadoopConf)
+ val credentialsPath = new Path(credentialsFile)
+ val thresholdTime = System.currentTimeMillis() - (daysToKeepFiles.days).toMillis
+ hadoopUtil.listFilesSorted(
+ remoteFs, credentialsPath.getParent,
+ credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
+ .dropRight(numFilesToKeep)
+ .takeWhile(_.getModificationTime < thresholdTime)
+ .foreach(x => remoteFs.delete(x.getPath, true))
+ } catch {
+ // Such errors are not fatal, so don't throw. Make sure they are logged though
+ case e: Exception =>
+ logWarning("Error while attempting to cleanup old credentials. If you are seeing many " +
+ "such warnings there may be an issue with your HDFS cluster.", e)
+ }
+ }
+
+ private def writeNewCredentialsToHDFS(principal: String, keytab: String): Unit = {
+ // Keytab is copied by YARN to the working directory of the AM, so full path is
+ // not needed.
+
+ // HACK:
+ // HDFS will not issue new delegation tokens, if the Credentials object
+ // passed in already has tokens for that FS even if the tokens are expired (it really only
+ // checks if there are tokens for the service, and not if they are valid). So the only real
+ // way to get new tokens is to make sure a different Credentials object is used each time to
+ // get new tokens and then the new tokens are copied over the current user's Credentials.
+ // So:
+ // - we login as a different user and get the UGI
+ // - use that UGI to get the tokens (see doAs block below)
+ // - copy the tokens over to the current user's credentials (this will overwrite the tokens
+ // in the current user's Credentials object for this FS).
+ // The login to KDC happens each time new tokens are required, but this is rare enough to not
+ // have to worry about (like once every day or so). This makes this code clearer than having
+ // to login and then relogin every time (the HDFS API may not relogin since we don't use this
+ // UGI directly for HDFS communication.
+ logInfo(s"Attempting to login to KDC using principal: $principal")
+ val keytabLoggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)
+ logInfo("Successfully logged into KDC.")
+ val tempCreds = keytabLoggedInUGI.getCredentials
+ val credentialsPath = new Path(credentialsFile)
+ val dst = credentialsPath.getParent
+ var nearestNextRenewalTime = Long.MaxValue
+ keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] {
+ // Get a copy of the credentials
+ override def run(): Void = {
+ nearestNextRenewalTime = credentialManager.obtainCredentials(freshHadoopConf, tempCreds)
+ null
+ }
+ })
+
+ val currTime = System.currentTimeMillis()
+ val timeOfNextUpdate = if (nearestNextRenewalTime <= currTime) {
+ // If next renewal time is earlier than current time, we set next renewal time to current
+ // time, this will trigger next renewal immediately. Also set next update time to current
+ // time. There still has a gap between token renewal and update will potentially introduce
+ // issue.
+ logWarning(s"Next credential renewal time ($nearestNextRenewalTime) is earlier than " +
+ s"current time ($currTime), which is unexpected, please check your credential renewal " +
+ "related configurations in the target services.")
+ timeOfNextRenewal = currTime
+ currTime
+ } else {
+ // Next valid renewal time is about 75% of credential renewal time, and update time is
+ // slightly later than valid renewal time (80% of renewal time).
+ timeOfNextRenewal = ((nearestNextRenewalTime - currTime) * 0.75 + currTime).toLong
+ ((nearestNextRenewalTime - currTime) * 0.8 + currTime).toLong
+ }
+
+ // Add the temp credentials back to the original ones.
+ UserGroupInformation.getCurrentUser.addCredentials(tempCreds)
+ val remoteFs = FileSystem.get(freshHadoopConf)
+ // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM
+ // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file
+ // and update the lastCredentialsFileSuffix.
+ if (lastCredentialsFileSuffix == 0) {
+ hadoopUtil.listFilesSorted(
+ remoteFs, credentialsPath.getParent,
+ credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
+ .lastOption.foreach { status =>
+ lastCredentialsFileSuffix = hadoopUtil.getSuffixForCredentialsPath(status.getPath)
+ }
+ }
+ val nextSuffix = lastCredentialsFileSuffix + 1
+
+ val tokenPathStr =
+ credentialsFile + SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM +
+ timeOfNextUpdate.toLong.toString + SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM +
+ nextSuffix
+ val tokenPath = new Path(tokenPathStr)
+ val tempTokenPath = new Path(tokenPathStr + SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
+
+ logInfo("Writing out delegation tokens to " + tempTokenPath.toString)
+ val credentials = UserGroupInformation.getCurrentUser.getCredentials
+ credentials.writeTokenStorageFile(tempTokenPath, freshHadoopConf)
+ logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr")
+ remoteFs.rename(tempTokenPath, tokenPath)
+ logInfo("Delegation token file rename complete.")
+ lastCredentialsFileSuffix = nextSuffix
+ }
+
+ def stop(): Unit = {
+ credentialRenewer.shutdown()
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala
new file mode 100644
index 0000000000..c4c07b4930
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala
@@ -0,0 +1,105 @@
+/*
+ * 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.security
+
+import java.util.ServiceLoader
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.Credentials
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+/**
+ * A ConfigurableCredentialManager to manage all the registered credential providers and offer
+ * APIs for other modules to obtain credentials as well as renewal time. By default
+ * [[HDFSCredentialProvider]], [[HiveCredentialProvider]] and [[HBaseCredentialProvider]] will
+ * be loaded in if not explicitly disabled, any plugged-in credential provider wants to be
+ * managed by ConfigurableCredentialManager needs to implement [[ServiceCredentialProvider]]
+ * interface and put into resources/META-INF/services to be loaded by ServiceLoader.
+ *
+ * Also each credential provider is controlled by
+ * spark.yarn.security.credentials.{service}.enabled, it will not be loaded in if set to false.
+ */
+private[yarn] final class ConfigurableCredentialManager(
+ sparkConf: SparkConf, hadoopConf: Configuration) extends Logging {
+ private val deprecatedProviderEnabledConfig = "spark.yarn.security.tokens.%s.enabled"
+ private val providerEnabledConfig = "spark.yarn.security.credentials.%s.enabled"
+
+ // Maintain all the registered credential providers
+ private val credentialProviders = {
+ val providers = ServiceLoader.load(classOf[ServiceCredentialProvider],
+ Utils.getContextOrSparkClassLoader).asScala
+
+ // Filter out credentials in which spark.yarn.security.credentials.{service}.enabled is false.
+ providers.filter { p =>
+ sparkConf.getOption(providerEnabledConfig.format(p.serviceName))
+ .orElse {
+ sparkConf.getOption(deprecatedProviderEnabledConfig.format(p.serviceName)).map { c =>
+ logWarning(s"${deprecatedProviderEnabledConfig.format(p.serviceName)} is deprecated, " +
+ s"using ${providerEnabledConfig.format(p.serviceName)} instead")
+ c
+ }
+ }.map(_.toBoolean).getOrElse(true)
+ }.map { p => (p.serviceName, p) }.toMap
+ }
+
+ /**
+ * Get credential provider for the specified service.
+ */
+ def getServiceCredentialProvider(service: String): Option[ServiceCredentialProvider] = {
+ credentialProviders.get(service)
+ }
+
+ /**
+ * Obtain credentials from all the registered providers.
+ * @return nearest time of next renewal, Long.MaxValue if all the credentials aren't renewable,
+ * otherwise the nearest renewal time of any credentials will be returned.
+ */
+ def obtainCredentials(hadoopConf: Configuration, creds: Credentials): Long = {
+ credentialProviders.values.flatMap { provider =>
+ if (provider.credentialsRequired(hadoopConf)) {
+ provider.obtainCredentials(hadoopConf, sparkConf, creds)
+ } else {
+ logDebug(s"Service ${provider.serviceName} does not require a token." +
+ s" Check your configuration to see if security is disabled or not.")
+ None
+ }
+ }.foldLeft(Long.MaxValue)(math.min)
+ }
+
+ /**
+ * Create an [[AMCredentialRenewer]] instance, caller should be responsible to stop this
+ * instance when it is not used. AM will use it to renew credentials periodically.
+ */
+ def credentialRenewer(): AMCredentialRenewer = {
+ new AMCredentialRenewer(sparkConf, hadoopConf, this)
+ }
+
+ /**
+ * Create an [[CredentialUpdater]] instance, caller should be resposible to stop this intance
+ * when it is not used. Executors and driver (client mode) will use it to update credentials.
+ * periodically.
+ */
+ def credentialUpdater(): CredentialUpdater = {
+ new CredentialUpdater(sparkConf, hadoopConf, this)
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala
new file mode 100644
index 0000000000..5df4fbd9c1
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala
@@ -0,0 +1,130 @@
+/*
+ * 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.security
+
+import java.util.concurrent.{Executors, TimeUnit}
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.{ThreadUtils, Utils}
+
+private[spark] class CredentialUpdater(
+ sparkConf: SparkConf,
+ hadoopConf: Configuration,
+ credentialManager: ConfigurableCredentialManager) extends Logging {
+
+ @volatile private var lastCredentialsFileSuffix = 0
+
+ private val credentialsFile = sparkConf.get(CREDENTIALS_FILE_PATH)
+ private val freshHadoopConf =
+ SparkHadoopUtil.get.getConfBypassingFSCache(
+ hadoopConf, new Path(credentialsFile).toUri.getScheme)
+
+ private val credentialUpdater =
+ Executors.newSingleThreadScheduledExecutor(
+ ThreadUtils.namedThreadFactory("Credential Refresh Thread"))
+
+ // This thread wakes up and picks up new credentials from HDFS, if any.
+ private val credentialUpdaterRunnable =
+ new Runnable {
+ override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired())
+ }
+
+ /** Start the credential updater task */
+ def start(): Unit = {
+ val startTime = sparkConf.get(CREDENTIALS_RENEWAL_TIME)
+ val remainingTime = startTime - System.currentTimeMillis()
+ if (remainingTime <= 0) {
+ credentialUpdater.schedule(credentialUpdaterRunnable, 1, TimeUnit.MINUTES)
+ } else {
+ logInfo(s"Scheduling credentials refresh from HDFS in $remainingTime millis.")
+ credentialUpdater.schedule(credentialUpdaterRunnable, remainingTime, TimeUnit.MILLISECONDS)
+ }
+ }
+
+ private def updateCredentialsIfRequired(): Unit = {
+ val timeToNextUpdate = try {
+ val credentialsFilePath = new Path(credentialsFile)
+ val remoteFs = FileSystem.get(freshHadoopConf)
+ SparkHadoopUtil.get.listFilesSorted(
+ remoteFs, credentialsFilePath.getParent,
+ credentialsFilePath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
+ .lastOption.map { credentialsStatus =>
+ val suffix = SparkHadoopUtil.get.getSuffixForCredentialsPath(credentialsStatus.getPath)
+ if (suffix > lastCredentialsFileSuffix) {
+ logInfo("Reading new credentials from " + credentialsStatus.getPath)
+ val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath)
+ lastCredentialsFileSuffix = suffix
+ UserGroupInformation.getCurrentUser.addCredentials(newCredentials)
+ logInfo("Credentials updated from credentials file.")
+
+ val remainingTime = getTimeOfNextUpdateFromFileName(credentialsStatus.getPath)
+ - System.currentTimeMillis()
+ if (remainingTime <= 0) TimeUnit.MINUTES.toMillis(1) else remainingTime
+ } else {
+ // If current credential file is older than expected, sleep 1 hour and check again.
+ TimeUnit.HOURS.toMillis(1)
+ }
+ }.getOrElse {
+ // Wait for 1 minute to check again if there's no credential file currently
+ TimeUnit.MINUTES.toMillis(1)
+ }
+ } catch {
+ // Since the file may get deleted while we are reading it, catch the Exception and come
+ // back in an hour to try again
+ case NonFatal(e) =>
+ logWarning("Error while trying to update credentials, will try again in 1 hour", e)
+ TimeUnit.HOURS.toMillis(1)
+ }
+
+ credentialUpdater.schedule(
+ credentialUpdaterRunnable, timeToNextUpdate, TimeUnit.MILLISECONDS)
+ }
+
+ private def getCredentialsFromHDFSFile(remoteFs: FileSystem, tokenPath: Path): Credentials = {
+ val stream = remoteFs.open(tokenPath)
+ try {
+ val newCredentials = new Credentials()
+ newCredentials.readTokenStorageStream(stream)
+ newCredentials
+ } finally {
+ stream.close()
+ }
+ }
+
+ private def getTimeOfNextUpdateFromFileName(credentialsPath: Path): Long = {
+ val name = credentialsPath.getName
+ val index = name.lastIndexOf(SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM)
+ val slice = name.substring(0, index)
+ val last2index = slice.lastIndexOf(SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM)
+ name.substring(last2index + 1, index).toLong
+ }
+
+ def stop(): Unit = {
+ credentialUpdater.shutdown()
+ }
+
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala
new file mode 100644
index 0000000000..5571df09a2
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala
@@ -0,0 +1,74 @@
+/*
+ * 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.security
+
+import scala.reflect.runtime.universe
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.token.{Token, TokenIdentifier}
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+
+private[security] class HBaseCredentialProvider extends ServiceCredentialProvider with Logging {
+
+ override def serviceName: String = "hbase"
+
+ override def obtainCredentials(
+ hadoopConf: Configuration,
+ sparkConf: SparkConf,
+ creds: Credentials): Option[Long] = {
+ try {
+ val mirror = universe.runtimeMirror(getClass.getClassLoader)
+ val obtainToken = mirror.classLoader.
+ loadClass("org.apache.hadoop.hbase.security.token.TokenUtil").
+ getMethod("obtainToken", classOf[Configuration])
+
+ logDebug("Attempting to fetch HBase security token.")
+ val token = obtainToken.invoke(null, hbaseConf(hadoopConf))
+ .asInstanceOf[Token[_ <: TokenIdentifier]]
+ logInfo(s"Get token from HBase: ${token.toString}")
+ creds.addToken(token.getService, token)
+ } catch {
+ case NonFatal(e) =>
+ logDebug(s"Failed to get token from service $serviceName", e)
+ }
+
+ None
+ }
+
+ override def credentialsRequired(hadoopConf: Configuration): Boolean = {
+ hbaseConf(hadoopConf).get("hbase.security.authentication") == "kerberos"
+ }
+
+ private def hbaseConf(conf: Configuration): Configuration = {
+ try {
+ val mirror = universe.runtimeMirror(getClass.getClassLoader)
+ val confCreate = mirror.classLoader.
+ loadClass("org.apache.hadoop.hbase.HBaseConfiguration").
+ getMethod("create", classOf[Configuration])
+ confCreate.invoke(null, conf).asInstanceOf[Configuration]
+ } catch {
+ case NonFatal(e) =>
+ logDebug("Fail to invoke HBaseConfiguration", e)
+ conf
+ }
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala
new file mode 100644
index 0000000000..8d06d735ba
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProvider.scala
@@ -0,0 +1,110 @@
+/*
+ * 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.security
+
+import java.io.{ByteArrayInputStream, DataInputStream}
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier
+import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.security.Credentials
+
+import org.apache.spark.{SparkConf, SparkException}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+
+private[security] class HDFSCredentialProvider extends ServiceCredentialProvider with Logging {
+ // Token renewal interval, this value will be set in the first call,
+ // if None means no token renewer specified, so cannot get token renewal interval.
+ private var tokenRenewalInterval: Option[Long] = null
+
+ override val serviceName: String = "hdfs"
+
+ override def obtainCredentials(
+ hadoopConf: Configuration,
+ sparkConf: SparkConf,
+ creds: Credentials): Option[Long] = {
+ // NameNode to access, used to get tokens from different FileSystems
+ nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
+ val dstFs = dst.getFileSystem(hadoopConf)
+ logInfo("getting token for namenode: " + dst)
+ dstFs.addDelegationTokens(getTokenRenewer(hadoopConf), creds)
+ }
+
+ // Get the token renewal interval if it is not set. It will only be called once.
+ if (tokenRenewalInterval == null) {
+ tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf)
+ }
+
+ // Get the time of next renewal.
+ tokenRenewalInterval.map { interval =>
+ creds.getAllTokens.asScala
+ .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND)
+ .map { t =>
+ val identifier = new DelegationTokenIdentifier()
+ identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier)))
+ identifier.getIssueDate + interval
+ }.foldLeft(0L)(math.max)
+ }
+ }
+
+ private def getTokenRenewalInterval(
+ hadoopConf: Configuration, sparkConf: SparkConf): Option[Long] = {
+ // We cannot use the tokens generated with renewer yarn. Trying to renew
+ // those will fail with an access control issue. So create new tokens with the logged in
+ // user as renewer.
+ sparkConf.get(PRINCIPAL).map { renewer =>
+ val creds = new Credentials()
+ nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
+ val dstFs = dst.getFileSystem(hadoopConf)
+ dstFs.addDelegationTokens(renewer, creds)
+ }
+ val t = creds.getAllTokens.asScala
+ .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND)
+ .head
+ val newExpiration = t.renew(hadoopConf)
+ val identifier = new DelegationTokenIdentifier()
+ identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier)))
+ val interval = newExpiration - identifier.getIssueDate
+ logInfo(s"Renewal Interval is $interval")
+ interval
+ }
+ }
+
+ private def getTokenRenewer(conf: Configuration): String = {
+ val delegTokenRenewer = Master.getMasterPrincipal(conf)
+ logDebug("delegation token renewer is: " + delegTokenRenewer)
+ if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+ val errorMessage = "Can't get Master Kerberos principal for use as renewer"
+ logError(errorMessage)
+ throw new SparkException(errorMessage)
+ }
+
+ delegTokenRenewer
+ }
+
+ private def nnsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = {
+ sparkConf.get(NAMENODES_TO_ACCESS).map(new Path(_)).toSet +
+ sparkConf.get(STAGING_DIR).map(new Path(_))
+ .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory)
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala
new file mode 100644
index 0000000000..16d8fc32bb
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala
@@ -0,0 +1,129 @@
+/*
+ * 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.security
+
+import java.lang.reflect.UndeclaredThrowableException
+import java.security.PrivilegedExceptionAction
+
+import scala.reflect.runtime.universe
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+import org.apache.hadoop.security.token.Token
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+private[security] class HiveCredentialProvider extends ServiceCredentialProvider with Logging {
+
+ override def serviceName: String = "hive"
+
+ private def hiveConf(hadoopConf: Configuration): Configuration = {
+ try {
+ val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+ // the hive configuration class is a subclass of Hadoop Configuration, so can be cast down
+ // to a Configuration and used without reflection
+ val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")
+ // using the (Configuration, Class) constructor allows the current configuration to be
+ // included in the hive config.
+ val ctor = hiveConfClass.getDeclaredConstructor(classOf[Configuration],
+ classOf[Object].getClass)
+ ctor.newInstance(hadoopConf, hiveConfClass).asInstanceOf[Configuration]
+ } catch {
+ case NonFatal(e) =>
+ logDebug("Fail to create Hive Configuration", e)
+ hadoopConf
+ }
+ }
+
+ override def credentialsRequired(hadoopConf: Configuration): Boolean = {
+ UserGroupInformation.isSecurityEnabled &&
+ hiveConf(hadoopConf).getTrimmed("hive.metastore.uris", "").nonEmpty
+ }
+
+ override def obtainCredentials(
+ hadoopConf: Configuration,
+ sparkConf: SparkConf,
+ creds: Credentials): Option[Long] = {
+ val conf = hiveConf(hadoopConf)
+
+ val principalKey = "hive.metastore.kerberos.principal"
+ val principal = conf.getTrimmed(principalKey, "")
+ require(principal.nonEmpty, s"Hive principal $principalKey undefined")
+ val metastoreUri = conf.getTrimmed("hive.metastore.uris", "")
+ require(metastoreUri.nonEmpty, "Hive metastore uri undefined")
+
+ val currentUser = UserGroupInformation.getCurrentUser()
+ logDebug(s"Getting Hive delegation token for ${currentUser.getUserName()} against " +
+ s"$principal at $metastoreUri")
+
+ val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader)
+ val hiveClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.ql.metadata.Hive")
+ val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")
+ val closeCurrent = hiveClass.getMethod("closeCurrent")
+
+ try {
+ // get all the instance methods before invoking any
+ val getDelegationToken = hiveClass.getMethod("getDelegationToken",
+ classOf[String], classOf[String])
+ val getHive = hiveClass.getMethod("get", hiveConfClass)
+
+ doAsRealUser {
+ val hive = getHive.invoke(null, conf)
+ val tokenStr = getDelegationToken.invoke(hive, currentUser.getUserName(), principal)
+ .asInstanceOf[String]
+ val hive2Token = new Token[DelegationTokenIdentifier]()
+ hive2Token.decodeFromUrlString(tokenStr)
+ logInfo(s"Get Token from hive metastore: ${hive2Token.toString}")
+ creds.addToken(new Text("hive.server2.delegation.token"), hive2Token)
+ }
+ } catch {
+ case NonFatal(e) =>
+ logDebug(s"Fail to get token from service $serviceName", e)
+ } finally {
+ Utils.tryLogNonFatalError {
+ closeCurrent.invoke(null)
+ }
+ }
+
+ None
+ }
+
+ /**
+ * Run some code as the real logged in user (which may differ from the current user, for
+ * example, when using proxying).
+ */
+ private def doAsRealUser[T](fn: => T): T = {
+ val currentUser = UserGroupInformation.getCurrentUser()
+ val realUser = Option(currentUser.getRealUser()).getOrElse(currentUser)
+
+ // For some reason the Scala-generated anonymous class ends up causing an
+ // UndeclaredThrowableException, even if you annotate the method with @throws.
+ try {
+ realUser.doAs(new PrivilegedExceptionAction[T]() {
+ override def run(): T = fn
+ })
+ } catch {
+ case e: UndeclaredThrowableException => throw Option(e.getCause()).getOrElse(e)
+ }
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
new file mode 100644
index 0000000000..4e3fcce8db
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala
@@ -0,0 +1,57 @@
+/*
+ * 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.security
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.spark.SparkConf
+
+/**
+ * A credential provider for a service. User must implement this if they need to access a
+ * secure service from Spark.
+ */
+trait ServiceCredentialProvider {
+
+ /**
+ * Name of the service to provide credentials. This name should unique, Spark internally will
+ * use this name to differentiate credential provider.
+ */
+ def serviceName: String
+
+ /**
+ * To decide whether credential is required for this service. By default it based on whether
+ * Hadoop security is enabled.
+ */
+ def credentialsRequired(hadoopConf: Configuration): Boolean = {
+ UserGroupInformation.isSecurityEnabled
+ }
+
+ /**
+ * Obtain credentials for this service and get the time of the next renewal.
+ * @param hadoopConf Configuration of current Hadoop Compatible system.
+ * @param sparkConf Spark configuration.
+ * @param creds Credentials to add tokens and security keys to.
+ * @return If this Credential is renewable and can be renewed, return the time of the next
+ * renewal, otherwise None should be returned.
+ */
+ def obtainCredentials(
+ hadoopConf: Configuration,
+ sparkConf: SparkConf,
+ creds: Credentials): Option[Long]
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
new file mode 100644
index 0000000000..6c3556a2ee
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.launcher
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+import scala.util.Properties
+
+/**
+ * Exposes methods from the launcher library that are used by the YARN backend.
+ */
+private[spark] object YarnCommandBuilderUtils {
+
+ def quoteForBatchScript(arg: String): String = {
+ CommandBuilderUtils.quoteForBatchScript(arg)
+ }
+
+ def findJarsDir(sparkHome: String): String = {
+ val scalaVer = Properties.versionNumberString
+ .split("\\.")
+ .take(2)
+ .mkString(".")
+ CommandBuilderUtils.findJarsDir(sparkHome, scalaVer, true)
+ }
+
+ /**
+ * Adds the perm gen configuration to the list of java options if needed and not yet added.
+ *
+ * Note that this method adds the option based on the local JVM version; if the node where
+ * the container is running has a different Java version, there's a risk that the option will
+ * not be added (e.g. if the AM is running Java 8 but the container's node is set up to use
+ * Java 7).
+ */
+ def addPermGenSizeOpt(args: ListBuffer[String]): Unit = {
+ CommandBuilderUtils.addPermGenSizeOpt(args.asJava)
+ }
+
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala
new file mode 100644
index 0000000000..4ed285230f
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala
@@ -0,0 +1,143 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicBoolean
+
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+
+import org.apache.spark.SparkContext
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+/**
+ * An extension service that can be loaded into a Spark YARN scheduler.
+ * A Service that can be started and stopped.
+ *
+ * 1. For implementations to be loadable by `SchedulerExtensionServices`,
+ * they must provide an empty constructor.
+ * 2. The `stop()` operation MUST be idempotent, and succeed even if `start()` was
+ * never invoked.
+ */
+trait SchedulerExtensionService {
+
+ /**
+ * Start the extension service. This should be a no-op if
+ * called more than once.
+ * @param binding binding to the spark application and YARN
+ */
+ def start(binding: SchedulerExtensionServiceBinding): Unit
+
+ /**
+ * Stop the service
+ * The `stop()` operation MUST be idempotent, and succeed even if `start()` was
+ * never invoked.
+ */
+ def stop(): Unit
+}
+
+/**
+ * Binding information for a [[SchedulerExtensionService]].
+ *
+ * The attempt ID will be set if the service is started within a YARN application master;
+ * there is then a different attempt ID for every time that AM is restarted.
+ * When the service binding is instantiated in client mode, there's no attempt ID, as it lacks
+ * this information.
+ * @param sparkContext current spark context
+ * @param applicationId YARN application ID
+ * @param attemptId YARN attemptID. This will always be unset in client mode, and always set in
+ * cluster mode.
+ */
+case class SchedulerExtensionServiceBinding(
+ sparkContext: SparkContext,
+ applicationId: ApplicationId,
+ attemptId: Option[ApplicationAttemptId] = None)
+
+/**
+ * Container for [[SchedulerExtensionService]] instances.
+ *
+ * Loads Extension Services from the configuration property
+ * `"spark.yarn.services"`, instantiates and starts them.
+ * When stopped, it stops all child entries.
+ *
+ * The order in which child extension services are started and stopped
+ * is undefined.
+ */
+private[spark] class SchedulerExtensionServices extends SchedulerExtensionService
+ with Logging {
+ private var serviceOption: Option[String] = None
+ private var services: List[SchedulerExtensionService] = Nil
+ private val started = new AtomicBoolean(false)
+ private var binding: SchedulerExtensionServiceBinding = _
+
+ /**
+ * Binding operation will load the named services and call bind on them too; the
+ * entire set of services are then ready for `init()` and `start()` calls.
+ *
+ * @param binding binding to the spark application and YARN
+ */
+ def start(binding: SchedulerExtensionServiceBinding): Unit = {
+ if (started.getAndSet(true)) {
+ logWarning("Ignoring re-entrant start operation")
+ return
+ }
+ require(binding.sparkContext != null, "Null context parameter")
+ require(binding.applicationId != null, "Null appId parameter")
+ this.binding = binding
+ val sparkContext = binding.sparkContext
+ val appId = binding.applicationId
+ val attemptId = binding.attemptId
+ logInfo(s"Starting Yarn extension services with app $appId and attemptId $attemptId")
+
+ services = sparkContext.conf.get(SCHEDULER_SERVICES).map { sClass =>
+ val instance = Utils.classForName(sClass)
+ .newInstance()
+ .asInstanceOf[SchedulerExtensionService]
+ // bind this service
+ instance.start(binding)
+ logInfo(s"Service $sClass started")
+ instance
+ }.toList
+ }
+
+ /**
+ * Get the list of services.
+ *
+ * @return a list of services; Nil until the service is started
+ */
+ def getServices: List[SchedulerExtensionService] = services
+
+ /**
+ * Stop the services; idempotent.
+ *
+ */
+ override def stop(): Unit = {
+ if (started.getAndSet(false)) {
+ logInfo(s"Stopping $this")
+ services.foreach { s =>
+ Utils.tryLogNonFatalError(s.stop())
+ }
+ }
+ }
+
+ override def toString(): String = s"""SchedulerExtensionServices
+ |(serviceOption=$serviceOption,
+ | services=$services,
+ | started=$started)""".stripMargin
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
new file mode 100644
index 0000000000..60da356ad1
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
@@ -0,0 +1,157 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+import org.apache.hadoop.yarn.api.records.YarnApplicationState
+
+import org.apache.spark.{SparkContext, SparkException}
+import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnSparkHadoopUtil}
+import org.apache.spark.internal.Logging
+import org.apache.spark.launcher.SparkAppHandle
+import org.apache.spark.scheduler.TaskSchedulerImpl
+
+private[spark] class YarnClientSchedulerBackend(
+ scheduler: TaskSchedulerImpl,
+ sc: SparkContext)
+ extends YarnSchedulerBackend(scheduler, sc)
+ with Logging {
+
+ private var client: Client = null
+ private var monitorThread: MonitorThread = null
+
+ /**
+ * Create a Yarn client to submit an application to the ResourceManager.
+ * This waits until the application is running.
+ */
+ override def start() {
+ val driverHost = conf.get("spark.driver.host")
+ val driverPort = conf.get("spark.driver.port")
+ val hostport = driverHost + ":" + driverPort
+ sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.webUrl) }
+
+ val argsArrayBuf = new ArrayBuffer[String]()
+ argsArrayBuf += ("--arg", hostport)
+
+ logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" "))
+ val args = new ClientArguments(argsArrayBuf.toArray)
+ totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(conf)
+ client = new Client(args, conf)
+ bindToYarn(client.submitApplication(), None)
+
+ // SPARK-8687: Ensure all necessary properties have already been set before
+ // we initialize our driver scheduler backend, which serves these properties
+ // to the executors
+ super.start()
+ waitForApplication()
+
+ // SPARK-8851: In yarn-client mode, the AM still does the credentials refresh. The driver
+ // reads the credentials from HDFS, just like the executors and updates its own credentials
+ // cache.
+ if (conf.contains("spark.yarn.credentials.file")) {
+ YarnSparkHadoopUtil.get.startCredentialUpdater(conf)
+ }
+ monitorThread = asyncMonitorApplication()
+ monitorThread.start()
+ }
+
+ /**
+ * Report the state of the application until it is running.
+ * If the application has finished, failed or been killed in the process, throw an exception.
+ * This assumes both `client` and `appId` have already been set.
+ */
+ private def waitForApplication(): Unit = {
+ assert(client != null && appId.isDefined, "Application has not been submitted yet!")
+ val (state, _) = client.monitorApplication(appId.get, returnOnRunning = true) // blocking
+ if (state == YarnApplicationState.FINISHED ||
+ state == YarnApplicationState.FAILED ||
+ state == YarnApplicationState.KILLED) {
+ throw new SparkException("Yarn application has already ended! " +
+ "It might have been killed or unable to launch application master.")
+ }
+ if (state == YarnApplicationState.RUNNING) {
+ logInfo(s"Application ${appId.get} has started running.")
+ }
+ }
+
+ /**
+ * We create this class for SPARK-9519. Basically when we interrupt the monitor thread it's
+ * because the SparkContext is being shut down(sc.stop() called by user code), but if
+ * monitorApplication return, it means the Yarn application finished before sc.stop() was called,
+ * which means we should call sc.stop() here, and we don't allow the monitor to be interrupted
+ * before SparkContext stops successfully.
+ */
+ private class MonitorThread extends Thread {
+ private var allowInterrupt = true
+
+ override def run() {
+ try {
+ val (state, _) = client.monitorApplication(appId.get, logApplicationReport = false)
+ logError(s"Yarn application has already exited with state $state!")
+ allowInterrupt = false
+ sc.stop()
+ } catch {
+ case e: InterruptedException => logInfo("Interrupting monitor thread")
+ }
+ }
+
+ def stopMonitor(): Unit = {
+ if (allowInterrupt) {
+ this.interrupt()
+ }
+ }
+ }
+
+ /**
+ * Monitor the application state in a separate thread.
+ * If the application has exited for any reason, stop the SparkContext.
+ * This assumes both `client` and `appId` have already been set.
+ */
+ private def asyncMonitorApplication(): MonitorThread = {
+ assert(client != null && appId.isDefined, "Application has not been submitted yet!")
+ val t = new MonitorThread
+ t.setName("Yarn application state monitor")
+ t.setDaemon(true)
+ t
+ }
+
+ /**
+ * Stop the scheduler. This assumes `start()` has already been called.
+ */
+ override def stop() {
+ assert(client != null, "Attempted to stop this scheduler before starting it!")
+ if (monitorThread != null) {
+ monitorThread.stopMonitor()
+ }
+
+ // Report a final state to the launcher if one is connected. This is needed since in client
+ // mode this backend doesn't let the app monitor loop run to completion, so it does not report
+ // the final state itself.
+ //
+ // Note: there's not enough information at this point to provide a better final state,
+ // so assume the application was successful.
+ client.reportLauncherState(SparkAppHandle.State.FINISHED)
+
+ super.stop()
+ YarnSparkHadoopUtil.get.stopCredentialUpdater()
+ client.stop()
+ logInfo("Stopped")
+ }
+
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala
new file mode 100644
index 0000000000..64cd1bd088
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala
@@ -0,0 +1,56 @@
+/*
+ * 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.{SparkContext, SparkException}
+import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
+
+/**
+ * Cluster Manager for creation of Yarn scheduler and backend
+ */
+private[spark] class YarnClusterManager extends ExternalClusterManager {
+
+ override def canCreate(masterURL: String): Boolean = {
+ masterURL == "yarn"
+ }
+
+ override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = {
+ sc.deployMode match {
+ case "cluster" => new YarnClusterScheduler(sc)
+ case "client" => new YarnScheduler(sc)
+ case _ => throw new SparkException(s"Unknown deploy mode '${sc.deployMode}' for Yarn")
+ }
+ }
+
+ override def createSchedulerBackend(sc: SparkContext,
+ masterURL: String,
+ scheduler: TaskScheduler): SchedulerBackend = {
+ sc.deployMode match {
+ case "cluster" =>
+ new YarnClusterSchedulerBackend(scheduler.asInstanceOf[TaskSchedulerImpl], sc)
+ case "client" =>
+ new YarnClientSchedulerBackend(scheduler.asInstanceOf[TaskSchedulerImpl], sc)
+ case _ =>
+ throw new SparkException(s"Unknown deploy mode '${sc.deployMode}' for Yarn")
+ }
+ }
+
+ override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = {
+ scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend)
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
new file mode 100644
index 0000000000..96c9151fc3
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
@@ -0,0 +1,37 @@
+/*
+ * 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
+
+/**
+ * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of
+ * ApplicationMaster, etc is done
+ */
+private[spark] class YarnClusterScheduler(sc: SparkContext) extends YarnScheduler(sc) {
+
+ logInfo("Created YarnClusterScheduler")
+
+ override def postStartHook() {
+ ApplicationMaster.sparkContextInitialized(sc)
+ super.postStartHook()
+ logInfo("YarnClusterScheduler.postStartHook done")
+ }
+
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
new file mode 100644
index 0000000000..4f3d5ebf40
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala
@@ -0,0 +1,67 @@
+/*
+ * 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.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+
+import org.apache.spark.SparkContext
+import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnSparkHadoopUtil}
+import org.apache.spark.scheduler.TaskSchedulerImpl
+import org.apache.spark.util.Utils
+
+private[spark] class YarnClusterSchedulerBackend(
+ scheduler: TaskSchedulerImpl,
+ sc: SparkContext)
+ extends YarnSchedulerBackend(scheduler, sc) {
+
+ override def start() {
+ val attemptId = ApplicationMaster.getAttemptId
+ bindToYarn(attemptId.getApplicationId(), Some(attemptId))
+ super.start()
+ totalExpectedExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sc.conf)
+ }
+
+ override def getDriverLogUrls: Option[Map[String, String]] = {
+ var driverLogs: Option[Map[String, String]] = None
+ try {
+ val yarnConf = new YarnConfiguration(sc.hadoopConfiguration)
+ val containerId = YarnSparkHadoopUtil.get.getContainerId
+
+ val httpAddress = System.getenv(Environment.NM_HOST.name()) +
+ ":" + System.getenv(Environment.NM_HTTP_PORT.name())
+ // lookup appropriate http scheme for container log urls
+ val yarnHttpPolicy = yarnConf.get(
+ YarnConfiguration.YARN_HTTP_POLICY_KEY,
+ YarnConfiguration.YARN_HTTP_POLICY_DEFAULT
+ )
+ val user = Utils.getCurrentUserName()
+ val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://"
+ val baseUrl = s"$httpScheme$httpAddress/node/containerlogs/$containerId/$user"
+ logDebug(s"Base URL for logs: $baseUrl")
+ driverLogs = Some(Map(
+ "stdout" -> s"$baseUrl/stdout?start=-4096",
+ "stderr" -> s"$baseUrl/stderr?start=-4096"))
+ } catch {
+ case e: Exception =>
+ logInfo("Error while building AM log links, so AM" +
+ " logs link will not appear in application UI", e)
+ }
+ driverLogs
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala
new file mode 100644
index 0000000000..029382133d
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.util.RackResolver
+import org.apache.log4j.{Level, Logger}
+
+import org.apache.spark._
+import org.apache.spark.scheduler.TaskSchedulerImpl
+import org.apache.spark.util.Utils
+
+private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) {
+
+ // RackResolver logs an INFO message whenever it resolves a rack, which is way too often.
+ if (Logger.getLogger(classOf[RackResolver]).getLevel == null) {
+ Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN)
+ }
+
+ // By default, rack is unknown
+ override def getRackForHost(hostPort: String): Option[String] = {
+ val host = Utils.parseHostPort(hostPort)._1
+ Option(RackResolver.resolve(sc.hadoopConfiguration, host).getNetworkLocation)
+ }
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
new file mode 100644
index 0000000000..2f9ea1911f
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
@@ -0,0 +1,315 @@
+/*
+ * 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 scala.concurrent.{ExecutionContext, Future}
+import scala.util.{Failure, Success}
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
+
+import org.apache.spark.SparkContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rpc._
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
+import org.apache.spark.ui.JettyUtils
+import org.apache.spark.util.{RpcUtils, ThreadUtils}
+
+/**
+ * Abstract Yarn scheduler backend that contains common logic
+ * between the client and cluster Yarn scheduler backends.
+ */
+private[spark] abstract class YarnSchedulerBackend(
+ scheduler: TaskSchedulerImpl,
+ sc: SparkContext)
+ extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) {
+
+ override val minRegisteredRatio =
+ if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+ 0.8
+ } else {
+ super.minRegisteredRatio
+ }
+
+ protected var totalExpectedExecutors = 0
+
+ private val yarnSchedulerEndpoint = new YarnSchedulerEndpoint(rpcEnv)
+
+ private val yarnSchedulerEndpointRef = rpcEnv.setupEndpoint(
+ YarnSchedulerBackend.ENDPOINT_NAME, yarnSchedulerEndpoint)
+
+ private implicit val askTimeout = RpcUtils.askRpcTimeout(sc.conf)
+
+ /** Application ID. */
+ protected var appId: Option[ApplicationId] = None
+
+ /** Attempt ID. This is unset for client-mode schedulers */
+ private var attemptId: Option[ApplicationAttemptId] = None
+
+ /** Scheduler extension services. */
+ private val services: SchedulerExtensionServices = new SchedulerExtensionServices()
+
+ // Flag to specify whether this schedulerBackend should be reset.
+ private var shouldResetOnAmRegister = false
+
+ /**
+ * Bind to YARN. This *must* be done before calling [[start()]].
+ *
+ * @param appId YARN application ID
+ * @param attemptId Optional YARN attempt ID
+ */
+ protected def bindToYarn(appId: ApplicationId, attemptId: Option[ApplicationAttemptId]): Unit = {
+ this.appId = Some(appId)
+ this.attemptId = attemptId
+ }
+
+ override def start() {
+ require(appId.isDefined, "application ID unset")
+ val binding = SchedulerExtensionServiceBinding(sc, appId.get, attemptId)
+ services.start(binding)
+ super.start()
+ }
+
+ override def stop(): Unit = {
+ try {
+ // SPARK-12009: To prevent Yarn allocator from requesting backup for the executors which
+ // was Stopped by SchedulerBackend.
+ requestTotalExecutors(0, 0, Map.empty)
+ super.stop()
+ } finally {
+ services.stop()
+ }
+ }
+
+ /**
+ * Get the attempt ID for this run, if the cluster manager supports multiple
+ * attempts. Applications run in client mode will not have attempt IDs.
+ * This attempt ID only includes attempt counter, like "1", "2".
+ *
+ * @return The application attempt id, if available.
+ */
+ override def applicationAttemptId(): Option[String] = {
+ attemptId.map(_.getAttemptId.toString)
+ }
+
+ /**
+ * Get an application ID associated with the job.
+ * This returns the string value of [[appId]] if set, otherwise
+ * the locally-generated ID from the superclass.
+ * @return The application ID
+ */
+ override def applicationId(): String = {
+ appId.map(_.toString).getOrElse {
+ logWarning("Application ID is not initialized yet.")
+ super.applicationId
+ }
+ }
+
+ /**
+ * Request executors from the ApplicationMaster by specifying the total number desired.
+ * This includes executors already pending or running.
+ */
+ override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = {
+ yarnSchedulerEndpointRef.ask[Boolean](
+ RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount))
+ }
+
+ /**
+ * Request that the ApplicationMaster kill the specified executors.
+ */
+ override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = {
+ yarnSchedulerEndpointRef.ask[Boolean](KillExecutors(executorIds))
+ }
+
+ override def sufficientResourcesRegistered(): Boolean = {
+ totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio
+ }
+
+ /**
+ * Add filters to the SparkUI.
+ */
+ private def addWebUIFilter(
+ filterName: String,
+ filterParams: Map[String, String],
+ proxyBase: String): Unit = {
+ if (proxyBase != null && proxyBase.nonEmpty) {
+ System.setProperty("spark.ui.proxyBase", proxyBase)
+ }
+
+ val hasFilter =
+ filterName != null && filterName.nonEmpty &&
+ filterParams != null && filterParams.nonEmpty
+ if (hasFilter) {
+ logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase")
+ conf.set("spark.ui.filters", filterName)
+ filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) }
+ scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) }
+ }
+ }
+
+ override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
+ new YarnDriverEndpoint(rpcEnv, properties)
+ }
+
+ /**
+ * Reset the state of SchedulerBackend to the initial state. This is happened when AM is failed
+ * and re-registered itself to driver after a failure. The stale state in driver should be
+ * cleaned.
+ */
+ override protected def reset(): Unit = {
+ super.reset()
+ sc.executorAllocationManager.foreach(_.reset())
+ }
+
+ /**
+ * Override the DriverEndpoint to add extra logic for the case when an executor is disconnected.
+ * This endpoint communicates with the executors and queries the AM for an executor's exit
+ * status when the executor is disconnected.
+ */
+ private class YarnDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)])
+ extends DriverEndpoint(rpcEnv, sparkProperties) {
+
+ /**
+ * When onDisconnected is received at the driver endpoint, the superclass DriverEndpoint
+ * handles it by assuming the Executor was lost for a bad reason and removes the executor
+ * immediately.
+ *
+ * In YARN's case however it is crucial to talk to the application master and ask why the
+ * executor had exited. If the executor exited for some reason unrelated to the running tasks
+ * (e.g., preemption), according to the application master, then we pass that information down
+ * to the TaskSetManager to inform the TaskSetManager that tasks on that lost executor should
+ * not count towards a job failure.
+ */
+ override def onDisconnected(rpcAddress: RpcAddress): Unit = {
+ addressToExecutorId.get(rpcAddress).foreach { executorId =>
+ if (disableExecutor(executorId)) {
+ yarnSchedulerEndpoint.handleExecutorDisconnectedFromDriver(executorId, rpcAddress)
+ }
+ }
+ }
+ }
+
+ /**
+ * An [[RpcEndpoint]] that communicates with the ApplicationMaster.
+ */
+ private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv)
+ extends ThreadSafeRpcEndpoint with Logging {
+ private var amEndpoint: Option[RpcEndpointRef] = None
+
+ private[YarnSchedulerBackend] def handleExecutorDisconnectedFromDriver(
+ executorId: String,
+ executorRpcAddress: RpcAddress): Unit = {
+ val removeExecutorMessage = amEndpoint match {
+ case Some(am) =>
+ val lossReasonRequest = GetExecutorLossReason(executorId)
+ am.ask[ExecutorLossReason](lossReasonRequest, askTimeout)
+ .map { reason => RemoveExecutor(executorId, reason) }(ThreadUtils.sameThread)
+ .recover {
+ case NonFatal(e) =>
+ logWarning(s"Attempted to get executor loss reason" +
+ s" for executor id ${executorId} at RPC address ${executorRpcAddress}," +
+ s" but got no response. Marking as slave lost.", e)
+ RemoveExecutor(executorId, SlaveLost())
+ }(ThreadUtils.sameThread)
+ case None =>
+ logWarning("Attempted to check for an executor loss reason" +
+ " before the AM has registered!")
+ Future.successful(RemoveExecutor(executorId, SlaveLost("AM is not yet registered.")))
+ }
+
+ removeExecutorMessage
+ .flatMap { message =>
+ driverEndpoint.ask[Boolean](message)
+ }(ThreadUtils.sameThread)
+ .onFailure {
+ case NonFatal(e) => logError(
+ s"Error requesting driver to remove executor $executorId after disconnection.", e)
+ }(ThreadUtils.sameThread)
+ }
+
+ override def receive: PartialFunction[Any, Unit] = {
+ case RegisterClusterManager(am) =>
+ logInfo(s"ApplicationMaster registered as $am")
+ amEndpoint = Option(am)
+ if (!shouldResetOnAmRegister) {
+ shouldResetOnAmRegister = true
+ } else {
+ // AM is already registered before, this potentially means that AM failed and
+ // a new one registered after the failure. This will only happen in yarn-client mode.
+ reset()
+ }
+
+ case AddWebUIFilter(filterName, filterParams, proxyBase) =>
+ addWebUIFilter(filterName, filterParams, proxyBase)
+
+ case r @ RemoveExecutor(executorId, reason) =>
+ logWarning(reason.toString)
+ driverEndpoint.ask[Boolean](r).onFailure {
+ case e =>
+ logError("Error requesting driver to remove executor" +
+ s" $executorId for reason $reason", e)
+ }(ThreadUtils.sameThread)
+ }
+
+
+ override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
+ case r: RequestExecutors =>
+ amEndpoint match {
+ case Some(am) =>
+ am.ask[Boolean](r).andThen {
+ case Success(b) => context.reply(b)
+ case Failure(NonFatal(e)) =>
+ logError(s"Sending $r to AM was unsuccessful", e)
+ context.sendFailure(e)
+ }(ThreadUtils.sameThread)
+ case None =>
+ logWarning("Attempted to request executors before the AM has registered!")
+ context.reply(false)
+ }
+
+ case k: KillExecutors =>
+ amEndpoint match {
+ case Some(am) =>
+ am.ask[Boolean](k).andThen {
+ case Success(b) => context.reply(b)
+ case Failure(NonFatal(e)) =>
+ logError(s"Sending $k to AM was unsuccessful", e)
+ context.sendFailure(e)
+ }(ThreadUtils.sameThread)
+ case None =>
+ logWarning("Attempted to kill executors before the AM has registered!")
+ context.reply(false)
+ }
+
+ case RetrieveLastAllocatedExecutorId =>
+ context.reply(currentExecutorIdCounter)
+ }
+
+ override def onDisconnected(remoteAddress: RpcAddress): Unit = {
+ if (amEndpoint.exists(_.address == remoteAddress)) {
+ logWarning(s"ApplicationMaster has disassociated: $remoteAddress")
+ amEndpoint = None
+ }
+ }
+ }
+}
+
+private[spark] object YarnSchedulerBackend {
+ val ENDPOINT_NAME = "YarnScheduler"
+}
diff --git a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
new file mode 100644
index 0000000000..d0ef5efa36
--- /dev/null
+++ b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider
@@ -0,0 +1 @@
+org.apache.spark.deploy.yarn.security.TestCredentialProvider
diff --git a/resource-managers/yarn/src/test/resources/log4j.properties b/resource-managers/yarn/src/test/resources/log4j.properties
new file mode 100644
index 0000000000..d13454d5ae
--- /dev/null
+++ b/resource-managers/yarn/src/test/resources/log4j.properties
@@ -0,0 +1,31 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file target/unit-tests.log
+log4j.rootCategory=DEBUG, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=true
+log4j.appender.file.file=target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
+
+# Ignore messages below warning level from a few verbose libraries.
+log4j.logger.com.sun.jersey=WARN
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.eclipse.jetty=WARN
+log4j.logger.org.mortbay=WARN
+log4j.logger.org.spark_project.jetty=WARN
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
new file mode 100644
index 0000000000..9c3b18e4ec
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala
@@ -0,0 +1,241 @@
+/*
+ * 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.{File, FileOutputStream, OutputStreamWriter}
+import java.nio.charset.StandardCharsets
+import java.util.Properties
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.google.common.io.Files
+import org.apache.commons.lang3.SerializationUtils
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.server.MiniYARNCluster
+import org.scalatest.{BeforeAndAfterAll, Matchers}
+import org.scalatest.concurrent.Eventually._
+
+import org.apache.spark._
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.launcher._
+import org.apache.spark.util.Utils
+
+abstract class BaseYarnClusterSuite
+ extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging {
+
+ // log4j configuration for the YARN containers, so that their output is collected
+ // by YARN instead of trying to overwrite unit-tests.log.
+ protected val LOG4J_CONF = """
+ |log4j.rootCategory=DEBUG, console
+ |log4j.appender.console=org.apache.log4j.ConsoleAppender
+ |log4j.appender.console.target=System.err
+ |log4j.appender.console.layout=org.apache.log4j.PatternLayout
+ |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
+ |log4j.logger.org.apache.hadoop=WARN
+ |log4j.logger.org.eclipse.jetty=WARN
+ |log4j.logger.org.mortbay=WARN
+ |log4j.logger.org.spark_project.jetty=WARN
+ """.stripMargin
+
+ private var yarnCluster: MiniYARNCluster = _
+ protected var tempDir: File = _
+ private var fakeSparkJar: File = _
+ protected var hadoopConfDir: File = _
+ private var logConfDir: File = _
+
+ var oldSystemProperties: Properties = null
+
+ def newYarnConfig(): YarnConfiguration
+
+ override def beforeAll() {
+ super.beforeAll()
+ oldSystemProperties = SerializationUtils.clone(System.getProperties)
+
+ tempDir = Utils.createTempDir()
+ logConfDir = new File(tempDir, "log4j")
+ logConfDir.mkdir()
+ System.setProperty("SPARK_YARN_MODE", "true")
+
+ val logConfFile = new File(logConfDir, "log4j.properties")
+ Files.write(LOG4J_CONF, logConfFile, StandardCharsets.UTF_8)
+
+ // Disable the disk utilization check to avoid the test hanging when people's disks are
+ // getting full.
+ val yarnConf = newYarnConfig()
+ yarnConf.set("yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage",
+ "100.0")
+
+ yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1)
+ yarnCluster.init(yarnConf)
+ yarnCluster.start()
+
+ // There's a race in MiniYARNCluster in which start() may return before the RM has updated
+ // its address in the configuration. You can see this in the logs by noticing that when
+ // MiniYARNCluster prints the address, it still has port "0" assigned, although later the
+ // test works sometimes:
+ //
+ // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0
+ //
+ // That log message prints the contents of the RM_ADDRESS config variable. If you check it
+ // later on, it looks something like this:
+ //
+ // INFO YarnClusterSuite: RM address in configuration is blah:42631
+ //
+ // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't
+ // done so in a timely manner (defined to be 10 seconds).
+ val config = yarnCluster.getConfig()
+ val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10)
+ while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") {
+ if (System.currentTimeMillis() > deadline) {
+ throw new IllegalStateException("Timed out waiting for RM to come up.")
+ }
+ logDebug("RM address still not set in configuration, waiting...")
+ TimeUnit.MILLISECONDS.sleep(100)
+ }
+
+ logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}")
+
+ fakeSparkJar = File.createTempFile("sparkJar", null, tempDir)
+ hadoopConfDir = new File(tempDir, Client.LOCALIZED_CONF_DIR)
+ assert(hadoopConfDir.mkdir())
+ File.createTempFile("token", ".txt", hadoopConfDir)
+ }
+
+ override def afterAll() {
+ try {
+ yarnCluster.stop()
+ } finally {
+ System.setProperties(oldSystemProperties)
+ super.afterAll()
+ }
+ }
+
+ protected def runSpark(
+ clientMode: Boolean,
+ klass: String,
+ appArgs: Seq[String] = Nil,
+ sparkArgs: Seq[(String, String)] = Nil,
+ extraClassPath: Seq[String] = Nil,
+ extraJars: Seq[String] = Nil,
+ extraConf: Map[String, String] = Map(),
+ extraEnv: Map[String, String] = Map()): SparkAppHandle.State = {
+ val deployMode = if (clientMode) "client" else "cluster"
+ val propsFile = createConfFile(extraClassPath = extraClassPath, extraConf = extraConf)
+ val env = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath()) ++ extraEnv
+
+ val launcher = new SparkLauncher(env.asJava)
+ if (klass.endsWith(".py")) {
+ launcher.setAppResource(klass)
+ } else {
+ launcher.setMainClass(klass)
+ launcher.setAppResource(fakeSparkJar.getAbsolutePath())
+ }
+ launcher.setSparkHome(sys.props("spark.test.home"))
+ .setMaster("yarn")
+ .setDeployMode(deployMode)
+ .setConf("spark.executor.instances", "1")
+ .setPropertiesFile(propsFile)
+ .addAppArgs(appArgs.toArray: _*)
+
+ sparkArgs.foreach { case (name, value) =>
+ if (value != null) {
+ launcher.addSparkArg(name, value)
+ } else {
+ launcher.addSparkArg(name)
+ }
+ }
+ extraJars.foreach(launcher.addJar)
+
+ val handle = launcher.startApplication()
+ try {
+ eventually(timeout(2 minutes), interval(1 second)) {
+ assert(handle.getState().isFinal())
+ }
+ } finally {
+ handle.kill()
+ }
+
+ handle.getState()
+ }
+
+ /**
+ * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide
+ * any sort of error when the job process finishes successfully, but the job itself fails. So
+ * the tests enforce that something is written to a file after everything is ok to indicate
+ * that the job succeeded.
+ */
+ protected def checkResult(finalState: SparkAppHandle.State, result: File): Unit = {
+ checkResult(finalState, result, "success")
+ }
+
+ protected def checkResult(
+ finalState: SparkAppHandle.State,
+ result: File,
+ expected: String): Unit = {
+ finalState should be (SparkAppHandle.State.FINISHED)
+ val resultString = Files.toString(result, StandardCharsets.UTF_8)
+ resultString should be (expected)
+ }
+
+ protected def mainClassName(klass: Class[_]): String = {
+ klass.getName().stripSuffix("$")
+ }
+
+ protected def createConfFile(
+ extraClassPath: Seq[String] = Nil,
+ extraConf: Map[String, String] = Map()): String = {
+ val props = new Properties()
+ props.put(SPARK_JARS.key, "local:" + fakeSparkJar.getAbsolutePath())
+
+ val testClasspath = new TestClasspathBuilder()
+ .buildClassPath(
+ logConfDir.getAbsolutePath() +
+ File.pathSeparator +
+ extraClassPath.mkString(File.pathSeparator))
+ .asScala
+ .mkString(File.pathSeparator)
+
+ props.put("spark.driver.extraClassPath", testClasspath)
+ props.put("spark.executor.extraClassPath", testClasspath)
+
+ // SPARK-4267: make sure java options are propagated correctly.
+ props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"")
+ props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"")
+
+ yarnCluster.getConfig().asScala.foreach { e =>
+ props.setProperty("spark.hadoop." + e.getKey(), e.getValue())
+ }
+ sys.props.foreach { case (k, v) =>
+ if (k.startsWith("spark.")) {
+ props.setProperty(k, v)
+ }
+ }
+ extraConf.foreach { case (k, v) => props.setProperty(k, v) }
+
+ val propsFile = File.createTempFile("spark", ".properties", tempDir)
+ val writer = new OutputStreamWriter(new FileOutputStream(propsFile), StandardCharsets.UTF_8)
+ props.store(writer, "Spark properties.")
+ writer.close()
+ propsFile.getAbsolutePath()
+ }
+
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
new file mode 100644
index 0000000000..b696e080ce
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
@@ -0,0 +1,204 @@
+/*
+ * 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 scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
+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.yarn.api.records.LocalResource
+import org.apache.hadoop.yarn.api.records.LocalResourceType
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility
+import org.apache.hadoop.yarn.util.ConverterUtils
+import org.mockito.Mockito.when
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.yarn.config._
+
+class ClientDistributedCacheManagerSuite extends SparkFunSuite with MockitoSugar {
+
+ class MockClientDistributedCacheManager extends ClientDistributedCacheManager {
+ override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]):
+ LocalResourceVisibility = {
+ 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 sparkConf = new SparkConf(false)
+ distMgr.updateConfiguration(sparkConf)
+ assert(sparkConf.get(CACHED_FILES) === Seq("file:/foo.invalid.com:8080/tmp/testing#link"))
+ assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Seq(0L))
+ assert(sparkConf.get(CACHED_FILES_SIZES) === Seq(0L))
+ assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Seq(LocalResourceVisibility.PRIVATE.name()))
+ assert(sparkConf.get(CACHED_FILES_TYPES) === Seq(LocalResourceType.FILE.name()))
+
+ // 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 sparkConf2 = new SparkConf(false)
+ distMgr.updateConfiguration(sparkConf2)
+
+ val files = sparkConf2.get(CACHED_FILES)
+ val sizes = sparkConf2.get(CACHED_FILES_SIZES)
+ val timestamps = sparkConf2.get(CACHED_FILES_TIMESTAMPS)
+ val visibilities = sparkConf2.get(CACHED_FILES_VISIBILITIES)
+
+ 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 sparkConf = new SparkConf(false)
+ distMgr.updateConfiguration(sparkConf)
+ assert(sparkConf.get(CACHED_FILES) === Nil)
+ assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Nil)
+ assert(sparkConf.get(CACHED_FILES_SIZES) === Nil)
+ assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Nil)
+ assert(sparkConf.get(CACHED_FILES_TYPES) === Nil)
+ }
+
+ 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 sparkConf = new SparkConf(false)
+ distMgr.updateConfiguration(sparkConf)
+ assert(sparkConf.get(CACHED_FILES) === Seq("file:/foo.invalid.com:8080/tmp/testing#link"))
+ assert(sparkConf.get(CACHED_FILES_SIZES) === Seq(20L))
+ assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Seq(10L))
+ assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Seq(LocalResourceVisibility.PRIVATE.name()))
+ assert(sparkConf.get(CACHED_FILES_TYPES) === Seq(LocalResourceType.ARCHIVE.name()))
+ }
+
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
new file mode 100644
index 0000000000..7deaf0af94
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
@@ -0,0 +1,462 @@
+/*
+ * 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.{File, FileInputStream, FileOutputStream}
+import java.net.URI
+import java.util.Properties
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{HashMap => MutableHashMap}
+import scala.reflect.ClassTag
+import scala.util.Try
+
+import org.apache.commons.lang3.SerializationUtils
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.MRJobConfig
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.YarnClientApplication
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.util.Records
+import org.mockito.Matchers.{eq => meq, _}
+import org.mockito.Mockito._
+import org.scalatest.{BeforeAndAfterAll, Matchers}
+
+import org.apache.spark.{SparkConf, SparkFunSuite, TestUtils}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils}
+
+class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll
+ with ResetSystemProperties {
+
+ import Client._
+
+ var oldSystemProperties: Properties = null
+
+ override def beforeAll(): Unit = {
+ super.beforeAll()
+ oldSystemProperties = SerializationUtils.clone(System.getProperties)
+ System.setProperty("SPARK_YARN_MODE", "true")
+ }
+
+ override def afterAll(): Unit = {
+ try {
+ System.setProperties(oldSystemProperties)
+ oldSystemProperties = null
+ } finally {
+ super.afterAll()
+ }
+ }
+
+ test("default Yarn application classpath") {
+ getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP))
+ }
+
+ test("default MR application classpath") {
+ getDefaultMRApplicationClasspath should be(Some(Fixtures.knownDefMRAppCP))
+ }
+
+ test("resultant classpath for an application that defines a classpath for YARN") {
+ withAppConf(Fixtures.mapYARNAppConf) { conf =>
+ val env = newEnv
+ populateHadoopClasspath(conf, env)
+ classpath(env) should be(
+ flatten(Fixtures.knownYARNAppCP, getDefaultMRApplicationClasspath))
+ }
+ }
+
+ test("resultant classpath for an application that defines a classpath for MR") {
+ withAppConf(Fixtures.mapMRAppConf) { conf =>
+ val env = newEnv
+ populateHadoopClasspath(conf, env)
+ classpath(env) should be(
+ flatten(getDefaultYarnApplicationClasspath, Fixtures.knownMRAppCP))
+ }
+ }
+
+ test("resultant classpath for an application that defines both classpaths, YARN and MR") {
+ withAppConf(Fixtures.mapAppConf) { conf =>
+ val env = newEnv
+ populateHadoopClasspath(conf, env)
+ classpath(env) should be(flatten(Fixtures.knownYARNAppCP, Fixtures.knownMRAppCP))
+ }
+ }
+
+ private val SPARK = "local:/sparkJar"
+ private val USER = "local:/userJar"
+ private val ADDED = "local:/addJar1,local:/addJar2,/addJar3"
+
+ private val PWD =
+ if (classOf[Environment].getMethods().exists(_.getName == "$$")) {
+ "{{PWD}}"
+ } else if (Utils.isWindows) {
+ "%PWD%"
+ } else {
+ Environment.PWD.$()
+ }
+
+ test("Local jar URIs") {
+ val conf = new Configuration()
+ val sparkConf = new SparkConf()
+ .set(SPARK_JARS, Seq(SPARK))
+ .set(USER_CLASS_PATH_FIRST, true)
+ .set("spark.yarn.dist.jars", ADDED)
+ val env = new MutableHashMap[String, String]()
+ val args = new ClientArguments(Array("--jar", USER))
+
+ populateClasspath(args, conf, sparkConf, env)
+
+ val cp = env("CLASSPATH").split(":|;|<CPS>")
+ s"$SPARK,$USER,$ADDED".split(",").foreach({ entry =>
+ val uri = new URI(entry)
+ if (LOCAL_SCHEME.equals(uri.getScheme())) {
+ cp should contain (uri.getPath())
+ } else {
+ cp should not contain (uri.getPath())
+ }
+ })
+ cp should contain(PWD)
+ cp should contain (s"$PWD${Path.SEPARATOR}${LOCALIZED_CONF_DIR}")
+ cp should not contain (APP_JAR)
+ }
+
+ test("Jar path propagation through SparkConf") {
+ val conf = new Configuration()
+ val sparkConf = new SparkConf()
+ .set(SPARK_JARS, Seq(SPARK))
+ .set("spark.yarn.dist.jars", ADDED)
+ val client = createClient(sparkConf, args = Array("--jar", USER))
+ doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]),
+ any(classOf[Path]), anyShort(), anyBoolean(), any())
+
+ val tempDir = Utils.createTempDir()
+ try {
+ // Because we mocked "copyFileToRemote" above to avoid having to create fake local files,
+ // we need to create a fake config archive in the temp dir to avoid having
+ // prepareLocalResources throw an exception.
+ new FileOutputStream(new File(tempDir, LOCALIZED_CONF_ARCHIVE)).close()
+
+ client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
+ sparkConf.get(APP_JAR) should be (Some(USER))
+
+ // The non-local path should be propagated by name only, since it will end up in the app's
+ // staging dir.
+ val expected = ADDED.split(",")
+ .map(p => {
+ val uri = new URI(p)
+ if (LOCAL_SCHEME == uri.getScheme()) {
+ p
+ } else {
+ Option(uri.getFragment()).getOrElse(new File(p).getName())
+ }
+ })
+ .mkString(",")
+
+ sparkConf.get(SECONDARY_JARS) should be (Some(expected.split(",").toSeq))
+ } finally {
+ Utils.deleteRecursively(tempDir)
+ }
+ }
+
+ test("Cluster path translation") {
+ val conf = new Configuration()
+ val sparkConf = new SparkConf()
+ .set(SPARK_JARS, Seq("local:/localPath/spark.jar"))
+ .set(GATEWAY_ROOT_PATH, "/localPath")
+ .set(REPLACEMENT_ROOT_PATH, "/remotePath")
+
+ getClusterPath(sparkConf, "/localPath") should be ("/remotePath")
+ getClusterPath(sparkConf, "/localPath/1:/localPath/2") should be (
+ "/remotePath/1:/remotePath/2")
+
+ val env = new MutableHashMap[String, String]()
+ populateClasspath(null, conf, sparkConf, env, extraClassPath = Some("/localPath/my1.jar"))
+ val cp = classpath(env)
+ cp should contain ("/remotePath/spark.jar")
+ cp should contain ("/remotePath/my1.jar")
+ }
+
+ test("configuration and args propagate through createApplicationSubmissionContext") {
+ val conf = new Configuration()
+ // When parsing tags, duplicates and leading/trailing whitespace should be removed.
+ // Spaces between non-comma strings should be preserved as single tags. Empty strings may or
+ // may not be removed depending on the version of Hadoop being used.
+ val sparkConf = new SparkConf()
+ .set(APPLICATION_TAGS.key, ",tag1, dup,tag2 , ,multi word , dup")
+ .set(MAX_APP_ATTEMPTS, 42)
+ .set("spark.app.name", "foo-test-app")
+ .set(QUEUE_NAME, "staging-queue")
+ val args = new ClientArguments(Array())
+
+ val appContext = Records.newRecord(classOf[ApplicationSubmissionContext])
+ val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse])
+ val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext])
+
+ val client = new Client(args, conf, sparkConf)
+ client.createApplicationSubmissionContext(
+ new YarnClientApplication(getNewApplicationResponse, appContext),
+ containerLaunchContext)
+
+ appContext.getApplicationName should be ("foo-test-app")
+ appContext.getQueue should be ("staging-queue")
+ appContext.getAMContainerSpec should be (containerLaunchContext)
+ appContext.getApplicationType should be ("SPARK")
+ appContext.getClass.getMethods.filter(_.getName.equals("getApplicationTags")).foreach{ method =>
+ val tags = method.invoke(appContext).asInstanceOf[java.util.Set[String]]
+ tags should contain allOf ("tag1", "dup", "tag2", "multi word")
+ tags.asScala.count(_.nonEmpty) should be (4)
+ }
+ appContext.getMaxAppAttempts should be (42)
+ }
+
+ test("spark.yarn.jars with multiple paths and globs") {
+ val libs = Utils.createTempDir()
+ val single = Utils.createTempDir()
+ val jar1 = TestUtils.createJarWithFiles(Map(), libs)
+ val jar2 = TestUtils.createJarWithFiles(Map(), libs)
+ val jar3 = TestUtils.createJarWithFiles(Map(), single)
+ val jar4 = TestUtils.createJarWithFiles(Map(), single)
+
+ val jarsConf = Seq(
+ s"${libs.getAbsolutePath()}/*",
+ jar3.getPath(),
+ s"local:${jar4.getPath()}",
+ s"local:${single.getAbsolutePath()}/*")
+
+ val sparkConf = new SparkConf().set(SPARK_JARS, jarsConf)
+ val client = createClient(sparkConf)
+
+ val tempDir = Utils.createTempDir()
+ client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
+
+ assert(sparkConf.get(SPARK_JARS) ===
+ Some(Seq(s"local:${jar4.getPath()}", s"local:${single.getAbsolutePath()}/*")))
+
+ verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar1.toURI())), anyShort(),
+ anyBoolean(), any())
+ verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar2.toURI())), anyShort(),
+ anyBoolean(), any())
+ verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar3.toURI())), anyShort(),
+ anyBoolean(), any())
+
+ val cp = classpath(client)
+ cp should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
+ cp should not contain (jar3.getPath())
+ cp should contain (jar4.getPath())
+ cp should contain (buildPath(single.getAbsolutePath(), "*"))
+ }
+
+ test("distribute jars archive") {
+ val temp = Utils.createTempDir()
+ val archive = TestUtils.createJarWithFiles(Map(), temp)
+
+ val sparkConf = new SparkConf().set(SPARK_ARCHIVE, archive.getPath())
+ val client = createClient(sparkConf)
+ client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil)
+
+ verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(archive.toURI())), anyShort(),
+ anyBoolean(), any())
+ classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
+
+ sparkConf.set(SPARK_ARCHIVE, LOCAL_SCHEME + ":" + archive.getPath())
+ intercept[IllegalArgumentException] {
+ client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil)
+ }
+ }
+
+ test("distribute archive multiple times") {
+ val libs = Utils.createTempDir()
+ // Create jars dir and RELEASE file to avoid IllegalStateException.
+ val jarsDir = new File(libs, "jars")
+ assert(jarsDir.mkdir())
+ new FileOutputStream(new File(libs, "RELEASE")).close()
+
+ val userLib1 = Utils.createTempDir()
+ val testJar = TestUtils.createJarWithFiles(Map(), userLib1)
+
+ // Case 1: FILES_TO_DISTRIBUTE and ARCHIVES_TO_DISTRIBUTE can't have duplicate files
+ val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+ .set(FILES_TO_DISTRIBUTE, Seq(testJar.getPath))
+ .set(ARCHIVES_TO_DISTRIBUTE, Seq(testJar.getPath))
+
+ val client = createClient(sparkConf)
+ val tempDir = Utils.createTempDir()
+ intercept[IllegalArgumentException] {
+ client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
+ }
+
+ // Case 2: FILES_TO_DISTRIBUTE can't have duplicate files.
+ val sparkConfFiles = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+ .set(FILES_TO_DISTRIBUTE, Seq(testJar.getPath, testJar.getPath))
+
+ val clientFiles = createClient(sparkConfFiles)
+ val tempDirForFiles = Utils.createTempDir()
+ intercept[IllegalArgumentException] {
+ clientFiles.prepareLocalResources(new Path(tempDirForFiles.getAbsolutePath()), Nil)
+ }
+
+ // Case 3: ARCHIVES_TO_DISTRIBUTE can't have duplicate files.
+ val sparkConfArchives = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+ .set(ARCHIVES_TO_DISTRIBUTE, Seq(testJar.getPath, testJar.getPath))
+
+ val clientArchives = createClient(sparkConfArchives)
+ val tempDirForArchives = Utils.createTempDir()
+ intercept[IllegalArgumentException] {
+ clientArchives.prepareLocalResources(new Path(tempDirForArchives.getAbsolutePath()), Nil)
+ }
+
+ // Case 4: FILES_TO_DISTRIBUTE can have unique file.
+ val sparkConfFilesUniq = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+ .set(FILES_TO_DISTRIBUTE, Seq(testJar.getPath))
+
+ val clientFilesUniq = createClient(sparkConfFilesUniq)
+ val tempDirForFilesUniq = Utils.createTempDir()
+ clientFilesUniq.prepareLocalResources(new Path(tempDirForFilesUniq.getAbsolutePath()), Nil)
+
+ // Case 5: ARCHIVES_TO_DISTRIBUTE can have unique file.
+ val sparkConfArchivesUniq = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+ .set(ARCHIVES_TO_DISTRIBUTE, Seq(testJar.getPath))
+
+ val clientArchivesUniq = createClient(sparkConfArchivesUniq)
+ val tempDirArchivesUniq = Utils.createTempDir()
+ clientArchivesUniq.prepareLocalResources(new Path(tempDirArchivesUniq.getAbsolutePath()), Nil)
+
+ }
+
+ test("distribute local spark jars") {
+ val temp = Utils.createTempDir()
+ val jarsDir = new File(temp, "jars")
+ assert(jarsDir.mkdir())
+ val jar = TestUtils.createJarWithFiles(Map(), jarsDir)
+ new FileOutputStream(new File(temp, "RELEASE")).close()
+
+ val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> temp.getAbsolutePath()))
+ val client = createClient(sparkConf)
+ client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil)
+ classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*"))
+ }
+
+ test("ignore same name jars") {
+ val libs = Utils.createTempDir()
+ val jarsDir = new File(libs, "jars")
+ assert(jarsDir.mkdir())
+ new FileOutputStream(new File(libs, "RELEASE")).close()
+ val userLib1 = Utils.createTempDir()
+ val userLib2 = Utils.createTempDir()
+
+ val jar1 = TestUtils.createJarWithFiles(Map(), jarsDir)
+ val jar2 = TestUtils.createJarWithFiles(Map(), userLib1)
+ // Copy jar2 to jar3 with same name
+ val jar3 = {
+ val target = new File(userLib2, new File(jar2.toURI).getName)
+ val input = new FileInputStream(jar2.getPath)
+ val output = new FileOutputStream(target)
+ Utils.copyStream(input, output, closeStreams = true)
+ target.toURI.toURL
+ }
+
+ val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath))
+ .set(JARS_TO_DISTRIBUTE, Seq(jar2.getPath, jar3.getPath))
+
+ val client = createClient(sparkConf)
+ val tempDir = Utils.createTempDir()
+ client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil)
+
+ // Only jar2 will be added to SECONDARY_JARS, jar3 which has the same name with jar2 will be
+ // ignored.
+ sparkConf.get(SECONDARY_JARS) should be (Some(Seq(new File(jar2.toURI).getName)))
+ }
+
+ object Fixtures {
+
+ val knownDefYarnAppCP: Seq[String] =
+ getFieldValue[Array[String], Seq[String]](classOf[YarnConfiguration],
+ "DEFAULT_YARN_APPLICATION_CLASSPATH",
+ Seq[String]())(a => a.toSeq)
+
+
+ val knownDefMRAppCP: Seq[String] =
+ getFieldValue2[String, Array[String], Seq[String]](
+ classOf[MRJobConfig],
+ "DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH",
+ Seq[String]())(a => a.split(","))(a => a.toSeq)
+
+ val knownYARNAppCP = Some(Seq("/known/yarn/path"))
+
+ val knownMRAppCP = Some(Seq("/known/mr/path"))
+
+ val mapMRAppConf =
+ Map("mapreduce.application.classpath" -> knownMRAppCP.map(_.mkString(":")).get)
+
+ val mapYARNAppConf =
+ Map(YarnConfiguration.YARN_APPLICATION_CLASSPATH -> knownYARNAppCP.map(_.mkString(":")).get)
+
+ val mapAppConf = mapYARNAppConf ++ mapMRAppConf
+ }
+
+ def withAppConf(m: Map[String, String] = Map())(testCode: (Configuration) => Any) {
+ val conf = new Configuration
+ m.foreach { case (k, v) => conf.set(k, v, "ClientSpec") }
+ testCode(conf)
+ }
+
+ def newEnv: MutableHashMap[String, String] = MutableHashMap[String, String]()
+
+ def classpath(env: MutableHashMap[String, String]): Array[String] =
+ env(Environment.CLASSPATH.name).split(":|;|<CPS>")
+
+ def flatten(a: Option[Seq[String]], b: Option[Seq[String]]): Array[String] =
+ (a ++ b).flatten.toArray
+
+ def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = {
+ Try(clazz.getField(field))
+ .map(_.get(null).asInstanceOf[A])
+ .toOption
+ .map(mapTo)
+ .getOrElse(defaults)
+ }
+
+ def getFieldValue2[A: ClassTag, A1: ClassTag, B](
+ clazz: Class[_],
+ field: String,
+ defaults: => B)(mapTo: A => B)(mapTo1: A1 => B): B = {
+ Try(clazz.getField(field)).map(_.get(null)).map {
+ case v: A => mapTo(v)
+ case v1: A1 => mapTo1(v1)
+ case _ => defaults
+ }.toOption.getOrElse(defaults)
+ }
+
+ private def createClient(
+ sparkConf: SparkConf,
+ conf: Configuration = new Configuration(),
+ args: Array[String] = Array()): Client = {
+ val clientArgs = new ClientArguments(args)
+ spy(new Client(clientArgs, conf, sparkConf))
+ }
+
+ private def classpath(client: Client): Array[String] = {
+ val env = new MutableHashMap[String, String]()
+ populateClasspath(null, client.hadoopConf, client.sparkConf, env)
+ classpath(env)
+ }
+
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
new file mode 100644
index 0000000000..afb4b691b5
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala
@@ -0,0 +1,153 @@
+/*
+ * 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.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.scalatest.{BeforeAndAfterEach, Matchers}
+
+import org.apache.spark.SparkFunSuite
+
+class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with BeforeAndAfterEach {
+
+ private val yarnAllocatorSuite = new YarnAllocatorSuite
+ import yarnAllocatorSuite._
+
+ def createContainerRequest(nodes: Array[String]): ContainerRequest =
+ new ContainerRequest(containerResource, nodes, null, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)
+
+ override def beforeEach() {
+ yarnAllocatorSuite.beforeEach()
+ }
+
+ override def afterEach() {
+ yarnAllocatorSuite.afterEach()
+ }
+
+ test("allocate locality preferred containers with enough resource and no matched existed " +
+ "containers") {
+ // 1. All the locations of current containers cannot satisfy the new requirements
+ // 2. Current requested container number can fully satisfy the pending tasks.
+
+ val handler = createAllocator(2)
+ handler.updateResourceRequests()
+ handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2")))
+
+ val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+ 3, 15, Map("host3" -> 15, "host4" -> 15, "host5" -> 10),
+ handler.allocatedHostToContainersMap, Seq.empty)
+
+ assert(localities.map(_.nodes) === Array(
+ Array("host3", "host4", "host5"),
+ Array("host3", "host4", "host5"),
+ Array("host3", "host4")))
+ }
+
+ test("allocate locality preferred containers with enough resource and partially matched " +
+ "containers") {
+ // 1. Parts of current containers' locations can satisfy the new requirements
+ // 2. Current requested container number can fully satisfy the pending tasks.
+
+ val handler = createAllocator(3)
+ handler.updateResourceRequests()
+ handler.handleAllocatedContainers(Array(
+ createContainer("host1"),
+ createContainer("host1"),
+ createContainer("host2")
+ ))
+
+ val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+ 3, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
+ handler.allocatedHostToContainersMap, Seq.empty)
+
+ assert(localities.map(_.nodes) ===
+ Array(null, Array("host2", "host3"), Array("host2", "host3")))
+ }
+
+ test("allocate locality preferred containers with limited resource and partially matched " +
+ "containers") {
+ // 1. Parts of current containers' locations can satisfy the new requirements
+ // 2. Current requested container number cannot fully satisfy the pending tasks.
+
+ val handler = createAllocator(3)
+ handler.updateResourceRequests()
+ handler.handleAllocatedContainers(Array(
+ createContainer("host1"),
+ createContainer("host1"),
+ createContainer("host2")
+ ))
+
+ val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+ 1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
+ handler.allocatedHostToContainersMap, Seq.empty)
+
+ assert(localities.map(_.nodes) === Array(Array("host2", "host3")))
+ }
+
+ test("allocate locality preferred containers with fully matched containers") {
+ // Current containers' locations can fully satisfy the new requirements
+
+ val handler = createAllocator(5)
+ handler.updateResourceRequests()
+ handler.handleAllocatedContainers(Array(
+ createContainer("host1"),
+ createContainer("host1"),
+ createContainer("host2"),
+ createContainer("host2"),
+ createContainer("host3")
+ ))
+
+ val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+ 3, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
+ handler.allocatedHostToContainersMap, Seq.empty)
+
+ assert(localities.map(_.nodes) === Array(null, null, null))
+ }
+
+ test("allocate containers with no locality preference") {
+ // Request new container without locality preference
+
+ val handler = createAllocator(2)
+ handler.updateResourceRequests()
+ handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2")))
+
+ val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+ 1, 0, Map.empty, handler.allocatedHostToContainersMap, Seq.empty)
+
+ assert(localities.map(_.nodes) === Array(null))
+ }
+
+ test("allocate locality preferred containers by considering the localities of pending requests") {
+ val handler = createAllocator(3)
+ handler.updateResourceRequests()
+ handler.handleAllocatedContainers(Array(
+ createContainer("host1"),
+ createContainer("host1"),
+ createContainer("host2")
+ ))
+
+ val pendingAllocationRequests = Seq(
+ createContainerRequest(Array("host2", "host3")),
+ createContainerRequest(Array("host1", "host4")))
+
+ val localities = handler.containerPlacementStrategy.localityOfRequestedContainers(
+ 1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10),
+ handler.allocatedHostToContainersMap, pendingAllocationRequests)
+
+ assert(localities.map(_.nodes) === Array(Array("host3")))
+ }
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
new file mode 100644
index 0000000000..994dc75d34
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
@@ -0,0 +1,344 @@
+/*
+ * 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.util.{Arrays, List => JList}
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic
+import org.apache.hadoop.net.DNSToSwitchMapping
+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.mockito.Mockito._
+import org.scalatest.{BeforeAndAfterEach, Matchers}
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.deploy.yarn.YarnAllocator._
+import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._
+import org.apache.spark.rpc.RpcEndpointRef
+import org.apache.spark.scheduler.SplitInfo
+import org.apache.spark.util.ManualClock
+
+class MockResolver extends DNSToSwitchMapping {
+
+ override def resolve(names: JList[String]): JList[String] = {
+ if (names.size > 0 && names.get(0) == "host3") Arrays.asList("/rack2")
+ else Arrays.asList("/rack1")
+ }
+
+ override def reloadCachedMappings() {}
+
+ def reloadCachedMappings(names: JList[String]) {}
+}
+
+class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach {
+ val conf = new YarnConfiguration()
+ conf.setClass(
+ CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
+ classOf[MockResolver], classOf[DNSToSwitchMapping])
+
+ val sparkConf = new SparkConf()
+ sparkConf.set("spark.driver.host", "localhost")
+ sparkConf.set("spark.driver.port", "4040")
+ sparkConf.set(SPARK_JARS, Seq("notarealjar.jar"))
+ sparkConf.set("spark.yarn.launchContainers", "false")
+
+ val appAttemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0)
+
+ // Resource returned by YARN. YARN can give larger containers than requested, so give 6 cores
+ // instead of the 5 requested and 3 GB instead of the 2 requested.
+ val containerResource = Resource.newInstance(3072, 6)
+
+ var rmClient: AMRMClient[ContainerRequest] = _
+
+ var containerNum = 0
+
+ override def beforeEach() {
+ super.beforeEach()
+ rmClient = AMRMClient.createAMRMClient()
+ rmClient.init(conf)
+ rmClient.start()
+ }
+
+ override def afterEach() {
+ try {
+ rmClient.stop()
+ } finally {
+ super.afterEach()
+ }
+ }
+
+ class MockSplitInfo(host: String) extends SplitInfo(null, host, null, 1, null) {
+ override def hashCode(): Int = 0
+ override def equals(other: Any): Boolean = false
+ }
+
+ def createAllocator(maxExecutors: Int = 5): YarnAllocator = {
+ val args = Array(
+ "--jar", "somejar.jar",
+ "--class", "SomeClass")
+ val sparkConfClone = sparkConf.clone()
+ sparkConfClone
+ .set("spark.executor.instances", maxExecutors.toString)
+ .set("spark.executor.cores", "5")
+ .set("spark.executor.memory", "2048")
+ new YarnAllocator(
+ "not used",
+ mock(classOf[RpcEndpointRef]),
+ conf,
+ sparkConfClone,
+ rmClient,
+ appAttemptId,
+ new SecurityManager(sparkConf),
+ Map())
+ }
+
+ def createContainer(host: String): Container = {
+ // When YARN 2.6+ is required, avoid deprecation by using version with long second arg
+ val containerId = ContainerId.newInstance(appAttemptId, containerNum)
+ containerNum += 1
+ val nodeId = NodeId.newInstance(host, 1000)
+ Container.newInstance(containerId, nodeId, "", containerResource, RM_REQUEST_PRIORITY, null)
+ }
+
+ test("single container allocated") {
+ // request a single container and receive it
+ val handler = createAllocator(1)
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (1)
+
+ val container = createContainer("host1")
+ handler.handleAllocatedContainers(Array(container))
+
+ handler.getNumExecutorsRunning should be (1)
+ handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1")
+ handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId)
+
+ val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size
+ size should be (0)
+ }
+
+ test("container should not be created if requested number if met") {
+ // request a single container and receive it
+ val handler = createAllocator(1)
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (1)
+
+ val container = createContainer("host1")
+ handler.handleAllocatedContainers(Array(container))
+
+ handler.getNumExecutorsRunning should be (1)
+ handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1")
+ handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId)
+
+ val container2 = createContainer("host2")
+ handler.handleAllocatedContainers(Array(container2))
+ handler.getNumExecutorsRunning should be (1)
+ }
+
+ test("some containers allocated") {
+ // request a few containers and receive some of them
+ val handler = createAllocator(4)
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (4)
+
+ val container1 = createContainer("host1")
+ val container2 = createContainer("host1")
+ val container3 = createContainer("host2")
+ handler.handleAllocatedContainers(Array(container1, container2, container3))
+
+ handler.getNumExecutorsRunning should be (3)
+ handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1")
+ handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host1")
+ handler.allocatedContainerToHostMap.get(container3.getId).get should be ("host2")
+ handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId)
+ handler.allocatedHostToContainersMap.get("host1").get should contain (container2.getId)
+ handler.allocatedHostToContainersMap.get("host2").get should contain (container3.getId)
+ }
+
+ test("receive more containers than requested") {
+ val handler = createAllocator(2)
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (2)
+
+ val container1 = createContainer("host1")
+ val container2 = createContainer("host2")
+ val container3 = createContainer("host4")
+ handler.handleAllocatedContainers(Array(container1, container2, container3))
+
+ handler.getNumExecutorsRunning should be (2)
+ handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1")
+ handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host2")
+ handler.allocatedContainerToHostMap.contains(container3.getId) should be (false)
+ handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId)
+ handler.allocatedHostToContainersMap.get("host2").get should contain (container2.getId)
+ handler.allocatedHostToContainersMap.contains("host4") should be (false)
+ }
+
+ test("decrease total requested executors") {
+ val handler = createAllocator(4)
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (4)
+
+ handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty)
+ handler.updateResourceRequests()
+ handler.getPendingAllocate.size should be (3)
+
+ val container = createContainer("host1")
+ handler.handleAllocatedContainers(Array(container))
+
+ handler.getNumExecutorsRunning should be (1)
+ handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1")
+ handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId)
+
+ handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty)
+ handler.updateResourceRequests()
+ handler.getPendingAllocate.size should be (1)
+ }
+
+ test("decrease total requested executors to less than currently running") {
+ val handler = createAllocator(4)
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (4)
+
+ handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty)
+ handler.updateResourceRequests()
+ handler.getPendingAllocate.size should be (3)
+
+ val container1 = createContainer("host1")
+ val container2 = createContainer("host2")
+ handler.handleAllocatedContainers(Array(container1, container2))
+
+ handler.getNumExecutorsRunning should be (2)
+
+ handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty)
+ handler.updateResourceRequests()
+ handler.getPendingAllocate.size should be (0)
+ handler.getNumExecutorsRunning should be (2)
+ }
+
+ test("kill executors") {
+ val handler = createAllocator(4)
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (4)
+
+ val container1 = createContainer("host1")
+ val container2 = createContainer("host2")
+ handler.handleAllocatedContainers(Array(container1, container2))
+
+ handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty)
+ handler.executorIdToContainer.keys.foreach { id => handler.killExecutor(id ) }
+
+ val statuses = Seq(container1, container2).map { c =>
+ ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Finished", 0)
+ }
+ handler.updateResourceRequests()
+ handler.processCompletedContainers(statuses.toSeq)
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (1)
+ }
+
+ test("lost executor removed from backend") {
+ val handler = createAllocator(4)
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (4)
+
+ val container1 = createContainer("host1")
+ val container2 = createContainer("host2")
+ handler.handleAllocatedContainers(Array(container1, container2))
+
+ handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map())
+
+ val statuses = Seq(container1, container2).map { c =>
+ ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Failed", -1)
+ }
+ handler.updateResourceRequests()
+ handler.processCompletedContainers(statuses.toSeq)
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (2)
+ handler.getNumExecutorsFailed should be (2)
+ handler.getNumUnexpectedContainerRelease should be (2)
+ }
+
+ test("memory exceeded diagnostic regexes") {
+ val diagnostics =
+ "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " +
+ "beyond physical memory limits. Current usage: 2.1 MB of 2 GB physical memory used; " +
+ "5.8 GB of 4.2 GB virtual memory used. Killing container."
+ val vmemMsg = memLimitExceededLogMessage(diagnostics, VMEM_EXCEEDED_PATTERN)
+ val pmemMsg = memLimitExceededLogMessage(diagnostics, PMEM_EXCEEDED_PATTERN)
+ assert(vmemMsg.contains("5.8 GB of 4.2 GB virtual memory used."))
+ assert(pmemMsg.contains("2.1 MB of 2 GB physical memory used."))
+ }
+
+ test("window based failure executor counting") {
+ sparkConf.set("spark.yarn.executor.failuresValidityInterval", "100s")
+ val handler = createAllocator(4)
+ val clock = new ManualClock(0L)
+ handler.setClock(clock)
+
+ handler.updateResourceRequests()
+ handler.getNumExecutorsRunning should be (0)
+ handler.getPendingAllocate.size should be (4)
+
+ val containers = Seq(
+ createContainer("host1"),
+ createContainer("host2"),
+ createContainer("host3"),
+ createContainer("host4")
+ )
+ handler.handleAllocatedContainers(containers)
+
+ val failedStatuses = containers.map { c =>
+ ContainerStatus.newInstance(c.getId, ContainerState.COMPLETE, "Failed", -1)
+ }
+
+ handler.getNumExecutorsFailed should be (0)
+
+ clock.advance(100 * 1000L)
+ handler.processCompletedContainers(failedStatuses.slice(0, 1))
+ handler.getNumExecutorsFailed should be (1)
+
+ clock.advance(101 * 1000L)
+ handler.getNumExecutorsFailed should be (0)
+
+ handler.processCompletedContainers(failedStatuses.slice(1, 3))
+ handler.getNumExecutorsFailed should be (2)
+
+ clock.advance(50 * 1000L)
+ handler.processCompletedContainers(failedStatuses.slice(3, 4))
+ handler.getNumExecutorsFailed should be (3)
+
+ clock.advance(51 * 1000L)
+ handler.getNumExecutorsFailed should be (1)
+
+ clock.advance(50 * 1000L)
+ handler.getNumExecutorsFailed should be (0)
+ }
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
new file mode 100644
index 0000000000..99fb58a289
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -0,0 +1,493 @@
+/*
+ * 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.File
+import java.net.URL
+import java.nio.charset.StandardCharsets
+import java.util.{HashMap => JHashMap}
+
+import scala.collection.mutable
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import com.google.common.io.{ByteStreams, Files}
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.scalatest.Matchers
+import org.scalatest.concurrent.Eventually._
+
+import org.apache.spark._
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.launcher._
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart,
+ SparkListenerExecutorAdded}
+import org.apache.spark.scheduler.cluster.ExecutorInfo
+import org.apache.spark.tags.ExtendedYarnTest
+import org.apache.spark.util.Utils
+
+/**
+ * Integration tests for YARN; these tests use a mini Yarn cluster to run Spark-on-YARN
+ * applications, and require the Spark assembly to be built before they can be successfully
+ * run.
+ */
+@ExtendedYarnTest
+class YarnClusterSuite extends BaseYarnClusterSuite {
+
+ override def newYarnConfig(): YarnConfiguration = new YarnConfiguration()
+
+ private val TEST_PYFILE = """
+ |import mod1, mod2
+ |import sys
+ |from operator import add
+ |
+ |from pyspark import SparkConf , SparkContext
+ |if __name__ == "__main__":
+ | if len(sys.argv) != 2:
+ | print >> sys.stderr, "Usage: test.py [result file]"
+ | exit(-1)
+ | sc = SparkContext(conf=SparkConf())
+ | status = open(sys.argv[1],'w')
+ | result = "failure"
+ | rdd = sc.parallelize(range(10)).map(lambda x: x * mod1.func() * mod2.func())
+ | cnt = rdd.count()
+ | if cnt == 10:
+ | result = "success"
+ | status.write(result)
+ | status.close()
+ | sc.stop()
+ """.stripMargin
+
+ private val TEST_PYMODULE = """
+ |def func():
+ | return 42
+ """.stripMargin
+
+ test("run Spark in yarn-client mode") {
+ testBasicYarnApp(true)
+ }
+
+ test("run Spark in yarn-cluster mode") {
+ testBasicYarnApp(false)
+ }
+
+ test("run Spark in yarn-client mode with different configurations") {
+ testBasicYarnApp(true,
+ Map(
+ "spark.driver.memory" -> "512m",
+ "spark.executor.cores" -> "1",
+ "spark.executor.memory" -> "512m",
+ "spark.executor.instances" -> "2"
+ ))
+ }
+
+ test("run Spark in yarn-cluster mode with different configurations") {
+ testBasicYarnApp(false,
+ Map(
+ "spark.driver.memory" -> "512m",
+ "spark.driver.cores" -> "1",
+ "spark.executor.cores" -> "1",
+ "spark.executor.memory" -> "512m",
+ "spark.executor.instances" -> "2"
+ ))
+ }
+
+ test("run Spark in yarn-cluster mode with using SparkHadoopUtil.conf") {
+ testYarnAppUseSparkHadoopUtilConf()
+ }
+
+ test("run Spark in yarn-client mode with additional jar") {
+ testWithAddJar(true)
+ }
+
+ test("run Spark in yarn-cluster mode with additional jar") {
+ testWithAddJar(false)
+ }
+
+ test("run Spark in yarn-cluster mode unsuccessfully") {
+ // Don't provide arguments so the driver will fail.
+ val finalState = runSpark(false, mainClassName(YarnClusterDriver.getClass))
+ finalState should be (SparkAppHandle.State.FAILED)
+ }
+
+ test("run Spark in yarn-cluster mode failure after sc initialized") {
+ val finalState = runSpark(false, mainClassName(YarnClusterDriverWithFailure.getClass))
+ finalState should be (SparkAppHandle.State.FAILED)
+ }
+
+ test("run Python application in yarn-client mode") {
+ testPySpark(true)
+ }
+
+ test("run Python application in yarn-cluster mode") {
+ testPySpark(false)
+ }
+
+ test("run Python application in yarn-cluster mode using " +
+ " spark.yarn.appMasterEnv to override local envvar") {
+ testPySpark(
+ clientMode = false,
+ extraConf = Map(
+ "spark.yarn.appMasterEnv.PYSPARK_DRIVER_PYTHON"
+ -> sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python"),
+ "spark.yarn.appMasterEnv.PYSPARK_PYTHON"
+ -> sys.env.getOrElse("PYSPARK_PYTHON", "python")),
+ extraEnv = Map(
+ "PYSPARK_DRIVER_PYTHON" -> "not python",
+ "PYSPARK_PYTHON" -> "not python"))
+ }
+
+ test("user class path first in client mode") {
+ testUseClassPathFirst(true)
+ }
+
+ test("user class path first in cluster mode") {
+ testUseClassPathFirst(false)
+ }
+
+ test("monitor app using launcher library") {
+ val env = new JHashMap[String, String]()
+ env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath())
+
+ val propsFile = createConfFile()
+ val handle = new SparkLauncher(env)
+ .setSparkHome(sys.props("spark.test.home"))
+ .setConf("spark.ui.enabled", "false")
+ .setPropertiesFile(propsFile)
+ .setMaster("yarn")
+ .setDeployMode("client")
+ .setAppResource(SparkLauncher.NO_RESOURCE)
+ .setMainClass(mainClassName(YarnLauncherTestApp.getClass))
+ .startApplication()
+
+ try {
+ eventually(timeout(30 seconds), interval(100 millis)) {
+ handle.getState() should be (SparkAppHandle.State.RUNNING)
+ }
+
+ handle.getAppId() should not be (null)
+ handle.getAppId() should startWith ("application_")
+ handle.stop()
+
+ eventually(timeout(30 seconds), interval(100 millis)) {
+ handle.getState() should be (SparkAppHandle.State.KILLED)
+ }
+ } finally {
+ handle.kill()
+ }
+ }
+
+ test("timeout to get SparkContext in cluster mode triggers failure") {
+ val timeout = 2000
+ val finalState = runSpark(false, mainClassName(SparkContextTimeoutApp.getClass),
+ appArgs = Seq((timeout * 4).toString),
+ extraConf = Map(AM_MAX_WAIT_TIME.key -> timeout.toString))
+ finalState should be (SparkAppHandle.State.FAILED)
+ }
+
+ private def testBasicYarnApp(clientMode: Boolean, conf: Map[String, String] = Map()): Unit = {
+ val result = File.createTempFile("result", null, tempDir)
+ val finalState = runSpark(clientMode, mainClassName(YarnClusterDriver.getClass),
+ appArgs = Seq(result.getAbsolutePath()),
+ extraConf = conf)
+ checkResult(finalState, result)
+ }
+
+ private def testYarnAppUseSparkHadoopUtilConf(): Unit = {
+ val result = File.createTempFile("result", null, tempDir)
+ val finalState = runSpark(false,
+ mainClassName(YarnClusterDriverUseSparkHadoopUtilConf.getClass),
+ appArgs = Seq("key=value", result.getAbsolutePath()),
+ extraConf = Map("spark.hadoop.key" -> "value"))
+ checkResult(finalState, result)
+ }
+
+ private def testWithAddJar(clientMode: Boolean): Unit = {
+ val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir)
+ val driverResult = File.createTempFile("driver", null, tempDir)
+ val executorResult = File.createTempFile("executor", null, tempDir)
+ val finalState = runSpark(clientMode, mainClassName(YarnClasspathTest.getClass),
+ appArgs = Seq(driverResult.getAbsolutePath(), executorResult.getAbsolutePath()),
+ extraClassPath = Seq(originalJar.getPath()),
+ extraJars = Seq("local:" + originalJar.getPath()))
+ checkResult(finalState, driverResult, "ORIGINAL")
+ checkResult(finalState, executorResult, "ORIGINAL")
+ }
+
+ private def testPySpark(
+ clientMode: Boolean,
+ extraConf: Map[String, String] = Map(),
+ extraEnv: Map[String, String] = Map()): Unit = {
+ val primaryPyFile = new File(tempDir, "test.py")
+ Files.write(TEST_PYFILE, primaryPyFile, StandardCharsets.UTF_8)
+
+ // When running tests, let's not assume the user has built the assembly module, which also
+ // creates the pyspark archive. Instead, let's use PYSPARK_ARCHIVES_PATH to point at the
+ // needed locations.
+ val sparkHome = sys.props("spark.test.home")
+ val pythonPath = Seq(
+ s"$sparkHome/python/lib/py4j-0.10.4-src.zip",
+ s"$sparkHome/python")
+ val extraEnvVars = Map(
+ "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator),
+ "PYTHONPATH" -> pythonPath.mkString(File.pathSeparator)) ++ extraEnv
+
+ val moduleDir =
+ if (clientMode) {
+ // In client-mode, .py files added with --py-files are not visible in the driver.
+ // This is something that the launcher library would have to handle.
+ tempDir
+ } else {
+ val subdir = new File(tempDir, "pyModules")
+ subdir.mkdir()
+ subdir
+ }
+ val pyModule = new File(moduleDir, "mod1.py")
+ Files.write(TEST_PYMODULE, pyModule, StandardCharsets.UTF_8)
+
+ val mod2Archive = TestUtils.createJarWithFiles(Map("mod2.py" -> TEST_PYMODULE), moduleDir)
+ val pyFiles = Seq(pyModule.getAbsolutePath(), mod2Archive.getPath()).mkString(",")
+ val result = File.createTempFile("result", null, tempDir)
+
+ val finalState = runSpark(clientMode, primaryPyFile.getAbsolutePath(),
+ sparkArgs = Seq("--py-files" -> pyFiles),
+ appArgs = Seq(result.getAbsolutePath()),
+ extraEnv = extraEnvVars,
+ extraConf = extraConf)
+ checkResult(finalState, result)
+ }
+
+ private def testUseClassPathFirst(clientMode: Boolean): Unit = {
+ // Create a jar file that contains a different version of "test.resource".
+ val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir)
+ val userJar = TestUtils.createJarWithFiles(Map("test.resource" -> "OVERRIDDEN"), tempDir)
+ val driverResult = File.createTempFile("driver", null, tempDir)
+ val executorResult = File.createTempFile("executor", null, tempDir)
+ val finalState = runSpark(clientMode, mainClassName(YarnClasspathTest.getClass),
+ appArgs = Seq(driverResult.getAbsolutePath(), executorResult.getAbsolutePath()),
+ extraClassPath = Seq(originalJar.getPath()),
+ extraJars = Seq("local:" + userJar.getPath()),
+ extraConf = Map(
+ "spark.driver.userClassPathFirst" -> "true",
+ "spark.executor.userClassPathFirst" -> "true"))
+ checkResult(finalState, driverResult, "OVERRIDDEN")
+ checkResult(finalState, executorResult, "OVERRIDDEN")
+ }
+
+}
+
+private[spark] class SaveExecutorInfo extends SparkListener {
+ val addedExecutorInfos = mutable.Map[String, ExecutorInfo]()
+ var driverLogs: Option[collection.Map[String, String]] = None
+
+ override def onExecutorAdded(executor: SparkListenerExecutorAdded) {
+ addedExecutorInfos(executor.executorId) = executor.executorInfo
+ }
+
+ override def onApplicationStart(appStart: SparkListenerApplicationStart): Unit = {
+ driverLogs = appStart.driverLogs
+ }
+}
+
+private object YarnClusterDriverWithFailure extends Logging with Matchers {
+ def main(args: Array[String]): Unit = {
+ val sc = new SparkContext(new SparkConf()
+ .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
+ .setAppName("yarn test with failure"))
+
+ throw new Exception("exception after sc initialized")
+ }
+}
+
+private object YarnClusterDriverUseSparkHadoopUtilConf extends Logging with Matchers {
+ def main(args: Array[String]): Unit = {
+ if (args.length != 2) {
+ // scalastyle:off println
+ System.err.println(
+ s"""
+ |Invalid command line: ${args.mkString(" ")}
+ |
+ |Usage: YarnClusterDriverUseSparkHadoopUtilConf [hadoopConfKey=value] [result file]
+ """.stripMargin)
+ // scalastyle:on println
+ System.exit(1)
+ }
+
+ val sc = new SparkContext(new SparkConf()
+ .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
+ .setAppName("yarn test using SparkHadoopUtil's conf"))
+
+ val kv = args(0).split("=")
+ val status = new File(args(1))
+ var result = "failure"
+ try {
+ SparkHadoopUtil.get.conf.get(kv(0)) should be (kv(1))
+ result = "success"
+ } finally {
+ Files.write(result, status, StandardCharsets.UTF_8)
+ sc.stop()
+ }
+ }
+}
+
+private object YarnClusterDriver extends Logging with Matchers {
+
+ val WAIT_TIMEOUT_MILLIS = 10000
+
+ def main(args: Array[String]): Unit = {
+ if (args.length != 1) {
+ // scalastyle:off println
+ System.err.println(
+ s"""
+ |Invalid command line: ${args.mkString(" ")}
+ |
+ |Usage: YarnClusterDriver [result file]
+ """.stripMargin)
+ // scalastyle:on println
+ System.exit(1)
+ }
+
+ val sc = new SparkContext(new SparkConf()
+ .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
+ .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns"))
+ val conf = sc.getConf
+ val status = new File(args(0))
+ var result = "failure"
+ try {
+ val data = sc.parallelize(1 to 4, 4).collect().toSet
+ sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
+ data should be (Set(1, 2, 3, 4))
+ result = "success"
+
+ // Verify that the config archive is correctly placed in the classpath of all containers.
+ val confFile = "/" + Client.SPARK_CONF_FILE
+ assert(getClass().getResource(confFile) != null)
+ val configFromExecutors = sc.parallelize(1 to 4, 4)
+ .map { _ => Option(getClass().getResource(confFile)).map(_.toString).orNull }
+ .collect()
+ assert(configFromExecutors.find(_ == null) === None)
+ } finally {
+ Files.write(result, status, StandardCharsets.UTF_8)
+ sc.stop()
+ }
+
+ // verify log urls are present
+ val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo]
+ assert(listeners.size === 1)
+ val listener = listeners(0)
+ val executorInfos = listener.addedExecutorInfos.values
+ assert(executorInfos.nonEmpty)
+ executorInfos.foreach { info =>
+ assert(info.logUrlMap.nonEmpty)
+ }
+
+ // If we are running in yarn-cluster mode, verify that driver logs links and present and are
+ // in the expected format.
+ if (conf.get("spark.submit.deployMode") == "cluster") {
+ assert(listener.driverLogs.nonEmpty)
+ val driverLogs = listener.driverLogs.get
+ assert(driverLogs.size === 2)
+ assert(driverLogs.contains("stderr"))
+ assert(driverLogs.contains("stdout"))
+ val urlStr = driverLogs("stderr")
+ // Ensure that this is a valid URL, else this will throw an exception
+ new URL(urlStr)
+ val containerId = YarnSparkHadoopUtil.get.getContainerId
+ val user = Utils.getCurrentUserName()
+ assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096"))
+ }
+ }
+
+}
+
+private object YarnClasspathTest extends Logging {
+ def error(m: String, ex: Throwable = null): Unit = {
+ logError(m, ex)
+ // scalastyle:off println
+ System.out.println(m)
+ if (ex != null) {
+ ex.printStackTrace(System.out)
+ }
+ // scalastyle:on println
+ }
+
+ def main(args: Array[String]): Unit = {
+ if (args.length != 2) {
+ error(
+ s"""
+ |Invalid command line: ${args.mkString(" ")}
+ |
+ |Usage: YarnClasspathTest [driver result file] [executor result file]
+ """.stripMargin)
+ // scalastyle:on println
+ }
+
+ readResource(args(0))
+ val sc = new SparkContext(new SparkConf())
+ try {
+ sc.parallelize(Seq(1)).foreach { x => readResource(args(1)) }
+ } finally {
+ sc.stop()
+ }
+ }
+
+ private def readResource(resultPath: String): Unit = {
+ var result = "failure"
+ try {
+ val ccl = Thread.currentThread().getContextClassLoader()
+ val resource = ccl.getResourceAsStream("test.resource")
+ val bytes = ByteStreams.toByteArray(resource)
+ result = new String(bytes, 0, bytes.length, StandardCharsets.UTF_8)
+ } catch {
+ case t: Throwable =>
+ error(s"loading test.resource to $resultPath", t)
+ } finally {
+ Files.write(result, new File(resultPath), StandardCharsets.UTF_8)
+ }
+ }
+
+}
+
+private object YarnLauncherTestApp {
+
+ def main(args: Array[String]): Unit = {
+ // Do not stop the application; the test will stop it using the launcher lib. Just run a task
+ // that will prevent the process from exiting.
+ val sc = new SparkContext(new SparkConf())
+ sc.parallelize(Seq(1)).foreach { i =>
+ this.synchronized {
+ wait()
+ }
+ }
+ }
+
+}
+
+/**
+ * Used to test code in the AM that detects the SparkContext instance. Expects a single argument
+ * with the duration to sleep for, in ms.
+ */
+private object SparkContextTimeoutApp {
+
+ def main(args: Array[String]): Unit = {
+ val Array(sleepTime) = args
+ Thread.sleep(java.lang.Long.parseLong(sleepTime))
+ }
+
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
new file mode 100644
index 0000000000..950ebd9a2d
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala
@@ -0,0 +1,112 @@
+/*
+* 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.File
+import java.nio.charset.StandardCharsets
+
+import com.google.common.io.Files
+import org.apache.commons.io.FileUtils
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.scalatest.Matchers
+
+import org.apache.spark._
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.shuffle.ShuffleTestAccessor
+import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor}
+import org.apache.spark.tags.ExtendedYarnTest
+
+/**
+ * Integration test for the external shuffle service with a yarn mini-cluster
+ */
+@ExtendedYarnTest
+class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite {
+
+ override def newYarnConfig(): YarnConfiguration = {
+ val yarnConfig = new YarnConfiguration()
+ yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle")
+ yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"),
+ classOf[YarnShuffleService].getCanonicalName)
+ yarnConfig.set("spark.shuffle.service.port", "0")
+ yarnConfig
+ }
+
+ test("external shuffle service") {
+ val shuffleServicePort = YarnTestAccessor.getShuffleServicePort
+ val shuffleService = YarnTestAccessor.getShuffleServiceInstance
+
+ val registeredExecFile = YarnTestAccessor.getRegisteredExecutorFile(shuffleService)
+
+ logInfo("Shuffle service port = " + shuffleServicePort)
+ val result = File.createTempFile("result", null, tempDir)
+ val finalState = runSpark(
+ false,
+ mainClassName(YarnExternalShuffleDriver.getClass),
+ appArgs = Seq(result.getAbsolutePath(), registeredExecFile.getAbsolutePath),
+ extraConf = Map(
+ "spark.shuffle.service.enabled" -> "true",
+ "spark.shuffle.service.port" -> shuffleServicePort.toString
+ )
+ )
+ checkResult(finalState, result)
+ assert(YarnTestAccessor.getRegisteredExecutorFile(shuffleService).exists())
+ }
+}
+
+private object YarnExternalShuffleDriver extends Logging with Matchers {
+
+ val WAIT_TIMEOUT_MILLIS = 10000
+
+ def main(args: Array[String]): Unit = {
+ if (args.length != 2) {
+ // scalastyle:off println
+ System.err.println(
+ s"""
+ |Invalid command line: ${args.mkString(" ")}
+ |
+ |Usage: ExternalShuffleDriver [result file] [registered exec file]
+ """.stripMargin)
+ // scalastyle:on println
+ System.exit(1)
+ }
+
+ val sc = new SparkContext(new SparkConf()
+ .setAppName("External Shuffle Test"))
+ val conf = sc.getConf
+ val status = new File(args(0))
+ val registeredExecFile = new File(args(1))
+ logInfo("shuffle service executor file = " + registeredExecFile)
+ var result = "failure"
+ val execStateCopy = new File(registeredExecFile.getAbsolutePath + "_dup")
+ try {
+ val data = sc.parallelize(0 until 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }.
+ collect().toSet
+ sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
+ data should be ((0 until 10).map{x => x -> (x * 10 + 450)}.toSet)
+ result = "success"
+ // only one process can open a leveldb file at a time, so we copy the files
+ FileUtils.copyDirectory(registeredExecFile, execStateCopy)
+ assert(!ShuffleTestAccessor.reloadRegisteredExecutors(execStateCopy).isEmpty)
+ } finally {
+ sc.stop()
+ FileUtils.deleteDirectory(execStateCopy)
+ Files.write(result, status, StandardCharsets.UTF_8)
+ }
+ }
+
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
new file mode 100644
index 0000000000..7fbbe12609
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
@@ -0,0 +1,213 @@
+/*
+ * 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.{File, IOException}
+import java.nio.charset.StandardCharsets
+
+import com.google.common.io.{ByteStreams, Files}
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.yarn.api.ApplicationConstants
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.scalatest.Matchers
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.{ResetSystemProperties, Utils}
+
+class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging
+ with ResetSystemProperties {
+
+ val hasBash =
+ try {
+ val exitCode = Runtime.getRuntime().exec(Array("bash", "--version")).waitFor()
+ exitCode == 0
+ } catch {
+ case e: IOException =>
+ false
+ }
+
+ if (!hasBash) {
+ logWarning("Cannot execute bash, skipping bash tests.")
+ }
+
+ def bashTest(name: String)(fn: => Unit): Unit =
+ if (hasBash) test(name)(fn) else ignore(name)(fn)
+
+ bashTest("shell script escaping") {
+ val scriptFile = File.createTempFile("script.", ".sh", Utils.createTempDir())
+ val args = Array("arg1", "${arg.2}", "\"arg3\"", "'arg4'", "$arg5", "\\arg6")
+ try {
+ val argLine = args.map(a => YarnSparkHadoopUtil.escapeForShell(a)).mkString(" ")
+ Files.write(("bash -c \"echo " + argLine + "\"").getBytes(StandardCharsets.UTF_8), scriptFile)
+ scriptFile.setExecutable(true)
+
+ val proc = Runtime.getRuntime().exec(Array(scriptFile.getAbsolutePath()))
+ val out = new String(ByteStreams.toByteArray(proc.getInputStream())).trim()
+ val err = new String(ByteStreams.toByteArray(proc.getErrorStream()))
+ val exitCode = proc.waitFor()
+ exitCode should be (0)
+ out should be (args.mkString(" "))
+ } finally {
+ scriptFile.delete()
+ }
+ }
+
+ test("Yarn configuration override") {
+ val key = "yarn.nodemanager.hostname"
+ val default = new YarnConfiguration()
+
+ val sparkConf = new SparkConf()
+ .set("spark.hadoop." + key, "someHostName")
+ val yarnConf = new YarnSparkHadoopUtil().newConfiguration(sparkConf)
+
+ yarnConf.getClass() should be (classOf[YarnConfiguration])
+ yarnConf.get(key) should not be default.get(key)
+ }
+
+
+ test("test getApplicationAclsForYarn acls on") {
+
+ // spark acls on, just pick up default user
+ val sparkConf = new SparkConf()
+ sparkConf.set("spark.acls.enable", "true")
+
+ val securityMgr = new SecurityManager(sparkConf)
+ val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)
+
+ val viewAcls = acls.get(ApplicationAccessType.VIEW_APP)
+ val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP)
+
+ viewAcls match {
+ case Some(vacls) =>
+ val aclSet = vacls.split(',').map(_.trim).toSet
+ assert(aclSet.contains(System.getProperty("user.name", "invalid")))
+ case None =>
+ fail()
+ }
+ modifyAcls match {
+ case Some(macls) =>
+ val aclSet = macls.split(',').map(_.trim).toSet
+ assert(aclSet.contains(System.getProperty("user.name", "invalid")))
+ case None =>
+ fail()
+ }
+ }
+
+ test("test getApplicationAclsForYarn acls on and specify users") {
+
+ // default spark acls are on and specify acls
+ val sparkConf = new SparkConf()
+ sparkConf.set("spark.acls.enable", "true")
+ sparkConf.set("spark.ui.view.acls", "user1,user2")
+ sparkConf.set("spark.modify.acls", "user3,user4")
+
+ val securityMgr = new SecurityManager(sparkConf)
+ val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)
+
+ val viewAcls = acls.get(ApplicationAccessType.VIEW_APP)
+ val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP)
+
+ viewAcls match {
+ case Some(vacls) =>
+ val aclSet = vacls.split(',').map(_.trim).toSet
+ assert(aclSet.contains("user1"))
+ assert(aclSet.contains("user2"))
+ assert(aclSet.contains(System.getProperty("user.name", "invalid")))
+ case None =>
+ fail()
+ }
+ modifyAcls match {
+ case Some(macls) =>
+ val aclSet = macls.split(',').map(_.trim).toSet
+ assert(aclSet.contains("user3"))
+ assert(aclSet.contains("user4"))
+ assert(aclSet.contains(System.getProperty("user.name", "invalid")))
+ case None =>
+ fail()
+ }
+
+ }
+
+ test("test expandEnvironment result") {
+ val target = Environment.PWD
+ if (classOf[Environment].getMethods().exists(_.getName == "$$")) {
+ YarnSparkHadoopUtil.expandEnvironment(target) should be ("{{" + target + "}}")
+ } else if (Utils.isWindows) {
+ YarnSparkHadoopUtil.expandEnvironment(target) should be ("%" + target + "%")
+ } else {
+ YarnSparkHadoopUtil.expandEnvironment(target) should be ("$" + target)
+ }
+
+ }
+
+ test("test getClassPathSeparator result") {
+ if (classOf[ApplicationConstants].getFields().exists(_.getName == "CLASS_PATH_SEPARATOR")) {
+ YarnSparkHadoopUtil.getClassPathSeparator() should be ("<CPS>")
+ } else if (Utils.isWindows) {
+ YarnSparkHadoopUtil.getClassPathSeparator() should be (";")
+ } else {
+ YarnSparkHadoopUtil.getClassPathSeparator() should be (":")
+ }
+ }
+
+ test("check different hadoop utils based on env variable") {
+ try {
+ System.setProperty("SPARK_YARN_MODE", "true")
+ assert(SparkHadoopUtil.get.getClass === classOf[YarnSparkHadoopUtil])
+ System.setProperty("SPARK_YARN_MODE", "false")
+ assert(SparkHadoopUtil.get.getClass === classOf[SparkHadoopUtil])
+ } finally {
+ System.clearProperty("SPARK_YARN_MODE")
+ }
+ }
+
+
+
+ // This test needs to live here because it depends on isYarnMode returning true, which can only
+ // happen in the YARN module.
+ test("security manager token generation") {
+ try {
+ System.setProperty("SPARK_YARN_MODE", "true")
+ val initial = SparkHadoopUtil.get
+ .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY)
+ assert(initial === null || initial.length === 0)
+
+ val conf = new SparkConf()
+ .set(SecurityManager.SPARK_AUTH_CONF, "true")
+ .set(SecurityManager.SPARK_AUTH_SECRET_CONF, "unused")
+ val sm = new SecurityManager(conf)
+
+ val generated = SparkHadoopUtil.get
+ .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY)
+ assert(generated != null)
+ val genString = new Text(generated).toString()
+ assert(genString != "unused")
+ assert(sm.getSecretKey() === genString)
+ } finally {
+ // removeSecretKey() was only added in Hadoop 2.6, so instead we just set the secret
+ // to an empty string.
+ SparkHadoopUtil.get.addSecretKeyToUserCredentials(SecurityManager.SECRET_LOOKUP_KEY, "")
+ System.clearProperty("SPARK_YARN_MODE")
+ }
+ }
+
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala
new file mode 100644
index 0000000000..db4619e80c
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala
@@ -0,0 +1,150 @@
+/*
+ * 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.security
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.token.Token
+import org.scalatest.{BeforeAndAfter, Matchers}
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.yarn.config._
+
+class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter {
+ private var credentialManager: ConfigurableCredentialManager = null
+ private var sparkConf: SparkConf = null
+ private var hadoopConf: Configuration = null
+
+ override def beforeAll(): Unit = {
+ super.beforeAll()
+
+ sparkConf = new SparkConf()
+ hadoopConf = new Configuration()
+ System.setProperty("SPARK_YARN_MODE", "true")
+ }
+
+ override def afterAll(): Unit = {
+ System.clearProperty("SPARK_YARN_MODE")
+
+ super.afterAll()
+ }
+
+ test("Correctly load default credential providers") {
+ credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+
+ credentialManager.getServiceCredentialProvider("hdfs") should not be (None)
+ credentialManager.getServiceCredentialProvider("hbase") should not be (None)
+ credentialManager.getServiceCredentialProvider("hive") should not be (None)
+ }
+
+ test("disable hive credential provider") {
+ sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false")
+ credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+
+ credentialManager.getServiceCredentialProvider("hdfs") should not be (None)
+ credentialManager.getServiceCredentialProvider("hbase") should not be (None)
+ credentialManager.getServiceCredentialProvider("hive") should be (None)
+ }
+
+ test("using deprecated configurations") {
+ sparkConf.set("spark.yarn.security.tokens.hdfs.enabled", "false")
+ sparkConf.set("spark.yarn.security.tokens.hive.enabled", "false")
+ credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+
+ credentialManager.getServiceCredentialProvider("hdfs") should be (None)
+ credentialManager.getServiceCredentialProvider("hive") should be (None)
+ credentialManager.getServiceCredentialProvider("test") should not be (None)
+ credentialManager.getServiceCredentialProvider("hbase") should not be (None)
+ }
+
+ test("verify obtaining credentials from provider") {
+ credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+ val creds = new Credentials()
+
+ // Tokens can only be obtained from TestTokenProvider, for hdfs, hbase and hive tokens cannot
+ // be obtained.
+ credentialManager.obtainCredentials(hadoopConf, creds)
+ val tokens = creds.getAllTokens
+ tokens.size() should be (1)
+ tokens.iterator().next().getService should be (new Text("test"))
+ }
+
+ test("verify getting credential renewal info") {
+ credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf)
+ val creds = new Credentials()
+
+ val testCredentialProvider = credentialManager.getServiceCredentialProvider("test").get
+ .asInstanceOf[TestCredentialProvider]
+ // Only TestTokenProvider can get the time of next token renewal
+ val nextRenewal = credentialManager.obtainCredentials(hadoopConf, creds)
+ nextRenewal should be (testCredentialProvider.timeOfNextTokenRenewal)
+ }
+
+ test("obtain tokens For HiveMetastore") {
+ val hadoopConf = new Configuration()
+ hadoopConf.set("hive.metastore.kerberos.principal", "bob")
+ // thrift picks up on port 0 and bails out, without trying to talk to endpoint
+ hadoopConf.set("hive.metastore.uris", "http://localhost:0")
+
+ val hiveCredentialProvider = new HiveCredentialProvider()
+ val credentials = new Credentials()
+ hiveCredentialProvider.obtainCredentials(hadoopConf, sparkConf, credentials)
+
+ credentials.getAllTokens.size() should be (0)
+ }
+
+ test("Obtain tokens For HBase") {
+ val hadoopConf = new Configuration()
+ hadoopConf.set("hbase.security.authentication", "kerberos")
+
+ val hbaseTokenProvider = new HBaseCredentialProvider()
+ val creds = new Credentials()
+ hbaseTokenProvider.obtainCredentials(hadoopConf, sparkConf, creds)
+
+ creds.getAllTokens.size should be (0)
+ }
+}
+
+class TestCredentialProvider extends ServiceCredentialProvider {
+ val tokenRenewalInterval = 86400 * 1000L
+ var timeOfNextTokenRenewal = 0L
+
+ override def serviceName: String = "test"
+
+ override def credentialsRequired(conf: Configuration): Boolean = true
+
+ override def obtainCredentials(
+ hadoopConf: Configuration,
+ sparkConf: SparkConf,
+ creds: Credentials): Option[Long] = {
+ if (creds == null) {
+ // Guard out other unit test failures.
+ return None
+ }
+
+ val emptyToken = new Token()
+ emptyToken.setService(new Text("test"))
+ creds.addToken(emptyToken.getService, emptyToken)
+
+ val currTime = System.currentTimeMillis()
+ timeOfNextTokenRenewal = (currTime - currTime % tokenRenewalInterval) + tokenRenewalInterval
+
+ Some(timeOfNextTokenRenewal)
+ }
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala
new file mode 100644
index 0000000000..7b2da3f26e
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HDFSCredentialProviderSuite.scala
@@ -0,0 +1,71 @@
+/*
+ * 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.security
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.scalatest.{Matchers, PrivateMethodTester}
+
+import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
+
+class HDFSCredentialProviderSuite
+ extends SparkFunSuite
+ with PrivateMethodTester
+ with Matchers {
+ private val _getTokenRenewer = PrivateMethod[String]('getTokenRenewer)
+
+ private def getTokenRenewer(
+ hdfsCredentialProvider: HDFSCredentialProvider, conf: Configuration): String = {
+ hdfsCredentialProvider invokePrivate _getTokenRenewer(conf)
+ }
+
+ private var hdfsCredentialProvider: HDFSCredentialProvider = null
+
+ override def beforeAll() {
+ super.beforeAll()
+
+ if (hdfsCredentialProvider == null) {
+ hdfsCredentialProvider = new HDFSCredentialProvider()
+ }
+ }
+
+ override def afterAll() {
+ if (hdfsCredentialProvider != null) {
+ hdfsCredentialProvider = null
+ }
+
+ super.afterAll()
+ }
+
+ test("check token renewer") {
+ val hadoopConf = new Configuration()
+ hadoopConf.set("yarn.resourcemanager.address", "myrm:8033")
+ hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM")
+ val renewer = getTokenRenewer(hdfsCredentialProvider, hadoopConf)
+ renewer should be ("yarn/myrm:8032@SPARKTEST.COM")
+ }
+
+ test("check token renewer default") {
+ val hadoopConf = new Configuration()
+ val caught =
+ intercept[SparkException] {
+ getTokenRenewer(hdfsCredentialProvider, hadoopConf)
+ }
+ assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer")
+ }
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala
new file mode 100644
index 0000000000..da9e8e21a2
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala
@@ -0,0 +1,36 @@
+/*
+ * 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.launcher
+
+import java.util.{List => JList, Map => JMap}
+
+/**
+ * Exposes AbstractCommandBuilder to the YARN tests, so that they can build classpaths the same
+ * way other cluster managers do.
+ */
+private[spark] class TestClasspathBuilder extends AbstractCommandBuilder {
+
+ childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, sys.props("spark.test.home"))
+
+ override def buildClassPath(extraCp: String): JList[String] = super.buildClassPath(extraCp)
+
+ /** Not used by the YARN tests. */
+ override def buildCommand(env: JMap[String, String]): JList[String] =
+ throw new UnsupportedOperationException()
+
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala
new file mode 100644
index 0000000000..1fed2562fc
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala
@@ -0,0 +1,70 @@
+/*
+ * 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.network.shuffle
+
+import java.io.File
+import java.util.concurrent.ConcurrentMap
+
+import org.apache.hadoop.yarn.api.records.ApplicationId
+import org.fusesource.leveldbjni.JniDBFactory
+import org.iq80.leveldb.{DB, Options}
+
+import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
+
+/**
+ * just a cheat to get package-visible members in tests
+ */
+object ShuffleTestAccessor {
+
+ def getBlockResolver(handler: ExternalShuffleBlockHandler): ExternalShuffleBlockResolver = {
+ handler.blockManager
+ }
+
+ def getExecutorInfo(
+ appId: ApplicationId,
+ execId: String,
+ resolver: ExternalShuffleBlockResolver
+ ): Option[ExecutorShuffleInfo] = {
+ val id = new AppExecId(appId.toString, execId)
+ Option(resolver.executors.get(id))
+ }
+
+ def registeredExecutorFile(resolver: ExternalShuffleBlockResolver): File = {
+ resolver.registeredExecutorFile
+ }
+
+ def shuffleServiceLevelDB(resolver: ExternalShuffleBlockResolver): DB = {
+ resolver.db
+ }
+
+ def reloadRegisteredExecutors(
+ file: File): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = {
+ val options: Options = new Options
+ options.createIfMissing(true)
+ val factory = new JniDBFactory
+ val db = factory.open(file, options)
+ val result = ExternalShuffleBlockResolver.reloadRegisteredExecutors(db)
+ db.close()
+ result
+ }
+
+ def reloadRegisteredExecutors(
+ db: DB): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = {
+ ExternalShuffleBlockResolver.reloadRegisteredExecutors(db)
+ }
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
new file mode 100644
index 0000000000..a58784f596
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala
@@ -0,0 +1,372 @@
+/*
+ * 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.network.yarn
+
+import java.io.{DataOutputStream, File, FileOutputStream, IOException}
+import java.nio.ByteBuffer
+import java.nio.file.Files
+import java.nio.file.attribute.PosixFilePermission._
+import java.util.EnumSet
+
+import scala.annotation.tailrec
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.service.ServiceStateException
+import org.apache.hadoop.yarn.api.records.ApplicationId
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.server.api.{ApplicationInitializationContext, ApplicationTerminationContext}
+import org.scalatest.{BeforeAndAfterEach, Matchers}
+import org.scalatest.concurrent.Eventually._
+
+import org.apache.spark.SecurityManager
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.network.shuffle.ShuffleTestAccessor
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
+import org.apache.spark.util.Utils
+
+class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach {
+ private[yarn] var yarnConfig: YarnConfiguration = null
+ private[yarn] val SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"
+
+ override def beforeEach(): Unit = {
+ super.beforeEach()
+ yarnConfig = new YarnConfiguration()
+ yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle")
+ yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"),
+ classOf[YarnShuffleService].getCanonicalName)
+ yarnConfig.setInt("spark.shuffle.service.port", 0)
+ yarnConfig.setBoolean(YarnShuffleService.STOP_ON_FAILURE_KEY, true)
+ val localDir = Utils.createTempDir()
+ yarnConfig.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath)
+ }
+
+ var s1: YarnShuffleService = null
+ var s2: YarnShuffleService = null
+ var s3: YarnShuffleService = null
+
+ override def afterEach(): Unit = {
+ try {
+ if (s1 != null) {
+ s1.stop()
+ s1 = null
+ }
+ if (s2 != null) {
+ s2.stop()
+ s2 = null
+ }
+ if (s3 != null) {
+ s3.stop()
+ s3 = null
+ }
+ } finally {
+ super.afterEach()
+ }
+ }
+
+ test("executor state kept across NM restart") {
+ s1 = new YarnShuffleService
+ // set auth to true to test the secrets recovery
+ yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, true)
+ s1.init(yarnConfig)
+ val app1Id = ApplicationId.newInstance(0, 1)
+ val app1Data = makeAppInfo("user", app1Id)
+ s1.initializeApplication(app1Data)
+ val app2Id = ApplicationId.newInstance(0, 2)
+ val app2Data = makeAppInfo("user", app2Id)
+ s1.initializeApplication(app2Data)
+
+ val execStateFile = s1.registeredExecutorFile
+ execStateFile should not be (null)
+ val secretsFile = s1.secretsFile
+ secretsFile should not be (null)
+ val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
+ val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
+
+ val blockHandler = s1.blockHandler
+ val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
+ ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
+
+ blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
+ blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
+ ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", blockResolver) should
+ be (Some(shuffleInfo1))
+ ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should
+ be (Some(shuffleInfo2))
+
+ if (!execStateFile.exists()) {
+ @tailrec def findExistingParent(file: File): File = {
+ if (file == null) file
+ else if (file.exists()) file
+ else findExistingParent(file.getParentFile())
+ }
+ val existingParent = findExistingParent(execStateFile)
+ assert(false, s"$execStateFile does not exist -- closest existing parent is $existingParent")
+ }
+ assert(execStateFile.exists(), s"$execStateFile did not exist")
+
+ // now we pretend the shuffle service goes down, and comes back up
+ s1.stop()
+ s2 = new YarnShuffleService
+ s2.init(yarnConfig)
+ s2.secretsFile should be (secretsFile)
+ s2.registeredExecutorFile should be (execStateFile)
+
+ val handler2 = s2.blockHandler
+ val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2)
+
+ // now we reinitialize only one of the apps, and expect yarn to tell us that app2 was stopped
+ // during the restart
+ s2.initializeApplication(app1Data)
+ s2.stopApplication(new ApplicationTerminationContext(app2Id))
+ ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1))
+ ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None)
+
+ // Act like the NM restarts one more time
+ s2.stop()
+ s3 = new YarnShuffleService
+ s3.init(yarnConfig)
+ s3.registeredExecutorFile should be (execStateFile)
+ s3.secretsFile should be (secretsFile)
+
+ val handler3 = s3.blockHandler
+ val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3)
+
+ // app1 is still running
+ s3.initializeApplication(app1Data)
+ ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (Some(shuffleInfo1))
+ ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None)
+ s3.stop()
+ }
+
+ test("removed applications should not be in registered executor file") {
+ s1 = new YarnShuffleService
+ yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, false)
+ s1.init(yarnConfig)
+ val secretsFile = s1.secretsFile
+ secretsFile should be (null)
+ val app1Id = ApplicationId.newInstance(0, 1)
+ val app1Data = makeAppInfo("user", app1Id)
+ s1.initializeApplication(app1Data)
+ val app2Id = ApplicationId.newInstance(0, 2)
+ val app2Data = makeAppInfo("user", app2Id)
+ s1.initializeApplication(app2Data)
+
+ val execStateFile = s1.registeredExecutorFile
+ execStateFile should not be (null)
+ val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
+ val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
+
+ val blockHandler = s1.blockHandler
+ val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
+ ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
+
+ blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
+ blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
+
+ val db = ShuffleTestAccessor.shuffleServiceLevelDB(blockResolver)
+ ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty
+
+ s1.stopApplication(new ApplicationTerminationContext(app1Id))
+ ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty
+ s1.stopApplication(new ApplicationTerminationContext(app2Id))
+ ShuffleTestAccessor.reloadRegisteredExecutors(db) shouldBe empty
+ }
+
+ test("shuffle service should be robust to corrupt registered executor file") {
+ s1 = new YarnShuffleService
+ s1.init(yarnConfig)
+ val app1Id = ApplicationId.newInstance(0, 1)
+ val app1Data = makeAppInfo("user", app1Id)
+ s1.initializeApplication(app1Data)
+
+ val execStateFile = s1.registeredExecutorFile
+ val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
+
+ val blockHandler = s1.blockHandler
+ val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
+ ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
+
+ blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
+
+ // now we pretend the shuffle service goes down, and comes back up. But we'll also
+ // make a corrupt registeredExecutor File
+ s1.stop()
+
+ execStateFile.listFiles().foreach{_.delete()}
+
+ val out = new DataOutputStream(new FileOutputStream(execStateFile + "/CURRENT"))
+ out.writeInt(42)
+ out.close()
+
+ s2 = new YarnShuffleService
+ s2.init(yarnConfig)
+ s2.registeredExecutorFile should be (execStateFile)
+
+ val handler2 = s2.blockHandler
+ val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2)
+
+ // we re-initialize app1, but since the file was corrupt there is nothing we can do about it ...
+ s2.initializeApplication(app1Data)
+ // however, when we initialize a totally new app2, everything is still happy
+ val app2Id = ApplicationId.newInstance(0, 2)
+ val app2Data = makeAppInfo("user", app2Id)
+ s2.initializeApplication(app2Data)
+ val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
+ resolver2.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
+ ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (Some(shuffleInfo2))
+ s2.stop()
+
+ // another stop & restart should be fine though (eg., we recover from previous corruption)
+ s3 = new YarnShuffleService
+ s3.init(yarnConfig)
+ s3.registeredExecutorFile should be (execStateFile)
+ val handler3 = s3.blockHandler
+ val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3)
+
+ s3.initializeApplication(app2Data)
+ ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (Some(shuffleInfo2))
+ s3.stop()
+ }
+
+ test("get correct recovery path") {
+ // Test recovery path is set outside the shuffle service, this is to simulate NM recovery
+ // enabled scenario, where recovery path will be set by yarn.
+ s1 = new YarnShuffleService
+ val recoveryPath = new Path(Utils.createTempDir().toURI)
+ s1.setRecoveryPath(recoveryPath)
+
+ s1.init(yarnConfig)
+ s1._recoveryPath should be (recoveryPath)
+ s1.stop()
+
+ // Test recovery path is set inside the shuffle service, this will be happened when NM
+ // recovery is not enabled or there's no NM recovery (Hadoop 2.5-).
+ s2 = new YarnShuffleService
+ s2.init(yarnConfig)
+ s2._recoveryPath should be
+ (new Path(yarnConfig.getTrimmedStrings("yarn.nodemanager.local-dirs")(0)))
+ s2.stop()
+ }
+
+ test("moving recovery file from NM local dir to recovery path") {
+ // This is to test when Hadoop is upgrade to 2.5+ and NM recovery is enabled, we should move
+ // old recovery file to the new path to keep compatibility
+
+ // Simulate s1 is running on old version of Hadoop in which recovery file is in the NM local
+ // dir.
+ s1 = new YarnShuffleService
+ // set auth to true to test the secrets recovery
+ yarnConfig.setBoolean(SecurityManager.SPARK_AUTH_CONF, true)
+ s1.init(yarnConfig)
+ val app1Id = ApplicationId.newInstance(0, 1)
+ val app1Data = makeAppInfo("user", app1Id)
+ s1.initializeApplication(app1Data)
+ val app2Id = ApplicationId.newInstance(0, 2)
+ val app2Data = makeAppInfo("user", app2Id)
+ s1.initializeApplication(app2Data)
+
+ assert(s1.secretManager.getSecretKey(app1Id.toString()) != null)
+ assert(s1.secretManager.getSecretKey(app2Id.toString()) != null)
+
+ val execStateFile = s1.registeredExecutorFile
+ execStateFile should not be (null)
+ val secretsFile = s1.secretsFile
+ secretsFile should not be (null)
+ val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, SORT_MANAGER)
+ val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, SORT_MANAGER)
+
+ val blockHandler = s1.blockHandler
+ val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler)
+ ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile)
+
+ blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1)
+ blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2)
+ ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", blockResolver) should
+ be (Some(shuffleInfo1))
+ ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should
+ be (Some(shuffleInfo2))
+
+ assert(execStateFile.exists(), s"$execStateFile did not exist")
+
+ s1.stop()
+
+ // Simulate s2 is running on Hadoop 2.5+ with NM recovery is enabled.
+ assert(execStateFile.exists())
+ val recoveryPath = new Path(Utils.createTempDir().toURI)
+ s2 = new YarnShuffleService
+ s2.setRecoveryPath(recoveryPath)
+ s2.init(yarnConfig)
+
+ // Ensure that s2 has loaded known apps from the secrets db.
+ assert(s2.secretManager.getSecretKey(app1Id.toString()) != null)
+ assert(s2.secretManager.getSecretKey(app2Id.toString()) != null)
+
+ val execStateFile2 = s2.registeredExecutorFile
+ val secretsFile2 = s2.secretsFile
+
+ recoveryPath.toString should be (new Path(execStateFile2.getParentFile.toURI).toString)
+ recoveryPath.toString should be (new Path(secretsFile2.getParentFile.toURI).toString)
+ eventually(timeout(10 seconds), interval(5 millis)) {
+ assert(!execStateFile.exists())
+ }
+ eventually(timeout(10 seconds), interval(5 millis)) {
+ assert(!secretsFile.exists())
+ }
+
+ val handler2 = s2.blockHandler
+ val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2)
+
+ // now we reinitialize only one of the apps, and expect yarn to tell us that app2 was stopped
+ // during the restart
+ // Since recovery file is got from old path, so the previous state should be stored.
+ s2.initializeApplication(app1Data)
+ s2.stopApplication(new ApplicationTerminationContext(app2Id))
+ ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1))
+ ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None)
+
+ s2.stop()
+ }
+
+ test("service throws error if cannot start") {
+ // Set up a read-only local dir.
+ val roDir = Utils.createTempDir()
+ Files.setPosixFilePermissions(roDir.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE))
+ yarnConfig.set(YarnConfiguration.NM_LOCAL_DIRS, roDir.getAbsolutePath())
+
+ // Try to start the shuffle service, it should fail.
+ val service = new YarnShuffleService()
+
+ try {
+ val error = intercept[ServiceStateException] {
+ service.init(yarnConfig)
+ }
+ assert(error.getCause().isInstanceOf[IOException])
+ } finally {
+ service.stop()
+ Files.setPosixFilePermissions(roDir.toPath(),
+ EnumSet.of(OWNER_READ, OWNER_WRITE, OWNER_EXECUTE))
+ }
+ }
+
+ private def makeAppInfo(user: String, appId: ApplicationId): ApplicationInitializationContext = {
+ val secret = ByteBuffer.wrap(new Array[Byte](0))
+ new ApplicationInitializationContext(user, appId, secret)
+ }
+
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala
new file mode 100644
index 0000000000..db322cd18e
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.network.yarn
+
+import java.io.File
+
+/**
+ * just a cheat to get package-visible members in tests
+ */
+object YarnTestAccessor {
+ def getShuffleServicePort: Int = {
+ YarnShuffleService.boundPort
+ }
+
+ def getShuffleServiceInstance: YarnShuffleService = {
+ YarnShuffleService.instance
+ }
+
+ def getRegisteredExecutorFile(service: YarnShuffleService): File = {
+ service.registeredExecutorFile
+ }
+
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala
new file mode 100644
index 0000000000..6ea7984c64
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala
@@ -0,0 +1,72 @@
+/*
+ * 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.scalatest.BeforeAndAfter
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.deploy.yarn.config._
+import org.apache.spark.internal.Logging
+
+/**
+ * Test the integration with [[SchedulerExtensionServices]]
+ */
+class ExtensionServiceIntegrationSuite extends SparkFunSuite
+ with LocalSparkContext with BeforeAndAfter
+ with Logging {
+
+ val applicationId = new StubApplicationId(0, 1111L)
+ val attemptId = new StubApplicationAttemptId(applicationId, 1)
+
+ /*
+ * Setup phase creates the spark context
+ */
+ before {
+ val sparkConf = new SparkConf()
+ sparkConf.set(SCHEDULER_SERVICES, Seq(classOf[SimpleExtensionService].getName()))
+ sparkConf.setMaster("local").setAppName("ExtensionServiceIntegrationSuite")
+ sc = new SparkContext(sparkConf)
+ }
+
+ test("Instantiate") {
+ val services = new SchedulerExtensionServices()
+ assertResult(Nil, "non-nil service list") {
+ services.getServices
+ }
+ services.start(SchedulerExtensionServiceBinding(sc, applicationId))
+ services.stop()
+ }
+
+ test("Contains SimpleExtensionService Service") {
+ val services = new SchedulerExtensionServices()
+ try {
+ services.start(SchedulerExtensionServiceBinding(sc, applicationId))
+ val serviceList = services.getServices
+ assert(serviceList.nonEmpty, "empty service list")
+ val (service :: Nil) = serviceList
+ val simpleService = service.asInstanceOf[SimpleExtensionService]
+ assert(simpleService.started.get, "service not started")
+ services.stop()
+ assert(!simpleService.started.get, "service not stopped")
+ } finally {
+ services.stop()
+ }
+ }
+}
+
+
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala
new file mode 100644
index 0000000000..9b8c98cda8
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala
@@ -0,0 +1,34 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicBoolean
+
+private[spark] class SimpleExtensionService extends SchedulerExtensionService {
+
+ /** started flag; set in the `start()` call, stopped in `stop()`. */
+ val started = new AtomicBoolean(false)
+
+ override def start(binding: SchedulerExtensionServiceBinding): Unit = {
+ started.set(true)
+ }
+
+ override def stop(): Unit = {
+ started.set(false)
+ }
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala
new file mode 100644
index 0000000000..4b57b9509a
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala
@@ -0,0 +1,48 @@
+/*
+ * 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.{ApplicationAttemptId, ApplicationId}
+
+/**
+ * A stub application ID; can be set in constructor and/or updated later.
+ * @param applicationId application ID
+ * @param attempt an attempt counter
+ */
+class StubApplicationAttemptId(var applicationId: ApplicationId, var attempt: Int)
+ extends ApplicationAttemptId {
+
+ override def setApplicationId(appID: ApplicationId): Unit = {
+ applicationId = appID
+ }
+
+ override def getAttemptId: Int = {
+ attempt
+ }
+
+ override def setAttemptId(attemptId: Int): Unit = {
+ attempt = attemptId
+ }
+
+ override def getApplicationId: ApplicationId = {
+ applicationId
+ }
+
+ override def build(): Unit = {
+ }
+}
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala
new file mode 100644
index 0000000000..bffa0e09be
--- /dev/null
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala
@@ -0,0 +1,42 @@
+/*
+ * 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
+
+/**
+ * Simple Testing Application Id; ID and cluster timestamp are set in constructor
+ * and cannot be updated.
+ * @param id app id
+ * @param clusterTimestamp timestamp
+ */
+private[spark] class StubApplicationId(id: Int, clusterTimestamp: Long) extends ApplicationId {
+ override def getId: Int = {
+ id
+ }
+
+ override def getClusterTimestamp: Long = {
+ clusterTimestamp
+ }
+
+ override def setId(id: Int): Unit = {}
+
+ override def setClusterTimestamp(clusterTimestamp: Long): Unit = {}
+
+ override def build(): Unit = {}
+}