+ spark.yarn.hardKillTimeout |
+ 60s |
+
+ Number of milliseconds to wait before the job client kills the application.
+ After the wait, client will attempt to terminate the YARN application.
+ |
+
# Important notes
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
index 5f2da036ff9f7..e7d9041a51126 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
@@ -442,6 +442,7 @@ protected boolean handle(String opt, String value) {
}
break;
case KILL_SUBMISSION:
+ case UPLOAD_CRED_SUBMISSION:
case STATUS:
isAppResourceReq = false;
sparkArgs.add(opt);
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java
index 6767cc5079649..f0db4a1769564 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java
@@ -48,6 +48,7 @@ class SparkSubmitOptionParser {
protected final String FILES = "--files";
protected final String JARS = "--jars";
protected final String KILL_SUBMISSION = "--kill";
+ protected final String UPLOAD_CRED_SUBMISSION = "--uploadCred";
protected final String MASTER = "--master";
protected final String NAME = "--name";
protected final String PACKAGES = "--packages";
@@ -102,6 +103,7 @@ class SparkSubmitOptionParser {
{ JARS },
{ KEYTAB },
{ KILL_SUBMISSION },
+ { UPLOAD_CRED_SUBMISSION },
{ MASTER },
{ NAME },
{ NUM_EXECUTORS },
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index e227bff88f71d..5a0728d88c2db 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -17,22 +17,29 @@
package org.apache.spark.deploy.yarn
-import java.io.{File, IOException}
+import java.io.{ByteArrayInputStream, DataInputStream, File, IOException}
import java.lang.reflect.InvocationTargetException
import java.net.{Socket, URI, URL}
import java.util.concurrent.{TimeoutException, TimeUnit}
+import javax.crypto.SecretKey
+import javax.crypto.spec.SecretKeySpec
import scala.collection.mutable.HashMap
import scala.concurrent.Promise
import scala.concurrent.duration.Duration
import scala.util.control.NonFatal
+import com.google.common.base.Charsets
+import io.netty.buffer.ByteBuf
+import io.netty.buffer.Unpooled
+import io.netty.handler.codec.base64.Base64
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.yarn.api._
import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException
import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
@@ -41,9 +48,16 @@ import org.apache.spark.deploy.yarn.config._
import org.apache.spark.deploy.yarn.security.{AMCredentialRenewer, YARNHadoopDelegationTokenManager}
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
+import org.apache.spark.network.{BlockDataManager, TransportContext}
+import org.apache.spark.network.client.TransportClientBootstrap
+import org.apache.spark.network.netty.{NettyBlockRpcServer, SparkTransportConf}
+import org.apache.spark.network.sasl.{SaslClientBootstrap, SaslServerBootstrap}
+import org.apache.spark.network.server.{TransportServer, TransportServerBootstrap}
import org.apache.spark.rpc._
+import org.apache.spark.rpc.netty.NettyRpcCallContext
import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend}
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
+import org.apache.spark.serializer.JavaSerializer
import org.apache.spark.util._
/**
@@ -89,6 +103,7 @@ private[spark] class ApplicationMaster(
@volatile private var reporterThread: Thread = _
@volatile private var allocator: YarnAllocator = _
+ @volatile private var clientToAMPort: Int = _
// A flag to check whether user has initialized spark context
@volatile private var registered = false
@@ -247,7 +262,9 @@ private[spark] class ApplicationMaster(
if (!unregistered) {
// we only want to unregister if we don't want the RM to retry
- if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) {
+ if (finalStatus == FinalApplicationStatus.SUCCEEDED ||
+ finalStatus == FinalApplicationStatus.KILLED ||
+ isLastAttempt) {
unregister(finalStatus, finalMsg)
cleanupStagingDir()
}
@@ -283,6 +300,7 @@ private[spark] class ApplicationMaster(
credentialRenewerThread.start()
credentialRenewerThread.join()
}
+ clientToAMPort = sparkConf.getInt("spark.yarn.clientToAM.port", 0)
if (isClusterMode) {
runDriver(securityMgr)
@@ -402,7 +420,8 @@ private[spark] class ApplicationMaster(
uiAddress,
historyAddress,
securityMgr,
- localResources)
+ localResources,
+ clientToAMPort)
// Initialize the AM endpoint *after* the allocator has been initialized. This ensures
// that when the driver sends an initial executor request (e.g. after an AM restart),
@@ -422,6 +441,89 @@ private[spark] class ApplicationMaster(
YarnSchedulerBackend.ENDPOINT_NAME)
}
+ /**
+ * Create an [[RpcEndpoint]] that communicates with the client.
+ *
+ * @return A reference to the application master's RPC endpoint.
+ */
+ private def runClientAMEndpoint(
+ port: Int,
+ driverRef: RpcEndpointRef,
+ securityManager: SecurityManager): RpcEndpointRef = {
+ val serversparkConf = new SparkConf()
+ serversparkConf.set("spark.rpc.connectionUsingTokens", "true")
+
+ val amRpcEnv =
+ RpcEnv.create(ApplicationMaster.SYSTEM_NAME, Utils.localHostName(), port, serversparkConf,
+ securityManager)
+ clientToAMPort = amRpcEnv.address.port
+
+ val clientAMEndpoint =
+ amRpcEnv.setupEndpoint(ApplicationMaster.ENDPOINT_NAME,
+ new ClientToAMEndpoint(amRpcEnv, driverRef, securityManager))
+ clientAMEndpoint
+ }
+
+ /** RpcEndpoint class for ClientToAM */
+ private[spark] class ClientToAMEndpoint(
+ override val rpcEnv: RpcEnv,
+ driverRef: RpcEndpointRef,
+ securityManager: SecurityManager)
+ extends RpcEndpoint with Logging {
+
+ override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
+ case ApplicationMasterMessages.KillApplication =>
+ if (securityManager.checkModifyPermissions(context.senderUserName)) {
+ driverRef.send(StopSparkContext)
+ finish(FinalApplicationStatus.KILLED, ApplicationMaster.EXIT_KILLED)
+ context.reply(true)
+ } else {
+ context.reply(false)
+ }
+ case ApplicationMasterMessages.UploadCredentials(c) =>
+ if (securityManager.checkModifyPermissions(context.senderUserName)) {
+ context.reply(true)
+ val dataInput = new DataInputStream(new ByteArrayInputStream(c))
+ val credentials = new Credentials
+ credentials.readFields(dataInput)
+ if (credentials != null) {
+ logInfo(YarnSparkHadoopUtil.get.dumpTokens(credentials).mkString("\n"))
+ }
+ UserGroupInformation.getCurrentUser.addCredentials(credentials)
+
+ try {
+ val timeout = RpcUtils.askRpcTimeout(sparkConf)
+ val success = timeout.awaitResult(driverRef.ask[Boolean](DelegateCredentials(c)))
+ if (!success) {
+ throw new SparkException(s"Current user doesn't have modify ACL")
+ context.reply(false)
+ }
+ } catch {
+ case e: TimeoutException =>
+ throw new SparkException(s"Timed out waiting to upload credential")
+ context.reply(false)
+ }
+// sc.schedulerBackend match {
+// case s: CoarseGrainedSchedulerBackend =>
+// logInfo(s"Update credentials in driver")
+// val f = s.updateCredentials(c)
+// f onSuccess {
+// case b => context.reply(b)
+// }
+// f onFailure {
+// case NonFatal(e) => context.sendFailure(e)
+// e
+// }
+// case _ =>
+// throw new SparkException(s"Update credentials on" +
+// s" ${sc.schedulerBackend.getClass.getSimpleName} is not supported")
+// }
+ } else {
+ context.reply(false)
+ }
+ }
+ }
+
private def runDriver(securityMgr: SecurityManager): Unit = {
addAmIpFilter(None)
userClassThread = startUserApplication()
@@ -438,8 +540,12 @@ private[spark] class ApplicationMaster(
val driverRef = createSchedulerRef(
sc.getConf.get("spark.driver.host"),
sc.getConf.get("spark.driver.port"))
+ val clientToAMSecurityManager = new SecurityManager(sparkConf)
+ runClientAMEndpoint(clientToAMPort, driverRef, clientToAMSecurityManager)
registerAM(sc.getConf, rpcEnv, driverRef, sc.ui.map(_.webUrl), securityMgr)
registered = true
+ clientToAMSecurityManager.setSecretKey(Base64.encode(
+ Unpooled.wrappedBuffer(client.getMasterKey)).toString(Charsets.UTF_8));
} else {
// Sanity check; should never happen in normal operation, since sc should only be null
// if the user app did not create a SparkContext.
@@ -464,10 +570,13 @@ private[spark] class ApplicationMaster(
amCores, true)
val driverRef = waitForSparkDriver()
addAmIpFilter(Some(driverRef))
+ val clientToAMSecurityManager = new SecurityManager(sparkConf)
+ runClientAMEndpoint(clientToAMPort, driverRef, clientToAMSecurityManager)
registerAM(sparkConf, rpcEnv, driverRef, sparkConf.getOption("spark.driver.appUIAddress"),
securityMgr)
registered = true
-
+ clientToAMSecurityManager.setSecretKey(Base64.encode(
+ Unpooled.wrappedBuffer(client.getMasterKey)).toString(Charsets.UTF_8));
// In client mode the actor will stop the reporter thread.
reporterThread.join()
}
@@ -749,8 +858,20 @@ private[spark] class ApplicationMaster(
}
+sealed trait ApplicationMasterMessage extends Serializable
+
+private [spark] object ApplicationMasterMessages {
+
+ case class KillApplication() extends ApplicationMasterMessage
+
+ case class UploadCredentials(credentials: Array[Byte]) extends ApplicationMasterMessage
+}
+
object ApplicationMaster extends Logging {
+ val SYSTEM_NAME = "sparkYarnAM"
+ val ENDPOINT_NAME = "clientToAM"
+
// exit codes for different causes, no reason behind the values
private val EXIT_SUCCESS = 0
private val EXIT_UNCAUGHT_EXCEPTION = 10
@@ -760,6 +881,7 @@ object ApplicationMaster extends Logging {
private val EXIT_SECURITY = 14
private val EXIT_EXCEPTION_USER_CLASS = 15
private val EXIT_EARLY = 16
+ private val EXIT_KILLED = 17
private var master: ApplicationMaster = _
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index d408ca90a5d1c..87ef59337a6f5 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -18,19 +18,25 @@
package org.apache.spark.deploy.yarn
import java.io.{File, FileOutputStream, IOException, OutputStreamWriter}
-import java.net.{InetAddress, UnknownHostException, URI}
+import java.net.{InetAddress, InetSocketAddress, UnknownHostException, URI}
import java.nio.ByteBuffer
import java.nio.charset.StandardCharsets
import java.security.PrivilegedExceptionAction
import java.util.{Locale, Properties, UUID}
+import java.util.concurrent.TimeoutException
import java.util.zip.{ZipEntry, ZipOutputStream}
import scala.collection.JavaConverters._
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map}
+import scala.concurrent.ExecutionContext
+import scala.util.control.Breaks._
import scala.util.control.NonFatal
+import com.google.common.base.Charsets.UTF_8
import com.google.common.base.Objects
import com.google.common.io.Files
+import io.netty.buffer.Unpooled
+import io.netty.handler.codec.base64.Base64
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs._
import org.apache.hadoop.fs.permission.FsPermission
@@ -45,16 +51,18 @@ import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication}
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException
-import org.apache.hadoop.yarn.util.Records
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
import org.apache.spark.{SecurityManager, SparkConf, SparkException}
import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.yarn.ApplicationMasterMessages.{KillApplication, UploadCredentials}
import org.apache.spark.deploy.yarn.config._
import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils}
-import org.apache.spark.util.{CallerContext, Utils}
+import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint}
+import org.apache.spark.util.{CallerContext, RpcUtils, SparkExitCode, ThreadUtils, Utils}
private[spark] class Client(
val args: ClientArguments,
@@ -1149,6 +1157,129 @@ private[spark] class Client(
}
}
+ def killSparkApplication(securityManager: SecurityManager): Unit = {
+ setupCredentials()
+ yarnClient.init(yarnConf)
+ yarnClient.start
+ val appId = ConverterUtils.toApplicationId(args.userArgs(0))
+ val AMEndpoint = setupAMConnection(appId, securityManager)
+ try {
+ val timeout = RpcUtils.askRpcTimeout(sparkConf)
+ val success = timeout.awaitResult(AMEndpoint.ask[Boolean](KillApplication))
+ if (!success) {
+ throw new SparkException(s"Current user doesn't have modify ACL")
+ return
+ }
+ } catch {
+ case e: TimeoutException =>
+ throw new SparkException(s"Timed out waiting to kill the application: $appId")
+ }
+
+ var currentTime = System.currentTimeMillis
+ val timeKillIssued = currentTime
+
+ val killTimeOut = sparkConf.get(CLIENT_TO_AM_HARD_KILL_TIMEOUT)
+ while ((currentTime< timeKillIssued + killTimeOut)
+ && !isAppInTerminalState(appId)) {
+ try
+ Thread.sleep(1000L)
+ catch {
+ case ie: InterruptedException => break
+ }
+ currentTime = System.currentTimeMillis
+ }
+ if (!isAppInTerminalState(appId)) {
+ yarnClient.killApplication(appId)
+ }
+ }
+ def uploadCredentials(securityManager: SecurityManager): Unit = {
+ yarnClient.init(yarnConf)
+ yarnClient.start
+
+ setupCredentials()
+ credentialManager.obtainDelegationTokens(hadoopConf, credentials)
+
+ val dob = new DataOutputBuffer()
+ if (credentials != null) {
+ UserGroupInformation.getCurrentUser.addCredentials(credentials)
+ credentials.write(dob)
+ logDebug(YarnSparkHadoopUtil.get.dumpTokens(credentials).mkString("\n"))
+ }
+
+ val AMEndpoint = setupAMConnection(ConverterUtils.toApplicationId(args.userArgs(0)),
+ securityManager)
+// val timeout = RpcUtils.askRpcTimeout(sparkConf)
+// val success = timeout.awaitResult(
+// AMEndpoint.ask[Boolean](UploadCredential(dob.getData)))
+//
+// if (!success) {
+//
+// throw new SparkException(s"Timed out waiting to kill the application: $appId")
+// throw new SparkException(s"Current user doesn't have modifiy ACL for the Application : $appId")
+// }
+
+ try {
+ val timeout = RpcUtils.askRpcTimeout(sparkConf)
+ val success = timeout.awaitResult(AMEndpoint.ask[Boolean](UploadCredentials(dob.getData)))
+ if (!success) {
+ throw new SparkException(s"Current user doesn't have modify ACL")
+ return
+ }
+ } catch {
+ case e: TimeoutException =>
+ throw new SparkException(s"Timed out waiting to upload credential")
+ }
+ }
+ private def setupAMConnection(
+ appId: ApplicationId,
+ securityManager: SecurityManager): RpcEndpointRef = {
+ logInfo(s"APP ID $appId")
+ val report = getApplicationReport(appId)
+ val state = report.getYarnApplicationState
+ if (report.getHost() == null || "".equals(report.getHost()) || "N/A".equals(report.getHost())) {
+ throw new SparkException(s"AM for $appId not assigned or dont have view ACL for it")
+ }
+ if ( state != YarnApplicationState.RUNNING) {
+ throw new SparkException(s"Application $appId needs to be in RUNNING")
+ }
+
+ if (UserGroupInformation.isSecurityEnabled()) {
+ val serviceAddr = new InetSocketAddress(report.getHost(), report.getRpcPort())
+
+ val clientToAMToken = report.getClientToAMToken
+ val token = ConverterUtils.convertFromYarn(clientToAMToken, serviceAddr)
+
+ // Fetch Identifier, secretkey from the report, encode it and Set it in the Security Manager
+ val userName = token.getIdentifier
+ var userstring = Base64.encode(Unpooled.wrappedBuffer(userName)).toString(UTF_8);
+ securityManager.setSaslUser(userstring)
+ val secretkey = token.getPassword
+ var secretkeystring = Base64.encode(Unpooled.wrappedBuffer(secretkey)).toString(UTF_8);
+ securityManager.setSecretKey(secretkeystring)
+ }
+
+ sparkConf.set("spark.rpc.connectionUsingTokens", "true")
+ val rpcEnv =
+ RpcEnv.create("yarnDriverClient", Utils.localHostName(), 0, sparkConf, securityManager)
+ val AMHostPort = RpcAddress(report.getHost, report.getRpcPort)
+ val AMEndpoint = rpcEnv.setupEndpointRef(AMHostPort,
+ ApplicationMaster.ENDPOINT_NAME)
+
+ AMEndpoint
+ }
+
+ private def checkAppStatus(appId: ApplicationId): YarnApplicationState = {
+ val report = getApplicationReport(appId)
+ report.getYarnApplicationState
+ }
+
+ private def isAppInTerminalState(appId: ApplicationId): Boolean = {
+ var status = checkAppStatus(appId)
+ return (status == YarnApplicationState.KILLED
+ || status == YarnApplicationState.FAILED
+ || status == YarnApplicationState.FINISHED)
+ }
+
private def findPySparkArchives(): Seq[String] = {
sys.env.get("PYSPARK_ARCHIVES_PATH")
.map(_.split(",").toSeq)
@@ -1186,6 +1317,22 @@ private object Client extends Logging {
new Client(args, sparkConf).run()
}
+ def yarnKillSubmission(argStrings: Array[String]): Unit = {
+ System.setProperty("SPARK_YARN_MODE", "true")
+ val sparkConf = new SparkConf
+ val args = new ClientArguments(argStrings)
+
+ new Client(args, sparkConf).killSparkApplication(new SecurityManager(sparkConf))
+ }
+ def yarnUploadCredentials(argStrings: Array[String]): Unit = {
+ System.setProperty("SPARK_YARN_MODE", "true")
+ val sparkConf = new SparkConf
+ val args = new ClientArguments(argStrings)
+ logInfo("Came here")
+
+ new Client(args, sparkConf).uploadCredentials(new SecurityManager(sparkConf))
+ }
+
// Alias for the user jar
val APP_JAR_NAME: String = "__app__.jar"
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
index 72f4d273ab53b..68c8134d5527a 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
@@ -17,6 +17,8 @@
package org.apache.spark.deploy.yarn
+import java.nio.ByteBuffer
+
import scala.collection.JavaConverters._
import org.apache.hadoop.yarn.api.records._
@@ -39,6 +41,7 @@ private[spark] class YarnRMClient extends Logging {
private var amClient: AMRMClient[ContainerRequest] = _
private var uiHistoryAddress: String = _
private var registered: Boolean = false
+ private var masterkey: ByteBuffer = _
/**
* Registers the application master with the RM.
@@ -58,7 +61,8 @@ private[spark] class YarnRMClient extends Logging {
uiAddress: Option[String],
uiHistoryAddress: String,
securityMgr: SecurityManager,
- localResources: Map[String, LocalResource]
+ localResources: Map[String, LocalResource],
+ port: Int = 0
): YarnAllocator = {
amClient = AMRMClient.createAMRMClient()
amClient.init(conf)
@@ -71,8 +75,9 @@ private[spark] class YarnRMClient extends Logging {
logInfo("Registering the ApplicationMaster")
synchronized {
- amClient.registerApplicationMaster(Utils.localHostName(), 0, trackingUrl)
+ var response = amClient.registerApplicationMaster(Utils.localHostName(), port, trackingUrl)
registered = true
+ masterkey = response.getClientToAMTokenMasterKey()
}
new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), securityMgr,
localResources, new SparkRackResolver())
@@ -89,6 +94,9 @@ private[spark] class YarnRMClient extends Logging {
amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress)
}
}
+ /** Obtain the MasterKey reported back from YARN when Registering AM. */
+ def getMasterKey(): ByteBuffer = masterkey
+
/** Returns the attempt ID. */
def getAttemptId(): ApplicationAttemptId = {
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
index 187803cc6050b..26520578e066a 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
@@ -187,6 +187,11 @@ package object config {
.toSequence
.createWithDefault(Nil)
+ private[spark] val CLIENT_TO_AM_HARD_KILL_TIMEOUT = ConfigBuilder("spark.yarn.hardKillTimeout")
+ .internal()
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefaultString("60s")
+
/* Client-mode AM configuration. */
private[spark] val AM_CORES = ConfigBuilder("spark.yarn.am.cores")
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
index 415a29fd887e8..cb3a53bf02c6d 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
@@ -23,9 +23,11 @@ import scala.concurrent.{ExecutionContext, Future}
import scala.util.{Failure, Success}
import scala.util.control.NonFatal
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId}
-import org.apache.spark.SparkContext
+import org.apache.spark.{SparkContext, SparkException}
import org.apache.spark.internal.Logging
import org.apache.spark.rpc._
import org.apache.spark.scheduler._
@@ -271,6 +273,9 @@ private[spark] abstract class YarnSchedulerBackend(
logError("Error requesting driver to remove executor" +
s" $executorId for reason $reason", e)
}(ThreadUtils.sameThread)
+ case StopSparkContext =>
+ sc.stop
+
}
@@ -305,6 +310,24 @@ private[spark] abstract class YarnSchedulerBackend(
case RetrieveLastAllocatedExecutorId =>
context.reply(currentExecutorIdCounter)
+
+ case DelegateCredentials(c) =>
+ context.reply(true)
+ sc.schedulerBackend match {
+ case s: CoarseGrainedSchedulerBackend =>
+ logInfo(s"Update credentials in driver")
+ val f = s.updateCredentials(c)
+// f.onSuccess {
+// case b => context.reply(b)
+// }
+// f.onFailure {
+// case NonFatal(e) => context.sendFailure(e)
+// e
+// }
+ case _ =>
+ throw new SparkException(s"Update credentials on" +
+ s" ${sc.schedulerBackend.getClass.getSimpleName} is not supported")
+ }
}
override def onDisconnected(remoteAddress: RpcAddress): Unit = {