From 3f941dc0f3ba3ed43cc82e993c828ce8af28c93b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 8 Sep 2014 22:21:43 -0700 Subject: [PATCH 01/18] First cut at simplifying the Client (stable and alpha) Clearly, a lot of documentation and further work needs to be done in ClientBase.scala. This is only the first checkpoint. --- .../org/apache/spark/deploy/yarn/Client.scala | 146 +++++-------- .../apache/spark/deploy/yarn/ClientBase.scala | 203 ++++++++++++------ .../cluster/YarnClientSchedulerBackend.scala | 2 +- .../spark/deploy/yarn/ClientBaseSuite.scala | 18 +- .../org/apache/spark/deploy/yarn/Client.scala | 158 +++++--------- 5 files changed, 252 insertions(+), 275 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 10fc39bba87d1..779609f8b2c7e 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -23,13 +23,11 @@ import org.apache.hadoop.conf.Configuration 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.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.YarnClientImpl import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Apps, Records} +import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil @@ -37,7 +35,10 @@ import org.apache.spark.deploy.SparkHadoopUtil /** * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. */ -class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) +private[spark] class Client( + val args: ClientArguments, + val hadoopConf: Configuration, + val sparkConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { def this(clientArgs: ClientArguments, spConf: SparkConf) = @@ -45,120 +46,87 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf()) - val args = clientArgs - val conf = hadoopConf - val sparkConf = spConf - var rpc: YarnRPC = YarnRPC.create(conf) - val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + val yarnConf: YarnConfiguration = new YarnConfiguration(hadoopConf) - - // for client user who want to monitor app status by itself. - def runApp() = { + /** Submit an application running our ApplicationMaster to the ResourceManager. */ + override def submitApplication(): ApplicationId = { validateArgs() + // Initialize and start the client service. init(yarnConf) start() - logClusterResourceDetails() - val newApp = super.getNewApplication() - val appId = newApp.getApplicationId() + logInfo("Received cluster metric info from ResourceManager, number of NodeManagers: " + + getYarnClusterMetrics.getNumNodeManagers) - verifyClusterResources(newApp) - val appContext = createApplicationSubmissionContext(appId) - val appStagingDir = getAppStagingDir(appId) - val localResources = prepareLocalResources(appStagingDir) - val env = setupLaunchEnv(localResources, appStagingDir) - val amContainer = createContainerLaunchContext(newApp, localResources, env) + // Get a new application from our RM. + val newAppResponse = getNewApplication() + val appId = newAppResponse.getApplicationId() - val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] - // Memory for the ApplicationMaster. - capability.setMemory(args.amMemory + memoryOverhead) - amContainer.setResource(capability) + // Verify whether the cluster has enough resources for our AM. + verifyClusterResources(newAppResponse) - appContext.setQueue(args.amQueue) - appContext.setAMContainerSpec(amContainer) - appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) + // Set up ContainerLaunchContext to launch our AM container. + val containerContext = createContainerLaunchContext(newAppResponse) - submitApp(appContext) - appId - } + // Set up ApplicationSubmissionContext to submit our AM. + val appContext = createApplicationSubmissionContext(appId, containerContext) - def run() { - val appId = runApp() - monitorApplication(appId) + // Finally, submit and monitor the application. + logInfo(s"Submitting application ${appId.getId} to ResourceManager") + submitApplication(appContext) + appId } - def logClusterResourceDetails() { - val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics - logInfo("Got cluster metric info from ASM, numNodeManagers = " + - clusterMetrics.getNumNodeManagers) + /** + * + */ + override def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) + : ContainerLaunchContext = { + val containerContext = super.createContainerLaunchContext(newAppResponse) + val capability = Records.newRecord(classOf[Resource]) + capability.setMemory(getAMMemory(newAppResponse) + memoryOverhead) + containerContext.setResource(capability) + containerContext } - - def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { - logInfo("Setting up application submission context for ASM") + /** + * + */ + def createApplicationSubmissionContext( + appId: ApplicationId, + containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) appContext.setApplicationId(appId) appContext.setApplicationName(args.appName) + appContext.setQueue(args.amQueue) + appContext.setAMContainerSpec(containerContext) + appContext.setUser(UserGroupInformation.getCurrentUser.getShortUserName) appContext } - def calculateAMMemory(newApp: GetNewApplicationResponse): Int = { + /** + * + */ + override def getAMMemory(newApp: GetNewApplicationResponse): Int = { val minResMemory = newApp.getMinimumResourceCapability().getMemory() val amMemory = ((args.amMemory / minResMemory) * minResMemory) + - ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - - memoryOverhead) + ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - memoryOverhead) amMemory } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { - // Setup security tokens. + /** */ + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { val dob = new DataOutputBuffer() credentials.writeTokenStorageToStream(dob) amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) } - def submitApp(appContext: ApplicationSubmissionContext) = { - // Submit the application to the applications manager. - logInfo("Submitting application to ASM") - super.submitApplication(appContext) - } - - def monitorApplication(appId: ApplicationId): Boolean = { - val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - - while (true) { - Thread.sleep(interval) - val report = super.getApplicationReport(appId) - - logInfo("Application report from ASM: \n" + - "\t application identifier: " + appId.toString() + "\n" + - "\t appId: " + appId.getId() + "\n" + - "\t clientToken: " + report.getClientToken() + "\n" + - "\t appDiagnostics: " + report.getDiagnostics() + "\n" + - "\t appMasterHost: " + report.getHost() + "\n" + - "\t appQueue: " + report.getQueue() + "\n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + - "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" + - "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + - "\t appUser: " + report.getUser() - ) - - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - return true - } - } - true - } + /** */ + override def getClientToken(report: ApplicationReport): String = report.getClientToken } -object Client { - +private[spark] object Client { def main(argStrings: Array[String]) { if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a " + @@ -166,21 +134,17 @@ object Client { } // Set an env variable indicating we are running in YARN mode. - // Note that anything with SPARK prefix gets propagated to all (remote) processes + // 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 try { val args = new ClientArguments(argStrings, sparkConf) new Client(args, sparkConf).run() } catch { - case e: Exception => { + case e: Exception => Console.err.println(e.getMessage) System.exit(1) - } } - - System.exit(0) } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8075b7a7fb837..81113918fd21f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -43,13 +43,14 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, Spar * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The * Client submits an application to the YARN ResourceManager. */ -trait ClientBase extends Logging { +private[spark] trait ClientBase extends Logging { + import ClientBase._ + val args: ClientArguments - val conf: Configuration + val hadoopConf: Configuration val sparkConf: SparkConf val yarnConf: YarnConfiguration - val credentials = UserGroupInformation.getCurrentUser().getCredentials() - private val SPARK_STAGING: String = ".sparkStaging" + val credentials = UserGroupInformation.getCurrentUser.getCredentials private val distCacheMgr = new ClientDistributedCacheManager() // Staging directory is private! -> rwx-------- @@ -150,20 +151,20 @@ trait ClientBase extends Logging { originalPath: Path, replication: Short, setPerms: Boolean = false): Path = { - val fs = FileSystem.get(conf) - val remoteFs = originalPath.getFileSystem(conf) + val fs = FileSystem.get(hadoopConf) + val remoteFs = originalPath.getFileSystem(hadoopConf) var newPath = originalPath if (!compareFs(remoteFs, fs)) { newPath = new Path(dstDir, originalPath.getName()) logInfo("Uploading " + originalPath + " to " + newPath) - FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) + FileUtil.copy(remoteFs, originalPath, fs, newPath, false, hadoopConf) fs.setReplication(newPath, replication) if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) } // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific // version shows the specific version in the distributed cache configuration val qualPath = fs.makeQualified(newPath) - val fc = FileContext.getFileContext(qualPath.toUri(), conf) + val fc = FileContext.getFileContext(qualPath.toUri(), hadoopConf) val destPath = fc.resolvePath(qualPath) destPath } @@ -172,7 +173,7 @@ trait ClientBase extends Logging { var qualifiedURI = localURI // If not specified, assume these are in the local filesystem to keep behavior like Hadoop if (qualifiedURI.getScheme() == null) { - qualifiedURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(qualifiedURI)).toString) + qualifiedURI = new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(qualifiedURI)).toString) } new Path(qualifiedURI) } @@ -181,10 +182,10 @@ trait ClientBase extends Logging { logInfo("Preparing Local resources") // Upload Spark and the application JAR to the remote file system if necessary. Add them as // local resources to the application master. - val fs = FileSystem.get(conf) + val fs = FileSystem.get(hadoopConf) val dst = new Path(fs.getHomeDirectory(), appStagingDir) val nns = ClientBase.getNameNodesToAccess(sparkConf) + dst - ClientBase.obtainTokensForNamenodes(nns, conf, credentials) + ClientBase.obtainTokensForNamenodes(nns, hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort val localResources = HashMap[String, LocalResource]() @@ -211,8 +212,8 @@ trait ClientBase extends Logging { if (!ClientBase.LOCAL_SCHEME.equals(localURI.getScheme())) { val setPermissions = destName.equals(ClientBase.APP_JAR) val destPath = copyRemoteFile(dst, qualifyForLocal(localURI), replication, setPermissions) - val destFs = FileSystem.get(destPath.toUri(), conf) - distCacheMgr.addResource(destFs, conf, destPath, localResources, LocalResourceType.FILE, + val destFs = FileSystem.get(destPath.toUri(), hadoopConf) + distCacheMgr.addResource(destFs, hadoopConf, destPath, localResources, LocalResourceType.FILE, destName, statCache) } else if (confKey != null) { sparkConf.set(confKey, localPath) @@ -232,7 +233,7 @@ trait ClientBase extends Logging { val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, resType, + distCacheMgr.addResource(fs, hadoopConf, destPath, localResources, resType, linkname, statCache) if (addToClasspath) { cachedSecondaryJarLinks += linkname @@ -257,14 +258,12 @@ trait ClientBase extends Logging { .map{case (k, v) => (k.substring(prefix.length), v)} } - - def setupLaunchEnv( - localResources: HashMap[String, LocalResource], - stagingDir: String): HashMap[String, String] = { + /** + * + */ + def setupLaunchEnv(stagingDir: String): HashMap[String, String] = { logInfo("Setting up the launch environment") - val env = new HashMap[String, String]() - val extraCp = sparkConf.getOption("spark.driver.extraClassPath") ClientBase.populateClasspath(args, yarnConf, sparkConf, env, extraCp) env("SPARK_YARN_MODE") = "true" @@ -283,36 +282,9 @@ trait ClientBase extends Logging { sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => // Allow users to specify some environment variables. YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs, File.pathSeparator) - // 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 } - env - } - - def userArgsToString(clientArgs: ClientArguments): String = { - val prefix = " --arg " - val args = clientArgs.userArgs - val retval = new StringBuilder() - for (arg <- args) { - retval.append(prefix).append(" ").append(YarnSparkHadoopUtil.escapeForShell(arg)) - } - retval.toString - } - - def calculateAMMemory(newApp: GetNewApplicationResponse): Int - - def setupSecurityToken(amContainer: ContainerLaunchContext) - - def createContainerLaunchContext( - newApp: GetNewApplicationResponse, - localResources: HashMap[String, LocalResource], - env: HashMap[String, String]): ContainerLaunchContext = { - logInfo("Setting up container launch context") - val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) - amContainer.setLocalResources(localResources) - - val isLaunchingDriver = args.userClass != null // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's @@ -322,6 +294,7 @@ trait ClientBase extends Logging { // Note that to warn the user about the deprecation in cluster mode, some code from // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition // described above). + val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sys.env.get("SPARK_JAVA_OPTS").foreach { value => val warning = @@ -344,14 +317,40 @@ trait ClientBase extends Logging { env("SPARK_JAVA_OPTS") = value } } - amContainer.setEnvironment(env) - val amMemory = calculateAMMemory(newApp) + env + } + + def userArgsToString(clientArgs: ClientArguments): String = { + val prefix = " --arg " + val args = clientArgs.userArgs + val retval = new StringBuilder() + for (arg <- args) { + retval.append(prefix).append(" ").append(YarnSparkHadoopUtil.escapeForShell(arg)) + } + retval.toString + } + + /** + * Prepare a ContainerLaunchContext to launch our AM container. + * This sets up the launch environment, java options, and the command for launching the AM. + */ + def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) + : ContainerLaunchContext = { + logInfo("Setting up container launch context") + + val appId = newAppResponse.getApplicationId + val appStagingDir = getAppStagingDir(appId) + val localResources = prepareLocalResources(appStagingDir) + val launchEnv = setupLaunchEnv(appStagingDir) + val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) + amContainer.setLocalResources(localResources) + amContainer.setEnvironment(launchEnv) val javaOpts = ListBuffer[String]() // Add Xmx for AM memory - javaOpts += "-Xmx" + amMemory + "m" + javaOpts += "-Xmx" + getAMMemory(newAppResponse) + "m" val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) javaOpts += "-Djava.io.tmpdir=" + tmpDir @@ -363,8 +362,7 @@ trait ClientBase extends Logging { // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset // of cores on a node. - val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && - java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC")) + 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 @@ -382,6 +380,8 @@ trait ClientBase extends Logging { javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } + // Include driver-specific java options if we are launching a driver + val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) @@ -419,39 +419,110 @@ trait ClientBase extends Logging { logInfo("Yarn AM launch context:") logInfo(s" user class: ${args.userClass}") - logInfo(s" env: $env") + logInfo(s" env: $launchEnv") logInfo(s" command: ${commands.mkString(" ")}") // TODO: it would be nicer to just make sure there are no null commands here val printableCommands = commands.map(s => if (s == null) "null" else s).toList amContainer.setCommands(printableCommands) - setupSecurityToken(amContainer) - // send the acl settings into YARN to control who has access via YARN interfaces val securityManager = new SecurityManager(sparkConf) amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager)) - + setupSecurityToken(amContainer) amContainer } + + /** + * Report the state of an application until it has exited, either successfully or + * due to some failure. + */ + def monitorApplication(appId: ApplicationId): Unit = { + val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) + while (true) { + Thread.sleep(interval) + val report = getApplicationReport(appId) + val state = report.getYarnApplicationState + + logInfo(s"Application report from ResourceManager for ${appId.getId} (state: $state)") + + logDebug( + s"\t full application identifier: $appId\n" + + s"\t clientToken: ${getClientToken(report)}\n" + + s"\t appDiagnostics: ${report.getDiagnostics}\n" + + s"\t appMasterHost: ${report.getHost}\n" + + s"\t appQueue: ${report.getQueue}\n" + + s"\t appMasterRpcPort: ${report.getRpcPort}\n" + + s"\t appStartTime: ${report.getStartTime}\n" + + s"\t yarnAppState: $state\n" + + s"\t distributedFinalState: ${report.getFinalApplicationStatus}\n" + + s"\t appTrackingUrl: ${report.getTrackingUrl}\n" + + s"\t appUser: ${report.getUser}") + + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + return + } + } + } + + /** + * Submit an application to the ResourceManager and monitor its state. + * This continues until the application has exited for any reason. + */ + def run(): Unit = monitorApplication(submitApplication()) + + /* --------------------------------------------------------------------------------------- * + | Methods that cannot be implemented here due to API differences across hadoop versions | + * --------------------------------------------------------------------------------------- */ + + /** Submit an application running our ApplicationMaster to the ResourceManager. */ + def submitApplication(): ApplicationId + + /** */ + def setupSecurityToken(containerContext: ContainerLaunchContext): Unit + + /** */ + def getApplicationReport(appId: ApplicationId): ApplicationReport + + /** */ + def getClientToken(report: ApplicationReport): String + + /** Return the amount of memory for launching the ApplicationMaster container (MB). */ + def getAMMemory(newAppResponse: GetNewApplicationResponse): Int = args.amMemory } -object ClientBase extends Logging { +private[spark] object ClientBase extends Logging { + + // Alias for the Spark assembly jar and the user jar val SPARK_JAR: String = "__spark__.jar" val APP_JAR: 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" + + // Location of any user-defined Spark jars val CONF_SPARK_JAR = "spark.yarn.jar" - /** - * This is an internal config used to propagate the location of the user's jar file to the - * driver/executors. - */ + val ENV_SPARK_JAR = "SPARK_JAR" + + // Internal config to propagate the location of the user's jar to the driver/executors val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" - /** - * This is an internal config used to propagate the list of extra jars to add to the classpath - * of executors. - */ + + // Internal config to propagate the locations of any extra jars to add to the classpath + // of the executors val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" - val ENV_SPARK_JAR = "SPARK_JAR" + + // 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) /** * Find the user-defined Spark jar if configured, or return the jar containing this diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 41c662cd7a6de..acbdad46c9ea0 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -82,7 +82,7 @@ private[spark] class YarnClientSchedulerBackend( val args = new ClientArguments(argsArrayBuf.toArray, conf) totalExpectedExecutors = args.numExecutors client = new Client(args, conf) - appId = client.runApp() + appId = client.submitApplication() waitForApp() checkerThread = yarnApplicationStateCheckerThread() } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 68cc2890f3a22..2c2bb1baeafaa 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext +import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.Matchers._ import org.mockito.Mockito._ @@ -116,7 +116,7 @@ class ClientBaseSuite extends FunSuite with Matchers { doReturn(new Path("/")).when(client).copyRemoteFile(any(classOf[Path]), any(classOf[Path]), anyShort(), anyBoolean()) - var tempDir = Files.createTempDir(); + val tempDir = Files.createTempDir() try { client.prepareLocalResources(tempDir.getAbsolutePath()) sparkConf.getOption(ClientBase.CONF_SPARK_USER_JAR) should be (Some(USER)) @@ -234,16 +234,14 @@ class ClientBaseSuite extends FunSuite with Matchers { private class DummyClient( val args: ClientArguments, - val conf: Configuration, + val hadoopConf: Configuration, val sparkConf: SparkConf, val yarnConf: YarnConfiguration) extends ClientBase { - - override def calculateAMMemory(newApp: GetNewApplicationResponse): Int = - throw new UnsupportedOperationException() - - override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = - throw new UnsupportedOperationException() - + override def getAMMemory(newApp: GetNewApplicationResponse): Int = ??? + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = ??? + override def submitApplication(): ApplicationId = ??? + override def getApplicationReport(appId: ApplicationId): ApplicationReport = ??? + override def getClientToken(report: ApplicationReport): String = ??? } } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 313a0d21ce181..c222641f4866d 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -21,11 +21,9 @@ import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.YarnClient +import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SparkConf} @@ -34,137 +32,88 @@ import org.apache.spark.deploy.SparkHadoopUtil /** * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's stable API. */ -class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) +private[spark] class Client( + val args: ClientArguments, + val hadoopConf: Configuration, + val sparkConf: SparkConf) extends ClientBase with Logging { - val yarnClient = YarnClient.createYarnClient - def this(clientArgs: ClientArguments, spConf: SparkConf) = this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf()) - val args = clientArgs - val conf = hadoopConf - val sparkConf = spConf - var rpc: YarnRPC = YarnRPC.create(conf) - val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + val yarnClient = YarnClient.createYarnClient + val yarnConf = new YarnConfiguration(hadoopConf) - def runApp(): ApplicationId = { + /** Submit an application running our ApplicationMaster to the ResourceManager. */ + override def submitApplication(): ApplicationId = { validateArgs() + // Initialize and start the client service. yarnClient.init(yarnConf) yarnClient.start() - // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers). - logClusterResourceDetails() - - // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM) - // interface). + logInfo("Received cluster metric info from ResourceManager, number of NodeManagers: " + + yarnClient.getYarnClusterMetrics.getNumNodeManagers) - // Get a new client application. + // Get a new application from our RM. val newApp = yarnClient.createApplication() val newAppResponse = newApp.getNewApplicationResponse() val appId = newAppResponse.getApplicationId() + // Verify whether the cluster has enough resources for our AM. verifyClusterResources(newAppResponse) - // Set up resource and environment variables. - val appStagingDir = getAppStagingDir(appId) - val localResources = prepareLocalResources(appStagingDir) - val launchEnv = setupLaunchEnv(localResources, appStagingDir) - val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv) - - // Set up an application submission context. - val appContext = newApp.getApplicationSubmissionContext() - appContext.setApplicationName(args.appName) - appContext.setQueue(args.amQueue) - appContext.setAMContainerSpec(amContainer) - appContext.setApplicationType("SPARK") + // Set up ContainerLaunchContext to launch our AM container. + val containerContext = createContainerLaunchContext(newAppResponse) - // Memory for the ApplicationMaster. - val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] - memoryResource.setMemory(args.amMemory + memoryOverhead) - appContext.setResource(memoryResource) + // Set up ApplicationSubmissionContext to submit our AM. + val appContext = createApplicationSubmissionContext(newApp, containerContext) // Finally, submit and monitor the application. - submitApp(appContext) + logInfo(s"Submitting application ${appId.getId} to ResourceManager") + yarnClient.submitApplication(appContext) appId } - def run() { - val appId = runApp() - monitorApplication(appId) - } + /** Stop this client. */ + def stop(): Unit = yarnClient.stop() - def logClusterResourceDetails() { - val clusterMetrics: YarnClusterMetrics = yarnClient.getYarnClusterMetrics - logInfo("Got cluster metric info from ResourceManager, number of NodeManagers: " + - clusterMetrics.getNumNodeManagers) - } - - def calculateAMMemory(newApp: GetNewApplicationResponse) :Int = { - // TODO: Need a replacement for the following code to fix -Xmx? - // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() - // var amMemory = ((args.amMemory / minResMemory) * minResMemory) + - // ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - - // memoryOverhead ) - args.amMemory + /** + * + */ + def createApplicationSubmissionContext( + newApp: YarnClientApplication, + containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { + val appContext = newApp.getApplicationSubmissionContext + appContext.setApplicationName(args.appName) + appContext.setQueue(args.amQueue) + appContext.setAMContainerSpec(containerContext) + appContext.setApplicationType("SPARK") + val capability = Records.newRecord(classOf[Resource]) + capability.setMemory(args.amMemory + memoryOverhead) + appContext.setResource(capability) + appContext } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { - // Setup security tokens. - val dob = new DataOutputBuffer() + /** */ + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { + val dob = new DataOutputBuffer credentials.writeTokenStorageToStream(dob) - amContainer.setTokens(ByteBuffer.wrap(dob.getData())) + amContainer.setTokens(ByteBuffer.wrap(dob.getData)) } - def submitApp(appContext: ApplicationSubmissionContext) = { - // Submit the application to the applications manager. - logInfo("Submitting application to ResourceManager") - yarnClient.submitApplication(appContext) - } + /** */ + override def getApplicationReport(appId: ApplicationId): ApplicationReport = + yarnClient.getApplicationReport(appId) - def getApplicationReport(appId: ApplicationId) = - yarnClient.getApplicationReport(appId) - - def stop = yarnClient.stop - - def monitorApplication(appId: ApplicationId): Boolean = { - val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - - while (true) { - Thread.sleep(interval) - val report = yarnClient.getApplicationReport(appId) - - logInfo("Application report from ResourceManager: \n" + - "\t application identifier: " + appId.toString() + "\n" + - "\t appId: " + appId.getId() + "\n" + - "\t clientToAMToken: " + report.getClientToAMToken() + "\n" + - "\t appDiagnostics: " + report.getDiagnostics() + "\n" + - "\t appMasterHost: " + report.getHost() + "\n" + - "\t appQueue: " + report.getQueue() + "\n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + - "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" + - "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + - "\t appUser: " + report.getUser() - ) - - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - return true - } - } - true - } + /** */ + override def getClientToken(report: ApplicationReport): String = + report.getClientToAMToken.toString } -object Client { - +private[spark] object Client { def main(argStrings: Array[String]) { if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a " + @@ -172,22 +121,17 @@ object Client { } // Set an env variable indicating we are running in YARN mode. - // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - - // see Client#setupLaunchEnv(). + // 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() + val sparkConf = new SparkConf try { val args = new ClientArguments(argStrings, sparkConf) new Client(args, sparkConf).run() } catch { - case e: Exception => { + case e: Exception => Console.err.println(e.getMessage) System.exit(1) - } } - - System.exit(0) } - } From fabe4c43780c6d11e70d6ce11cfb7b94e54d56fd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 9 Sep 2014 12:26:35 -0700 Subject: [PATCH 02/18] Reuse more code in YarnClientSchedulerBackend We implement a while loop to monitor an application's state in four separate places (stable/Client, alpha/Client, and twice in YarnClientSchedulerBackend). This commit reduces this to one. --- .../apache/spark/deploy/yarn/ClientBase.scala | 49 +++++++---- .../cluster/YarnClientSchedulerBackend.scala | 83 +++++++++---------- .../org/apache/spark/deploy/yarn/Client.scala | 1 + 3 files changed, 72 insertions(+), 61 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 81113918fd21f..a0153afcb4290 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -435,36 +435,51 @@ private[spark] trait ClientBase extends Logging { /** * Report the state of an application until it has exited, either successfully or - * due to some failure. + * due to some failure, then return the application state. + * + * @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 state of the application, one of FINISHED, FAILED, KILLED, and RUNNING. */ - def monitorApplication(appId: ApplicationId): Unit = { + def monitorApplication( + appId: ApplicationId, + returnOnRunning: Boolean = false, + logApplicationReport: Boolean = true): YarnApplicationState = { val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) while (true) { Thread.sleep(interval) val report = getApplicationReport(appId) val state = report.getYarnApplicationState - logInfo(s"Application report from ResourceManager for ${appId.getId} (state: $state)") - - logDebug( - s"\t full application identifier: $appId\n" + - s"\t clientToken: ${getClientToken(report)}\n" + - s"\t appDiagnostics: ${report.getDiagnostics}\n" + - s"\t appMasterHost: ${report.getHost}\n" + - s"\t appQueue: ${report.getQueue}\n" + - s"\t appMasterRpcPort: ${report.getRpcPort}\n" + - s"\t appStartTime: ${report.getStartTime}\n" + - s"\t yarnAppState: $state\n" + - s"\t distributedFinalState: ${report.getFinalApplicationStatus}\n" + - s"\t appTrackingUrl: ${report.getTrackingUrl}\n" + - s"\t appUser: ${report.getUser}") + if (logApplicationReport) { + logInfo(s"Application report from ResourceManager for application ${appId.getId} " + + s"(state: $state)") + logDebug( + s"\t full application identifier: $appId\n" + + s"\t clientToken: ${getClientToken(report)}\n" + + s"\t appDiagnostics: ${report.getDiagnostics}\n" + + s"\t appMasterHost: ${report.getHost}\n" + + s"\t appQueue: ${report.getQueue}\n" + + s"\t appMasterRpcPort: ${report.getRpcPort}\n" + + s"\t appStartTime: ${report.getStartTime}\n" + + s"\t yarnAppState: $state\n" + + s"\t distributedFinalState: ${report.getFinalApplicationStatus}\n" + + s"\t appTrackingUrl: ${report.getTrackingUrl}\n" + + s"\t appUser: ${report.getUser}") + } if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { - return + return state + } + + if (returnOnRunning && state == YarnApplicationState.RUNNING) { + return state } } + // Never reached, but keeps compiler happy + throw new SparkException("While loop is depleted! This should never happen...") } /** diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index acbdad46c9ea0..a825c41667e25 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnSparkHadoopUtil} +import org.apache.spark.deploy.yarn.{Client, ClientArguments} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -36,7 +36,6 @@ private[spark] class YarnClientSchedulerBackend( var client: Client = null var appId: ApplicationId = null - var checkerThread: Thread = null var stopping: Boolean = false var totalExpectedExecutors = 0 @@ -83,66 +82,62 @@ private[spark] class YarnClientSchedulerBackend( totalExpectedExecutors = args.numExecutors client = new Client(args, conf) appId = client.submitApplication() - waitForApp() - checkerThread = yarnApplicationStateCheckerThread() + waitForApplication() + asyncMonitorApplication() } - def waitForApp() { - - // TODO : need a better way to find out whether the executors are ready or not - // maybe by resource usage report? - while(true) { - val report = client.getApplicationReport(appId) - - logInfo("Application report from ASM: \n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" - ) - - // Ready to go, or already gone. - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.RUNNING) { - return - } else if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - throw new SparkException("Yarn application already ended," + - "might be killed or not able to launch application master.") - } - - Thread.sleep(1000) + /** + * 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 != null, "Application has not been submitted yet!") + val state = client.monitorApplication(appId, 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.getId} has started running.") } } - private def yarnApplicationStateCheckerThread(): Thread = { + /** + * 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(): Thread = { + assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { - while (!stopping) { - val report = client.getApplicationReport(appId) - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.KILLED - || state == YarnApplicationState.FAILED) { - logError(s"Yarn application already ended: $state") - sc.stop() - stopping = true - } - Thread.sleep(1000L) + val state = client.monitorApplication(appId, logApplicationReport = false) // blocking + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.KILLED || + state == YarnApplicationState.FAILED) { + logWarning(s"Yarn application has exited: $state") + sc.stop() + stopping = true } - checkerThread = null - Thread.currentThread().interrupt() } } - t.setName("Yarn Application State Checker") + t.setName("Yarn Application State Monitor") t.setDaemon(true) t.start() 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!") stopping = true super.stop() - client.stop + client.stop() logInfo("Stopped") } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index c222641f4866d..ee6c0adb083a7 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -109,6 +109,7 @@ private[spark] class Client( yarnClient.getApplicationReport(appId) /** */ + // FIXME: This could throw NPE override def getClientToken(report: ApplicationReport): String = report.getClientToAMToken.toString } From 6de9072d8cc3efbbc1c29a7fc0be6dab51e11a04 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 9 Sep 2014 12:48:49 -0700 Subject: [PATCH 03/18] Guard against potential NPE in debug logging mode The getClientToken (or getClientToAMToken) method in Hadoop apparently returns null sometimes. We need to prevent NPEs that result from this. Yay documentation. --- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 8 ++++++-- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 8 +++++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 779609f8b2c7e..d0a888e81f8fe 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -122,8 +122,12 @@ private[spark] class Client( amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) } - /** */ - override def getClientToken(report: ApplicationReport): String = report.getClientToken + /** + * 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. + */ + override def getClientToken(report: ApplicationReport): String = + Option(report.getClientToken).getOrElse("") } private[spark] object Client { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index ee6c0adb083a7..e9b95d9c2a510 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -108,10 +108,12 @@ private[spark] class Client( override def getApplicationReport(appId: ApplicationId): ApplicationReport = yarnClient.getApplicationReport(appId) - /** */ - // FIXME: This could throw NPE + /** + * 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. + */ override def getClientToken(report: ApplicationReport): String = - report.getClientToAMToken.toString + Option(report.getClientToAMToken).map(_.toString).getOrElse("") } private[spark] object Client { From ef7069aee933a97ae60c112d4d2b581678f27e87 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 9 Sep 2014 13:32:22 -0700 Subject: [PATCH 04/18] Clean up YarnClientSchedulerBackend more --- .../spark/deploy/yarn/ClientArguments.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 68 ++++++++++--------- 2 files changed, 38 insertions(+), 32 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 40d8d6d6e6961..83781b3e7515b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -147,7 +147,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { "Options:\n" + " --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster mode)\n" + " --class CLASS_NAME Name of your application's main class (required)\n" + - " --arg ARGS Argument to be passed to your application's main class.\n" + + " --arg ARG Argument to be passed to your application's main class.\n" + " Multiple invocations are possible, each will be passed in order.\n" + " --num-executors NUM Number of executors to start (Default: 2)\n" + " --executor-cores NUM Number of cores for the executors (Default: 1).\n" + diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index a825c41667e25..da4196843629b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -39,45 +39,22 @@ private[spark] class YarnClientSchedulerBackend( var stopping: Boolean = false var totalExpectedExecutors = 0 - private[spark] def addArg(optionName: String, envVar: String, sysProp: String, - arrayBuf: ArrayBuffer[String]) { - if (System.getenv(envVar) != null) { - arrayBuf += (optionName, System.getenv(envVar)) - } else if (sc.getConf.contains(sysProp)) { - arrayBuf += (optionName, sc.getConf.get(sysProp)) - } - } - + /** + * Create a Yarn client to submit an application to the ResourceManager. + * This waits until the application is running. + */ override def start() { super.start() - val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort conf.set("spark.driver.appUIAddress", sc.ui.appUIHostPort) val argsArrayBuf = new ArrayBuffer[String]() - argsArrayBuf += ( - "--args", hostport - ) - - // process any optional arguments, given either as environment variables - // or system properties. use the defaults already defined in ClientArguments - // if things aren't specified. system properties override environment - // variables. - List(("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), - ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), - ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.executor.instances"), - ("--num-executors", "SPARK_EXECUTOR_INSTANCES", "spark.executor.instances"), - ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), - ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), - ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), - ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), - ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), - ("--name", "SPARK_YARN_APP_NAME", "spark.app.name")) - .foreach { case (optName, envVar, sysProp) => addArg(optName, envVar, sysProp, argsArrayBuf) } - - logDebug("ClientArguments called with: " + argsArrayBuf) + argsArrayBuf += ("--arg", hostport) + argsArrayBuf ++= getExtraClientArguments + + logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" ")) val args = new ClientArguments(argsArrayBuf.toArray, conf) totalExpectedExecutors = args.numExecutors client = new Client(args, conf) @@ -86,6 +63,35 @@ private[spark] class YarnClientSchedulerBackend( asyncMonitorApplication() } + /** + * Return any extra command line arguments to be passed to Client provided in the form of + * environment variables or Spark properties. + */ + private def getExtraClientArguments: Seq[String] = { + val extraArgs = new ArrayBuffer[String] + val optionTuples = // List of (target Client argument, environment variable, Spark property) + List( + ("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), + ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), + ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.executor.instances"), + ("--num-executors", "SPARK_EXECUTOR_INSTANCES", "spark.executor.instances"), + ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), + ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), + ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), + ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), + ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), + ("--name", "SPARK_YARN_APP_NAME", "spark.app.name") + ) + optionTuples.foreach { case (argName, envVar, sparkProp) => + if (System.getenv(envVar) != null) { + extraArgs += (optionName, System.getenv(envVar)) + } else if (sc.getConf.contains(sysProp)) { + extraArgs += (optionName, sc.getConf.get(sysProp)) + } + } + extraArgs + } + /** * 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. From 6c94d79cfe6c007c1ac4f21aeb7e7ed37dc8316c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 9 Sep 2014 16:00:59 -0700 Subject: [PATCH 05/18] Various cleanups in ClientBase and ClientArguments --- .../org/apache/spark/deploy/yarn/Client.scala | 2 - .../spark/deploy/yarn/ClientArguments.scala | 55 ++++-- .../apache/spark/deploy/yarn/ClientBase.scala | 178 ++++++++---------- .../cluster/YarnClientSchedulerBackend.scala | 6 +- .../org/apache/spark/deploy/yarn/Client.scala | 2 - 5 files changed, 122 insertions(+), 121 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index d0a888e81f8fe..83a7cc2692a03 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -50,8 +50,6 @@ private[spark] class Client( /** Submit an application running our ApplicationMaster to the ResourceManager. */ override def submitApplication(): ApplicationId = { - validateArgs() - // Initialize and start the client service. init(yarnConf) start() diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 83781b3e7515b..b63d0b5f95ee9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -17,15 +17,14 @@ package org.apache.spark.deploy.yarn -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf -import org.apache.spark.scheduler.InputFormatInfo import org.apache.spark.util.{Utils, IntParam, MemoryParam} // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! -class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { +private[spark] class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var addJars: String = null var files: String = null var archives: String = null @@ -35,28 +34,48 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var executorMemory = 1024 // MB var executorCores = 1 var numExecutors = 2 - var amQueue = sparkConf.get("QUEUE", "default") + var amQueue = sparkConf.get("spark.yarn.queue", "default") var amMemory: Int = 512 // MB var appName: String = "Spark" var priority = 0 - parseArgs(args.toList) + // Additional memory to allocate to containers + val memoryOverhead = sparkConf.getInt( + "spark.yarn.driver.memoryOverhead", YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - // env variable SPARK_YARN_DIST_ARCHIVES/SPARK_YARN_DIST_FILES set in yarn-client then - // it should default to hdfs:// - files = Option(files).getOrElse(sys.env.get("SPARK_YARN_DIST_FILES").orNull) - archives = Option(archives).getOrElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES").orNull) + parseArgs(args.toList) + loadDefaultArgs() + validateArgs() + + private def loadDefaultArgs(): Unit = { + // For backward compatibility, SPARK_YARN_DIST_{ARCHIVES/FILES} should be resolved to hdfs://, + // while spark.yarn.dist.{archives/files} should be resolved to file:// (SPARK-2051). + files = Option(files).orElse(sys.env.get("SPARK_YARN_DIST_FILES")).orNull + files = Option(files) + .orElse(sparkConf.getOption("spark.yarn.dist.files")) + .map(p => Utils.resolveURIs(p)) + .orNull + archives = Option(archives).orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")).orNull + archives = Option(archives) + .orElse(sparkConf.getOption("spark.yarn.dist.archives")) + .map(p => Utils.resolveURIs(p)) + .orNull + } - // spark.yarn.dist.archives/spark.yarn.dist.files defaults to use file:// if not specified, - // for both yarn-client and yarn-cluster - files = Option(files).getOrElse(sparkConf.getOption("spark.yarn.dist.files"). - map(p => Utils.resolveURIs(p)).orNull) - archives = Option(archives).getOrElse(sparkConf.getOption("spark.yarn.dist.archives"). - map(p => Utils.resolveURIs(p)).orNull) + private def validateArgs(): Unit = { + Map[Boolean, String]( + (numExecutors <= 0) -> "You must specify at least 1 executor!", + (amMemory <= memoryOverhead) -> s"AM memory must be > $memoryOverhead MB", + (executorMemory <= memoryOverhead) -> s"Executor memory must be > $memoryOverhead MB" + ).foreach { case (errorCondition, errorMessage) => + if (errorCondition) { + throw new IllegalArgumentException(errorMessage + "\n" + getUsageMessage()) + } + } + } private def parseArgs(inputArgs: List[String]): Unit = { val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() - var args = inputArgs while (!args.isEmpty) { @@ -138,10 +157,8 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { userArgs = userArgsBuffer.readOnly } - - def getUsageMessage(unknownParam: Any = null): String = { + private def getUsageMessage(unknownParam: List[String] = null): String = { val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else "" - message + "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + "Options:\n" + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index a0153afcb4290..df0d8591ef8c8 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -40,68 +40,38 @@ import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} /** - * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the YARN ResourceManager. + * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. + * The Client submits an application to the YARN ResourceManager. */ private[spark] trait ClientBase extends Logging { import ClientBase._ - val args: ClientArguments - val hadoopConf: Configuration - val sparkConf: SparkConf - val yarnConf: YarnConfiguration - val credentials = UserGroupInformation.getCurrentUser.getCredentials + protected val args: ClientArguments + protected val hadoopConf: Configuration + protected val sparkConf: SparkConf + protected val yarnConf: YarnConfiguration + protected val credentials = UserGroupInformation.getCurrentUser.getCredentials + protected val memoryOverhead = args.memoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() - // 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) + /** Verify that we have not requested more resources than is available in the cluster. */ + protected def verifyClusterResources(newAppResponse: GetNewApplicationResponse) = { + val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() + logInfo(s"Max memory capability of a single resource in this cluster: $maxMem MB") - // Additional memory overhead - in mb. - protected def memoryOverhead: Int = sparkConf.getInt("spark.yarn.driver.memoryOverhead", - YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - - // TODO(harvey): This could just go in ClientArguments. - def validateArgs() = { - Map( - (args.numExecutors <= 0) -> "Error: You must specify at least 1 executor!", - (args.amMemory <= memoryOverhead) -> ("Error: AM memory size must be" + - "greater than: " + memoryOverhead), - (args.executorMemory <= memoryOverhead) -> ("Error: Executor memory size" + - "must be greater than: " + memoryOverhead.toString) - ).foreach { case(cond, errStr) => - if (cond) { - logError(errStr) - throw new IllegalArgumentException(args.getUsageMessage()) - } - } - } - - def getAppStagingDir(appId: ApplicationId): String = { - SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR - } - - def verifyClusterResources(app: GetNewApplicationResponse) = { - val maxMem = app.getMaximumResourceCapability().getMemory() - logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) - - // If we have requested more then the clusters max for a single resource then exit. + // If we have requested more than the cluster maximum for a single resource, exit. if (args.executorMemory > maxMem) { val errorMessage = "Required executor memory (%d MB), is above the max threshold (%d MB) of this cluster." .format(args.executorMemory, maxMem) - logError(errorMessage) throw new IllegalArgumentException(errorMessage) } - val amMem = args.amMemory + memoryOverhead + val amMem = getAMMemory(newAppResponse) + memoryOverhead if (amMem > maxMem) { - - val errorMessage = "Required AM memory (%d) is above the max threshold (%d) of this cluster." - .format(amMem, maxMem) + val errorMessage = + "Required AM memory (%d MB) is above the max threshold (%d MB) of this cluster." + .format(amMem, maxMem) logError(errorMessage) throw new IllegalArgumentException(errorMessage) } @@ -145,8 +115,8 @@ private[spark] trait ClientBase extends Logging { } } - /** Copy the file into HDFS if needed. */ - private[yarn] def copyRemoteFile( + /** Copy the file into HDFS if needed. Exposed for testing. */ + def copyRemoteFile( dstDir: Path, originalPath: Path, replication: Short, @@ -205,7 +175,7 @@ private[spark] trait ClientBase extends Logging { (ClientBase.SPARK_JAR, ClientBase.sparkJar(sparkConf), ClientBase.CONF_SPARK_JAR), (ClientBase.APP_JAR, args.userJar, ClientBase.CONF_SPARK_USER_JAR), ("log4j.properties", oldLog4jConf.getOrElse(null), null) - ).foreach { case(destName, _localPath, confKey) => + ).foreach { case (destName, _localPath, confKey) => val localPath: String = if (_localPath != null) _localPath.trim() else "" if (! localPath.isEmpty()) { val localURI = new URI(localPath) @@ -251,11 +221,12 @@ private[spark] trait ClientBase extends Logging { localResources } - /** Get all application master environment variables set on this SparkConf */ + /** Get all application master environment variables set on this SparkConf. */ def getAppMasterEnv: Seq[(String, String)] = { val prefix = "spark.yarn.appMasterEnv." - sparkConf.getAll.filter{case (k, v) => k.startsWith(prefix)} - .map{case (k, v) => (k.substring(prefix.length), v)} + sparkConf.getAll + .filter { case (k, v) => k.startsWith(prefix) } + .map { case (k, v) => (k.substring(prefix.length), v) } } /** @@ -321,16 +292,6 @@ private[spark] trait ClientBase extends Logging { env } - def userArgsToString(clientArgs: ClientArguments): String = { - val prefix = " --arg " - val args = clientArgs.userArgs - val retval = new StringBuilder() - for (arg <- args) { - retval.append(prefix).append(" ").append(YarnSparkHadoopUtil.escapeForShell(arg)) - } - retval.toString - } - /** * Prepare a ContainerLaunchContext to launch our AM container. * This sets up the launch environment, java options, and the command for launching the AM. @@ -396,19 +357,27 @@ private[spark] trait ClientBase extends Logging { } else { Nil } + val userJar = + if (args.userJar != null) { + Seq("--jar", args.userJar) + } else { + Nil + } val amClass = if (isLaunchingDriver) { classOf[ApplicationMaster].getName() } else { classOf[ApplicationMaster].getName().replace("ApplicationMaster", "ExecutorLauncher") } + val userArgs = args.userArgs.flatMap { arg => + Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) + } val amArgs = - Seq(amClass) ++ userClass ++ - (if (args.userJar != null) Seq("--jar", args.userJar) else Nil) ++ - Seq("--executor-memory", args.executorMemory.toString, + Seq(amClass) ++ userClass ++ userJar ++ userArgs ++ + Seq( + "--executor-memory", args.executorMemory.toString, "--executor-cores", args.executorCores.toString, - "--num-executors ", args.numExecutors.toString, - userArgsToString(args)) + "--num-executors ", args.numExecutors.toString) // Command for the ApplicationMaster val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ @@ -417,15 +386,19 @@ private[spark] trait ClientBase extends Logging { "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - logInfo("Yarn AM launch context:") - logInfo(s" user class: ${args.userClass}") - logInfo(s" env: $launchEnv") - logInfo(s" command: ${commands.mkString(" ")}") - // TODO: it would be nicer to just make sure there are no null commands here val printableCommands = commands.map(s => if (s == null) "null" else s).toList amContainer.setCommands(printableCommands) + logDebug("===============================================================================") + logDebug("Yarn AM launch context:") + logDebug(s" user class: ${args.userClass}") + logDebug(" env:") + launchEnv.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)) @@ -454,7 +427,7 @@ private[spark] trait ClientBase extends Logging { if (logApplicationReport) { logInfo(s"Application report from ResourceManager for application ${appId.getId} " + s"(state: $state)") - logDebug( + logDebug("\n" + s"\t full application identifier: $appId\n" + s"\t clientToken: ${getClientToken(report)}\n" + s"\t appDiagnostics: ${report.getDiagnostics}\n" + @@ -559,6 +532,10 @@ private[spark] object ClientBase extends Logging { } } + def getAppStagingDir(appId: ApplicationId): String = { + SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR + } + def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) = { val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf) for (c <- classPathElementsToAdd.flatten) { @@ -630,8 +607,12 @@ private[spark] object ClientBase extends Logging { triedDefault.toOption } - def populateClasspath(args: ClientArguments, conf: Configuration, sparkConf: SparkConf, - env: HashMap[String, String], extraClassPath: Option[String] = None) { + def populateClasspath( + args: ClientArguments, + conf: Configuration, + sparkConf: SparkConf, + env: HashMap[String, String], + extraClassPath: Option[String] = None): Unit = { extraClassPath.foreach(addClasspathEntry(_, env)) addClasspathEntry(Environment.PWD.$(), env) @@ -654,8 +635,10 @@ private[spark] object ClientBase extends Logging { * Adds the user jars which have local: URIs (or alternate names, such as APP_JAR) explicitly * to the classpath. */ - private def addUserClasspath(args: ClientArguments, conf: SparkConf, - env: HashMap[String, String]) = { + private def addUserClasspath( + args: ClientArguments, + conf: SparkConf, + env: HashMap[String, String]): Unit = { if (args != null) { addFileToClasspath(args.userJar, APP_JAR, env) if (args.addJars != null) { @@ -684,8 +667,10 @@ private[spark] object ClientBase extends Logging { * @param fileName Alternate name for the file (optional). * @param env Map holding the environment variables. */ - private def addFileToClasspath(path: String, fileName: String, - env: HashMap[String, String]) : Unit = { + private def addFileToClasspath( + path: String, + fileName: String, + env: HashMap[String, String]): Unit = { if (path != null) { scala.util.control.Exception.ignoring(classOf[URISyntaxException]) { val localPath = getLocalPath(path) @@ -711,19 +696,22 @@ private[spark] object ClientBase extends Logging { null } - private def addClasspathEntry(path: String, env: HashMap[String, String]) = - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, path, - File.pathSeparator) + private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit = + YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, path, File.pathSeparator) /** * Get the list of namenodes the user may access. */ - private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { - sparkConf.get("spark.yarn.access.namenodes", "").split(",").map(_.trim()).filter(!_.isEmpty) - .map(new Path(_)).toSet + def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { + sparkConf.get("spark.yarn.access.namenodes", "") + .split(",") + .map(_.trim()) + .filter(!_.isEmpty) + .map(new Path(_)) + .toSet } - private[yarn] def getTokenRenewer(conf: Configuration): String = { + def getTokenRenewer(conf: Configuration): String = { val delegTokenRenewer = Master.getMasterPrincipal(conf) logDebug("delegation token renewer is: " + delegTokenRenewer) if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { @@ -737,16 +725,16 @@ private[spark] object ClientBase extends Logging { /** * Obtains tokens for the namenodes passed in and adds them to the credentials. */ - private[yarn] def obtainTokensForNamenodes(paths: Set[Path], conf: Configuration, - creds: Credentials) { + def obtainTokensForNamenodes( + paths: Set[Path], + conf: Configuration, + creds: Credentials): Unit = { if (UserGroupInformation.isSecurityEnabled()) { val delegTokenRenewer = getTokenRenewer(conf) - - paths.foreach { - dst => - val dstFs = dst.getFileSystem(conf) - logDebug("getting token for namenode: " + dst) - dstFs.addDelegationTokens(delegTokenRenewer, creds) + paths.foreach { dst => + val dstFs = dst.getFileSystem(conf) + logDebug("getting token for namenode: " + dst) + dstFs.addDelegationTokens(delegTokenRenewer, creds) } } } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index da4196843629b..9c78cc3137b5d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -82,11 +82,11 @@ private[spark] class YarnClientSchedulerBackend( ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), ("--name", "SPARK_YARN_APP_NAME", "spark.app.name") ) - optionTuples.foreach { case (argName, envVar, sparkProp) => + optionTuples.foreach { case (optionName, envVar, sparkProp) => if (System.getenv(envVar) != null) { extraArgs += (optionName, System.getenv(envVar)) - } else if (sc.getConf.contains(sysProp)) { - extraArgs += (optionName, sc.getConf.get(sysProp)) + } else if (sc.getConf.contains(sparkProp)) { + extraArgs += (optionName, sc.getConf.get(sparkProp)) } } extraArgs diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index e9b95d9c2a510..a647f053d6ed0 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -48,8 +48,6 @@ private[spark] class Client( /** Submit an application running our ApplicationMaster to the ResourceManager. */ override def submitApplication(): ApplicationId = { - validateArgs() - // Initialize and start the client service. yarnClient.init(yarnConf) yarnClient.start() From 8766d377e24904c12c820175f8db17e8549e9835 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 9 Sep 2014 19:05:51 -0700 Subject: [PATCH 06/18] Heavily add documentation to Client* classes + various clean-ups --- .../org/apache/spark/deploy/yarn/Client.scala | 42 +-- .../spark/deploy/yarn/ClientArguments.scala | 7 +- .../apache/spark/deploy/yarn/ClientBase.scala | 255 ++++++++++-------- .../yarn/ClientDistributedCacheManager.scala | 97 +++---- .../spark/deploy/yarn/ClientBaseSuite.scala | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 31 ++- 6 files changed, 234 insertions(+), 200 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 83a7cc2692a03..a86a8dd143be0 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -48,9 +48,14 @@ private[spark] class Client( val yarnConf: YarnConfiguration = new YarnConfiguration(hadoopConf) + /* ------------------------------------------------------------------------------------- * + | The following methods have much in common in the stable and alpha versions of Client, | + | but cannot be implemented in the parent trait due to subtle API differences across | + | hadoop versions. | + * ------------------------------------------------------------------------------------- */ + /** Submit an application running our ApplicationMaster to the ResourceManager. */ override def submitApplication(): ApplicationId = { - // Initialize and start the client service. init(yarnConf) start() @@ -64,10 +69,8 @@ private[spark] class Client( // Verify whether the cluster has enough resources for our AM. verifyClusterResources(newAppResponse) - // Set up ContainerLaunchContext to launch our AM container. + // Set up the appropriate contexts to launch our AM. val containerContext = createContainerLaunchContext(newAppResponse) - - // Set up ApplicationSubmissionContext to submit our AM. val appContext = createApplicationSubmissionContext(appId, containerContext) // Finally, submit and monitor the application. @@ -77,7 +80,9 @@ private[spark] class Client( } /** - * + * Set up a context for launching our ApplicationMaster container. + * In the Yarn alpha API, the memory requirements of this container must be set in + * the ContainerLaunchContext instead of the ApplicationSubmissionContext. */ override def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) : ContainerLaunchContext = { @@ -88,9 +93,7 @@ private[spark] class Client( containerContext } - /** - * - */ + /** Set up the context for submitting our ApplicationMaster. */ def createApplicationSubmissionContext( appId: ApplicationId, containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { @@ -104,25 +107,30 @@ private[spark] class Client( } /** - * + * Set up security tokens for launching our ApplicationMaster container. + * ContainerLaunchContext#setContainerTokens is renamed `setTokens` in the stable API. */ - override def getAMMemory(newApp: GetNewApplicationResponse): Int = { - val minResMemory = newApp.getMinimumResourceCapability().getMemory() - val amMemory = ((args.amMemory / minResMemory) * minResMemory) + - ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - memoryOverhead) - amMemory - } - - /** */ override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { val dob = new DataOutputBuffer() credentials.writeTokenStorageToStream(dob) amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) } + /** + * Return the amount of memory for launching the ApplicationMaster container (MB). + * GetNewApplicationResponse#getMinimumResourceCapability does not exist in the stable API. + */ + override def getAMMemory(newAppResponse: GetNewApplicationResponse): Int = { + val minResMemory = newAppResponse.getMinimumResourceCapability().getMemory() + val amMemory = ((args.amMemory / minResMemory) * minResMemory) + + ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - memoryOverhead) + amMemory + } + /** * 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. + * ApplicationReport#getClientToken is renamed `getClientToAMToken` in the stable API. */ override def getClientToken(report: ApplicationReport): String = Option(report.getClientToken).getOrElse("") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index b63d0b5f95ee9..a6158759817e2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -24,7 +24,7 @@ import org.apache.spark.util.{Utils, IntParam, MemoryParam} // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! -private[spark] class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { +private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) { var addJars: String = null var files: String = null var archives: String = null @@ -47,6 +47,7 @@ private[spark] class ClientArguments(val args: Array[String], val sparkConf: Spa loadDefaultArgs() validateArgs() + /** Load any default arguments provided through environment variables and Spark properties. */ private def loadDefaultArgs(): Unit = { // For backward compatibility, SPARK_YARN_DIST_{ARCHIVES/FILES} should be resolved to hdfs://, // while spark.yarn.dist.{archives/files} should be resolved to file:// (SPARK-2051). @@ -62,6 +63,10 @@ private[spark] class ClientArguments(val args: Array[String], val sparkConf: Spa .orNull } + /** + * Fail fast if any arguments provided are invalid. + * This is intended to be called only after the provided arguments have been parsed. + */ private def validateArgs(): Unit = { Map[Boolean, String]( (numExecutors <= 0) -> "You must specify at least 1 executor!", diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index df0d8591ef8c8..dece10d5af075 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -54,7 +54,9 @@ private[spark] trait ClientBase extends Logging { protected val memoryOverhead = args.memoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() - /** Verify that we have not requested more resources than is available in the cluster. */ + /** + * Verify that we have not requested more resources than is available in the cluster. + */ protected def verifyClusterResources(newAppResponse: GetNewApplicationResponse) = { val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() logInfo(s"Max memory capability of a single resource in this cluster: $maxMem MB") @@ -80,43 +82,10 @@ private[spark] trait ClientBase extends Logging { // getting all the node reports and computing ourselves. } - /** See if two file systems are the same or not. */ - private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { - val srcUri = srcFs.getUri() - val dstUri = destFs.getUri() - if (srcUri.getScheme() == null) { - return false - } - if (!srcUri.getScheme().equals(dstUri.getScheme())) { - return false - } - var srcHost = srcUri.getHost() - var dstHost = dstUri.getHost() - if ((srcHost != null) && (dstHost != null)) { - try { - srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() - dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() - } catch { - case e: UnknownHostException => - return false - } - if (!srcHost.equals(dstHost)) { - return false - } - } else if (srcHost == null && dstHost != null) { - return false - } else if (srcHost != null && dstHost == null) { - return false - } - if (srcUri.getPort() != dstUri.getPort()) { - false - } else { - true - } - } - - /** Copy the file into HDFS if needed. Exposed for testing. */ - def copyRemoteFile( + /** + * Copy the file to a remote file system if needed. Exposed for testing. + */ + def copyFileToRemote( dstDir: Path, originalPath: Path, replication: Short, @@ -139,23 +108,35 @@ private[spark] trait ClientBase extends Logging { destPath } - private def qualifyForLocal(localURI: URI): Path = { - var qualifiedURI = localURI - // If not specified, assume these are in the local filesystem to keep behavior like Hadoop - if (qualifiedURI.getScheme() == null) { - qualifiedURI = new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(qualifiedURI)).toString) - } + /** + * 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): 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) } + /** + * 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(appStagingDir: String): HashMap[String, LocalResource] = { - logInfo("Preparing Local resources") // Upload Spark and the application JAR to the remote file system if necessary. Add them as // local resources to the application master. val fs = FileSystem.get(hadoopConf) val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val nns = ClientBase.getNameNodesToAccess(sparkConf) + dst - ClientBase.obtainTokensForNamenodes(nns, hadoopConf, credentials) + val nns = getNameNodesToAccess(sparkConf) + dst + obtainTokensForNamenodes(nns, hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort val localResources = HashMap[String, LocalResource]() @@ -171,72 +152,86 @@ private[spark] trait ClientBase extends Logging { "for alternatives.") } + /** + * Copy the given main resource to the distributed cache if the scheme is not "local". + * Otherwise, set the corresponding key in our SparkConf to handle it downstream. + * Each resource is represented by a 4-tuple of: + * (1) destination resource name, + * (2) local path to the resource, + * (3) Spark property key to set if the scheme is not local, and + * (4) whether to set permissions for this resource + */ List( - (ClientBase.SPARK_JAR, ClientBase.sparkJar(sparkConf), ClientBase.CONF_SPARK_JAR), - (ClientBase.APP_JAR, args.userJar, ClientBase.CONF_SPARK_USER_JAR), - ("log4j.properties", oldLog4jConf.getOrElse(null), null) - ).foreach { case (destName, _localPath, confKey) => + (SPARK_JAR, sparkJar(sparkConf), CONF_SPARK_JAR, false), + (APP_JAR, args.userJar, CONF_SPARK_USER_JAR, true), + ("log4j.properties", oldLog4jConf.orNull, null, false) + ).foreach { case (destName, _localPath, confKey, setPermissions) => val localPath: String = if (_localPath != null) _localPath.trim() else "" - if (! localPath.isEmpty()) { + if (!localPath.isEmpty()) { val localURI = new URI(localPath) - if (!ClientBase.LOCAL_SCHEME.equals(localURI.getScheme())) { - val setPermissions = destName.equals(ClientBase.APP_JAR) - val destPath = copyRemoteFile(dst, qualifyForLocal(localURI), replication, setPermissions) + if (localURI.getScheme != LOCAL_SCHEME) { + val src = getQualifiedLocalPath(localURI) + val destPath = copyFileToRemote(dst, src, replication, setPermissions) val destFs = FileSystem.get(destPath.toUri(), hadoopConf) - distCacheMgr.addResource(destFs, hadoopConf, destPath, localResources, LocalResourceType.FILE, - destName, statCache) + distCacheMgr.addResource(destFs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, destName, statCache) } else if (confKey != null) { + // If the resource is intended for local use only, handle this downstream + // by setting the appropriate property sparkConf.set(confKey, localPath) } } } + /** + * 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] - val fileLists = List( (args.addJars, LocalResourceType.FILE, true), + List( + (args.addJars, LocalResourceType.FILE, true), (args.files, LocalResourceType.FILE, false), - (args.archives, LocalResourceType.ARCHIVE, false) ) - fileLists.foreach { case (flist, resType, addToClasspath) => + (args.archives, LocalResourceType.ARCHIVE, false) + ).foreach { case (flist, resType, addToClasspath) => if (flist != null && !flist.isEmpty()) { - flist.split(',').foreach { case file: String => + flist.split(',').foreach { file => val localURI = new URI(file.trim()) - if (!ClientBase.LOCAL_SCHEME.equals(localURI.getScheme())) { + if (localURI.getScheme != LOCAL_SCHEME) { val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) - val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, hadoopConf, destPath, localResources, resType, - linkname, statCache) + val destPath = copyFileToRemote(dst, localPath, replication) + distCacheMgr.addResource( + fs, hadoopConf, destPath, localResources, resType, linkname, statCache) if (addToClasspath) { cachedSecondaryJarLinks += linkname } } else if (addToClasspath) { + // Resource is intended for local use only and should be added to the class path cachedSecondaryJarLinks += file.trim() } } } } logInfo("Prepared Local resources " + localResources) - sparkConf.set(ClientBase.CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) + if (cachedSecondaryJarLinks.nonEmpty) { + sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) + } UserGroupInformation.getCurrentUser().addCredentials(credentials) localResources } - /** Get all application master environment variables set on this SparkConf. */ - def getAppMasterEnv: Seq[(String, String)] = { - val prefix = "spark.yarn.appMasterEnv." - sparkConf.getAll - .filter { case (k, v) => k.startsWith(prefix) } - .map { case (k, v) => (k.substring(prefix.length), v) } - } - /** - * + * Set up the environment for launching our ApplicationMaster container. */ - def setupLaunchEnv(stagingDir: String): HashMap[String, String] = { + private def setupLaunchEnv(stagingDir: String): HashMap[String, String] = { logInfo("Setting up the launch environment") val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - ClientBase.populateClasspath(args, yarnConf, sparkConf, env, extraCp) + populateClasspath(args, yarnConf, sparkConf, env, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -245,9 +240,14 @@ private[spark] trait ClientBase extends Logging { distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) - getAppMasterEnv.foreach { case (key, value) => - YarnSparkHadoopUtil.addToEnvironment(env, key, value, File.pathSeparator) - } + // 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.addToEnvironment(env, k, v, File.pathSeparator) + } // Keep this for backwards compatibility but users should move to the config sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => @@ -293,10 +293,10 @@ private[spark] trait ClientBase extends Logging { } /** - * Prepare a ContainerLaunchContext to launch our AM container. + * Set up a ContainerLaunchContext to launch our ApplicationMaster container. * This sets up the launch environment, java options, and the command for launching the AM. */ - def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) + protected def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) : ContainerLaunchContext = { logInfo("Setting up container launch context") @@ -468,17 +468,20 @@ private[spark] trait ClientBase extends Logging { /** Submit an application running our ApplicationMaster to the ResourceManager. */ def submitApplication(): ApplicationId - /** */ - def setupSecurityToken(containerContext: ContainerLaunchContext): Unit + /** Set up security tokens for launching our ApplicationMaster container. */ + protected def setupSecurityToken(containerContext: ContainerLaunchContext): Unit - /** */ - def getApplicationReport(appId: ApplicationId): ApplicationReport + /** Get the application report from the ResourceManager for an application we have submitted. */ + protected def getApplicationReport(appId: ApplicationId): ApplicationReport - /** */ - def getClientToken(report: ApplicationReport): String + /** + * 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. + */ + protected def getClientToken(report: ApplicationReport): String /** Return the amount of memory for launching the ApplicationMaster container (MB). */ - def getAMMemory(newAppResponse: GetNewApplicationResponse): Int = args.amMemory + protected def getAMMemory(newAppResponse: GetNewApplicationResponse): Int = args.amMemory } private[spark] object ClientBase extends Logging { @@ -519,7 +522,7 @@ private[spark] object ClientBase extends Logging { * This method first looks in the SparkConf object for the CONF_SPARK_JAR key, and in the * user environment if that is not found (for backwards compatibility). */ - def sparkJar(conf: SparkConf) = { + private def sparkJar(conf: SparkConf) = { if (conf.contains(CONF_SPARK_JAR)) { conf.get(CONF_SPARK_JAR) } else if (System.getenv(ENV_SPARK_JAR) != null) { @@ -532,7 +535,10 @@ private[spark] object ClientBase extends Logging { } } - def getAppStagingDir(appId: ApplicationId): String = { + /** + * Return the path to the given application's staging directory. + */ + private def getAppStagingDir(appId: ApplicationId): String = { SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR } @@ -581,7 +587,7 @@ private[spark] object ClientBase extends Logging { /** * In Hadoop 0.23, the MR application classpath comes with the YARN application - * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. + * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. * So we need to use reflection to retrieve it. */ def getDefaultMRApplicationClasspath: Option[Seq[String]] = { @@ -620,10 +626,10 @@ private[spark] object ClientBase extends Logging { if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { addUserClasspath(args, sparkConf, env) addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - ClientBase.populateHadoopClasspath(conf, env) + populateHadoopClasspath(conf, env) } else { addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - ClientBase.populateHadoopClasspath(conf, env) + populateHadoopClasspath(conf, env) addUserClasspath(args, sparkConf, env) } @@ -650,8 +656,11 @@ private[spark] object ClientBase extends Logging { val userJar = conf.get(CONF_SPARK_USER_JAR, null) addFileToClasspath(userJar, APP_JAR, env) - val cachedSecondaryJarLinks = conf.get(CONF_SPARK_YARN_SECONDARY_JARS, "").split(",") - cachedSecondaryJarLinks.foreach(jar => addFileToClasspath(jar, null, env)) + // Add any secondary jars to the classpath + conf.get(CONF_SPARK_YARN_SECONDARY_JARS, "") + .split(",") + .filter(_.nonEmpty) + .foreach(jar => addFileToClasspath(jar, null, env)) } } @@ -673,27 +682,16 @@ private[spark] object ClientBase extends Logging { env: HashMap[String, String]): Unit = { if (path != null) { scala.util.control.Exception.ignoring(classOf[URISyntaxException]) { - val localPath = getLocalPath(path) - if (localPath != null) { - addClasspathEntry(localPath, env) + val uri = new URI(path) + if (uri.getScheme == LOCAL_SCHEME) { + addClasspathEntry(uri.getPath, env) return } } } if (fileName != null) { - addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env); - } - } - - /** - * Returns the local path if the URI is a "local:" URI, or null otherwise. - */ - private def getLocalPath(resource: String): String = { - val uri = new URI(resource) - if (LOCAL_SCHEME.equals(uri.getScheme())) { - return uri.getPath() + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env) } - null } private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit = @@ -739,4 +737,39 @@ private[spark] object ClientBase extends Logging { } } + /** 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) { + return false + } + if (!srcUri.getScheme().equals(dstUri.getScheme())) { + return false + } + var srcHost = srcUri.getHost() + var dstHost = dstUri.getHost() + if ((srcHost != null) && (dstHost != null)) { + try { + srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() + dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() + } catch { + case e: UnknownHostException => + return false + } + if (!srcHost.equals(dstHost)) { + return false + } + } else if (srcHost == null && dstHost != null) { + return false + } else if (srcHost != null && dstHost == null) { + return false + } + if (srcUri.getPort() != dstUri.getPort()) { + false + } else { + true + } + } + } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 9b7f1fca96c6d..c592ecfdfce06 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -19,29 +19,24 @@ package org.apache.spark.deploy.yarn import java.net.URI +import scala.collection.mutable.{HashMap, LinkedHashMap, 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.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.fs.permission.FsAction -import org.apache.hadoop.yarn.api.records.LocalResource -import org.apache.hadoop.yarn.api.records.LocalResourceVisibility -import org.apache.hadoop.yarn.api.records.LocalResourceType +import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.util.{Records, ConverterUtils} -import org.apache.spark.Logging - -import scala.collection.mutable.HashMap -import scala.collection.mutable.LinkedHashMap -import scala.collection.mutable.Map - +import org.apache.spark.Logging /** Client side methods to setup the Hadoop distributed cache */ -class ClientDistributedCacheManager() extends Logging { - private val distCacheFiles: Map[String, Tuple3[String, String, String]] = - LinkedHashMap[String, Tuple3[String, String, String]]() - private val distCacheArchives: Map[String, Tuple3[String, String, String]] = - LinkedHashMap[String, Tuple3[String, String, String]]() +private[spark] class ClientDistributedCacheManager() extends Logging { + + // Mappings from remote URI to (file status, modification time, visibility) + private val distCacheFiles: Map[String, (String, String, String)] = + LinkedHashMap[String, (String, String, String)]() + private val distCacheArchives: Map[String, (String, String, String)] = + LinkedHashMap[String, (String, String, String)]() /** @@ -68,9 +63,9 @@ class ClientDistributedCacheManager() extends Logging { resourceType: LocalResourceType, link: String, statCache: Map[URI, FileStatus], - appMasterOnly: Boolean = false) = { + appMasterOnly: Boolean = false): Unit = { val destStatus = fs.getFileStatus(destPath) - val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + val amJarRsrc = Records.newRecord(classOf[LocalResource]) amJarRsrc.setType(resourceType) val visibility = getVisibility(conf, destPath.toUri(), statCache) amJarRsrc.setVisibility(visibility) @@ -80,7 +75,7 @@ class ClientDistributedCacheManager() extends Logging { if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name") localResources(link) = amJarRsrc - if (appMasterOnly == false) { + if (!appMasterOnly) { val uri = destPath.toUri() val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link) if (resourceType == LocalResourceType.FILE) { @@ -95,12 +90,10 @@ class ClientDistributedCacheManager() extends Logging { /** * Adds the necessary cache file env variables to the env passed in - * @param env */ - def setDistFilesEnv(env: Map[String, String]) = { + def setDistFilesEnv(env: Map[String, String]): Unit = { val (keys, tupleValues) = distCacheFiles.unzip val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = @@ -114,12 +107,10 @@ class ClientDistributedCacheManager() extends Logging { /** * Adds the necessary cache archive env variables to the env passed in - * @param env */ - def setDistArchivesEnv(env: Map[String, String]) = { + def setDistArchivesEnv(env: Map[String, String]): Unit = { val (keys, tupleValues) = distCacheArchives.unzip val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = @@ -133,25 +124,21 @@ class ClientDistributedCacheManager() extends Logging { /** * Returns the local resource visibility depending on the cache file permissions - * @param conf - * @param uri - * @param statCache * @return LocalResourceVisibility */ - def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): - LocalResourceVisibility = { + def getVisibility( + conf: Configuration, + uri: URI, + statCache: Map[URI, FileStatus]): LocalResourceVisibility = { if (isPublic(conf, uri, statCache)) { - return LocalResourceVisibility.PUBLIC - } - LocalResourceVisibility.PRIVATE + LocalResourceVisibility.PUBLIC + } else { + LocalResourceVisibility.PRIVATE + } } /** - * Returns a boolean to denote whether a cache file is visible to all(public) - * or not - * @param conf - * @param uri - * @param statCache + * 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 */ def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { @@ -167,13 +154,12 @@ class ClientDistributedCacheManager() extends Logging { /** * Returns true if all ancestors of the specified path have the 'execute' * permission set for all users (i.e. that other users can traverse - * the directory heirarchy to the given path) - * @param fs - * @param path - * @param statCache + * the directory hierarchy to the given path) * @return true if all ancestors have the 'execute' permission set for all users */ - def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, + def ancestorsHaveExecutePermissions( + fs: FileSystem, + path: Path, statCache: Map[URI, FileStatus]): Boolean = { var current = path while (current != null) { @@ -187,32 +173,25 @@ class ClientDistributedCacheManager() extends Logging { } /** - * Checks for a given path whether the Other permissions on it + * Checks for a given path whether the Other permissions on it * imply the permission in the passed FsAction - * @param fs - * @param path - * @param action - * @param statCache * @return true if the path in the uri is visible to all, false otherwise */ - def checkPermissionOfOther(fs: FileSystem, path: Path, - action: FsAction, statCache: Map[URI, FileStatus]): Boolean = { + def checkPermissionOfOther( + fs: FileSystem, + path: Path, + action: FsAction, + statCache: Map[URI, FileStatus]): Boolean = { val status = getFileStatus(fs, path.toUri(), statCache) val perms = status.getPermission() val otherAction = perms.getOtherAction() - if (otherAction.implies(action)) { - return true - } - false + otherAction.implies(action) } /** - * Checks to see if the given uri exists in the cache, if it does it + * Checks to see if the given uri exists in the cache, if it does it * returns the existing FileStatus, otherwise it stats the uri, stores * it in the cache, and returns the FileStatus. - * @param fs - * @param uri - * @param statCache * @return FileStatus */ def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = { diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 2c2bb1baeafaa..5e308afa8182e 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -113,7 +113,7 @@ class ClientBaseSuite extends FunSuite with Matchers { val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) - doReturn(new Path("/")).when(client).copyRemoteFile(any(classOf[Path]), + doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), any(classOf[Path]), anyShort(), anyBoolean()) val tempDir = Files.createTempDir() diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index a647f053d6ed0..3547144a9862c 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -46,9 +46,22 @@ private[spark] class Client( val yarnClient = YarnClient.createYarnClient val yarnConf = new YarnConfiguration(hadoopConf) - /** Submit an application running our ApplicationMaster to the ResourceManager. */ + def stop(): Unit = yarnClient.stop() + + /* ------------------------------------------------------------------------------------- * + | The following methods have much in common in the stable and alpha versions of Client, | + | but cannot be implemented in the parent trait due to subtle API differences across | + | hadoop versions. | + * ------------------------------------------------------------------------------------- */ + + /** + * 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. + */ override def submitApplication(): ApplicationId = { - // Initialize and start the client service. yarnClient.init(yarnConf) yarnClient.start() @@ -63,10 +76,8 @@ private[spark] class Client( // Verify whether the cluster has enough resources for our AM. verifyClusterResources(newAppResponse) - // Set up ContainerLaunchContext to launch our AM container. + // Set up the appropriate contexts to launch our AM. val containerContext = createContainerLaunchContext(newAppResponse) - - // Set up ApplicationSubmissionContext to submit our AM. val appContext = createApplicationSubmissionContext(newApp, containerContext) // Finally, submit and monitor the application. @@ -75,11 +86,9 @@ private[spark] class Client( appId } - /** Stop this client. */ - def stop(): Unit = yarnClient.stop() - /** - * + * Set up the context for submitting our ApplicationMaster. + * This uses the YarnClientApplication not available in the Yarn alpha API. */ def createApplicationSubmissionContext( newApp: YarnClientApplication, @@ -95,14 +104,14 @@ private[spark] class Client( appContext } - /** */ + /** Set up security tokens for launching our ApplicationMaster container. */ override 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. */ override def getApplicationReport(appId: ApplicationId): ApplicationReport = yarnClient.getApplicationReport(appId) From e4779b6846c9c6228513d03db9f8dfcc595e53de Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Sep 2014 13:18:24 -0700 Subject: [PATCH 07/18] Clean up log messages + variable naming in ClientBase --- .../org/apache/spark/deploy/yarn/Client.scala | 8 +- .../spark/deploy/yarn/ClientArguments.scala | 12 ++- .../apache/spark/deploy/yarn/ClientBase.scala | 92 +++++++++---------- .../org/apache/spark/deploy/yarn/Client.scala | 6 +- 4 files changed, 61 insertions(+), 57 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index a86a8dd143be0..a33c807233353 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -59,8 +59,8 @@ private[spark] class Client( init(yarnConf) start() - logInfo("Received cluster metric info from ResourceManager, number of NodeManagers: " - + getYarnClusterMetrics.getNumNodeManagers) + logInfo("Requesting a new application from cluster with %d NodeManagers" + .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers)) // Get a new application from our RM. val newAppResponse = getNewApplication() @@ -88,7 +88,7 @@ private[spark] class Client( : ContainerLaunchContext = { val containerContext = super.createContainerLaunchContext(newAppResponse) val capability = Records.newRecord(classOf[Resource]) - capability.setMemory(getAMMemory(newAppResponse) + memoryOverhead) + capability.setMemory(getAMMemory(newAppResponse) + amMemoryOverhead) containerContext.setResource(capability) containerContext } @@ -123,7 +123,7 @@ private[spark] class Client( override def getAMMemory(newAppResponse: GetNewApplicationResponse): Int = { val minResMemory = newAppResponse.getMinimumResourceCapability().getMemory() val amMemory = ((args.amMemory / minResMemory) * minResMemory) + - ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - memoryOverhead) + ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - amMemoryOverhead) amMemory } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index a6158759817e2..b66f177218219 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -40,8 +40,11 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var priority = 0 // Additional memory to allocate to containers - val memoryOverhead = sparkConf.getInt( + // For now, use driver's memory overhead as our AM container's memory overhead + val amMemoryOverhead = sparkConf.getInt( "spark.yarn.driver.memoryOverhead", YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) + val executorMemoryOverhead = sparkConf.getInt( + "spark.yarn.executor.memoryOverhead", YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) parseArgs(args.toList) loadDefaultArgs() @@ -70,8 +73,9 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) private def validateArgs(): Unit = { Map[Boolean, String]( (numExecutors <= 0) -> "You must specify at least 1 executor!", - (amMemory <= memoryOverhead) -> s"AM memory must be > $memoryOverhead MB", - (executorMemory <= memoryOverhead) -> s"Executor memory must be > $memoryOverhead MB" + (amMemory <= amMemoryOverhead) -> s"AM memory must be > $amMemoryOverhead MB", + (executorMemory <= executorMemoryOverhead) -> + s"Executor memory must be > $executorMemoryOverhead MB" ).foreach { case (errorCondition, errorMessage) => if (errorCondition) { throw new IllegalArgumentException(errorMessage + "\n" + getUsageMessage()) @@ -80,7 +84,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) } private def parseArgs(inputArgs: List[String]): Unit = { - val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() + val userArgsBuffer = new ArrayBuffer[String]() var args = inputArgs while (!args.isEmpty) { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index dece10d5af075..f5bd8cedc2324 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -51,61 +51,59 @@ private[spark] trait ClientBase extends Logging { protected val sparkConf: SparkConf protected val yarnConf: YarnConfiguration protected val credentials = UserGroupInformation.getCurrentUser.getCredentials - protected val memoryOverhead = args.memoryOverhead // MB + protected val amMemoryOverhead = args.amMemoryOverhead // MB + protected val executorMemoryOverhead = args.executorMemoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() /** - * Verify that we have not requested more resources than is available in the cluster. + * Fail fast if we have requested more resources per container than is available in the cluster. */ - protected def verifyClusterResources(newAppResponse: GetNewApplicationResponse) = { + protected def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = { val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() - logInfo(s"Max memory capability of a single resource in this cluster: $maxMem MB") - - // If we have requested more than the cluster maximum for a single resource, exit. - if (args.executorMemory > maxMem) { - val errorMessage = - "Required executor memory (%d MB), is above the max threshold (%d MB) of this cluster." - .format(args.executorMemory, maxMem) - logError(errorMessage) - throw new IllegalArgumentException(errorMessage) + logInfo("Verifying our application request has not exceeded the maximum " + + s"memory capability of the cluster ($maxMem MB per container)") + val executorMem = args.executorMemory + executorMemoryOverhead + if (executorMem > maxMem) { + throw new IllegalArgumentException(s"Required executor memory ($executorMem MB) " + + s"is above the max threshold ($maxMem MB) of this cluster!") } - val amMem = getAMMemory(newAppResponse) + memoryOverhead + val amMem = getAMMemory(newAppResponse) + amMemoryOverhead if (amMem > maxMem) { - val errorMessage = - "Required AM memory (%d MB) is above the max threshold (%d MB) of this cluster." - .format(amMem, maxMem) - logError(errorMessage) - throw new IllegalArgumentException(errorMessage) + throw new IllegalArgumentException(s"Required AM memory ($amMem MB) " + + s"is above the max threshold ($maxMem MB) of this cluster!") } - // 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 file to a remote file system if needed. Exposed for testing. + * Copy the given file to a remote file system if needed. This is used for preparing + * resources for launching the ApplicationMaster container. Exposed for testing. */ def copyFileToRemote( - dstDir: Path, - originalPath: Path, + destDir: Path, + srcPath: Path, replication: Short, setPerms: Boolean = false): Path = { - val fs = FileSystem.get(hadoopConf) - val remoteFs = originalPath.getFileSystem(hadoopConf) - var newPath = originalPath - if (!compareFs(remoteFs, fs)) { - newPath = new Path(dstDir, originalPath.getName()) - logInfo("Uploading " + originalPath + " to " + newPath) - FileUtil.copy(remoteFs, originalPath, fs, newPath, false, hadoopConf) - fs.setReplication(newPath, replication) - if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) + val destFs = destDir.getFileSystem(hadoopConf) + val srcFs = srcPath.getFileSystem(hadoopConf) + var destPath = srcPath + if (!compareFs(srcFs, destFs)) { + destPath = new Path(destDir, srcPath.getName()) + logInfo(s"Uploading resource $srcPath -> $destPath") + FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf) + destFs.setReplication(destPath, replication) + if (setPerms) { + 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 qualPath = fs.makeQualified(newPath) - val fc = FileContext.getFileContext(qualPath.toUri(), hadoopConf) - val destPath = fc.resolvePath(qualPath) - destPath + val qualifiedDestPath = destFs.makeQualified(destPath) + val fc = FileContext.getFileContext(qualifiedDestPath.toUri(), hadoopConf) + fc.resolvePath(qualifiedDestPath) } /** @@ -131,8 +129,9 @@ private[spark] trait ClientBase extends Logging { * Exposed for testing. */ def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { - // Upload Spark and the application JAR to the remote file system if necessary. Add them as - // local resources to the application master. + 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 = FileSystem.get(hadoopConf) val dst = new Path(fs.getHomeDirectory(), appStagingDir) val nns = getNameNodesToAccess(sparkConf) + dst @@ -215,7 +214,6 @@ private[spark] trait ClientBase extends Logging { } } } - logInfo("Prepared Local resources " + localResources) if (cachedSecondaryJarLinks.nonEmpty) { sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) } @@ -228,7 +226,7 @@ private[spark] trait ClientBase extends Logging { * Set up the environment for launching our ApplicationMaster container. */ private def setupLaunchEnv(stagingDir: String): HashMap[String, String] = { - logInfo("Setting up the launch environment") + logInfo("Setting up the launch environment for our AM container") val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.driver.extraClassPath") populateClasspath(args, yarnConf, sparkConf, env, extraCp) @@ -298,7 +296,7 @@ private[spark] trait ClientBase extends Logging { */ protected def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) : ContainerLaunchContext = { - logInfo("Setting up container launch context") + logInfo("Setting up container launch context for our AM") val appId = newAppResponse.getApplicationId val appStagingDir = getAppStagingDir(appId) @@ -392,11 +390,13 @@ private[spark] trait ClientBase extends Logging { logDebug("===============================================================================") logDebug("Yarn AM launch context:") - logDebug(s" user class: ${args.userClass}") - logDebug(" env:") - launchEnv.foreach { case (k, v) => logDebug(s" $k -> $v") } - logDebug(" command:") - logDebug(s" ${printableCommands.mkString(" ")}") + logDebug(s" user class: ${args.userClass}") + 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 @@ -522,7 +522,7 @@ private[spark] object ClientBase extends Logging { * This method first looks in the SparkConf object for the CONF_SPARK_JAR key, and in the * user environment if that is not found (for backwards compatibility). */ - private def sparkJar(conf: SparkConf) = { + private def sparkJar(conf: SparkConf): String = { if (conf.contains(CONF_SPARK_JAR)) { conf.get(CONF_SPARK_JAR) } else if (System.getenv(ENV_SPARK_JAR) != null) { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 3547144a9862c..b4a8e3340300e 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -65,8 +65,8 @@ private[spark] class Client( yarnClient.init(yarnConf) yarnClient.start() - logInfo("Received cluster metric info from ResourceManager, number of NodeManagers: " - + yarnClient.getYarnClusterMetrics.getNumNodeManagers) + 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() @@ -99,7 +99,7 @@ private[spark] class Client( appContext.setAMContainerSpec(containerContext) appContext.setApplicationType("SPARK") val capability = Records.newRecord(classOf[Resource]) - capability.setMemory(args.amMemory + memoryOverhead) + capability.setMemory(args.amMemory + amMemoryOverhead) appContext.setResource(capability) appContext } From 6573c1d4d175ed3c103a9979f87cf9853421b8e0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Sep 2014 15:30:54 -0700 Subject: [PATCH 08/18] Clean up, simplify and document code for setting classpaths --- .../apache/spark/deploy/yarn/ClientBase.scala | 67 ++++++++++--------- .../deploy/yarn/ExecutorRunnableUtil.scala | 11 ++- .../deploy/yarn/YarnSparkHadoopUtil.scala | 65 ++++++++---------- .../spark/deploy/yarn/ClientBaseSuite.scala | 2 +- 4 files changed, 71 insertions(+), 74 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index f5bd8cedc2324..44a63c9d44875 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -17,7 +17,6 @@ package org.apache.spark.deploy.yarn -import java.io.File import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import scala.collection.JavaConversions._ @@ -60,7 +59,7 @@ private[spark] trait ClientBase extends Logging { */ protected def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = { val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() - logInfo("Verifying our application request has not exceeded the maximum " + + logInfo("Verifying our application has not requested more than the maximum " + s"memory capability of the cluster ($maxMem MB per container)") val executorMem = args.executorMemory + executorMemoryOverhead if (executorMem > maxMem) { @@ -243,14 +242,12 @@ private[spark] trait ClientBase extends Logging { sparkConf.getAll .filter { case (k, v) => k.startsWith(amEnvPrefix) } .map { case (k, v) => (k.substring(amEnvPrefix.length), v) } - .foreach { case (k, v) => - YarnSparkHadoopUtil.addToEnvironment(env, k, v, File.pathSeparator) - } + .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, File.pathSeparator) + 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 } @@ -542,16 +539,15 @@ private[spark] object ClientBase extends Logging { SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR } - def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) = { + /** + * Populate the classpath entry in the given environment map with any application + * classpath specified through the Hadoop and Yarn configurations. + */ + def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]): Unit = { val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf) for (c <- classPathElementsToAdd.flatten) { - YarnSparkHadoopUtil.addToEnvironment( - env, - Environment.CLASSPATH.name, - c.trim, - File.pathSeparator) + YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, c.trim) } - classPathElementsToAdd } private def getYarnAppClasspath(conf: Configuration): Option[Seq[String]] = @@ -613,6 +609,10 @@ private[spark] object ClientBase extends Logging { triedDefault.toOption } + /** + * Populate the classpath entry in the given environment map. + * This includes the user jar, Spark jar, and any extra application jars. + */ def populateClasspath( args: ClientArguments, conf: Configuration, @@ -634,7 +634,7 @@ private[spark] object ClientBase extends Logging { } // Append all jar files under the working directory to the classpath. - addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env); + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env) } /** @@ -645,22 +645,21 @@ private[spark] object ClientBase extends Logging { args: ClientArguments, conf: SparkConf, env: HashMap[String, String]): Unit = { - if (args != null) { - addFileToClasspath(args.userJar, APP_JAR, env) - if (args.addJars != null) { - args.addJars.split(",").foreach { case file: String => - addFileToClasspath(file, null, env) - } + + // If `args` is not null, we are launching an AM container. + // Otherwise, we are launching executor containers. + val (mainJar, secondaryJars) = + if (args != null) { + (args.userJar, args.addJars) + } else { + (conf.get(CONF_SPARK_USER_JAR, null), conf.get(CONF_SPARK_YARN_SECONDARY_JARS, null)) + } + + addFileToClasspath(mainJar, APP_JAR, env) + if (secondaryJars != null) { + secondaryJars.split(",").filter(_.nonEmpty).foreach { jar => + addFileToClasspath(jar, null, env) } - } else { - val userJar = conf.get(CONF_SPARK_USER_JAR, null) - addFileToClasspath(userJar, APP_JAR, env) - - // Add any secondary jars to the classpath - conf.get(CONF_SPARK_YARN_SECONDARY_JARS, "") - .split(",") - .filter(_.nonEmpty) - .foreach(jar => addFileToClasspath(jar, null, env)) } } @@ -694,8 +693,12 @@ private[spark] object ClientBase extends Logging { } } + /** + * 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.addToEnvironment(env, Environment.CLASSPATH.name, path, File.pathSeparator) + YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path) /** * Get the list of namenodes the user may access. @@ -737,7 +740,9 @@ private[spark] object ClientBase extends Logging { } } - /** Return whether the two file systems are the same. */ + /** + * 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() diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 312d82a649792..0aa04b72d64b6 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -171,15 +171,14 @@ trait ExecutorRunnableUtil extends Logging { val extraCp = sparkConf.getOption("spark.executor.extraClassPath") ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp) - sparkConf.getExecutorEnv.foreach { case (key, value) => - YarnSparkHadoopUtil.addToEnvironment(env, key, value, File.pathSeparator) - } + sparkConf.getExecutorEnv.foreach { case (key, value) => env(key) = value } // Keep this for backwards compatibility but users should move to the config - YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), - File.pathSeparator) + sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) + } - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k, v) => env(k) = v } env } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 4a33e34c3bfc7..517b2597e3212 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import java.lang.{Boolean => JBoolean} +import java.io.File import java.util.{Collections, Set => JSet} import java.util.regex.Matcher import java.util.regex.Pattern @@ -29,21 +30,19 @@ 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.util.StringInterner import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records.ApplicationAccessType import org.apache.hadoop.yarn.util.RackResolver import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils /** * Contains util methods to interact with Hadoop from spark. */ -class YarnSparkHadoopUtil extends SparkHadoopUtil { +private[spark] class YarnSparkHadoopUtil extends SparkHadoopUtil { override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) @@ -84,7 +83,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } -object YarnSparkHadoopUtil { +private[spark] object YarnSparkHadoopUtil { // Additional memory overhead - in mb. val DEFAULT_MEMORY_OVERHEAD = 384 @@ -100,30 +99,26 @@ object YarnSparkHadoopUtil { private val hostToRack = new ConcurrentHashMap[String, String]() private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() - def addToEnvironment( - env: HashMap[String, String], - variable: String, - value: String, - classPathSeparator: String) = { - var envVariable = "" - if (env.get(variable) == None) { - envVariable = value - } else { - envVariable = env.get(variable).get + classPathSeparator + value - } - env put (StringInterner.weakIntern(variable), StringInterner.weakIntern(envVariable)) + /** + * 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) + File.pathSeparator + value } else value + env.put(key, newValue) } - def setEnvFromInputString( - env: HashMap[String, String], - envString: String, - classPathSeparator: String) = { - if (envString != null && envString.length() > 0) { - var childEnvs = envString.split(",") - var p = Pattern.compile(getEnvironmentVariableRegex()) + /** + * 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) { - var parts = cEnv.split("=") // split on '=' - var m = p.matcher(parts(1)) + val parts = cEnv.split("=") // split on '=' + val m = p.matcher(parts(1)) val sb = new StringBuffer while (m.find()) { val variable = m.group(1) @@ -131,8 +126,7 @@ object YarnSparkHadoopUtil { 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 + // 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 @@ -142,14 +136,13 @@ object YarnSparkHadoopUtil { m.appendReplacement(sb, Matcher.quoteReplacement(replace)) } m.appendTail(sb) - addToEnvironment(env, parts(0), sb.toString(), classPathSeparator) + env(parts(0)) = sb.toString } } } - private def getEnvironmentVariableRegex() : String = { - val osName = System.getProperty("os.name") - if (osName startsWith "Windows") { + private val environmentVariableRegex: String = { + if (Utils.isWindows) { "%([A-Za-z_][A-Za-z0-9_]*?)%" } else { "\\$([A-Za-z_][A-Za-z0-9_]*)" @@ -181,14 +174,14 @@ object YarnSparkHadoopUtil { } } - private[spark] def lookupRack(conf: Configuration, host: String): String = { + def lookupRack(conf: Configuration, host: String): String = { if (!hostToRack.contains(host)) { populateRackInfo(conf, host) } hostToRack.get(host) } - private[spark] def populateRackInfo(conf: Configuration, hostname: String) { + def populateRackInfo(conf: Configuration, hostname: String) { Utils.checkHost(hostname) if (!hostToRack.containsKey(hostname)) { @@ -212,8 +205,8 @@ object YarnSparkHadoopUtil { } } - private[spark] def getApplicationAclsForYarn(securityMgr: SecurityManager): - Map[ApplicationAccessType, String] = { + def getApplicationAclsForYarn(securityMgr: SecurityManager) + : Map[ApplicationAccessType, String] = { Map[ApplicationAccessType, String] ( ApplicationAccessType.VIEW_APP -> securityMgr.getViewAcls, ApplicationAccessType.MODIFY_APP -> securityMgr.getModifyAcls diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 5e308afa8182e..19ff5a0e8e893 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -89,7 +89,7 @@ class ClientBaseSuite extends FunSuite with Matchers { val env = new MutableHashMap[String, String]() val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - ClientBase.populateClasspath(args, conf, sparkConf, env, None) + ClientBase.populateClasspath(args, conf, sparkConf, env) val cp = env("CLASSPATH").split(File.pathSeparator) s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => From 6d74888478140a794be719d1dbe60aed9883ef9b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Sep 2014 15:36:22 -0700 Subject: [PATCH 09/18] Minor comment changes --- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 8 ++++---- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index a33c807233353..d0b6a3809a885 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -62,18 +62,18 @@ private[spark] class Client( logInfo("Requesting a new application from cluster with %d NodeManagers" .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers)) - // Get a new application from our RM. + // Get a new application from our RM val newAppResponse = getNewApplication() val appId = newAppResponse.getApplicationId() - // Verify whether the cluster has enough resources for our AM. + // Verify whether the cluster has enough resources for our AM verifyClusterResources(newAppResponse) - // Set up the appropriate contexts to launch our AM. + // Set up the appropriate contexts to launch our AM val containerContext = createContainerLaunchContext(newAppResponse) val appContext = createApplicationSubmissionContext(appId, containerContext) - // Finally, submit and monitor the application. + // Finally, submit and monitor the application logInfo(s"Submitting application ${appId.getId} to ResourceManager") submitApplication(appContext) appId diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index b4a8e3340300e..25ace7ca33ff5 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -68,19 +68,19 @@ private[spark] class Client( logInfo("Requesting a new application from cluster with %d NodeManagers" .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers)) - // Get a new application from our RM. + // Get a new application from our RM val newApp = yarnClient.createApplication() val newAppResponse = newApp.getNewApplicationResponse() val appId = newAppResponse.getApplicationId() - // Verify whether the cluster has enough resources for our AM. + // Verify whether the cluster has enough resources for our AM verifyClusterResources(newAppResponse) - // Set up the appropriate contexts to launch our AM. + // Set up the appropriate contexts to launch our AM val containerContext = createContainerLaunchContext(newAppResponse) val appContext = createApplicationSubmissionContext(newApp, containerContext) - // Finally, submit and monitor the application. + // Finally, submit and monitor the application logInfo(s"Submitting application ${appId.getId} to ResourceManager") yarnClient.submitApplication(appContext) appId From ed0b42d8f375fd2219f9626c280063a103aa15a0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Sep 2014 16:04:18 -0700 Subject: [PATCH 10/18] Fix alpha compilation error --- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index d0b6a3809a885..56667a8f29263 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -60,7 +60,7 @@ private[spark] class Client( start() logInfo("Requesting a new application from cluster with %d NodeManagers" - .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers)) + .format(getYarnClusterMetrics.getNumNodeManagers)) // Get a new application from our RM val newAppResponse = getNewApplication() From 45ccdea1acfb3b7aaff0522877d24699843cbc57 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Sep 2014 14:02:32 -0700 Subject: [PATCH 11/18] Remove usages of getAMMemory This merges the refator with commit ed1980ffa9ccb87d76694ba910ef22df034bca49. --- .../scala/org/apache/spark/deploy/yarn/Client.scala | 13 +------------ .../org/apache/spark/deploy/yarn/ClientBase.scala | 7 ++----- .../apache/spark/deploy/yarn/ClientBaseSuite.scala | 1 - 3 files changed, 3 insertions(+), 18 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 56667a8f29263..0e8372992cfb7 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -88,7 +88,7 @@ private[spark] class Client( : ContainerLaunchContext = { val containerContext = super.createContainerLaunchContext(newAppResponse) val capability = Records.newRecord(classOf[Resource]) - capability.setMemory(getAMMemory(newAppResponse) + amMemoryOverhead) + capability.setMemory(args.amMemory + amMemoryOverhead) containerContext.setResource(capability) containerContext } @@ -116,17 +116,6 @@ private[spark] class Client( amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) } - /** - * Return the amount of memory for launching the ApplicationMaster container (MB). - * GetNewApplicationResponse#getMinimumResourceCapability does not exist in the stable API. - */ - override def getAMMemory(newAppResponse: GetNewApplicationResponse): Int = { - val minResMemory = newAppResponse.getMinimumResourceCapability().getMemory() - val amMemory = ((args.amMemory / minResMemory) * minResMemory) + - ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - amMemoryOverhead) - amMemory - } - /** * 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. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 44a63c9d44875..5dd37accaf2f9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -66,7 +66,7 @@ private[spark] trait ClientBase extends Logging { throw new IllegalArgumentException(s"Required executor memory ($executorMem MB) " + s"is above the max threshold ($maxMem MB) of this cluster!") } - val amMem = getAMMemory(newAppResponse) + amMemoryOverhead + val amMem = args.amMemory + amMemoryOverhead if (amMem > maxMem) { throw new IllegalArgumentException(s"Required AM memory ($amMem MB) " + s"is above the max threshold ($maxMem MB) of this cluster!") @@ -306,7 +306,7 @@ private[spark] trait ClientBase extends Logging { val javaOpts = ListBuffer[String]() // Add Xmx for AM memory - javaOpts += "-Xmx" + getAMMemory(newAppResponse) + "m" + javaOpts += "-Xmx" + args.amMemory + "m" val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) javaOpts += "-Djava.io.tmpdir=" + tmpDir @@ -476,9 +476,6 @@ private[spark] trait ClientBase extends Logging { * If no security is enabled, the token returned by the report is null. */ protected def getClientToken(report: ApplicationReport): String - - /** Return the amount of memory for launching the ApplicationMaster container (MB). */ - protected def getAMMemory(newAppResponse: GetNewApplicationResponse): Int = args.amMemory } private[spark] object ClientBase extends Logging { diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 19ff5a0e8e893..96bbfda8fa6cd 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -237,7 +237,6 @@ class ClientBaseSuite extends FunSuite with Matchers { val hadoopConf: Configuration, val sparkConf: SparkConf, val yarnConf: YarnConfiguration) extends ClientBase { - override def getAMMemory(newApp: GetNewApplicationResponse): Int = ??? override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = ??? override def submitApplication(): ApplicationId = ??? override def getApplicationReport(appId: ApplicationId): ApplicationReport = ??? From 1590141ee753a7fb7b0358a8ef73fef553a1e16d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Sep 2014 18:20:09 -0700 Subject: [PATCH 12/18] Address review comments --- .../org/apache/spark/deploy/yarn/Client.scala | 4 ++- .../spark/deploy/yarn/ClientArguments.scala | 15 ++++----- .../apache/spark/deploy/yarn/ClientBase.scala | 32 +++++++++++++------ .../deploy/yarn/YarnSparkHadoopUtil.scala | 8 +++-- .../cluster/YarnClientSchedulerBackend.scala | 12 ++++--- .../org/apache/spark/deploy/yarn/Client.scala | 4 ++- 6 files changed, 48 insertions(+), 27 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 0e8372992cfb7..5a20532315e59 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -125,7 +125,7 @@ private[spark] class Client( Option(report.getClientToken).getOrElse("") } -private[spark] object Client { +object Client { def main(argStrings: Array[String]) { if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a " + @@ -145,5 +145,7 @@ private[spark] object Client { Console.err.println(e.getMessage) System.exit(1) } + + System.exit(0) } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index b66f177218219..201b742736c6e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -47,22 +47,20 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) "spark.yarn.executor.memoryOverhead", YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) parseArgs(args.toList) - loadDefaultArgs() + loadEnvironmentArgs() validateArgs() /** Load any default arguments provided through environment variables and Spark properties. */ - private def loadDefaultArgs(): Unit = { + private def loadEnvironmentArgs(): Unit = { // For backward compatibility, SPARK_YARN_DIST_{ARCHIVES/FILES} should be resolved to hdfs://, // while spark.yarn.dist.{archives/files} should be resolved to file:// (SPARK-2051). - files = Option(files).orElse(sys.env.get("SPARK_YARN_DIST_FILES")).orNull files = Option(files) - .orElse(sparkConf.getOption("spark.yarn.dist.files")) - .map(p => Utils.resolveURIs(p)) + .orElse(sys.env.get("SPARK_YARN_DIST_FILES")) + .orElse(sparkConf.getOption("spark.yarn.dist.files").map(p => Utils.resolveURIs(p))) .orNull - archives = Option(archives).orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")).orNull archives = Option(archives) - .orElse(sparkConf.getOption("spark.yarn.dist.archives")) - .map(p => Utils.resolveURIs(p)) + .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) + .orElse(sparkConf.getOption("spark.yarn.dist.archives").map(p => Utils.resolveURIs(p))) .orNull } @@ -71,6 +69,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) * This is intended to be called only after the provided arguments have been parsed. */ private def validateArgs(): Unit = { + // TODO: memory checks are outdated (SPARK-3476) Map[Boolean, String]( (numExecutors <= 0) -> "You must specify at least 1 executor!", (amMemory <= amMemoryOverhead) -> s"AM memory must be > $amMemoryOverhead MB", diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 5dd37accaf2f9..e874ed7e3dada 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -36,7 +36,9 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records + import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} +import org.apache.spark.util.Utils /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. @@ -217,7 +219,6 @@ private[spark] trait ClientBase extends Logging { sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) } - UserGroupInformation.getCurrentUser().addCredentials(credentials) localResources } @@ -360,9 +361,9 @@ private[spark] trait ClientBase extends Logging { } val amClass = if (isLaunchingDriver) { - classOf[ApplicationMaster].getName() + Utils.getFormattedClassName(ApplicationMaster) } else { - classOf[ApplicationMaster].getName().replace("ApplicationMaster", "ExecutorLauncher") + Utils.getFormattedClassName(ExecutorLauncher) } val userArgs = args.userArgs.flatMap { arg => Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) @@ -400,6 +401,8 @@ private[spark] trait ClientBase extends Logging { val securityManager = new SecurityManager(sparkConf) amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager)) setupSecurityToken(amContainer) + UserGroupInformation.getCurrentUser().addCredentials(credentials) + amContainer } @@ -409,22 +412,24 @@ private[spark] trait ClientBase extends Logging { * * @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. + * @param shouldKeepMonitoring The condition to keep monitoring. * @return state of the application, one of FINISHED, FAILED, KILLED, and RUNNING. */ def monitorApplication( appId: ApplicationId, returnOnRunning: Boolean = false, - logApplicationReport: Boolean = true): YarnApplicationState = { + logApplicationReport: Boolean = true, + shouldKeepMonitoring: () => Boolean = () => true): YarnApplicationState = { val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - while (true) { + var firstIteration = true + while (shouldKeepMonitoring()) { Thread.sleep(interval) val report = getApplicationReport(appId) val state = report.getYarnApplicationState if (logApplicationReport) { - logInfo(s"Application report from ResourceManager for application ${appId.getId} " + - s"(state: $state)") - logDebug("\n" + + logInfo(s"Application report from ResourceManager for app ${appId.getId} (state: $state)") + val details = "\n" + s"\t full application identifier: $appId\n" + s"\t clientToken: ${getClientToken(report)}\n" + s"\t appDiagnostics: ${report.getDiagnostics}\n" + @@ -435,7 +440,14 @@ private[spark] trait ClientBase extends Logging { s"\t yarnAppState: $state\n" + s"\t distributedFinalState: ${report.getFinalApplicationStatus}\n" + s"\t appTrackingUrl: ${report.getTrackingUrl}\n" + - s"\t appUser: ${report.getUser}") + s"\t appUser: ${report.getUser}" + + // Log report details every iteration if DEBUG is enabled, otherwise only the first + if (log.isDebugEnabled) { + logDebug(details) + } else if (firstIteration) { + logInfo(details) + } } if (state == YarnApplicationState.FINISHED || @@ -447,6 +459,8 @@ private[spark] trait ClientBase extends Logging { if (returnOnRunning && state == YarnApplicationState.RUNNING) { return state } + + firstIteration = false } // Never reached, but keeps compiler happy throw new SparkException("While loop is depleted! This should never happen...") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 517b2597e3212..0b712c201904a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.Utils /** * Contains util methods to interact with Hadoop from spark. */ -private[spark] class YarnSparkHadoopUtil extends SparkHadoopUtil { +class YarnSparkHadoopUtil extends SparkHadoopUtil { override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) @@ -83,7 +83,7 @@ private[spark] class YarnSparkHadoopUtil extends SparkHadoopUtil { } -private[spark] object YarnSparkHadoopUtil { +object YarnSparkHadoopUtil { // Additional memory overhead - in mb. val DEFAULT_MEMORY_OVERHEAD = 384 @@ -136,7 +136,9 @@ private[spark] object YarnSparkHadoopUtil { m.appendReplacement(sb, Matcher.quoteReplacement(replace)) } m.appendTail(sb) - env(parts(0)) = sb.toString + // 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) } } } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 35b2f4212bcd7..77f40ba5fc785 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -34,10 +34,11 @@ private[spark] class YarnClientSchedulerBackend( minRegisteredRatio = 0.8 } - var client: Client = null - var appId: ApplicationId = null - var stopping: Boolean = false - var totalExpectedExecutors = 0 + private var client: Client = null + private var appId: ApplicationId = null + private var stopping: Boolean = false + private var totalExpectedExecutors = 0 + private def isStopping(): Boolean = stopping /** * Create a Yarn client to submit an application to the ResourceManager. @@ -120,7 +121,8 @@ private[spark] class YarnClientSchedulerBackend( assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { - val state = client.monitorApplication(appId, logApplicationReport = false) // blocking + val state = client.monitorApplication( + appId, logApplicationReport = false, shouldKeepMonitoring = isStopping) // blocking if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.KILLED || state == YarnApplicationState.FAILED) { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 25ace7ca33ff5..0b43e6ee20538 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -123,7 +123,7 @@ private[spark] class Client( Option(report.getClientToAMToken).map(_.toString).getOrElse("") } -private[spark] object Client { +object Client { def main(argStrings: Array[String]) { if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a " + @@ -143,5 +143,7 @@ private[spark] object Client { Console.err.println(e.getMessage) System.exit(1) } + + System.exit(0) } } From a0ad1e9f778a85e7dee1eca4337574deddcece78 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Sep 2014 20:29:34 -0700 Subject: [PATCH 13/18] Fix class not found error --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index e874ed7e3dada..8dbf6dc6a4819 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -361,9 +361,9 @@ private[spark] trait ClientBase extends Logging { } val amClass = if (isLaunchingDriver) { - Utils.getFormattedClassName(ApplicationMaster) + Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName } else { - Utils.getFormattedClassName(ExecutorLauncher) + Class.forName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName } val userArgs = args.userArgs.flatMap { arg => Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) From 547487c135b59146bee5bb9175c050f6062e5a96 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Sep 2014 20:34:16 -0700 Subject: [PATCH 14/18] Provide default values for null application report entries --- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8dbf6dc6a4819..64a90badaa1d9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -429,10 +429,12 @@ private[spark] trait ClientBase extends Logging { if (logApplicationReport) { logInfo(s"Application report from ResourceManager for app ${appId.getId} (state: $state)") + val clientToken = Option(getClientToken(report)).getOrElse("N/A") + val appDiagnostics = Option(report.getDiagnostics).getOrElse("N/A") val details = "\n" + s"\t full application identifier: $appId\n" + - s"\t clientToken: ${getClientToken(report)}\n" + - s"\t appDiagnostics: ${report.getDiagnostics}\n" + + s"\t clientToken: $clientToken\n" + + s"\t appDiagnostics: $appDiagnostics\n" + s"\t appMasterHost: ${report.getHost}\n" + s"\t appQueue: ${report.getQueue}\n" + s"\t appMasterRpcPort: ${report.getRpcPort}\n" + From 7dd6298f279cd67ee3ea6d91d5dfbce956a6db8c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Sep 2014 21:10:09 -0700 Subject: [PATCH 15/18] Simplify ClientBase#monitorApplication There's only so much abstraction that you could do before it gets too complicated. --- .../apache/spark/deploy/yarn/ClientBase.scala | 6 ++--- .../cluster/YarnClientSchedulerBackend.scala | 25 +++++++++++-------- 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 64a90badaa1d9..ba6cdf923c7f0 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -412,17 +412,15 @@ private[spark] trait ClientBase extends Logging { * * @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. - * @param shouldKeepMonitoring The condition to keep monitoring. * @return state of the application, one of FINISHED, FAILED, KILLED, and RUNNING. */ def monitorApplication( appId: ApplicationId, returnOnRunning: Boolean = false, - logApplicationReport: Boolean = true, - shouldKeepMonitoring: () => Boolean = () => true): YarnApplicationState = { + logApplicationReport: Boolean = true): YarnApplicationState = { val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) var firstIteration = true - while (shouldKeepMonitoring()) { + while (true) { Thread.sleep(interval) val report = getApplicationReport(appId) val state = report.getYarnApplicationState diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 77f40ba5fc785..5f30afc97aa0a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -117,25 +117,28 @@ private[spark] class YarnClientSchedulerBackend( * If the application has exited for any reason, stop the SparkContext. * This assumes both `client` and `appId` have already been set. */ - private def asyncMonitorApplication(): Thread = { + private def asyncMonitorApplication(): Unit = { assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { - val state = client.monitorApplication( - appId, logApplicationReport = false, shouldKeepMonitoring = isStopping) // blocking - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.KILLED || - state == YarnApplicationState.FAILED) { - logWarning(s"Yarn application has exited: $state") - sc.stop() - stopping = true + while (!stopping) { + val report = client.getApplicationReport(appId) + val state = report.getYarnApplicationState() + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.KILLED || + state == YarnApplicationState.FAILED) { + logError(s"Yarn application has already exited with state $state!") + sc.stop() + stopping = true + } + Thread.sleep(1000L) } + Thread.currentThread().interrupt() } } - t.setName("Yarn Application State Monitor") + t.setName("Yarn application state monitor") t.setDaemon(true) t.start() - t } /** From a3b9693884b99622b626890fba08b2111d661e93 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Sep 2014 22:30:33 -0700 Subject: [PATCH 16/18] Minor changes --- .../org/apache/spark/deploy/yarn/ClientBase.scala | 2 +- .../spark/deploy/yarn/ExecutorRunnableUtil.scala | 11 +++++++---- .../cluster/YarnClientSchedulerBackend.scala | 1 - 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index ba6cdf923c7f0..3ae4b8882eedb 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -388,7 +388,7 @@ private[spark] trait ClientBase extends Logging { logDebug("===============================================================================") logDebug("Yarn AM launch context:") - logDebug(s" user class: ${args.userClass}") + logDebug(s" user class: ${Option(args.userClass).getOrElse("N/A")}") logDebug(" env:") launchEnv.foreach { case (k, v) => logDebug(s" $k -> $v") } logDebug(" resources:") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 0aa04b72d64b6..566756d319098 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -17,7 +17,6 @@ package org.apache.spark.deploy.yarn -import java.io.File import java.net.URI import scala.collection.JavaConversions._ @@ -125,9 +124,9 @@ trait ExecutorRunnableUtil extends Logging { localResources: HashMap[String, LocalResource], timestamp: String, size: String, - vis: String) = { + vis: String): Unit = { val uri = new URI(file) - val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + val amJarRsrc = Records.newRecord(classOf[LocalResource]) amJarRsrc.setType(rtype) amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) @@ -171,7 +170,11 @@ trait ExecutorRunnableUtil extends Logging { val extraCp = sparkConf.getOption("spark.executor.extraClassPath") ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp) - sparkConf.getExecutorEnv.foreach { case (key, value) => env(key) = value } + 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 => diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 5f30afc97aa0a..64a81a743d356 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -38,7 +38,6 @@ private[spark] class YarnClientSchedulerBackend( private var appId: ApplicationId = null private var stopping: Boolean = false private var totalExpectedExecutors = 0 - private def isStopping(): Boolean = stopping /** * Create a Yarn client to submit an application to the ResourceManager. From 2ca6d6414b3c5ada460d195098b1a2d29a265e17 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 17 Sep 2014 13:00:59 -0700 Subject: [PATCH 17/18] Improve logging in application monitor (1) Now it provides a more loggable format if the value is null or "" (2) The application report details can change over time. Instead of logging them once at the beginning, we log them every time the application state changes. --- .../apache/spark/deploy/yarn/ClientBase.scala | 50 +++++++++++-------- 1 file changed, 28 insertions(+), 22 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 3ae4b8882eedb..eaefdb6f5a53b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} -import org.apache.spark.util.Utils /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. @@ -419,7 +418,7 @@ private[spark] trait ClientBase extends Logging { returnOnRunning: Boolean = false, logApplicationReport: Boolean = true): YarnApplicationState = { val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - var firstIteration = true + var lastState: YarnApplicationState = null while (true) { Thread.sleep(interval) val report = getApplicationReport(appId) @@ -427,26 +426,32 @@ private[spark] trait ClientBase extends Logging { if (logApplicationReport) { logInfo(s"Application report from ResourceManager for app ${appId.getId} (state: $state)") - val clientToken = Option(getClientToken(report)).getOrElse("N/A") - val appDiagnostics = Option(report.getDiagnostics).getOrElse("N/A") - val details = "\n" + - s"\t full application identifier: $appId\n" + - s"\t clientToken: $clientToken\n" + - s"\t appDiagnostics: $appDiagnostics\n" + - s"\t appMasterHost: ${report.getHost}\n" + - s"\t appQueue: ${report.getQueue}\n" + - s"\t appMasterRpcPort: ${report.getRpcPort}\n" + - s"\t appStartTime: ${report.getStartTime}\n" + - s"\t yarnAppState: $state\n" + - s"\t distributedFinalState: ${report.getFinalApplicationStatus}\n" + - s"\t appTrackingUrl: ${report.getTrackingUrl}\n" + - s"\t appUser: ${report.getUser}" - - // Log report details every iteration if DEBUG is enabled, otherwise only the first + val details = Seq[(String, String)]( + ("full identifier", appId.toString), + ("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 + val formattedDetails = details + .map { case (k, v) => + val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") + s"\n\t $k: $newValue" } + .mkString("") + + // If DEBUG is enabled, log report details every iteration + // Otherwise, log them every time the application changes state if (log.isDebugEnabled) { - logDebug(details) - } else if (firstIteration) { - logInfo(details) + logDebug(formattedDetails) + } else if (lastState != state) { + logInfo(formattedDetails) } } @@ -460,8 +465,9 @@ private[spark] trait ClientBase extends Logging { return state } - firstIteration = false + lastState = state } + // Never reached, but keeps compiler happy throw new SparkException("While loop is depleted! This should never happen...") } From 39e8c7b2e7ba56467653d2c68cef835bb0b5c9c4 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 22 Sep 2014 13:56:39 -0700 Subject: [PATCH 18/18] Address review comments --- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 9 +++++---- .../scheduler/cluster/YarnClientSchedulerBackend.scala | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index eaefdb6f5a53b..0cd8a031e6c66 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -77,8 +77,9 @@ private[spark] trait ClientBase extends Logging { } /** - * Copy the given file to a remote file system if needed. This is used for preparing - * resources for launching the ApplicationMaster container. Exposed for testing. + * 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. */ def copyFileToRemote( destDir: Path, @@ -409,6 +410,7 @@ private[spark] trait ClientBase extends Logging { * Report the state of an application until it has exited, either successfully or * due to some failure, then return the application state. * + * @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 state of the application, one of FINISHED, FAILED, KILLED, and RUNNING. @@ -425,9 +427,8 @@ private[spark] trait ClientBase extends Logging { val state = report.getYarnApplicationState if (logApplicationReport) { - logInfo(s"Application report from ResourceManager for app ${appId.getId} (state: $state)") + logInfo(s"Application report for $appId (state: $state)") val details = Seq[(String, String)]( - ("full identifier", appId.toString), ("client token", getClientToken(report)), ("diagnostics", report.getDiagnostics), ("ApplicationMaster host", report.getHost), diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 64a81a743d356..200a30899290b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -107,7 +107,7 @@ private[spark] class YarnClientSchedulerBackend( "It might have been killed or unable to launch application master.") } if (state == YarnApplicationState.RUNNING) { - logInfo(s"Application ${appId.getId} has started running.") + logInfo(s"Application $appId has started running.") } }