Github user tgravescs commented on a diff in the pull request:
https://github.com/apache/spark/pull/2020#discussion_r16423913
--- Diff:
yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
---
@@ -0,0 +1,449 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.yarn
+
+import java.io.IOException
+import java.net.Socket
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference}
+
+import scala.collection.JavaConversions._
+import scala.util.Try
+
+import akka.actor._
+import akka.remote._
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.util.ShutdownHookManager
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+
+import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import
org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter
+import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils}
+
+/**
+ * Common application master functionality for Spark on Yarn.
+ */
+private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
+ client: YarnRMClient) extends Logging {
+ // TODO: Currently, task to container is computed once (TaskSetManager)
- which need not be
+ // optimal as more containers are available. Might need to handle this
better.
+ private val ALLOCATE_HEARTBEAT_INTERVAL = 100
+
+ private val sparkConf = new SparkConf()
+ private val yarnConf: YarnConfiguration = new YarnConfiguration(new
Configuration())
+ private val isDriver = args.userClass != null
+
+ // Default to numExecutors * 2, with minimum of 3
+ private val maxNumExecutorFailures =
sparkConf.getInt("spark.yarn.max.executor.failures",
+ sparkConf.getInt("spark.yarn.max.worker.failures",
math.max(args.numExecutors * 2, 3)))
+
+ private var finished = false
+ private var registered = false
+ private var reporterThread: Thread = _
+ private var allocator: YarnAllocator = _
+
+ // Fields used in client mode.
+ private var actorSystem: ActorSystem = null
+ private var actor: ActorRef = _
+
+ // Fields used in cluster mode.
+ private val sparkContextRef = new AtomicReference[SparkContext](null)
+ private val userResult = new AtomicBoolean(false)
+
+ final def run(): Unit = {
+ // Setup the directories so things go to YARN approved directories
rather
+ // than user specified and /tmp.
+ System.setProperty("spark.local.dir", getLocalDirs())
+
+ if (isDriver) {
+ // Set the web ui port to be ephemeral for yarn so we don't conflict
with
+ // other spark processes running on the same box
+ System.setProperty("spark.ui.port", "0")
+
+ // Set the master property to match the requested mode.
+ System.setProperty("spark.master", "yarn-cluster")
+ }
+
+ logInfo("ApplicationAttemptId: " + client.getAttemptId())
+
+ // If this is the last attempt, register a shutdown hook to cleanup
the staging dir
+ // after the app is finished, in case it does not exit through the
expected means.
+ // Use priority 30 as it's higher than HDFS. It's the same priority
MapReduce is using.
+ if (isLastAttempt()) {
+ val cleanupHook = new Runnable {
+ override def run() {
+ logInfo("AppMaster received a signal.")
+ if (!finished) {
+ cleanupStagingDir()
+ }
+ }
+ }
+ ShutdownHookManager.get().addShutdownHook(cleanupHook, 30)
+ }
+
+ // Call this to force generation of secret so it gets populated into
the
+ // Hadoop UGI. This has to happen before the startUserClass which does
a
+ // doAs in order for the credentials to be passed on to the executor
containers.
+ val securityMgr = new SecurityManager(sparkConf)
+
+ val success =
+ try {
+ if (isDriver) runDriver() else runExecutorLauncher(securityMgr)
+ } catch {
+ case e: Exception =>
+ logError("Exception while running AM main loop.", e)
+ false
+ }
+
+ finish(if (success) FinalApplicationStatus.SUCCEEDED else
FinalApplicationStatus.FAILED)
+ val shouldCleanup = success || isLastAttempt()
+ if (shouldCleanup) {
+ cleanupStagingDir()
+ }
+ }
+
+ final def finish(status: FinalApplicationStatus, diagnostics: String =
"") = synchronized {
+ if (!finished) {
+ logInfo(s"Finishing ApplicationMaster with $status")
--- End diff --
I know you didn't change this , but can you add the diagnostics string to
the log statement too?
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]