From 77914dd74b3a4af3501bda7a72c658ffcdd0682f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 30 Jan 2015 11:14:35 -0800 Subject: [PATCH 01/41] WIP: Add kerberos principal and keytab to YARN client. --- .../org/apache/spark/deploy/yarn/Client.scala | 21 ++++++++++++++++++- .../spark/deploy/yarn/ClientArguments.scala | 10 +++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 46d9df93488cb..1c1b4151c6754 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -17,8 +17,10 @@ package org.apache.spark.deploy.yarn +import java.io.File import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer +import java.nio.file.Files import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} @@ -61,7 +63,7 @@ private[spark] class Client( private val yarnClient = YarnClient.createYarnClient private val yarnConf = new YarnConfiguration(hadoopConf) - private val credentials = UserGroupInformation.getCurrentUser.getCredentials + private var credentials: Credentials = null private val amMemoryOverhead = args.amMemoryOverhead // MB private val executorMemoryOverhead = args.executorMemoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() @@ -540,6 +542,23 @@ private[spark] class Client( amContainer } + def setupCredentials(): Unit = { + Option(args.principal) match { + case Some(principal) => + Option(args.keytab) match { + case Some(keytabPath) => + File principalFile = Files.createTempF + val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytabPath) + credentials = ugi.getCredentials + case None => + } + case None => + credentials = UserGroupInformation.getCurrentUser.getCredentials + + } + + } + /** * Report the state of an application until it has exited, either successfully or * due to some failure, then return a pair of the yarn application state (FINISHED, FAILED, diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 3bc7eb1abf341..62311969d99af 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -41,6 +41,8 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var amCores: Int = 1 var appName: String = "Spark" var priority = 0 + var principal: String = null + var keytab: String = null def isClusterMode: Boolean = userClass != null private var driverMemory: Int = 512 // MB @@ -222,6 +224,14 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) archives = value args = tail + case ("--principal") :: value :: tail => + principal = value + args = tail + + case ("--keytab") :: value :: tail => + keytab = value + args = tail + case Nil => case _ => From ccba5bc3e7ceceb9b1f15072888454b88d1a2322 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 2 Feb 2015 15:06:30 -0800 Subject: [PATCH 02/41] WIP: More changes wrt kerberos --- .../org/apache/spark/deploy/yarn/Client.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 1c1b4151c6754..e4491d3addf2b 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -24,7 +24,7 @@ import java.nio.file.Files import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} -import scala.util.{Try, Success, Failure} +import scala.util.{Random, Try, Success, Failure} import com.google.common.base.Objects @@ -69,6 +69,9 @@ private[spark] class Client( private val distCacheMgr = new ClientDistributedCacheManager() private val isClusterMode = args.isClusterMode + private var loginFromKeytab = false + private var kerberosFileName: String = null + def stop(): Unit = yarnClient.stop() @@ -86,6 +89,7 @@ private[spark] class Client( * available in the alpha API. */ def submitApplication(): ApplicationId = { + setupCredentials() yarnClient.init(yarnConf) yarnClient.start() @@ -380,7 +384,6 @@ private[spark] class Client( private def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) : ContainerLaunchContext = { logInfo("Setting up container launch context for our AM") - val appId = newAppResponse.getApplicationId val appStagingDir = getAppStagingDir(appId) val localResources = prepareLocalResources(appStagingDir) @@ -547,16 +550,19 @@ private[spark] class Client( case Some(principal) => Option(args.keytab) match { case Some(keytabPath) => - File principalFile = Files.createTempF + // Generate a file name that can be used for the keytab file, that does not conflict + // with any user file. + val f = new File(keytabPath) + kerberosFileName = f.getName + "-" + System.currentTimeMillis() val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytabPath) credentials = ugi.getCredentials + loginFromKeytab = true case None => + throw new SparkException("Keytab must be specified when principal is specified.") } case None => credentials = UserGroupInformation.getCurrentUser.getCredentials - } - } /** From 2b0d745ec7b76c3dd992660c24ddac556ba1de6a Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 18 Feb 2015 21:47:14 -0800 Subject: [PATCH 03/41] [SPARK-5342][YARN] Allow long running Spark apps to run on secure YARN/HDFS. Current Spark apps running on Secure YARN/HDFS would not be able to write data to HDFS after 7 days, since delegation tokens cannot be renewed beyond that. This means Spark Streaming apps will not be able to run on Secure YARN. This commit adds basic functionality to fix this issue. In this patch: - new parameters are added - principal and keytab, which can be used to login to a KDC - the client logs in, and then get tokens to start the AM - the keytab is copied to the staging directory - the AM waits for 60% of the time till expiry of the tokens and then logs in using the keytab - each time after 60% of the time, new tokens are created and sent to the executors --- .../apache/spark/deploy/SparkHadoopUtil.scala | 115 +++++++++++++++++- .../CoarseGrainedExecutorBackend.scala | 8 ++ .../cluster/CoarseGrainedClusterMessage.scala | 4 + .../CoarseGrainedSchedulerBackend.scala | 11 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 5 + .../org/apache/spark/deploy/yarn/Client.scala | 45 ++++--- 6 files changed, 159 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index e0a32fb65cd51..0846b087eafe0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -17,20 +17,26 @@ package org.apache.spark.deploy +import java.io.{ByteArrayInputStream, DataInputStream, DataOutputStream, ByteArrayOutputStream} import java.lang.reflect.Method +import java.net.URI +import java.nio.ByteBuffer import java.security.PrivilegedExceptionAction +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.{TimeUnit, ThreadFactory, Executors} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.{FileUtil, FileStatus, FileSystem, Path} import org.apache.hadoop.fs.FileSystem.Statistics -import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier +import org.apache.hadoop.mapred.{Master, JobConf} import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.{Logging, SparkContext, SparkConf, SparkException} +import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.Utils +import org.apache.spark.util.{SerializableBuffer, Utils} import scala.collection.JavaConversions._ @@ -40,9 +46,14 @@ import scala.collection.JavaConversions._ */ @DeveloperApi class SparkHadoopUtil extends Logging { - val conf: Configuration = newConfiguration(new SparkConf()) + val sparkConf = new SparkConf() + val conf: Configuration = newConfiguration(sparkConf) UserGroupInformation.setConfiguration(conf) + private var keytabFile: Option[String] = None + private var loginPrincipal: Option[String] = None + private val loggedInViaKeytab = new AtomicBoolean(false) + /** * Runs the given function with a Hadoop UserGroupInformation as a thread local variable * (distributed to child threads), used for authenticating HDFS and YARN calls. @@ -121,6 +132,100 @@ class SparkHadoopUtil extends Logging { UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } + def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit ={ + loginPrincipal = Option(principal) + keytabFile = Option(keytab) + } + + private[spark] def scheduleLoginFromKeytab(callback: (SerializableBuffer) => Unit): Unit = { + + loginPrincipal match { + case Some(principal) => + val keytab = keytabFile.get + val remoteFs = FileSystem.get(conf) + val remoteKeytabPath = new Path( + remoteFs.getHomeDirectory, System.getenv("SPARK_STAGING_DIR") + Path.SEPARATOR + keytab) + val localFS = FileSystem.getLocal(conf) + // At this point, SparkEnv is likely no initialized, so create a dir, put the keytab there. + val tempDir = Utils.createTempDir() + val localURI = new URI(tempDir.getAbsolutePath + Path.SEPARATOR + keytab) + val qualifiedURI = new URI(localFS.makeQualified(new Path(localURI)).toString) + FileUtil.copy( + remoteFs, remoteKeytabPath, localFS, new Path(qualifiedURI), false, false, conf) + // Get the current credentials, find out when they expire. + val creds = UserGroupInformation.getCurrentUser.getCredentials + val credStream = new ByteArrayOutputStream() + creds.writeTokenStorageToStream(new DataOutputStream(credStream)) + val in = new DataInputStream(new ByteArrayInputStream(credStream.toByteArray)) + val tokenIdentifier = new DelegationTokenIdentifier() + tokenIdentifier.readFields(in) + val timeToRenewal = (0.6 * (tokenIdentifier.getMaxDate - System.currentTimeMillis())).toLong + Executors.newSingleThreadScheduledExecutor(new ThreadFactory { + override def newThread(r: Runnable): Thread = { + val t = new Thread(r) + t.setName("Delegation Token Refresh Thread") + t.setDaemon(true) + t + } + }).scheduleWithFixedDelay(new Runnable { + override def run(): Unit = { + if (!loggedInViaKeytab.get()) { + loginUserFromKeytab(principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab) + loggedInViaKeytab.set(true) + } + val nns = getNameNodesToAccess(sparkConf) + remoteKeytabPath + val newCredentials = new Credentials() + obtainTokensForNamenodes(nns, conf, newCredentials) + // Now write this out via Akka to executors. + val outputStream = new ByteArrayOutputStream() + newCredentials.writeTokenStorageToStream(new DataOutputStream(outputStream)) + callback(new SerializableBuffer(ByteBuffer.wrap(outputStream.toByteArray))) + } + }, timeToRenewal, timeToRenewal, TimeUnit.MILLISECONDS) + + } + } + + /** + * Get the list of namenodes the user may access. + */ + def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { + sparkConf.get("spark.yarn.access.namenodes", "") + .split(",") + .map(_.trim()) + .filter(!_.isEmpty) + .map(new Path(_)) + .toSet + } + + def getTokenRenewer(conf: Configuration): String = { + val delegTokenRenewer = Master.getMasterPrincipal(conf) + logDebug("delegation token renewer is: " + delegTokenRenewer) + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer" + logError(errorMessage) + throw new SparkException(errorMessage) + } + delegTokenRenewer + } + + /** + * Obtains tokens for the namenodes passed in and adds them to the 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) + } + } + } + /** * Returns a function that can be called to find Hadoop FileSystem bytes read. If * getFSBytesReadOnThreadCallback is called from thread r at time t, the returned callback will diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index dd19e4947db1e..2e086dc194843 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -18,6 +18,7 @@ package org.apache.spark.executor import java.net.URL +import java.io.{ByteArrayInputStream, DataInputStream} import java.nio.ByteBuffer import scala.collection.mutable @@ -26,6 +27,7 @@ import scala.concurrent.Await import akka.actor.{Actor, ActorSelection, Props} import akka.pattern.Patterns import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} +import org.apache.hadoop.security.Credentials import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState @@ -105,6 +107,12 @@ private[spark] class CoarseGrainedExecutorBackend( executor.stop() context.stop(self) context.system.shutdown() + + case UpdateCredentials(newCredentials) => + val credentials = new Credentials() + credentials.readTokenStorageStream( + new DataInputStream(new ByteArrayInputStream(newCredentials.value.array()))) + SparkHadoopUtil.get.addCurrentUserCredentials(credentials) } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 9bf74f4be198d..d1a931e8bdaac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -51,6 +51,10 @@ private[spark] object CoarseGrainedClusterMessages { case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer) extends CoarseGrainedClusterMessage + // Driver to all executors. + case class UpdateCredentials(newCredentials: SerializableBuffer) + extends CoarseGrainedClusterMessage + object StatusUpdate { /** Alternate factory method that takes a ByteBuffer directly for the data field */ def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6f77fa32ce37b..af5f794843ef6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,6 +27,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -75,6 +76,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste override protected def log = CoarseGrainedSchedulerBackend.this.log private val addressToExecutorId = new HashMap[Address, String] + // If a principal and keytab have been set, use that to create new credentials for executors + // periodically + SparkHadoopUtil.get.scheduleLoginFromKeytab(sendNewCredentialsToExecutors _) + override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -85,6 +90,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } + def sendNewCredentialsToExecutors(credentials: SerializableBuffer): Unit = { + executorDataMap.values.foreach{ x => + x.executorActor ! UpdateCredentials(credentials) + } + } + def receiveWithLogging = { case RegisterExecutor(executorId, hostPort, cores, logUrls) => Utils.checkHostPort(hostPort, "Host port expected " + hostPort) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a9bf861d160c1..882d4799d2078 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -576,6 +576,11 @@ object ApplicationMaster extends Logging { master = new ApplicationMaster(amArgs, new YarnRMClient(amArgs)) System.exit(master.run()) } + // At this point, we have tokens that will expire only after a while, so we now schedule a + // login for some time before the tokens expire. Since the SparkContext has already started, + // we can now get access to the driver actor as well. + SparkHadoopUtil.get.setPrincipalAndKeytabForLogin( + System.getenv("SPARK_PRINCIPAL"), System.getenv("SPARK_KEYTAB")) } private[spark] def sparkContextInitialized(sc: SparkContext) = { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index e4491d3addf2b..249febe9cc6cf 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -70,7 +70,7 @@ private[spark] class Client( private val isClusterMode = args.isClusterMode private var loginFromKeytab = false - private var kerberosFileName: String = null + private var keytabFileName: String = null def stop(): Unit = yarnClient.stop() @@ -89,6 +89,7 @@ private[spark] class Client( * available in the alpha API. */ def submitApplication(): ApplicationId = { + // Setup the credentials before doing anything else, so we have don't have issues at any point. setupCredentials() yarnClient.init(yarnConf) yarnClient.start() @@ -319,6 +320,21 @@ private[spark] class Client( env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() + // If we logged in from keytab, make sure we copy the keytab to the staging directory on + // HDFS, and setup the relevant environment vars, so the AM can login again. + if (loginFromKeytab) { + val fs = FileSystem.get(hadoopConf) + val stagingDirPath = new Path(fs.getHomeDirectory, stagingDir) + val localUri = new URI(args.keytab) + val localPath = getQualifiedLocalPath(localUri, hadoopConf) + val destinationPath = new Path(stagingDirPath, keytabFileName) + val replication = sparkConf.getInt("spark.yarn.submit.file.replication", + fs.getDefaultReplication(destinationPath)).toShort + copyFileToRemote(destinationPath, localPath, replication) + env("SPARK_PRINCIPAL") = args.principal + env("SPARK_KEYTAB") = keytabFileName + } + // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) @@ -553,7 +569,7 @@ private[spark] class Client( // Generate a file name that can be used for the keytab file, that does not conflict // with any user file. val f = new File(keytabPath) - kerberosFileName = f.getName + "-" + System.currentTimeMillis() + keytabFileName = f.getName + "-" + System.currentTimeMillis() val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytabPath) credentials = ugi.getCredentials loginFromKeytab = true @@ -891,23 +907,11 @@ object Client extends Logging { * 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 + SparkHadoopUtil.get.getNameNodesToAccess(sparkConf) } private[yarn] def getTokenRenewer(conf: Configuration): String = { - val delegTokenRenewer = Master.getMasterPrincipal(conf) - logDebug("delegation token renewer is: " + delegTokenRenewer) - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) - } - delegTokenRenewer + SparkHadoopUtil.get.getTokenRenewer(conf) } /** @@ -917,14 +921,7 @@ object Client extends Logging { 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) - } - } + SparkHadoopUtil.get.obtainTokensForNamenodes(paths, conf, creds) } /** From f8fe694efd117d707313748c02cef42240a3aec7 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 19 Feb 2015 10:46:46 -0800 Subject: [PATCH 04/41] Handle None if keytab-login is not scheduled. --- .../main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 0846b087eafe0..b96d793335117 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -182,7 +182,7 @@ class SparkHadoopUtil extends Logging { callback(new SerializableBuffer(ByteBuffer.wrap(outputStream.toByteArray))) } }, timeToRenewal, timeToRenewal, TimeUnit.MILLISECONDS) - + case None => } } From bcfc3747e14ce6f1a242ba9cd71c03017b77f36e Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 19 Feb 2015 23:08:15 -0800 Subject: [PATCH 05/41] Fix Hadoop-1 build by adding no-op methods in SparkHadoopUtil, with impl in YarnSparkHadoopUtil. --- .../apache/spark/deploy/SparkHadoopUtil.scala | 100 +-------------- .../org/apache/spark/deploy/yarn/Client.scala | 7 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 114 +++++++++++++++++- 3 files changed, 117 insertions(+), 104 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index b96d793335117..98901ed15b9a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileUtil, FileStatus, FileSystem, Path} import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier -import org.apache.hadoop.mapred.{Master, JobConf} +import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation @@ -50,10 +50,6 @@ class SparkHadoopUtil extends Logging { val conf: Configuration = newConfiguration(sparkConf) UserGroupInformation.setConfiguration(conf) - private var keytabFile: Option[String] = None - private var loginPrincipal: Option[String] = None - private val loggedInViaKeytab = new AtomicBoolean(false) - /** * Runs the given function with a Hadoop UserGroupInformation as a thread local variable * (distributed to child threads), used for authenticating HDFS and YARN calls. @@ -132,99 +128,9 @@ class SparkHadoopUtil extends Logging { UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } - def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit ={ - loginPrincipal = Option(principal) - keytabFile = Option(keytab) - } - - private[spark] def scheduleLoginFromKeytab(callback: (SerializableBuffer) => Unit): Unit = { - - loginPrincipal match { - case Some(principal) => - val keytab = keytabFile.get - val remoteFs = FileSystem.get(conf) - val remoteKeytabPath = new Path( - remoteFs.getHomeDirectory, System.getenv("SPARK_STAGING_DIR") + Path.SEPARATOR + keytab) - val localFS = FileSystem.getLocal(conf) - // At this point, SparkEnv is likely no initialized, so create a dir, put the keytab there. - val tempDir = Utils.createTempDir() - val localURI = new URI(tempDir.getAbsolutePath + Path.SEPARATOR + keytab) - val qualifiedURI = new URI(localFS.makeQualified(new Path(localURI)).toString) - FileUtil.copy( - remoteFs, remoteKeytabPath, localFS, new Path(qualifiedURI), false, false, conf) - // Get the current credentials, find out when they expire. - val creds = UserGroupInformation.getCurrentUser.getCredentials - val credStream = new ByteArrayOutputStream() - creds.writeTokenStorageToStream(new DataOutputStream(credStream)) - val in = new DataInputStream(new ByteArrayInputStream(credStream.toByteArray)) - val tokenIdentifier = new DelegationTokenIdentifier() - tokenIdentifier.readFields(in) - val timeToRenewal = (0.6 * (tokenIdentifier.getMaxDate - System.currentTimeMillis())).toLong - Executors.newSingleThreadScheduledExecutor(new ThreadFactory { - override def newThread(r: Runnable): Thread = { - val t = new Thread(r) - t.setName("Delegation Token Refresh Thread") - t.setDaemon(true) - t - } - }).scheduleWithFixedDelay(new Runnable { - override def run(): Unit = { - if (!loggedInViaKeytab.get()) { - loginUserFromKeytab(principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab) - loggedInViaKeytab.set(true) - } - val nns = getNameNodesToAccess(sparkConf) + remoteKeytabPath - val newCredentials = new Credentials() - obtainTokensForNamenodes(nns, conf, newCredentials) - // Now write this out via Akka to executors. - val outputStream = new ByteArrayOutputStream() - newCredentials.writeTokenStorageToStream(new DataOutputStream(outputStream)) - callback(new SerializableBuffer(ByteBuffer.wrap(outputStream.toByteArray))) - } - }, timeToRenewal, timeToRenewal, TimeUnit.MILLISECONDS) - case None => - } - } - - /** - * Get the list of namenodes the user may access. - */ - def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { - sparkConf.get("spark.yarn.access.namenodes", "") - .split(",") - .map(_.trim()) - .filter(!_.isEmpty) - .map(new Path(_)) - .toSet - } - - def getTokenRenewer(conf: Configuration): String = { - val delegTokenRenewer = Master.getMasterPrincipal(conf) - logDebug("delegation token renewer is: " + delegTokenRenewer) - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) - } - delegTokenRenewer - } + def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit = {} - /** - * Obtains tokens for the namenodes passed in and adds them to the 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) - } - } - } + private[spark] def scheduleLoginFromKeytab(callback: (SerializableBuffer) => Unit): Unit = {} /** * Returns a function that can be called to find Hadoop FileSystem bytes read. If diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 249febe9cc6cf..b31eb6f9b3ea1 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -907,11 +907,11 @@ object Client extends Logging { * Get the list of namenodes the user may access. */ private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { - SparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil].getNameNodesToAccess(sparkConf) } private[yarn] def getTokenRenewer(conf: Configuration): String = { - SparkHadoopUtil.get.getTokenRenewer(conf) + SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil].getTokenRenewer(conf) } /** @@ -921,7 +921,8 @@ object Client extends Logging { paths: Set[Path], conf: Configuration, creds: Credentials): Unit = { - SparkHadoopUtil.get.obtainTokensForNamenodes(paths, conf, creds) + SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil] + .obtainTokensForNamenodes(paths, conf, creds) } /** diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 146b2c0f1a302..daf69646737d0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -17,15 +17,21 @@ package org.apache.spark.deploy.yarn -import java.io.File +import java.io._ +import java.net.URI +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.{TimeUnit, ThreadFactory, Executors} import java.util.regex.Matcher import java.util.regex.Pattern import scala.collection.mutable.HashMap import scala.util.Try +import org.apache.hadoop.fs.{FileUtil, Path, FileSystem} +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.io.Text -import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.{Master, JobConf} import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration @@ -34,15 +40,19 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.records.{Priority, ApplicationAccessType} import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{SparkException, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.Utils +import org.apache.spark.util.{SerializableBuffer, Utils} /** * Contains util methods to interact with Hadoop from spark. */ class YarnSparkHadoopUtil extends SparkHadoopUtil { + private var keytabFile: Option[String] = None + private var loginPrincipal: Option[String] = None + private val loggedInViaKeytab = new AtomicBoolean(false) + override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) } @@ -82,6 +92,101 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { if (credentials != null) credentials.getSecretKey(new Text(key)) else null } + override def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit ={ + loginPrincipal = Option(principal) + keytabFile = Option(keytab) + } + + private[spark] override def scheduleLoginFromKeytab( + callback: (SerializableBuffer) => Unit): Unit = { + + loginPrincipal match { + case Some(principal) => + val keytab = keytabFile.get + val remoteFs = FileSystem.get(conf) + val remoteKeytabPath = new Path( + remoteFs.getHomeDirectory, System.getenv("SPARK_STAGING_DIR") + Path.SEPARATOR + keytab) + val localFS = FileSystem.getLocal(conf) + // At this point, SparkEnv is likely no initialized, so create a dir, put the keytab there. + val tempDir = Utils.createTempDir() + val localURI = new URI(tempDir.getAbsolutePath + Path.SEPARATOR + keytab) + val qualifiedURI = new URI(localFS.makeQualified(new Path(localURI)).toString) + FileUtil.copy( + remoteFs, remoteKeytabPath, localFS, new Path(qualifiedURI), false, false, conf) + // Get the current credentials, find out when they expire. + val creds = UserGroupInformation.getCurrentUser.getCredentials + val credStream = new ByteArrayOutputStream() + creds.writeTokenStorageToStream(new DataOutputStream(credStream)) + val in = new DataInputStream(new ByteArrayInputStream(credStream.toByteArray)) + val tokenIdentifier = new DelegationTokenIdentifier() + tokenIdentifier.readFields(in) + val timeToRenewal = (0.6 * (tokenIdentifier.getMaxDate - System.currentTimeMillis())).toLong + Executors.newSingleThreadScheduledExecutor(new ThreadFactory { + override def newThread(r: Runnable): Thread = { + val t = new Thread(r) + t.setName("Delegation Token Refresh Thread") + t.setDaemon(true) + t + } + }).scheduleWithFixedDelay(new Runnable { + override def run(): Unit = { + if (!loggedInViaKeytab.get()) { + loginUserFromKeytab(principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab) + loggedInViaKeytab.set(true) + } + val nns = getNameNodesToAccess(sparkConf) + remoteKeytabPath + val newCredentials = new Credentials() + obtainTokensForNamenodes(nns, conf, newCredentials) + // Now write this out via Akka to executors. + val outputStream = new ByteArrayOutputStream() + newCredentials.writeTokenStorageToStream(new DataOutputStream(outputStream)) + callback(new SerializableBuffer(ByteBuffer.wrap(outputStream.toByteArray))) + } + }, timeToRenewal, timeToRenewal, TimeUnit.MILLISECONDS) + case None => + } + } + + /** + * Get the list of namenodes the user may access. + */ + def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { + sparkConf.get("spark.yarn.access.namenodes", "") + .split(",") + .map(_.trim()) + .filter(!_.isEmpty) + .map(new Path(_)) + .toSet + } + + def getTokenRenewer(conf: Configuration): String = { + val delegTokenRenewer = Master.getMasterPrincipal(conf) + logDebug("delegation token renewer is: " + delegTokenRenewer) + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer" + logError(errorMessage) + throw new SparkException(errorMessage) + } + delegTokenRenewer + } + + /** + * Obtains tokens for the namenodes passed in and adds them to the 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) + } + } + } + } object YarnSparkHadoopUtil { @@ -211,4 +316,5 @@ object YarnSparkHadoopUtil { def getClassPathSeparator(): String = { classPathSeparatorField.get(null).asInstanceOf[String] } + } From d282d7a69d563604776f6985760bd2c367725d0d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 19 Feb 2015 23:41:00 -0800 Subject: [PATCH 06/41] Fix ClientSuite to set YARN mode, so that the correct class is used in tests. --- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index f8f8129d220e4..a6224486f3535 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -28,8 +28,7 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.Matchers._ import org.mockito.Mockito._ -import org.scalatest.FunSuite -import org.scalatest.Matchers +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite, Matchers} import scala.collection.JavaConversions._ import scala.collection.mutable.{ HashMap => MutableHashMap } @@ -39,7 +38,15 @@ import scala.util.Try import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.util.Utils -class ClientSuite extends FunSuite with Matchers { +class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { + + override def beforeAll(): Unit = { + System.setProperty("SPARK_YARN_MODE", "true") + } + + override def afterAll(): Unit = { + System.setProperty("SPARK_YARN_MODE", "") + } test("default Yarn application classpath") { Client.getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP)) From fb27f46f2f2b06cffe15a4728e828371f92e17a5 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 23 Feb 2015 17:38:51 -0800 Subject: [PATCH 07/41] Make sure principal and keytab are set before CoarseGrainedSchedulerBackend is started. Also schedule re-logins in CoarseGrainedSchedulerBackend#start() --- .../apache/spark/deploy/SparkHadoopUtil.scala | 10 ++-------- .../executor/CoarseGrainedExecutorBackend.scala | 2 ++ .../cluster/CoarseGrainedClusterMessage.scala | 3 ++- .../cluster/CoarseGrainedSchedulerBackend.scala | 17 ++++++++++++----- .../spark/deploy/yarn/ApplicationMaster.scala | 11 ++++++----- .../org/apache/spark/deploy/yarn/Client.scala | 3 +++ .../spark/deploy/yarn/YarnSparkHadoopUtil.scala | 2 +- 7 files changed, 28 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 98901ed15b9a6..7476f1d52c4e3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -17,20 +17,14 @@ package org.apache.spark.deploy -import java.io.{ByteArrayInputStream, DataInputStream, DataOutputStream, ByteArrayOutputStream} import java.lang.reflect.Method -import java.net.URI -import java.nio.ByteBuffer import java.security.PrivilegedExceptionAction -import java.util.concurrent.atomic.AtomicBoolean -import java.util.concurrent.{TimeUnit, ThreadFactory, Executors} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileUtil, FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.fs.FileSystem.Statistics -import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} +import org.apache.hadoop.mapreduce.JobContext import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 2e086dc194843..5f0309d605761 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -108,7 +108,9 @@ private[spark] class CoarseGrainedExecutorBackend( context.stop(self) context.system.shutdown() + // Add new credentials received from the driver to the current user. case UpdateCredentials(newCredentials) => + logInfo("New credentials received from driver, adding the credentials to the current user") val credentials = new Credentials() credentials.readTokenStorageStream( new DataInputStream(new ByteArrayInputStream(newCredentials.value.array()))) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index d1a931e8bdaac..214a4865e5f26 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -51,7 +51,8 @@ private[spark] object CoarseGrainedClusterMessages { case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer) extends CoarseGrainedClusterMessage - // Driver to all executors. + // When the delegation tokens are about expire, the driver creates new tokens and sends them to + // the executors via this message. case class UpdateCredentials(newCredentials: SerializableBuffer) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index af5f794843ef6..3c897a63308b9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -76,10 +76,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste override protected def log = CoarseGrainedSchedulerBackend.this.log private val addressToExecutorId = new HashMap[Address, String] - // If a principal and keytab have been set, use that to create new credentials for executors - // periodically - SparkHadoopUtil.get.scheduleLoginFromKeytab(sendNewCredentialsToExecutors _) - override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -90,6 +86,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } + /** + * Send new credentials to executors. This is the method that is called when the scheduled + * login completes, so the new credentials can be sent to the executors. + * @param credentials + */ def sendNewCredentialsToExecutors(credentials: SerializableBuffer): Unit = { executorDataMap.values.foreach{ x => x.executorActor ! UpdateCredentials(credentials) @@ -245,9 +246,15 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste properties += ((key, value)) } } + + val driver = new DriverActor(properties) // TODO (prashant) send conf instead of properties driverActor = actorSystem.actorOf( - Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) + Props(driver), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) + + // If a principal and keytab have been set, use that to create new credentials for executors + // periodically + SparkHadoopUtil.get.scheduleLoginFromKeytab(driver.sendNewCredentialsToExecutors _) } def stopExecutors() { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 882d4799d2078..9bdd0f9bcf59d 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -256,6 +256,12 @@ private[spark] class ApplicationMaster( private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() + + // This must be done before SparkContext is initialized, since the CoarseGrainedSchedulerBackend + // is started at that time. That is what schedules the re-logins. It is scheduled only if the + // principal is actually setup. So we make sure it is available. + SparkHadoopUtil.get.setPrincipalAndKeytabForLogin( + System.getenv("SPARK_PRINCIPAL"), System.getenv("SPARK_KEYTAB")) userClassThread = startUserApplication() // This a bit hacky, but we need to wait until the spark.driver.port property has @@ -576,11 +582,6 @@ object ApplicationMaster extends Logging { master = new ApplicationMaster(amArgs, new YarnRMClient(amArgs)) System.exit(master.run()) } - // At this point, we have tokens that will expire only after a while, so we now schedule a - // login for some time before the tokens expire. Since the SparkContext has already started, - // we can now get access to the driver actor as well. - SparkHadoopUtil.get.setPrincipalAndKeytabForLogin( - System.getenv("SPARK_PRINCIPAL"), System.getenv("SPARK_KEYTAB")) } private[spark] def sparkContextInitialized(sc: SparkContext) = { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index b31eb6f9b3ea1..73e7f797c8f70 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -568,11 +568,14 @@ private[spark] class Client( case Some(keytabPath) => // Generate a file name that can be used for the keytab file, that does not conflict // with any user file. + logInfo("Attempting to login to the Kerberos" + + s" using principal: $principal and keytab: $keytabPath") val f = new File(keytabPath) keytabFileName = f.getName + "-" + System.currentTimeMillis() val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytabPath) credentials = ugi.getCredentials loginFromKeytab = true + logInfo("Successfully logged into Kerberos.") case None => throw new SparkException("Keytab must be specified when principal is specified.") } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index daf69646737d0..4df07e8b2a1ce 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -92,7 +92,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { if (credentials != null) credentials.getSecretKey(new Text(key)) else null } - override def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit ={ + override def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit = { loginPrincipal = Option(principal) keytabFile = Option(keytab) } From 8c6928a1cf966136847e6035a0610781b17e2769 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 23 Feb 2015 19:29:46 -0800 Subject: [PATCH 08/41] Fix issue caused by direct creation of Actor object. --- .../CoarseGrainedSchedulerBackend.scala | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 3c897a63308b9..bc101a0342e69 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -72,6 +72,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Executors we have requested the cluster manager to kill that have not died yet private val executorsPendingToRemove = new HashSet[String] + /** + * Send new credentials to executors. This is the method that is called when the scheduled + * login completes, so the new credentials can be sent to the executors. + * @param credentials + */ + def sendNewCredentialsToExecutors(credentials: SerializableBuffer): Unit = { + // We don't care about the reply, so going to deadLetters is fine. + executorDataMap.values.foreach(_.executorActor ! UpdateCredentials(credentials)) + } + class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { override protected def log = CoarseGrainedSchedulerBackend.this.log private val addressToExecutorId = new HashMap[Address, String] @@ -86,17 +96,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } - /** - * Send new credentials to executors. This is the method that is called when the scheduled - * login completes, so the new credentials can be sent to the executors. - * @param credentials - */ - def sendNewCredentialsToExecutors(credentials: SerializableBuffer): Unit = { - executorDataMap.values.foreach{ x => - x.executorActor ! UpdateCredentials(credentials) - } - } - def receiveWithLogging = { case RegisterExecutor(executorId, hostPort, cores, logUrls) => Utils.checkHostPort(hostPort, "Host port expected " + hostPort) @@ -247,14 +246,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste } } - val driver = new DriverActor(properties) // TODO (prashant) send conf instead of properties driverActor = actorSystem.actorOf( - Props(driver), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) + Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) // If a principal and keytab have been set, use that to create new credentials for executors // periodically - SparkHadoopUtil.get.scheduleLoginFromKeytab(driver.sendNewCredentialsToExecutors _) + SparkHadoopUtil.get.scheduleLoginFromKeytab(sendNewCredentialsToExecutors _) } def stopExecutors() { From d79b2b98532b1b5133026fe095104c5dc5f52af9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 24 Feb 2015 11:36:00 -0800 Subject: [PATCH 09/41] Make sure correct credentials are passed to FileSystem#addDelegationTokens() --- .../apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 4df07e8b2a1ce..368560a3803f4 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.yarn import java.io._ import java.net.URI import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean} import java.util.concurrent.{TimeUnit, ThreadFactory, Executors} import java.util.regex.Matcher import java.util.regex.Pattern @@ -52,6 +52,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { private var keytabFile: Option[String] = None private var loginPrincipal: Option[String] = None private val loggedInViaKeytab = new AtomicBoolean(false) + private val loggedInUGI = new AtomicReference[UserGroupInformation](null) override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) @@ -131,11 +132,12 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { }).scheduleWithFixedDelay(new Runnable { override def run(): Unit = { if (!loggedInViaKeytab.get()) { - loginUserFromKeytab(principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab) + loggedInUGI.set(UserGroupInformation.loginUserFromKeytabAndReturnUGI( + principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab)) loggedInViaKeytab.set(true) } val nns = getNameNodesToAccess(sparkConf) + remoteKeytabPath - val newCredentials = new Credentials() + val newCredentials = loggedInUGI.get().getCredentials obtainTokensForNamenodes(nns, conf, newCredentials) // Now write this out via Akka to executors. val outputStream = new ByteArrayOutputStream() From 0985b4e2fb1c51247eb993c28718c9b86f242563 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 27 Feb 2015 14:00:29 -0800 Subject: [PATCH 10/41] Write tokens to HDFS and read them back when required, rather than sending them over the wire. --- bin/utils.sh | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 4 +- .../org/apache/spark/deploy/SparkSubmit.scala | 2 + .../spark/deploy/SparkSubmitArguments.scala | 10 ++ .../CoarseGrainedExecutorBackend.scala | 11 +- .../cluster/CoarseGrainedClusterMessage.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 9 +- .../org/apache/spark/deploy/yarn/Client.scala | 35 +++--- .../deploy/yarn/YarnSparkHadoopUtil.scala | 110 +++++++++--------- 9 files changed, 100 insertions(+), 85 deletions(-) diff --git a/bin/utils.sh b/bin/utils.sh index 748dbe345a74c..f243c45a10805 100755 --- a/bin/utils.sh +++ b/bin/utils.sh @@ -36,7 +36,7 @@ function gatherSparkSubmitOpts() { --conf | --repositories | --properties-file | --driver-memory | --driver-java-options | \ --driver-library-path | --driver-class-path | --executor-memory | --driver-cores | \ --total-executor-cores | --executor-cores | --queue | --num-executors | --archives | \ - --proxy-user) + --proxy-user | --principal | --keytab) if [[ $# -lt 2 ]]; then "$SUBMIT_USAGE_FUNCTION" exit 1; diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 7476f1d52c4e3..c67593f1196d7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -122,9 +122,9 @@ class SparkHadoopUtil extends Logging { UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } - def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit = {} + def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit = ??? - private[spark] def scheduleLoginFromKeytab(callback: (SerializableBuffer) => Unit): Unit = {} + private[spark] def scheduleLoginFromKeytab(callback: (String) => Unit): Unit = {} /** * Returns a function that can be called to find Hadoop FileSystem bytes read. If diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 4c4110812e0a1..fc7f9d6f61021 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -372,6 +372,8 @@ object SparkSubmit { OptionAssigner(args.files, YARN, CLUSTER, clOption = "--files"), OptionAssigner(args.archives, YARN, CLUSTER, clOption = "--archives"), OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), + OptionAssigner(args.principal, YARN, CLUSTER, clOption = "--principal"), + OptionAssigner(args.keytab, YARN, CLUSTER, clOption = "--keytab"), // Other options OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 82e66a374249c..f413b51283a29 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -58,6 +58,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St var action: SparkSubmitAction = null val sparkProperties: HashMap[String, String] = new HashMap[String, String]() var proxyUser: String = null + var principal: String = null + var keytab: String = null // Standalone cluster mode only var supervise: Boolean = false @@ -410,6 +412,14 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St proxyUser = value parse(tail) + case ("--principal") :: value :: tail => + principal = value + parse(tail) + + case ("--keytab") :: value :: tail => + keytab = value + parse(tail) + case ("--help" | "-h") :: tail => printUsageAndExit(0) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 5f0309d605761..e42d6f7a06faa 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -17,8 +17,8 @@ package org.apache.spark.executor -import java.net.URL import java.io.{ByteArrayInputStream, DataInputStream} +import java.net.URL import java.nio.ByteBuffer import scala.collection.mutable @@ -27,6 +27,7 @@ import scala.concurrent.Await import akka.actor.{Actor, ActorSelection, Props} import akka.pattern.Patterns import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} +import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.security.Credentials import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} @@ -109,12 +110,14 @@ private[spark] class CoarseGrainedExecutorBackend( context.system.shutdown() // Add new credentials received from the driver to the current user. - case UpdateCredentials(newCredentials) => + case UpdateCredentials(newCredentialsPath) => logInfo("New credentials received from driver, adding the credentials to the current user") val credentials = new Credentials() - credentials.readTokenStorageStream( - new DataInputStream(new ByteArrayInputStream(newCredentials.value.array()))) + val remoteFs = FileSystem.get(SparkHadoopUtil.get.conf) + val inStream = remoteFs.open(new Path(newCredentialsPath)) + credentials.readTokenStorageStream(inStream) SparkHadoopUtil.get.addCurrentUserCredentials(credentials) + inStream.close() } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 214a4865e5f26..24b04992d40e3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -53,7 +53,7 @@ private[spark] object CoarseGrainedClusterMessages { // When the delegation tokens are about expire, the driver creates new tokens and sends them to // the executors via this message. - case class UpdateCredentials(newCredentials: SerializableBuffer) + case class UpdateCredentials(newCredentialsLocation: String) extends CoarseGrainedClusterMessage object StatusUpdate { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index bc101a0342e69..d423d81127005 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,8 +27,9 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.deploy.SparkHadoopUtil + import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} @@ -75,11 +76,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste /** * Send new credentials to executors. This is the method that is called when the scheduled * login completes, so the new credentials can be sent to the executors. - * @param credentials + * @param credentialsPath */ - def sendNewCredentialsToExecutors(credentials: SerializableBuffer): Unit = { + def sendNewCredentialsToExecutors(credentialsPath: String): Unit = { // We don't care about the reply, so going to deadLetters is fine. - executorDataMap.values.foreach(_.executorActor ! UpdateCredentials(credentials)) + executorDataMap.values.foreach(_.executorActor ! UpdateCredentials(credentialsPath)) } class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 73e7f797c8f70..7337b9a336fe0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -562,25 +562,22 @@ private[spark] class Client( } def setupCredentials(): Unit = { - Option(args.principal) match { - case Some(principal) => - Option(args.keytab) match { - case Some(keytabPath) => - // Generate a file name that can be used for the keytab file, that does not conflict - // with any user file. - logInfo("Attempting to login to the Kerberos" + - s" using principal: $principal and keytab: $keytabPath") - val f = new File(keytabPath) - keytabFileName = f.getName + "-" + System.currentTimeMillis() - val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytabPath) - credentials = ugi.getCredentials - loginFromKeytab = true - logInfo("Successfully logged into Kerberos.") - case None => - throw new SparkException("Keytab must be specified when principal is specified.") - } - case None => - credentials = UserGroupInformation.getCurrentUser.getCredentials + if (args.principal != null) { + if (args.keytab == null) { + throw new SparkException("Keytab must be specified when principal is specified.") + } + logInfo("Attempting to login to the Kerberos" + + s" using principal: ${args.principal} and keytab: ${args.keytab}") + val f = new File(args.keytab) + // Generate a file name that can be used for the keytab file, that does not conflict + // with any user file. + keytabFileName = f.getName + "-" + System.currentTimeMillis() + val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(args.principal, args.keytab) + credentials = ugi.getCredentials + loginFromKeytab = true + logInfo("Successfully logged into Kerberos.") + } else { + credentials = UserGroupInformation.getCurrentUser.getCredentials } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 368560a3803f4..0b2d989299d0f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -20,8 +20,7 @@ package org.apache.spark.deploy.yarn import java.io._ import java.net.URI import java.nio.ByteBuffer -import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean} -import java.util.concurrent.{TimeUnit, ThreadFactory, Executors} +import java.util.concurrent.{TimeUnit, Executors} import java.util.regex.Matcher import java.util.regex.Pattern @@ -49,10 +48,10 @@ import org.apache.spark.util.{SerializableBuffer, Utils} */ class YarnSparkHadoopUtil extends SparkHadoopUtil { - private var keytabFile: Option[String] = None - private var loginPrincipal: Option[String] = None - private val loggedInViaKeytab = new AtomicBoolean(false) - private val loggedInUGI = new AtomicReference[UserGroupInformation](null) + private var keytab: String = null + private var principal: String = null + @volatile private var loggedInViaKeytab = false + @volatile private var loggedInUGI: UserGroupInformation = null override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) @@ -94,58 +93,61 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } override def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit = { - loginPrincipal = Option(principal) - keytabFile = Option(keytab) + this.principal = principal + this.keytab = keytab } private[spark] override def scheduleLoginFromKeytab( - callback: (SerializableBuffer) => Unit): Unit = { - - loginPrincipal match { - case Some(principal) => - val keytab = keytabFile.get - val remoteFs = FileSystem.get(conf) - val remoteKeytabPath = new Path( - remoteFs.getHomeDirectory, System.getenv("SPARK_STAGING_DIR") + Path.SEPARATOR + keytab) - val localFS = FileSystem.getLocal(conf) - // At this point, SparkEnv is likely no initialized, so create a dir, put the keytab there. - val tempDir = Utils.createTempDir() - val localURI = new URI(tempDir.getAbsolutePath + Path.SEPARATOR + keytab) - val qualifiedURI = new URI(localFS.makeQualified(new Path(localURI)).toString) - FileUtil.copy( - remoteFs, remoteKeytabPath, localFS, new Path(qualifiedURI), false, false, conf) - // Get the current credentials, find out when they expire. - val creds = UserGroupInformation.getCurrentUser.getCredentials - val credStream = new ByteArrayOutputStream() - creds.writeTokenStorageToStream(new DataOutputStream(credStream)) - val in = new DataInputStream(new ByteArrayInputStream(credStream.toByteArray)) - val tokenIdentifier = new DelegationTokenIdentifier() - tokenIdentifier.readFields(in) - val timeToRenewal = (0.6 * (tokenIdentifier.getMaxDate - System.currentTimeMillis())).toLong - Executors.newSingleThreadScheduledExecutor(new ThreadFactory { - override def newThread(r: Runnable): Thread = { - val t = new Thread(r) - t.setName("Delegation Token Refresh Thread") - t.setDaemon(true) - t - } - }).scheduleWithFixedDelay(new Runnable { - override def run(): Unit = { - if (!loggedInViaKeytab.get()) { - loggedInUGI.set(UserGroupInformation.loginUserFromKeytabAndReturnUGI( - principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab)) - loggedInViaKeytab.set(true) + callback: (String) => Unit): Unit = { + if (principal != null) { + val stagingDir = System.getenv("SPARK_YARN_STAGING_DIR") + val remoteFs = FileSystem.get(conf) + val remoteKeytabPath = new Path( + remoteFs.getHomeDirectory, stagingDir + Path.SEPARATOR + keytab) + val localFS = FileSystem.getLocal(conf) + // At this point, SparkEnv is likely no initialized, so create a dir, put the keytab there. + val tempDir = Utils.createTempDir() + Utils.chmod700(tempDir) + val localURI = new URI(tempDir.getAbsolutePath + Path.SEPARATOR + keytab) + val qualifiedURI = new URI(localFS.makeQualified(new Path(localURI)).toString) + FileUtil.copy( + remoteFs, remoteKeytabPath, localFS, new Path(qualifiedURI), false, false, conf) + // Get the current credentials, find out when they expire. + val creds = { + if (loggedInUGI == null) { + UserGroupInformation.getCurrentUser.getCredentials + } else { + loggedInUGI.getCredentials + } + } + val credStream = new ByteArrayOutputStream() + creds.writeTokenStorageToStream(new DataOutputStream(credStream)) + val in = new DataInputStream(new ByteArrayInputStream(credStream.toByteArray)) + val tokenIdentifier = new DelegationTokenIdentifier() + tokenIdentifier.readFields(in) + val timeToRenewal = (0.6 * (tokenIdentifier.getMaxDate - System.currentTimeMillis())).toLong + Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("Delegation Token Refresh Thread")).scheduleWithFixedDelay( + new Runnable { + override def run(): Unit = { + if (!loggedInViaKeytab) { + loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( + principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab) + loggedInViaKeytab = true + } + val nns = getNameNodesToAccess(sparkConf) + remoteKeytabPath + val newCredentials = loggedInUGI.getCredentials + obtainTokensForNamenodes(nns, conf, newCredentials) + val tokenPath = new Path(remoteFs.getHomeDirectory, stagingDir + Path.SEPARATOR + + "credentials - " + System.currentTimeMillis()) + val stream = remoteFs.create(tokenPath, true) + // Now write this out via Akka to executors. + newCredentials.writeTokenStorageToStream(stream) + stream.hflush() + stream.close() + callback(tokenPath.toString) } - val nns = getNameNodesToAccess(sparkConf) + remoteKeytabPath - val newCredentials = loggedInUGI.get().getCredentials - obtainTokensForNamenodes(nns, conf, newCredentials) - // Now write this out via Akka to executors. - val outputStream = new ByteArrayOutputStream() - newCredentials.writeTokenStorageToStream(new DataOutputStream(outputStream)) - callback(new SerializableBuffer(ByteBuffer.wrap(outputStream.toByteArray))) - } - }, timeToRenewal, timeToRenewal, TimeUnit.MILLISECONDS) - case None => + }, timeToRenewal, timeToRenewal, TimeUnit.MILLISECONDS) } } From b4cb917d8ed5e06b3470f43ec221dd7ecdba7ec8 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 27 Feb 2015 16:04:07 -0800 Subject: [PATCH 11/41] Send keytab to AM via DistributedCache rather than directly via HDFS --- .../org/apache/spark/deploy/yarn/Client.scala | 56 +++++++------------ .../deploy/yarn/YarnSparkHadoopUtil.scala | 20 ++----- 2 files changed, 26 insertions(+), 50 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 7337b9a336fe0..def629a15cc55 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -21,6 +21,7 @@ import java.io.File import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer import java.nio.file.Files +import java.util.UUID import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} @@ -222,8 +223,10 @@ private[spark] class Client( // 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 - obtainTokensForNamenodes(nns, hadoopConf, credentials) + val nns = + SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil].getNameNodesToAccess(sparkConf) + dst + SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil]. + obtainTokensForNamenodes(nns, hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", fs.getDefaultReplication(dst)).toShort @@ -240,6 +243,20 @@ private[spark] class Client( "for alternatives.") } + // If we passed in a keytab, make sure we copy the keytab to the staging directory on + // HDFS, and setup the relevant environment vars, so the AM can login again. + if (loginFromKeytab) { + val fs = FileSystem.get(hadoopConf) + val stagingDirPath = new Path(fs.getHomeDirectory, appStagingDir) + val localUri = new URI(args.keytab) + val localPath = getQualifiedLocalPath(localUri, hadoopConf) + val destinationPath = new Path(stagingDirPath, keytabFileName) + copyFileToRemote(destinationPath, localPath, replication) + distCacheMgr.addResource( + fs, hadoopConf, destinationPath, localResources, LocalResourceType.FILE, keytabFileName, + statCache, appMasterOnly = true) + } + /** * 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. @@ -320,22 +337,11 @@ private[spark] class Client( env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() - // If we logged in from keytab, make sure we copy the keytab to the staging directory on - // HDFS, and setup the relevant environment vars, so the AM can login again. if (loginFromKeytab) { - val fs = FileSystem.get(hadoopConf) - val stagingDirPath = new Path(fs.getHomeDirectory, stagingDir) - val localUri = new URI(args.keytab) - val localPath = getQualifiedLocalPath(localUri, hadoopConf) - val destinationPath = new Path(stagingDirPath, keytabFileName) - val replication = sparkConf.getInt("spark.yarn.submit.file.replication", - fs.getDefaultReplication(destinationPath)).toShort - copyFileToRemote(destinationPath, localPath, replication) env("SPARK_PRINCIPAL") = args.principal env("SPARK_KEYTAB") = keytabFileName } - // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) @@ -571,7 +577,7 @@ private[spark] class Client( val f = new File(args.keytab) // Generate a file name that can be used for the keytab file, that does not conflict // with any user file. - keytabFileName = f.getName + "-" + System.currentTimeMillis() + keytabFileName = f.getName + "-" + UUID.randomUUID() val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(args.principal, args.keytab) credentials = ugi.getCredentials loginFromKeytab = true @@ -903,28 +909,6 @@ object Client extends Logging { private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit = YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path) - /** - * Get the list of namenodes the user may access. - */ - private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { - SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil].getNameNodesToAccess(sparkConf) - } - - private[yarn] def getTokenRenewer(conf: Configuration): String = { - SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil].getTokenRenewer(conf) - } - - /** - * Obtains tokens for the namenodes passed in and adds them to the credentials. - */ - private def obtainTokensForNamenodes( - paths: Set[Path], - conf: Configuration, - creds: Credentials): Unit = { - SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil] - .obtainTokensForNamenodes(paths, conf, creds) - } - /** * Return whether the two file systems are the same. */ diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 0b2d989299d0f..97ed233bb2524 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -100,18 +100,6 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { private[spark] override def scheduleLoginFromKeytab( callback: (String) => Unit): Unit = { if (principal != null) { - val stagingDir = System.getenv("SPARK_YARN_STAGING_DIR") - val remoteFs = FileSystem.get(conf) - val remoteKeytabPath = new Path( - remoteFs.getHomeDirectory, stagingDir + Path.SEPARATOR + keytab) - val localFS = FileSystem.getLocal(conf) - // At this point, SparkEnv is likely no initialized, so create a dir, put the keytab there. - val tempDir = Utils.createTempDir() - Utils.chmod700(tempDir) - val localURI = new URI(tempDir.getAbsolutePath + Path.SEPARATOR + keytab) - val qualifiedURI = new URI(localFS.makeQualified(new Path(localURI)).toString) - FileUtil.copy( - remoteFs, remoteKeytabPath, localFS, new Path(qualifiedURI), false, false, conf) // Get the current credentials, find out when they expire. val creds = { if (loggedInUGI == null) { @@ -131,13 +119,17 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { new Runnable { override def run(): Unit = { if (!loggedInViaKeytab) { + // Keytab is copied by YARN to the working directory of the AM, so full path is + // not needed. loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( - principal, tempDir.getAbsolutePath + Path.SEPARATOR + keytab) + principal, keytab) loggedInViaKeytab = true } - val nns = getNameNodesToAccess(sparkConf) + remoteKeytabPath + val nns = getNameNodesToAccess(sparkConf) val newCredentials = loggedInUGI.getCredentials obtainTokensForNamenodes(nns, conf, newCredentials) + val remoteFs = FileSystem.get(conf) + val stagingDir = System.getenv("SPARK_YARN_STAGING_DIR") val tokenPath = new Path(remoteFs.getHomeDirectory, stagingDir + Path.SEPARATOR + "credentials - " + System.currentTimeMillis()) val stream = remoteFs.create(tokenPath, true) From 5c11c3e348fecdd070f5ab471314bce94bb4b66e Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 27 Feb 2015 22:28:39 -0800 Subject: [PATCH 12/41] Move tests to YarnSparkHadoopUtil to fix compile issues. --- .../spark/deploy/yarn/ClientSuite.scala | 51 --------------- .../yarn/YarnSparkHadoopUtilSuite.scala | 62 ++++++++++++++++++- 2 files changed, 61 insertions(+), 52 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 92f04b4b859b3..b2ea4c9bd65f9 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -148,57 +148,6 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { } } - test("check access nns empty") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "") - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set()) - } - - test("check access nns unset") { - val sparkConf = new SparkConf() - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set()) - } - - test("check access nns") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032") - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set(new Path("hdfs://nn1:8032"))) - } - - test("check access nns space") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032, ") - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set(new Path("hdfs://nn1:8032"))) - } - - test("check access two nns") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032,hdfs://nn2:8032") - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set(new Path("hdfs://nn1:8032"), new Path("hdfs://nn2:8032"))) - } - - test("check token renewer") { - val hadoopConf = new Configuration() - hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") - hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") - val renewer = Client.getTokenRenewer(hadoopConf) - renewer should be ("yarn/myrm:8032@SPARKTEST.COM") - } - - test("check token renewer default") { - val hadoopConf = new Configuration() - val caught = - intercept[SparkException] { - Client.getTokenRenewer(hadoopConf) - } - assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") - } - object Fixtures { val knownDefYarnAppCP: Seq[String] = diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index b5a2db8f6225c..b33aedfc5f5cc 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.deploy.yarn import java.io.{File, IOException} import com.google.common.io.{ByteStreams, Files} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.conf.YarnConfiguration @@ -27,7 +29,7 @@ import org.scalatest.{FunSuite, Matchers} import org.apache.hadoop.yarn.api.records.ApplicationAccessType -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} import org.apache.spark.util.Utils @@ -173,4 +175,62 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { YarnSparkHadoopUtil.getClassPathSeparator() should be (":") } } + + test("check access nns empty") { + val sparkConf = new SparkConf() + val util = new YarnSparkHadoopUtil + sparkConf.set("spark.yarn.access.namenodes", "") + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set()) + } + + test("check access nns unset") { + val sparkConf = new SparkConf() + val util = new YarnSparkHadoopUtil + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set()) + } + + test("check access nns") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032") + val util = new YarnSparkHadoopUtil + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"))) + } + + test("check access nns space") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032, ") + val util = new YarnSparkHadoopUtil + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"))) + } + + test("check access two nns") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032,hdfs://nn2:8032") + val util = new YarnSparkHadoopUtil + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"), new Path("hdfs://nn2:8032"))) + } + + test("check token renewer") { + val hadoopConf = new Configuration() + hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") + hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") + val util = new YarnSparkHadoopUtil + val renewer = util.getTokenRenewer(hadoopConf) + renewer should be ("yarn/myrm:8032@SPARKTEST.COM") + } + + test("check token renewer default") { + val hadoopConf = new Configuration() + val util = new YarnSparkHadoopUtil + val caught = + intercept[SparkException] { + util.getTokenRenewer(hadoopConf) + } + assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") + } } From f6954dab2c1d7ebc614093bbde80ae1ae59bf97e Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 5 Mar 2015 12:30:06 -0800 Subject: [PATCH 13/41] Got rid of Akka communication to renew, instead the executors check a known file's modification time to read the credentials. --- .../apache/spark/deploy/SparkHadoopUtil.scala | 13 +- .../CoarseGrainedExecutorBackend.scala | 9 - .../cluster/CoarseGrainedClusterMessage.scala | 5 - .../CoarseGrainedSchedulerBackend.scala | 12 +- .../spark/deploy/yarn/ApplicationMaster.scala | 5 - .../org/apache/spark/deploy/yarn/Client.scala | 12 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 166 +++++++++++++----- 7 files changed, 136 insertions(+), 86 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index c67593f1196d7..d6b4a9257ee82 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -55,6 +55,7 @@ class SparkHadoopUtil extends Logging { def runAsSparkUser(func: () => Unit) { val user = Utils.getCurrentUserName() logDebug("running as user: " + user) + updateCredentialsIfRequired() val ugi = UserGroupInformation.createRemoteUser(user) transferCredentials(UserGroupInformation.getCurrentUser(), ugi) ugi.doAs(new PrivilegedExceptionAction[Unit] { @@ -62,6 +63,8 @@ class SparkHadoopUtil extends Logging { }) } + def updateCredentialsIfRequired(): Unit = {} + def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { for (token <- source.getTokens()) { dest.addToken(token) @@ -122,9 +125,13 @@ class SparkHadoopUtil extends Logging { UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } - def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit = ??? - - private[spark] def scheduleLoginFromKeytab(callback: (String) => Unit): Unit = {} + /** + * Schedule a login from the keytab and principal set using the --principal and --keytab + * arguments to spark-submit. This login happens only when the credentials of the current user + * are about to expire. This method reads SPARK_PRINCIPAL and SPARK_KEYTAB from the environment + * to do the login. This method is a no-op in non-YARN mode. + */ + private[spark] def scheduleLoginFromKeytab(): Unit = {} /** * Returns a function that can be called to find Hadoop FileSystem bytes read. If diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e42d6f7a06faa..879307a2f0e22 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -109,15 +109,6 @@ private[spark] class CoarseGrainedExecutorBackend( context.stop(self) context.system.shutdown() - // Add new credentials received from the driver to the current user. - case UpdateCredentials(newCredentialsPath) => - logInfo("New credentials received from driver, adding the credentials to the current user") - val credentials = new Credentials() - val remoteFs = FileSystem.get(SparkHadoopUtil.get.conf) - val inStream = remoteFs.open(new Path(newCredentialsPath)) - credentials.readTokenStorageStream(inStream) - SparkHadoopUtil.get.addCurrentUserCredentials(credentials) - inStream.close() } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 24b04992d40e3..9bf74f4be198d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -51,11 +51,6 @@ private[spark] object CoarseGrainedClusterMessages { case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer) extends CoarseGrainedClusterMessage - // When the delegation tokens are about expire, the driver creates new tokens and sends them to - // the executors via this message. - case class UpdateCredentials(newCredentialsLocation: String) - extends CoarseGrainedClusterMessage - object StatusUpdate { /** Alternate factory method that takes a ByteBuffer directly for the data field */ def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index d423d81127005..d7714a6185663 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -73,16 +73,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Executors we have requested the cluster manager to kill that have not died yet private val executorsPendingToRemove = new HashSet[String] - /** - * Send new credentials to executors. This is the method that is called when the scheduled - * login completes, so the new credentials can be sent to the executors. - * @param credentialsPath - */ - def sendNewCredentialsToExecutors(credentialsPath: String): Unit = { - // We don't care about the reply, so going to deadLetters is fine. - executorDataMap.values.foreach(_.executorActor ! UpdateCredentials(credentialsPath)) - } - class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { override protected def log = CoarseGrainedSchedulerBackend.this.log private val addressToExecutorId = new HashMap[Address, String] @@ -253,7 +243,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // If a principal and keytab have been set, use that to create new credentials for executors // periodically - SparkHadoopUtil.get.scheduleLoginFromKeytab(sendNewCredentialsToExecutors _) + SparkHadoopUtil.get.scheduleLoginFromKeytab() } def stopExecutors() { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 9bdd0f9bcf59d..d6823ca729032 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -257,11 +257,6 @@ private[spark] class ApplicationMaster( private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() - // This must be done before SparkContext is initialized, since the CoarseGrainedSchedulerBackend - // is started at that time. That is what schedules the re-logins. It is scheduled only if the - // principal is actually setup. So we make sure it is available. - SparkHadoopUtil.get.setPrincipalAndKeytabForLogin( - System.getenv("SPARK_PRINCIPAL"), System.getenv("SPARK_KEYTAB")) userClassThread = startUserApplication() // This a bit hacky, but we need to wait until the spark.driver.port property has diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index def629a15cc55..d6581beb885ba 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -246,14 +246,12 @@ private[spark] class Client( // If we passed in a keytab, make sure we copy the keytab to the staging directory on // HDFS, and setup the relevant environment vars, so the AM can login again. if (loginFromKeytab) { - val fs = FileSystem.get(hadoopConf) - val stagingDirPath = new Path(fs.getHomeDirectory, appStagingDir) val localUri = new URI(args.keytab) val localPath = getQualifiedLocalPath(localUri, hadoopConf) - val destinationPath = new Path(stagingDirPath, keytabFileName) - copyFileToRemote(destinationPath, localPath, replication) + val destinationPath = copyFileToRemote(dst, localPath, replication) + val destFs = FileSystem.get(destinationPath.toUri(), hadoopConf) distCacheMgr.addResource( - fs, hadoopConf, destinationPath, localResources, LocalResourceType.FILE, keytabFileName, + destFs, hadoopConf, destinationPath, localResources, LocalResourceType.FILE, keytabFileName, statCache, appMasterOnly = true) } @@ -577,10 +575,12 @@ private[spark] class Client( val f = new File(args.keytab) // Generate a file name that can be used for the keytab file, that does not conflict // with any user file. - keytabFileName = f.getName + "-" + UUID.randomUUID() + keytabFileName = f.getName + "-" + UUID.randomUUID().toString val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(args.principal, args.keytab) credentials = ugi.getCredentials loginFromKeytab = true + val credentialsFile = "credentials-" + UUID.randomUUID().toString + sparkConf.set("spark.yarn.credentials.file", credentialsFile) logInfo("Successfully logged into Kerberos.") } else { credentials = UserGroupInformation.getCurrentUser.getCredentials diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 97ed233bb2524..5ec4c1e410ee6 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -20,14 +20,17 @@ package org.apache.spark.deploy.yarn import java.io._ import java.net.URI import java.nio.ByteBuffer -import java.util.concurrent.{TimeUnit, Executors} +import java.util.concurrent.{ TimeUnit, Executors} import java.util.regex.Matcher import java.util.regex.Pattern import scala.collection.mutable.HashMap +import scala.collection.JavaConversions._ import scala.util.Try +import org.apache.hadoop.fs.Options.Rename import org.apache.hadoop.fs.{FileUtil, Path, FileSystem} +import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.{Master, JobConf} @@ -41,7 +44,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkException, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.{SerializableBuffer, Utils} +import org.apache.spark.util.Utils /** * Contains util methods to interact with Hadoop from spark. @@ -52,6 +55,13 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { private var principal: String = null @volatile private var loggedInViaKeytab = false @volatile private var loggedInUGI: UserGroupInformation = null + @volatile private var lastCredentialsRefresh = 0l + private lazy val delegationTokenRenewer = + Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("Delegation Token Refresh Thread")) + private lazy val delegationTokenExecuterUpdaterThread = new Runnable { + override def run(): Unit = updateCredentialsIfRequired() + } override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) @@ -92,57 +102,118 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { if (credentials != null) credentials.getSecretKey(new Text(key)) else null } - override def setPrincipalAndKeytabForLogin(principal: String, keytab: String): Unit = { - this.principal = principal - this.keytab = keytab + private[spark] override def scheduleLoginFromKeytab(): Unit = { + val principal = System.getenv("SPARK_PRINCIPAL") + val keytab = System.getenv("SPARK_KEYTAB") + if (principal != null) { + val delegationTokenRenewerThread = + new Runnable { + override def run(): Unit = { + if (!loggedInViaKeytab) { + // Keytab is copied by YARN to the working directory of the AM, so full path is + // not needed. + loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( + principal, keytab) + loggedInViaKeytab = true + } + val nns = getNameNodesToAccess(sparkConf) + val newCredentials = loggedInUGI.getCredentials + obtainTokensForNamenodes(nns, conf, newCredentials) + val remoteFs = FileSystem.get(conf) + val stagingDirPath = + new Path(remoteFs.getHomeDirectory, System.getenv("SPARK_YARN_STAGING_DIR")) + val tokenPathStr = sparkConf.get("spark.yarn.credentials.file") + val tokenPath = new Path(stagingDirPath.toString, tokenPathStr) + val tempTokenPath = new Path(stagingDirPath.toString, tokenPathStr + ".tmp") + val stream = remoteFs.create(tempTokenPath, true) + // Now write this out to HDFS + newCredentials.writeTokenStorageToStream(stream) + stream.hflush() + stream.close() + remoteFs.delete(tokenPath, true) + remoteFs.rename(tempTokenPath, tokenPath) + delegationTokenRenewer.schedule( + this, (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong, + TimeUnit.MILLISECONDS) + } + } + val timeToRenewal = (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong + delegationTokenRenewer.schedule( + delegationTokenRenewerThread, timeToRenewal, TimeUnit.MILLISECONDS) + } } - private[spark] override def scheduleLoginFromKeytab( - callback: (String) => Unit): Unit = { - if (principal != null) { - // Get the current credentials, find out when they expire. - val creds = { - if (loggedInUGI == null) { - UserGroupInformation.getCurrentUser.getCredentials - } else { - loggedInUGI.getCredentials + override def updateCredentialsIfRequired(): Unit = { + try { + val credentialsFile = sparkConf.get("spark.yarn.credentials.file") + if (credentialsFile != null && !credentialsFile.isEmpty) { + val remoteFs = FileSystem.get(conf) + val sparkStagingDir = System.getenv("SPARK_YARN_STAGING_DIR") + val stagingDirPath = new Path(remoteFs.getHomeDirectory, sparkStagingDir) + val credentialsFilePath = new Path(stagingDirPath, credentialsFile) + if (remoteFs.exists(credentialsFilePath)) { + val status = remoteFs.getFileStatus(credentialsFilePath) + val modTimeAtStart = status.getModificationTime + if (modTimeAtStart > lastCredentialsRefresh) { + val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsFilePath) + val newStatus = remoteFs.getFileStatus(credentialsFilePath) + // File was updated after we started reading it, lets come back later and try to read it. + if (newStatus.getModificationTime != modTimeAtStart) { + delegationTokenRenewer + .schedule(delegationTokenExecuterUpdaterThread, 1, TimeUnit.HOURS) + } else { + UserGroupInformation.getCurrentUser.addCredentials(newCredentials) + lastCredentialsRefresh = status.getModificationTime + val totalValidity = getLatestValidity - lastCredentialsRefresh + val timeToRunRenewal = lastCredentialsRefresh + (0.8 * totalValidity).toLong + val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() + delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterThread, + timeFromNowToRenewal, TimeUnit.MILLISECONDS) + } + } else { + // Check every hour to see if new credentials arrived. + delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterThread, 1, TimeUnit.HOURS) + } } } - val credStream = new ByteArrayOutputStream() - creds.writeTokenStorageToStream(new DataOutputStream(credStream)) - val in = new DataInputStream(new ByteArrayInputStream(credStream.toByteArray)) - val tokenIdentifier = new DelegationTokenIdentifier() - tokenIdentifier.readFields(in) - val timeToRenewal = (0.6 * (tokenIdentifier.getMaxDate - System.currentTimeMillis())).toLong - Executors.newSingleThreadScheduledExecutor( - Utils.namedThreadFactory("Delegation Token Refresh Thread")).scheduleWithFixedDelay( - new Runnable { - override def run(): Unit = { - if (!loggedInViaKeytab) { - // Keytab is copied by YARN to the working directory of the AM, so full path is - // not needed. - loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( - principal, keytab) - loggedInViaKeytab = true - } - val nns = getNameNodesToAccess(sparkConf) - val newCredentials = loggedInUGI.getCredentials - obtainTokensForNamenodes(nns, conf, newCredentials) - val remoteFs = FileSystem.get(conf) - val stagingDir = System.getenv("SPARK_YARN_STAGING_DIR") - val tokenPath = new Path(remoteFs.getHomeDirectory, stagingDir + Path.SEPARATOR + - "credentials - " + System.currentTimeMillis()) - val stream = remoteFs.create(tokenPath, true) - // Now write this out via Akka to executors. - newCredentials.writeTokenStorageToStream(stream) - stream.hflush() - stream.close() - callback(tokenPath.toString) - } - }, timeToRenewal, timeToRenewal, TimeUnit.MILLISECONDS) + } catch { + // Since the file may get deleted while we are reading it, + case e: Exception => + logWarning( + "Error encountered while trying to update credentials, will try again in 1 hour", e) + delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterThread, 1, TimeUnit.HOURS) } } + private[spark] def getCredentialsFromHDFSFile( + remoteFs: FileSystem, + tokenPath: Path + ): Credentials = { + val stream = remoteFs.open(tokenPath) + val newCredentials = new Credentials() + newCredentials.readFields(stream) + newCredentials + } + + private[spark] def getLatestValidity: Long = { + val creds = UserGroupInformation.getCurrentUser.getCredentials + var latestValidity: Long = 0 + creds.getAllTokens + .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) + .foreach { t => + val identifier = new DelegationTokenIdentifier() + identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) + latestValidity = { + if (latestValidity < identifier.getMaxDate) { + identifier.getMaxDate + } else { + latestValidity + } + } + } + latestValidity + } + /** * Get the list of namenodes the user may access. */ @@ -172,7 +243,8 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { def obtainTokensForNamenodes( paths: Set[Path], conf: Configuration, - creds: Credentials): Unit = { + creds: Credentials + ): Unit = { if (UserGroupInformation.isSecurityEnabled()) { val delegTokenRenewer = getTokenRenewer(conf) paths.foreach { dst => From f0f54cba1e579a0ee320dd27d0683ff4cd458cc4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 5 Mar 2015 13:19:09 -0800 Subject: [PATCH 14/41] Be more defensive when updating the credentials file. --- .../deploy/yarn/YarnSparkHadoopUtil.scala | 29 ++++++++++++++++--- 1 file changed, 25 insertions(+), 4 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 5ec4c1e410ee6..da84150b3ae26 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -55,7 +55,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { private var principal: String = null @volatile private var loggedInViaKeytab = false @volatile private var loggedInUGI: UserGroupInformation = null - @volatile private var lastCredentialsRefresh = 0l + @volatile private var lastCredentialsRefresh = 0L private lazy val delegationTokenRenewer = Executors.newSingleThreadScheduledExecutor( Utils.namedThreadFactory("Delegation Token Refresh Thread")) @@ -130,8 +130,22 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { newCredentials.writeTokenStorageToStream(stream) stream.hflush() stream.close() - remoteFs.delete(tokenPath, true) + // HDFS does reads by inodes now, so just doing a rename should be fine. But I could + // not find a clear explanation of when the blocks on HDFS are deleted. Ideally, we + // would not need this, but just be defensive to ensure we don't mess up the + // credentials. So create a file to show that we are currently updating - if the + // reader sees this file, they go away and come back later. Then delete old token and + // rename the old to new. + val updatingPath = new Path(stagingDirPath, "_UPDATING") + if (remoteFs.exists(updatingPath)) { + remoteFs.delete(updatingPath, true) + } + remoteFs.create(updatingPath).close() + if (remoteFs.exists(tokenPath)) { + remoteFs.delete(tokenPath, true) + } remoteFs.rename(tempTokenPath, tokenPath) + remoteFs.delete(updatingPath, true) delegationTokenRenewer.schedule( this, (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong, TimeUnit.MILLISECONDS) @@ -151,13 +165,19 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { val sparkStagingDir = System.getenv("SPARK_YARN_STAGING_DIR") val stagingDirPath = new Path(remoteFs.getHomeDirectory, sparkStagingDir) val credentialsFilePath = new Path(stagingDirPath, credentialsFile) + // If an update is currently in progress, come back later! + if (remoteFs.exists( new Path(stagingDirPath, "_UPDATING"))) { + delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterThread, 1, TimeUnit.HOURS) + } + // Now check if the file exists, if it does go get the credentials from there if (remoteFs.exists(credentialsFilePath)) { val status = remoteFs.getFileStatus(credentialsFilePath) val modTimeAtStart = status.getModificationTime if (modTimeAtStart > lastCredentialsRefresh) { val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsFilePath) val newStatus = remoteFs.getFileStatus(credentialsFilePath) - // File was updated after we started reading it, lets come back later and try to read it. + // File was updated after we started reading it, lets come back later and try to read + // it. if (newStatus.getModificationTime != modTimeAtStart) { delegationTokenRenewer .schedule(delegationTokenExecuterUpdaterThread, 1, TimeUnit.HOURS) @@ -177,7 +197,8 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } } } catch { - // Since the file may get deleted while we are reading it, + // Since the file may get deleted while we are reading it, catch the Exception and come + // back in an hour to try again case e: Exception => logWarning( "Error encountered while trying to update credentials, will try again in 1 hour", e) From af6d5f0b2ca70f46507c98f1c930b21591a35e59 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 5 Mar 2015 15:41:22 -0800 Subject: [PATCH 15/41] Cleaning up files where changes weren't required. --- .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 4 ---- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 1 - yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 4 +--- 3 files changed, 1 insertion(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 879307a2f0e22..dd19e4947db1e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -17,7 +17,6 @@ package org.apache.spark.executor -import java.io.{ByteArrayInputStream, DataInputStream} import java.net.URL import java.nio.ByteBuffer @@ -27,8 +26,6 @@ import scala.concurrent.Await import akka.actor.{Actor, ActorSelection, Props} import akka.pattern.Patterns import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} -import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.hadoop.security.Credentials import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState @@ -108,7 +105,6 @@ private[spark] class CoarseGrainedExecutorBackend( executor.stop() context.stop(self) context.system.shutdown() - } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index d6823ca729032..a9bf861d160c1 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -256,7 +256,6 @@ private[spark] class ApplicationMaster( private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() - userClassThread = startUserApplication() // This a bit hacky, but we need to wait until the spark.driver.port property has diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index d6581beb885ba..cab6f75343722 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -18,9 +18,8 @@ package org.apache.spark.deploy.yarn import java.io.File -import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} +import java.net.{InetAddress, UnknownHostException, URI} import java.nio.ByteBuffer -import java.nio.file.Files import java.util.UUID import scala.collection.JavaConversions._ @@ -33,7 +32,6 @@ import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.util.StringUtils From 2debcea367aa7e54d49af2109b47568ba61f829b Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 6 Mar 2015 00:49:38 -0800 Subject: [PATCH 16/41] Change the file structure for credentials files. I will push a followup patch which adds a cleanup mechanism for old credentials files. The credentials files are small and few enough for it to cause issues on HDFS. --- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 6 +- .../spark/deploy/SparkSubmitArguments.scala | 4 + .../CoarseGrainedSchedulerBackend.scala | 3 +- .../org/apache/spark/deploy/yarn/Client.scala | 34 ++-- .../deploy/yarn/YarnSparkHadoopUtil.scala | 185 +++++++++--------- 6 files changed, 122 insertions(+), 112 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index d6b4a9257ee82..9ca6e07b3ae0f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -40,7 +40,7 @@ import scala.collection.JavaConversions._ */ @DeveloperApi class SparkHadoopUtil extends Logging { - val sparkConf = new SparkConf() + protected val sparkConf = new SparkConf() // YarnSparkHadoopUtil requires this val conf: Configuration = newConfiguration(sparkConf) UserGroupInformation.setConfiguration(conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index fc7f9d6f61021..ba638b6c3b974 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -372,8 +372,10 @@ object SparkSubmit { OptionAssigner(args.files, YARN, CLUSTER, clOption = "--files"), OptionAssigner(args.archives, YARN, CLUSTER, clOption = "--archives"), OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), - OptionAssigner(args.principal, YARN, CLUSTER, clOption = "--principal"), - OptionAssigner(args.keytab, YARN, CLUSTER, clOption = "--keytab"), + + // Yarn client or cluster + OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, clOption = "--principal"), + OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, clOption = "--keytab"), // Other options OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index f413b51283a29..5eec6b6f02cc9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -517,6 +517,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | --num-executors NUM Number of executors to launch (Default: 2). | --archives ARCHIVES Comma separated list of archives to be extracted into the | working directory of each executor. + | --principal PRINCIPAL Principal to be used to login to KDC, while running on + | secure HDFS. + | --keytab KEYTAB The full path to the file that contains the keytab for the + | principal specified above. """.stripMargin ) SparkSubmit.exitFn() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index d7714a6185663..1eac2f4a32acf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,9 +27,8 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} - -import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index cab6f75343722..0ca5d6d28a544 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -24,9 +24,9 @@ import java.util.UUID import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} -import scala.util.{Random, Try, Success, Failure} +import scala.util.{Try, Success, Failure} -import com.google.common.base.Objects +import com.google.common.base.{Preconditions, Objects} import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.conf.Configuration @@ -43,8 +43,8 @@ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} 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.deploy.SparkHadoopUtil +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} import org.apache.spark.util.Utils private[spark] class Client( @@ -67,10 +67,7 @@ private[spark] class Client( private val executorMemoryOverhead = args.executorMemoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() private val isClusterMode = args.isClusterMode - private var loginFromKeytab = false - private var keytabFileName: String = null - def stop(): Unit = yarnClient.stop() @@ -221,10 +218,8 @@ private[spark] class Client( // and add them as local resources to the application master. val fs = FileSystem.get(hadoopConf) val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val nns = - SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil].getNameNodesToAccess(sparkConf) + dst - SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil]. - obtainTokensForNamenodes(nns, hadoopConf, credentials) + val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + dst + YarnSparkHadoopUtil.get.obtainTokensForNamenodes(nns, hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", fs.getDefaultReplication(dst)).toShort @@ -249,8 +244,8 @@ private[spark] class Client( val destinationPath = copyFileToRemote(dst, localPath, replication) val destFs = FileSystem.get(destinationPath.toUri(), hadoopConf) distCacheMgr.addResource( - destFs, hadoopConf, destinationPath, localResources, LocalResourceType.FILE, keytabFileName, - statCache, appMasterOnly = true) + destFs, hadoopConf, destinationPath, localResources, LocalResourceType.FILE, + sparkConf.get("spark.yarn.keytab"), statCache, appMasterOnly = true) } /** @@ -334,8 +329,11 @@ private[spark] class Client( env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() if (loginFromKeytab) { - env("SPARK_PRINCIPAL") = args.principal - env("SPARK_KEYTAB") = keytabFileName + val remoteFs = FileSystem.get(hadoopConf) + val stagingDirPath = new Path(remoteFs.getHomeDirectory, stagingDir) + val credentialsFile = "credentials-" + UUID.randomUUID().toString + sparkConf.set( + "spark.yarn.credentials.file", new Path(stagingDirPath, credentialsFile).toString) } // Set the environment variables to be passed on to the executors. @@ -573,13 +571,13 @@ private[spark] class Client( val f = new File(args.keytab) // Generate a file name that can be used for the keytab file, that does not conflict // with any user file. - keytabFileName = f.getName + "-" + UUID.randomUUID().toString + val keytabFileName = f.getName + "-" + UUID.randomUUID().toString val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(args.principal, args.keytab) credentials = ugi.getCredentials loginFromKeytab = true - val credentialsFile = "credentials-" + UUID.randomUUID().toString - sparkConf.set("spark.yarn.credentials.file", credentialsFile) - logInfo("Successfully logged into Kerberos.") + sparkConf.set("spark.yarn.keytab", keytabFileName) + sparkConf.set("spark.yarn.principal", args.principal) + logInfo("Successfully logged into the KDC.") } else { credentials = UserGroupInformation.getCurrentUser.getCredentials } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index da84150b3ae26..cab905cd4a5fd 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -18,9 +18,9 @@ package org.apache.spark.deploy.yarn import java.io._ -import java.net.URI -import java.nio.ByteBuffer -import java.util.concurrent.{ TimeUnit, Executors} +import java.util +import java.util.Comparator +import java.util.concurrent.{Executors, TimeUnit} import java.util.regex.Matcher import java.util.regex.Pattern @@ -28,9 +28,9 @@ import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ import scala.util.Try -import org.apache.hadoop.fs.Options.Rename -import org.apache.hadoop.fs.{FileUtil, Path, FileSystem} -import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.{Master, JobConf} @@ -40,10 +40,9 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.records.{Priority, ApplicationAccessType} -import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SparkException, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.util.Utils /** @@ -51,17 +50,16 @@ import org.apache.spark.util.Utils */ class YarnSparkHadoopUtil extends SparkHadoopUtil { - private var keytab: String = null - private var principal: String = null @volatile private var loggedInViaKeytab = false @volatile private var loggedInUGI: UserGroupInformation = null - @volatile private var lastCredentialsRefresh = 0L + private var lastCredentialsFileSuffix = 0 private lazy val delegationTokenRenewer = Executors.newSingleThreadScheduledExecutor( Utils.namedThreadFactory("Delegation Token Refresh Thread")) - private lazy val delegationTokenExecuterUpdaterThread = new Runnable { - override def run(): Unit = updateCredentialsIfRequired() - } + private lazy val delegationTokenExecuterUpdaterRunnable = + new Runnable { + override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired()) + } override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) @@ -103,49 +101,13 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } private[spark] override def scheduleLoginFromKeytab(): Unit = { - val principal = System.getenv("SPARK_PRINCIPAL") - val keytab = System.getenv("SPARK_KEYTAB") + val principal = sparkConf.get("spark.yarn.principal") + val keytab = sparkConf.get("spark.yarn.keytab") if (principal != null) { - val delegationTokenRenewerThread = + val delegationTokenRenewerRunnable = new Runnable { override def run(): Unit = { - if (!loggedInViaKeytab) { - // Keytab is copied by YARN to the working directory of the AM, so full path is - // not needed. - loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( - principal, keytab) - loggedInViaKeytab = true - } - val nns = getNameNodesToAccess(sparkConf) - val newCredentials = loggedInUGI.getCredentials - obtainTokensForNamenodes(nns, conf, newCredentials) - val remoteFs = FileSystem.get(conf) - val stagingDirPath = - new Path(remoteFs.getHomeDirectory, System.getenv("SPARK_YARN_STAGING_DIR")) - val tokenPathStr = sparkConf.get("spark.yarn.credentials.file") - val tokenPath = new Path(stagingDirPath.toString, tokenPathStr) - val tempTokenPath = new Path(stagingDirPath.toString, tokenPathStr + ".tmp") - val stream = remoteFs.create(tempTokenPath, true) - // Now write this out to HDFS - newCredentials.writeTokenStorageToStream(stream) - stream.hflush() - stream.close() - // HDFS does reads by inodes now, so just doing a rename should be fine. But I could - // not find a clear explanation of when the blocks on HDFS are deleted. Ideally, we - // would not need this, but just be defensive to ensure we don't mess up the - // credentials. So create a file to show that we are currently updating - if the - // reader sees this file, they go away and come back later. Then delete old token and - // rename the old to new. - val updatingPath = new Path(stagingDirPath, "_UPDATING") - if (remoteFs.exists(updatingPath)) { - remoteFs.delete(updatingPath, true) - } - remoteFs.create(updatingPath).close() - if (remoteFs.exists(tokenPath)) { - remoteFs.delete(tokenPath, true) - } - remoteFs.rename(tempTokenPath, tokenPath) - remoteFs.delete(updatingPath, true) + renewCredentials(principal, keytab) delegationTokenRenewer.schedule( this, (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong, TimeUnit.MILLISECONDS) @@ -153,47 +115,85 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } val timeToRenewal = (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong delegationTokenRenewer.schedule( - delegationTokenRenewerThread, timeToRenewal, TimeUnit.MILLISECONDS) + delegationTokenRenewerRunnable, timeToRenewal, TimeUnit.MILLISECONDS) + } + } + + private def renewCredentials(principal: String, keytab: String): Unit = { + if (!loggedInViaKeytab) { + // Keytab is copied by YARN to the working directory of the AM, so full path is + // not needed. + loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( + principal, keytab) + loggedInViaKeytab = true + } + val nns = getNameNodesToAccess(sparkConf) + val newCredentials = loggedInUGI.getCredentials + obtainTokensForNamenodes(nns, conf, newCredentials) + val remoteFs = FileSystem.get(conf) + val nextSuffix = lastCredentialsFileSuffix + 1 + val tokenPathStr = + sparkConf.get("spark.yarn.credentials.file") + "-" + nextSuffix + val tokenPath = new Path(tokenPathStr) + val tempTokenPath = new Path(tokenPathStr + ".tmp") + val stream = Option(remoteFs.create(tempTokenPath, true)) + try { + stream.foreach { s => + newCredentials.writeTokenStorageToStream(s) + s.hflush() + s.close() + remoteFs.rename(tempTokenPath, tokenPath) + } + } catch { + case e: Exception => + } finally { + stream.foreach(_.close()) } + + lastCredentialsFileSuffix = nextSuffix } override def updateCredentialsIfRequired(): Unit = { try { val credentialsFile = sparkConf.get("spark.yarn.credentials.file") if (credentialsFile != null && !credentialsFile.isEmpty) { + val credentialsFilePath = new Path(credentialsFile) val remoteFs = FileSystem.get(conf) - val sparkStagingDir = System.getenv("SPARK_YARN_STAGING_DIR") - val stagingDirPath = new Path(remoteFs.getHomeDirectory, sparkStagingDir) - val credentialsFilePath = new Path(stagingDirPath, credentialsFile) - // If an update is currently in progress, come back later! - if (remoteFs.exists( new Path(stagingDirPath, "_UPDATING"))) { - delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterThread, 1, TimeUnit.HOURS) - } - // Now check if the file exists, if it does go get the credentials from there - if (remoteFs.exists(credentialsFilePath)) { - val status = remoteFs.getFileStatus(credentialsFilePath) - val modTimeAtStart = status.getModificationTime - if (modTimeAtStart > lastCredentialsRefresh) { - val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsFilePath) - val newStatus = remoteFs.getFileStatus(credentialsFilePath) - // File was updated after we started reading it, lets come back later and try to read - // it. - if (newStatus.getModificationTime != modTimeAtStart) { - delegationTokenRenewer - .schedule(delegationTokenExecuterUpdaterThread, 1, TimeUnit.HOURS) + val stagingDirPath = new Path(remoteFs.getHomeDirectory, credentialsFilePath.getParent) + val fileStatuses = + remoteFs.listStatus(stagingDirPath, + new PathFilter { + override def accept(path: Path): Boolean = { + val name = path.getName + name.startsWith(credentialsFilePath.getName) && !name.endsWith(".tmp") + } + }) + util.Arrays.sort(fileStatuses, new Comparator[FileStatus] { + override def compare(o1: FileStatus, o2: FileStatus): Int = { + // can't return this directly, as it might cause int to overflow + val diff = o1.getModificationTime - o2.getModificationTime + if (diff < 0) { + -1 } else { - UserGroupInformation.getCurrentUser.addCredentials(newCredentials) - lastCredentialsRefresh = status.getModificationTime - val totalValidity = getLatestValidity - lastCredentialsRefresh - val timeToRunRenewal = lastCredentialsRefresh + (0.8 * totalValidity).toLong - val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() - delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterThread, - timeFromNowToRenewal, TimeUnit.MILLISECONDS) + 1 } - } else { - // Check every hour to see if new credentials arrived. - delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterThread, 1, TimeUnit.HOURS) } + }) + val credentialsStatus = fileStatuses(fileStatuses.length - 1) + val credentials = credentialsStatus.getPath + val suffix = credentials.getName.substring(credentials.getName.lastIndexOf("-") + 1).toInt + if (suffix > lastCredentialsFileSuffix) { + val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentials) + UserGroupInformation.getCurrentUser.addCredentials(newCredentials) + val totalValidity = getLatestValidity - credentialsStatus.getModificationTime + val timeToRunRenewal = + credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong + val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() + delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterRunnable, + timeFromNowToRenewal, TimeUnit.MILLISECONDS) + } else { + // Check every hour to see if new credentials arrived. + delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterRunnable, 1, TimeUnit.HOURS) } } } catch { @@ -202,14 +202,13 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { case e: Exception => logWarning( "Error encountered while trying to update credentials, will try again in 1 hour", e) - delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterThread, 1, TimeUnit.HOURS) + delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterRunnable, 1, TimeUnit.HOURS) } } private[spark] def getCredentialsFromHDFSFile( - remoteFs: FileSystem, - tokenPath: Path - ): Credentials = { + remoteFs: FileSystem, + tokenPath: Path): Credentials = { val stream = remoteFs.open(tokenPath) val newCredentials = new Credentials() newCredentials.readFields(stream) @@ -294,6 +293,14 @@ object YarnSparkHadoopUtil { // request types (like map/reduce in hadoop for example) val RM_REQUEST_PRIORITY = Priority.newInstance(1) + def get: YarnSparkHadoopUtil = { + val yarnMode = java.lang.Boolean.valueOf( + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) + if (!yarnMode) { + throw new SparkException("YarnSparkHadoopUtil is not available in non-YARN mode!") + } + SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil] + } /** * Add a path variable to the given environment map. * If the map already contains this key, append the value to the existing value instead. @@ -405,5 +412,5 @@ object YarnSparkHadoopUtil { def getClassPathSeparator(): String = { classPathSeparatorField.get(null).asInstanceOf[String] } - } + From f4fd711f44c24224178ca1ad9c7b4529bf62fa47 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 6 Mar 2015 11:26:11 -0800 Subject: [PATCH 17/41] Fix SparkConf usage. --- .../apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala | 8 +++----- .../org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 2 ++ 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index cab905cd4a5fd..92f9215571d04 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -101,9 +101,8 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } private[spark] override def scheduleLoginFromKeytab(): Unit = { - val principal = sparkConf.get("spark.yarn.principal") - val keytab = sparkConf.get("spark.yarn.keytab") - if (principal != null) { + sparkConf.getOption("spark.yarn.principal").foreach { principal => + val keytab = sparkConf.get("spark.yarn.keytab") val delegationTokenRenewerRunnable = new Runnable { override def run(): Unit = { @@ -155,8 +154,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { override def updateCredentialsIfRequired(): Unit = { try { - val credentialsFile = sparkConf.get("spark.yarn.credentials.file") - if (credentialsFile != null && !credentialsFile.isEmpty) { + sparkConf.getOption("spark.yarn.credentials.file").foreach { credentialsFile => val credentialsFilePath = new Path(credentialsFile) val remoteFs = FileSystem.get(conf) val stagingDirPath = new Path(remoteFs.getHomeDirectory, credentialsFilePath.getParent) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 0e37276ba724b..83e9b0b4a06e9 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -85,6 +85,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit tempDir = Utils.createTempDir() logConfDir = new File(tempDir, "log4j") logConfDir.mkdir() + System.setProperty("SPARK_YARN_MODE", "true") val logConfFile = new File(logConfDir, "log4j.properties") Files.write(LOG4J_CONF, logConfFile, UTF_8) @@ -124,6 +125,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit override def afterAll() { yarnCluster.stop() + System.clearProperty("SPARK_YARN_MODE") super.afterAll() } From 9ef5f1b7558731762435524636ff5bc87552a355 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 6 Mar 2015 16:41:15 -0800 Subject: [PATCH 18/41] Added explanation of how the credentials refresh works, some other minor fixes. --- .../org/apache/spark/deploy/yarn/Client.scala | 8 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 113 ++++++++++-------- 2 files changed, 69 insertions(+), 52 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 0ca5d6d28a544..867c19313d0d9 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -26,8 +26,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} import scala.util.{Try, Success, Failure} -import com.google.common.base.{Preconditions, Objects} - +import com.google.common.base.{Objects, Preconditions} import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ @@ -563,9 +562,8 @@ private[spark] class Client( def setupCredentials(): Unit = { if (args.principal != null) { - if (args.keytab == null) { - throw new SparkException("Keytab must be specified when principal is specified.") - } + Preconditions.checkNotNull( + args.keytab, "Keytab must be specified when principal is specified.") logInfo("Attempting to login to the Kerberos" + s" using principal: ${args.principal} and keytab: ${args.keytab}") val f = new File(args.keytab) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 92f9215571d04..de4e59b0864b3 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.yarn import java.io._ import java.util +import java.util.Arrays import java.util.Comparator import java.util.concurrent.{Executors, TimeUnit} import java.util.regex.Matcher @@ -28,6 +29,7 @@ import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ import scala.util.Try +import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.fs.FileSystem @@ -56,7 +58,9 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { private lazy val delegationTokenRenewer = Executors.newSingleThreadScheduledExecutor( Utils.namedThreadFactory("Delegation Token Refresh Thread")) - private lazy val delegationTokenExecuterUpdaterRunnable = + + // On the executor, this thread wakes up and picks up new tokens from HDFS, if any. + private lazy val executorUpdaterRunnable = new Runnable { override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired()) } @@ -100,13 +104,41 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { if (credentials != null) credentials.getSecretKey(new Text(key)) else null } + /* + * The following methods are primarily meant to make sure long-running apps like Spark + * Streaming apps can run without interruption while writing to secure HDFS. The + * scheduleLoginFromKeytab method is called on the driver when the + * CoarseGrainedScheduledBackend starts up. This method wakes up a thread that logs into the KDC + * once 75% of the expiry time of the original delegation tokens used for the container + * has elapsed. It then creates new delegation tokens and writes them to HDFS in a + * pre-specified location - the prefix of which is specified in the sparkConf by + * spark.yarn.credentials.file (so the file(s) would be named c-1, c-2 etc. - each update goes + * to a new file, with a monotonically increasing suffix). After this, the credentials are + * updated once 75% of the new tokens validity has elapsed. + * + * On the executor side, the updateCredentialsIfRequired method is called once 80% of the + * validity of the original tokens has elapsed. At that time the executor finds the + * credentials file with the latest timestamp and checks if it has read those credentials + * before (by keeping track of the suffix of the last file it read). If a new file has + * appeared, it will read the credentials and update the currently running UGI with it. This + * process happens again once 80% of the validity of this has expired. + */ private[spark] override def scheduleLoginFromKeytab(): Unit = { sparkConf.getOption("spark.yarn.principal").foreach { principal => val keytab = sparkConf.get("spark.yarn.keytab") - val delegationTokenRenewerRunnable = + // This thread periodically runs on the driver to update the delegation tokens on HDFS. + val driverTokenRenewerRunnable = new Runnable { override def run(): Unit = { - renewCredentials(principal, keytab) + try { + renewCredentials(principal, keytab) + } catch { + case e: Exception => + logWarning("Failed to write out new credentials to HDFS, will try again in an " + + "hour! If this happens too often tasks will fail.", e) + delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) + return + } delegationTokenRenewer.schedule( this, (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong, TimeUnit.MILLISECONDS) @@ -114,7 +146,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } val timeToRenewal = (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong delegationTokenRenewer.schedule( - delegationTokenRenewerRunnable, timeToRenewal, TimeUnit.MILLISECONDS) + driverTokenRenewerRunnable, timeToRenewal, TimeUnit.MILLISECONDS) } } @@ -143,8 +175,6 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { s.close() remoteFs.rename(tempTokenPath, tokenPath) } - } catch { - case e: Exception => } finally { stream.foreach(_.close()) } @@ -166,32 +196,27 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { name.startsWith(credentialsFilePath.getName) && !name.endsWith(".tmp") } }) - util.Arrays.sort(fileStatuses, new Comparator[FileStatus] { + Arrays.sort(fileStatuses, new Comparator[FileStatus] { override def compare(o1: FileStatus, o2: FileStatus): Int = { - // can't return this directly, as it might cause int to overflow - val diff = o1.getModificationTime - o2.getModificationTime - if (diff < 0) { - -1 - } else { - 1 - } + Longs.compare(o1.getModificationTime, o2.getModificationTime) } }) - val credentialsStatus = fileStatuses(fileStatuses.length - 1) - val credentials = credentialsStatus.getPath - val suffix = credentials.getName.substring(credentials.getName.lastIndexOf("-") + 1).toInt - if (suffix > lastCredentialsFileSuffix) { - val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentials) - UserGroupInformation.getCurrentUser.addCredentials(newCredentials) - val totalValidity = getLatestValidity - credentialsStatus.getModificationTime - val timeToRunRenewal = - credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong - val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() - delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterRunnable, - timeFromNowToRenewal, TimeUnit.MILLISECONDS) - } else { - // Check every hour to see if new credentials arrived. - delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterRunnable, 1, TimeUnit.HOURS) + fileStatuses.lastOption.foreach { credentialsStatus => + val credentials = credentialsStatus.getPath + val suffix = credentials.getName.substring(credentials.getName.lastIndexOf("-") + 1).toInt + if (suffix > lastCredentialsFileSuffix) { + val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentials) + UserGroupInformation.getCurrentUser.addCredentials(newCredentials) + val totalValidity = getLatestValidity - credentialsStatus.getModificationTime + val timeToRunRenewal = + credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong + val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() + delegationTokenRenewer.schedule(executorUpdaterRunnable, + timeFromNowToRenewal, TimeUnit.MILLISECONDS) + } else { + // Check every hour to see if new credentials arrived. + delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) + } } } } catch { @@ -200,7 +225,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { case e: Exception => logWarning( "Error encountered while trying to update credentials, will try again in 1 hour", e) - delegationTokenRenewer.schedule(delegationTokenExecuterUpdaterRunnable, 1, TimeUnit.HOURS) + delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) } } @@ -208,30 +233,24 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { remoteFs: FileSystem, tokenPath: Path): Credentials = { val stream = remoteFs.open(tokenPath) - val newCredentials = new Credentials() - newCredentials.readFields(stream) - newCredentials + try { + val newCredentials = new Credentials() + newCredentials.readFields(stream) + newCredentials + } finally { + stream.close() + } } private[spark] def getLatestValidity: Long = { - val creds = UserGroupInformation.getCurrentUser.getCredentials - var latestValidity: Long = 0 - creds.getAllTokens + UserGroupInformation.getCurrentUser.getCredentials.getAllTokens .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) - .foreach { t => + .map { t => val identifier = new DelegationTokenIdentifier() identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) - latestValidity = { - if (latestValidity < identifier.getMaxDate) { - identifier.getMaxDate - } else { - latestValidity - } - } - } - latestValidity + identifier.getMaxDate + }.foldLeft(0L)(math.max) } - /** * Get the list of namenodes the user may access. */ From 55522e3733de3d6f1a9dc7d1206a6fe10f9fc8e9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 6 Mar 2015 17:28:23 -0800 Subject: [PATCH 19/41] Fix failure caused by Preconditions ambiguity. --- yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 867c19313d0d9..f30231a9d36d2 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -563,7 +563,7 @@ private[spark] class Client( def setupCredentials(): Unit = { if (args.principal != null) { Preconditions.checkNotNull( - args.keytab, "Keytab must be specified when principal is specified.") + args.keytab, "Keytab must be specified when principal is specified.", Array.empty[Any]) logInfo("Attempting to login to the Kerberos" + s" using principal: ${args.principal} and keytab: ${args.keytab}") val f = new File(args.keytab) From 42813b4d852e590aea742898d8603113f4bdb675 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sun, 22 Mar 2015 21:20:10 -0700 Subject: [PATCH 20/41] Remove utils.sh, which was re-added due to merge with master. --- bin/utils.sh | 60 ---------------------------------------------------- 1 file changed, 60 deletions(-) delete mode 100755 bin/utils.sh diff --git a/bin/utils.sh b/bin/utils.sh deleted file mode 100755 index f243c45a10805..0000000000000 --- a/bin/utils.sh +++ /dev/null @@ -1,60 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# Gather all spark-submit options into SUBMISSION_OPTS -function gatherSparkSubmitOpts() { - - if [ -z "$SUBMIT_USAGE_FUNCTION" ]; then - echo "Function for printing usage of $0 is not set." 1>&2 - echo "Please set usage function to shell variable 'SUBMIT_USAGE_FUNCTION' in $0" 1>&2 - exit 1 - fi - - # NOTE: If you add or remove spark-submit options, - # modify NOT ONLY this script but also SparkSubmitArgument.scala - SUBMISSION_OPTS=() - APPLICATION_OPTS=() - while (($#)); do - case "$1" in - --master | --deploy-mode | --class | --name | --jars | --packages | --py-files | --files | \ - --conf | --repositories | --properties-file | --driver-memory | --driver-java-options | \ - --driver-library-path | --driver-class-path | --executor-memory | --driver-cores | \ - --total-executor-cores | --executor-cores | --queue | --num-executors | --archives | \ - --proxy-user | --principal | --keytab) - if [[ $# -lt 2 ]]; then - "$SUBMIT_USAGE_FUNCTION" - exit 1; - fi - SUBMISSION_OPTS+=("$1"); shift - SUBMISSION_OPTS+=("$1"); shift - ;; - - --verbose | -v | --supervise) - SUBMISSION_OPTS+=("$1"); shift - ;; - - *) - APPLICATION_OPTS+=("$1"); shift - ;; - esac - done - - export SUBMISSION_OPTS - export APPLICATION_OPTS -} From fa233bd5c22078cc75f0ab5163974420e438e47f Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 24 Mar 2015 12:43:24 -0700 Subject: [PATCH 21/41] Adding logging, fixing minor formatting and ordering issues. --- .../launcher/SparkSubmitOptionParser.java | 10 +++--- .../org/apache/spark/deploy/yarn/Client.scala | 8 +++-- .../deploy/yarn/YarnSparkHadoopUtil.scala | 34 +++++++++++++------ 3 files changed, 34 insertions(+), 18 deletions(-) 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 7bb6ed80afd4e..229000087688f 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -57,8 +57,6 @@ class SparkSubmitOptionParser { protected final String REPOSITORIES = "--repositories"; protected final String STATUS = "--status"; protected final String TOTAL_EXECUTOR_CORES = "--total-executor-cores"; - protected final String PRINCIPAL = "--principal"; - protected final String KEYTAB = "--keytab"; // Options that do not take arguments. protected final String HELP = "--help"; @@ -71,8 +69,10 @@ class SparkSubmitOptionParser { // YARN-only options. protected final String ARCHIVES = "--archives"; protected final String EXECUTOR_CORES = "--executor-cores"; - protected final String QUEUE = "--queue"; + protected final String KEYTAB = "--keytab"; protected final String NUM_EXECUTORS = "--num-executors"; + protected final String PRINCIPAL = "--principal"; + protected final String QUEUE = "--queue"; /** * This is the canonical list of spark-submit options. Each entry in the array contains the @@ -98,11 +98,13 @@ class SparkSubmitOptionParser { { EXECUTOR_MEMORY }, { FILES }, { JARS }, + { KEYTAB }, { KILL_SUBMISSION }, { MASTER }, { NAME }, { NUM_EXECUTORS }, { PACKAGES }, + { PRINCIPAL }, { PROPERTIES_FILE }, { PROXY_USER }, { PY_FILES }, @@ -110,8 +112,6 @@ class SparkSubmitOptionParser { { REPOSITORIES }, { STATUS }, { TOTAL_EXECUTOR_CORES }, - { PRINCIPAL}, - { KEYTAB} }; /** diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index db5e9afefbee3..2a27237f9395a 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -26,7 +26,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} import scala.util.{Try, Success, Failure} -import com.google.common.base.{Objects, Preconditions} +import com.google.common.base.Objects import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ @@ -238,6 +238,8 @@ private[spark] class Client( // If we passed in a keytab, make sure we copy the keytab to the staging directory on // HDFS, and setup the relevant environment vars, so the AM can login again. if (loginFromKeytab) { + logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" + + " via the YARN Secure Distributed Cache.") val localUri = new URI(args.keytab) val localPath = getQualifiedLocalPath(localUri, hadoopConf) val destinationPath = copyFileToRemote(dst, localPath, replication) @@ -333,6 +335,7 @@ private[spark] class Client( val credentialsFile = "credentials-" + UUID.randomUUID().toString sparkConf.set( "spark.yarn.credentials.file", new Path(stagingDirPath, credentialsFile).toString) + logInfo(s"Credentials file set to: $credentialsFile") } // Set the environment variables to be passed on to the executors. @@ -562,8 +565,7 @@ private[spark] class Client( def setupCredentials(): Unit = { if (args.principal != null) { - Preconditions.checkNotNull( - args.keytab, "Keytab must be specified when principal is specified.", Array.empty[Any]) + require(args.keytab != null, "Keytab must be specified when principal is specified.") logInfo("Attempting to login to the Kerberos" + s" using principal: ${args.principal} and keytab: ${args.keytab}") val f = new File(args.keytab) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index f6fd99d3e8b23..05f2d637a007e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -126,6 +126,13 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { private[spark] override def scheduleLoginFromKeytab(): Unit = { sparkConf.getOption("spark.yarn.principal").foreach { principal => val keytab = sparkConf.get("spark.yarn.keytab") + + def scheduleRenewal(runnable: Runnable) = { + val renewalInterval = (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong + logInfo("Scheduling login from keytab in " + renewalInterval + "millis.") + delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) + } + // This thread periodically runs on the driver to update the delegation tokens on HDFS. val driverTokenRenewerRunnable = new Runnable { @@ -139,14 +146,10 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) return } - delegationTokenRenewer.schedule( - this, (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong, - TimeUnit.MILLISECONDS) + scheduleRenewal(this) } } - val timeToRenewal = (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong - delegationTokenRenewer.schedule( - driverTokenRenewerRunnable, timeToRenewal, TimeUnit.MILLISECONDS) + scheduleRenewal(driverTokenRenewerRunnable) } } @@ -154,8 +157,10 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { if (!loggedInViaKeytab) { // Keytab is copied by YARN to the working directory of the AM, so full path is // not needed. + logInfo(s"Attempting to login to KDC using principal: $principal") loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( principal, keytab) + logInfo("Successfully logged into KDC.") loggedInViaKeytab = true } val nns = getNameNodesToAccess(sparkConf) @@ -167,13 +172,16 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { sparkConf.get("spark.yarn.credentials.file") + "-" + nextSuffix val tokenPath = new Path(tokenPathStr) val tempTokenPath = new Path(tokenPathStr + ".tmp") + logInfo("Writing out delegation tokens to " + tempTokenPath.toString) val stream = Option(remoteFs.create(tempTokenPath, true)) try { stream.foreach { s => newCredentials.writeTokenStorageToStream(s) s.hflush() s.close() + logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") remoteFs.rename(tempTokenPath, tokenPath) + logInfo("Delegation token file rename complete.") } } finally { stream.foreach(_.close()) @@ -205,16 +213,23 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { val credentials = credentialsStatus.getPath val suffix = credentials.getName.substring(credentials.getName.lastIndexOf("-") + 1).toInt if (suffix > lastCredentialsFileSuffix) { + logInfo("Reading new delegation tokens from " + credentials.toString) val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentials) + lastCredentialsFileSuffix = suffix UserGroupInformation.getCurrentUser.addCredentials(newCredentials) + val totalValidity = getLatestValidity - credentialsStatus.getModificationTime val timeToRunRenewal = credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() - delegationTokenRenewer.schedule(executorUpdaterRunnable, - timeFromNowToRenewal, TimeUnit.MILLISECONDS) + logInfo("Updated delegation tokens, will check for new tokens in " + + timeFromNowToRenewal + " millis") + delegationTokenRenewer.schedule( + executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) } else { // Check every hour to see if new credentials arrived. + logInfo("Updated delegation tokens were expected, but the driver has not updated the " + + "tokens yet, will check again in an hour.") delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) } } @@ -223,8 +238,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { // Since the file may get deleted while we are reading it, catch the Exception and come // back in an hour to try again case e: Exception => - logWarning( - "Error encountered while trying to update credentials, will try again in 1 hour", e) + logWarning("Error while trying to update credentials, will try again in 1 hour", e) delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) } } From 62c45ce494cb242698db9a53c3e6068c86540e0d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 24 Mar 2015 16:59:11 -0700 Subject: [PATCH 22/41] Relogin from keytab periodically. --- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 05f2d637a007e..175e6aa02fef1 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy.yarn import java.io._ -import java.util import java.util.Arrays import java.util.Comparator import java.util.concurrent.{Executors, TimeUnit} @@ -162,6 +161,20 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { principal, keytab) logInfo("Successfully logged into KDC.") loggedInViaKeytab = true + // Not exactly sure when HDFS re-logs in, be safe and do it ourselves. + // Periodically check and relogin this keytab. The UGI will take care of not relogging in + // if it is not necessary to relogin. + val reloginRunnable = new Runnable { + override def run(): Unit = { + try { + loggedInUGI.checkTGTAndReloginFromKeytab() + } catch { + case e: Exception => + logError("Error while attempting tp relogin to KDC", e) + } + } + } + delegationTokenRenewer.schedule(reloginRunnable, 6, TimeUnit.HOURS) } val nns = getNameNodesToAccess(sparkConf) val newCredentials = loggedInUGI.getCredentials From 61b2b279e04fcbdde5c3206164a1605fe357abb3 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 25 Mar 2015 13:54:41 -0700 Subject: [PATCH 23/41] Account for AM restarts by making sure lastSuffix is read from the files on HDFS. --- .../deploy/yarn/YarnSparkHadoopUtil.scala | 62 ++++++++++++------- 1 file changed, 41 insertions(+), 21 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 175e6aa02fef1..84b6ff5007ed7 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -127,7 +127,9 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { val keytab = sparkConf.get("spark.yarn.keytab") def scheduleRenewal(runnable: Runnable) = { - val renewalInterval = (0.75 * (getLatestValidity - System.currentTimeMillis())).toLong + // Latest validity can be -ve if the original tokens expired, and then the AM died. + val renewalInterval = + math.max((0.75 * (getLatestValidity - System.currentTimeMillis())).toLong, 0L) logInfo("Scheduling login from keytab in " + renewalInterval + "millis.") delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) } @@ -180,6 +182,16 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { val newCredentials = loggedInUGI.getCredentials obtainTokensForNamenodes(nns, conf, newCredentials) val remoteFs = FileSystem.get(conf) + // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM + // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file + // and update the lastCredentialsFileSuffix. + if (lastCredentialsFileSuffix == 0) { + val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) + listCredentialsFilesSorted(remoteFs, credentialsPath) + .lastOption.foreach { status => + lastCredentialsFileSuffix = getSuffixForCredentialsPath(status) + } + } val nextSuffix = lastCredentialsFileSuffix + 1 val tokenPathStr = sparkConf.get("spark.yarn.credentials.file") + "-" + nextSuffix @@ -199,35 +211,43 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } finally { stream.foreach(_.close()) } - lastCredentialsFileSuffix = nextSuffix } + private def listCredentialsFilesSorted( + remoteFs: FileSystem, + credentialsFilePath: Path): Array[FileStatus] = { + val fileStatuses = remoteFs.listStatus(credentialsFilePath.getParent, + new PathFilter { + override def accept(path: Path): Boolean = { + val name = path.getName + name.startsWith(credentialsFilePath.getName) && !name.endsWith(".tmp") + } + }) + Arrays.sort(fileStatuses, new Comparator[FileStatus] { + override def compare(o1: FileStatus, o2: FileStatus): Int = { + Longs.compare(o1.getModificationTime, o2.getModificationTime) + } + }) + fileStatuses + } + + private def getSuffixForCredentialsPath(credentialsStatus: FileStatus): Int = { + val fileName = credentialsStatus.getPath.getName + fileName.substring(fileName.lastIndexOf("-") + 1).toInt + } override def updateCredentialsIfRequired(): Unit = { try { sparkConf.getOption("spark.yarn.credentials.file").foreach { credentialsFile => val credentialsFilePath = new Path(credentialsFile) val remoteFs = FileSystem.get(conf) - val stagingDirPath = new Path(remoteFs.getHomeDirectory, credentialsFilePath.getParent) - val fileStatuses = - remoteFs.listStatus(stagingDirPath, - new PathFilter { - override def accept(path: Path): Boolean = { - val name = path.getName - name.startsWith(credentialsFilePath.getName) && !name.endsWith(".tmp") - } - }) - Arrays.sort(fileStatuses, new Comparator[FileStatus] { - override def compare(o1: FileStatus, o2: FileStatus): Int = { - Longs.compare(o1.getModificationTime, o2.getModificationTime) - } - }) - fileStatuses.lastOption.foreach { credentialsStatus => - val credentials = credentialsStatus.getPath - val suffix = credentials.getName.substring(credentials.getName.lastIndexOf("-") + 1).toInt + + listCredentialsFilesSorted(remoteFs, credentialsFilePath) + .lastOption.foreach { credentialsStatus => + val suffix = getSuffixForCredentialsPath(credentialsStatus) if (suffix > lastCredentialsFileSuffix) { - logInfo("Reading new delegation tokens from " + credentials.toString) - val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentials) + logInfo("Reading new delegation tokens from " + credentialsStatus.getPath) + val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) lastCredentialsFileSuffix = suffix UserGroupInformation.getCurrentUser.addCredentials(newCredentials) From 2f9975c6169c11b4e33280b8604ddf3f87935618 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 25 Mar 2015 17:20:10 -0700 Subject: [PATCH 24/41] Ensure new tokens are written out immediately on AM restart. Also, pikc up the latest suffix from HDFS if the AM is restarted. --- .../deploy/yarn/YarnSparkHadoopUtil.scala | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 84b6ff5007ed7..c242b989ef76b 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -126,11 +126,12 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { sparkConf.getOption("spark.yarn.principal").foreach { principal => val keytab = sparkConf.get("spark.yarn.keytab") + def getRenewalInterval = + math.max((0.75 * (getLatestValidity - System.currentTimeMillis())).toLong, 0L) + def scheduleRenewal(runnable: Runnable) = { - // Latest validity can be -ve if the original tokens expired, and then the AM died. - val renewalInterval = - math.max((0.75 * (getLatestValidity - System.currentTimeMillis())).toLong, 0L) - logInfo("Scheduling login from keytab in " + renewalInterval + "millis.") + val renewalInterval = getRenewalInterval + logInfo(s"Scheduling login from keytab in $renewalInterval millis.") delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) } @@ -139,7 +140,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { new Runnable { override def run(): Unit = { try { - renewCredentials(principal, keytab) + writeNewTokensToHDFS(principal, keytab) } catch { case e: Exception => logWarning("Failed to write out new credentials to HDFS, will try again in an " + @@ -150,11 +151,15 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { scheduleRenewal(this) } } + // If this is an AM restart, it is possible that the original tokens have expired, which + // means we need to login immediately to get new tokens. + if (getRenewalInterval == 0) writeNewTokensToHDFS(principal, keytab) + // Schedule update of credentials scheduleRenewal(driverTokenRenewerRunnable) } } - private def renewCredentials(principal: String, keytab: String): Unit = { + private def writeNewTokensToHDFS(principal: String, keytab: String): Unit = { if (!loggedInViaKeytab) { // Keytab is copied by YARN to the working directory of the AM, so full path is // not needed. @@ -179,8 +184,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { delegationTokenRenewer.schedule(reloginRunnable, 6, TimeUnit.HOURS) } val nns = getNameNodesToAccess(sparkConf) - val newCredentials = loggedInUGI.getCredentials - obtainTokensForNamenodes(nns, conf, newCredentials) + obtainTokensForNamenodes(nns, conf, loggedInUGI.getCredentials) val remoteFs = FileSystem.get(conf) // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file @@ -201,7 +205,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { val stream = Option(remoteFs.create(tempTokenPath, true)) try { stream.foreach { s => - newCredentials.writeTokenStorageToStream(s) + loggedInUGI.getCredentials.writeTokenStorageToStream(s) s.hflush() s.close() logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") @@ -241,7 +245,6 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { sparkConf.getOption("spark.yarn.credentials.file").foreach { credentialsFile => val credentialsFilePath = new Path(credentialsFile) val remoteFs = FileSystem.get(conf) - listCredentialsFilesSorted(remoteFs, credentialsFilePath) .lastOption.foreach { credentialsStatus => val suffix = getSuffixForCredentialsPath(credentialsStatus) @@ -250,7 +253,6 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) lastCredentialsFileSuffix = suffix UserGroupInformation.getCurrentUser.addCredentials(newCredentials) - val totalValidity = getLatestValidity - credentialsStatus.getModificationTime val timeToRunRenewal = credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong From f74303c89c8fe542122680077e2a471d41b809c2 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 26 Mar 2015 17:50:56 -0700 Subject: [PATCH 25/41] Move the new logic into specialized classes. Add cleanup for old credentials files. --- .../ExecutorDelegationTokenUpdater.scala | 108 +++++++++ .../apache/spark/deploy/SparkHadoopUtil.scala | 64 ++++-- .../CoarseGrainedExecutorBackend.scala | 6 +- .../CoarseGrainedSchedulerBackend.scala | 3 - .../yarn/AMDelegationTokenRenewer.scala | 201 +++++++++++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 6 + .../deploy/yarn/YarnSparkHadoopUtil.scala | 210 ------------------ 7 files changed, 370 insertions(+), 228 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala create mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala new file mode 100644 index 0000000000000..8dce094928222 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala @@ -0,0 +1,108 @@ +/* + * 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 + +import java.util.concurrent.{Executors, TimeUnit} +import java.util.{Comparator, Arrays} + +import com.google.common.primitives.Longs +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{PathFilter, FileStatus, Path, FileSystem} +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SparkConf} + +private[spark] class ExecutorDelegationTokenUpdater( + sparkConf: SparkConf, + hadoopConf: Configuration) extends Logging { + + @volatile private var lastCredentialsFileSuffix = 0 + + private lazy val delegationTokenRenewer = + Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("Delegation Token Refresh Thread")) + + // On the executor, this thread wakes up and picks up new tokens from HDFS, if any. + private lazy val executorUpdaterRunnable = + new Runnable { + override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired()) + } + + def updateCredentialsIfRequired(): Unit = { + try { + sparkConf.getOption("spark.yarn.credentials.file").foreach { credentialsFile => + val credentials = UserGroupInformation.getCurrentUser.getCredentials + val credentialsFilePath = new Path(credentialsFile) + val remoteFs = FileSystem.get(hadoopConf) + SparkHadoopUtil.get.listFilesSorted( + remoteFs, credentialsFilePath.getParent, credentialsFilePath.getName, ".tmp") + .lastOption.foreach { credentialsStatus => + val suffix = getSuffixForCredentialsPath(credentialsStatus) + if (suffix > lastCredentialsFileSuffix) { + logInfo("Reading new delegation tokens from " + credentialsStatus.getPath) + val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) + lastCredentialsFileSuffix = suffix + UserGroupInformation.getCurrentUser.addCredentials(newCredentials) + val totalValidity = SparkHadoopUtil.get.getLatestTokenValidity(credentials) - + credentialsStatus.getModificationTime + val timeToRunRenewal = + credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong + val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() + logInfo("Updated delegation tokens, will check for new tokens in " + + timeFromNowToRenewal + " millis") + delegationTokenRenewer.schedule( + executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) + } else { + // Check every hour to see if new credentials arrived. + logInfo("Updated delegation tokens were expected, but the driver has not updated the " + + "tokens yet, will check again in an hour.") + delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) + } + } + } + } catch { + // Since the file may get deleted while we are reading it, catch the Exception and come + // back in an hour to try again + case e: Exception => + logWarning("Error while trying to update credentials, will try again in 1 hour", e) + delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) + } + } + + private def getCredentialsFromHDFSFile( + remoteFs: FileSystem, + tokenPath: Path): Credentials = { + val stream = remoteFs.open(tokenPath) + try { + val newCredentials = new Credentials() + newCredentials.readFields(stream) + newCredentials + } finally { + stream.close() + } + } + + def stop(): Unit = { + delegationTokenRenewer.shutdown() + } + + private def getSuffixForCredentialsPath(credentialsStatus: FileStatus): Int = { + val fileName = credentialsStatus.getPath.getName + fileName.substring(fileName.lastIndexOf("-") + 1).toInt + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 9ca6e07b3ae0f..5c7c33b588f9f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -17,12 +17,16 @@ package org.apache.spark.deploy +import java.io.{ByteArrayInputStream, DataInputStream} import java.lang.reflect.Method import java.security.PrivilegedExceptionAction +import java.util.{Comparator, Arrays} +import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.{PathFilter, FileStatus, FileSystem, Path} import org.apache.hadoop.fs.FileSystem.Statistics +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.JobContext import org.apache.hadoop.security.Credentials @@ -30,7 +34,7 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.{SerializableBuffer, Utils} +import org.apache.spark.util.Utils import scala.collection.JavaConversions._ @@ -40,7 +44,7 @@ import scala.collection.JavaConversions._ */ @DeveloperApi class SparkHadoopUtil extends Logging { - protected val sparkConf = new SparkConf() // YarnSparkHadoopUtil requires this + val sparkConf = new SparkConf() val conf: Configuration = newConfiguration(sparkConf) UserGroupInformation.setConfiguration(conf) @@ -55,7 +59,6 @@ class SparkHadoopUtil extends Logging { def runAsSparkUser(func: () => Unit) { val user = Utils.getCurrentUserName() logDebug("running as user: " + user) - updateCredentialsIfRequired() val ugi = UserGroupInformation.createRemoteUser(user) transferCredentials(UserGroupInformation.getCurrentUser(), ugi) ugi.doAs(new PrivilegedExceptionAction[Unit] { @@ -63,8 +66,6 @@ class SparkHadoopUtil extends Logging { }) } - def updateCredentialsIfRequired(): Unit = {} - def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { for (token <- source.getTokens()) { dest.addToken(token) @@ -125,14 +126,6 @@ class SparkHadoopUtil extends Logging { UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } - /** - * Schedule a login from the keytab and principal set using the --principal and --keytab - * arguments to spark-submit. This login happens only when the credentials of the current user - * are about to expire. This method reads SPARK_PRINCIPAL and SPARK_KEYTAB from the environment - * to do the login. This method is a no-op in non-YARN mode. - */ - private[spark] def scheduleLoginFromKeytab(): Unit = {} - /** * Returns a function that can be called to find Hadoop FileSystem bytes read. If * getFSBytesReadOnThreadCallback is called from thread r at time t, the returned callback will @@ -213,6 +206,49 @@ class SparkHadoopUtil extends Logging { val baseStatus = fs.getFileStatus(basePath) if (baseStatus.isDir) recurse(basePath) else Array(baseStatus) } + + /** + * Lists all the files in a directory with the specified prefix, and does not end with the + * given suffix. + * @param remoteFs + * @param prefix + * @return + */ + + def listFilesSorted( + remoteFs: FileSystem, + dir: Path, + prefix: String, + exclusionSuffix: String): Array[FileStatus] = { + val fileStatuses = remoteFs.listStatus(dir, + new PathFilter { + override def accept(path: Path): Boolean = { + val name = path.getName + name.startsWith(prefix) && !name.endsWith(exclusionSuffix) + } + }) + Arrays.sort(fileStatuses, new Comparator[FileStatus] { + override def compare(o1: FileStatus, o2: FileStatus): Int = { + Longs.compare(o1.getModificationTime, o2.getModificationTime) + } + }) + fileStatuses + } + + /** + * Get the latest validity of the HDFS token in the Credentials object. + * @param credentials + * @return + */ + def getLatestTokenValidity(credentials: Credentials): Long = { + credentials.getAllTokens.filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) + .map { t => + val identifier = new DelegationTokenIdentifier() + identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) + identifier.getMaxDate + }.foldLeft(0L)(math.max) + } + } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index dd19e4947db1e..e3b7e37344bf7 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -29,7 +29,7 @@ import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.{ExecutorDelegationTokenUpdater, SparkHadoopUtil} import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -155,6 +155,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { driverConf.set(key, value) } } + // Periodically update the credentials for this user to ensure HDFS tokens get updated. + val tokenUpdater = new ExecutorDelegationTokenUpdater(driverConf, SparkHadoopUtil.get.conf) + tokenUpdater.updateCredentialsIfRequired() val env = SparkEnv.createExecutorEnv( driverConf, executorId, hostname, port, cores, isLocal = false) @@ -172,6 +175,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { env.actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") } env.actorSystem.awaitTermination() + tokenUpdater.stop() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 3dfda81883a3e..dae0f50c27f56 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -241,9 +241,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste driverActor = actorSystem.actorOf( Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) - // If a principal and keytab have been set, use that to create new credentials for executors - // periodically - SparkHadoopUtil.get.scheduleLoginFromKeytab() } def stopExecutors() { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala new file mode 100644 index 0000000000000..54b5369393a82 --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.yarn + +import java.util.concurrent.{Executors, TimeUnit} + +import scala.collection.JavaConversions._ +import scala.concurrent.duration.Duration + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SparkConf} + +/* + * The following methods are primarily meant to make sure long-running apps like Spark + * Streaming apps can run without interruption while writing to secure HDFS. The + * scheduleLoginFromKeytab method is called on the driver when the + * CoarseGrainedScheduledBackend starts up. This method wakes up a thread that logs into the KDC + * once 75% of the expiry time of the original delegation tokens used for the container + * has elapsed. It then creates new delegation tokens and writes them to HDFS in a + * pre-specified location - the prefix of which is specified in the sparkConf by + * spark.yarn.credentials.file (so the file(s) would be named c-1, c-2 etc. - each update goes + * to a new file, with a monotonically increasing suffix). After this, the credentials are + * updated once 75% of the new tokens validity has elapsed. + * + * On the executor side, the updateCredentialsIfRequired method is called once 80% of the + * validity of the original tokens has elapsed. At that time the executor finds the + * credentials file with the latest timestamp and checks if it has read those credentials + * before (by keeping track of the suffix of the last file it read). If a new file has + * appeared, it will read the credentials and update the currently running UGI with it. This + * process happens again once 80% of the validity of this has expired. + */ +class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) extends Logging { + + private var lastCredentialsFileSuffix = 0 + + private lazy val delegationTokenRenewer = + Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("Delegation Token Refresh Thread")) + + private var loggedInViaKeytab = false + private var loggedInUGI: UserGroupInformation = null + + private lazy val hadoopUtil = YarnSparkHadoopUtil.get + + /** + * Schedule a login from the keytab and principal set using the --principal and --keytab + * arguments to spark-submit. This login happens only when the credentials of the current user + * are about to expire. This method reads SPARK_PRINCIPAL and SPARK_KEYTAB from the environment + * to do the login. This method is a no-op in non-YARN mode. + */ + private[spark] def scheduleLoginFromKeytab(): Unit = { + sparkConf.getOption("spark.yarn.principal").foreach { principal => + val keytab = sparkConf.get("spark.yarn.keytab") + + def getRenewalInterval = { + val credentials = UserGroupInformation.getCurrentUser.getCredentials + math.max((0.75 * (hadoopUtil.getLatestTokenValidity(credentials) - + System.currentTimeMillis())).toLong, 0L) + } + + def scheduleRenewal(runnable: Runnable) = { + val renewalInterval = getRenewalInterval + logInfo(s"Scheduling login from keytab in $renewalInterval millis.") + delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) + } + + // This thread periodically runs on the driver to update the delegation tokens on HDFS. + val driverTokenRenewerRunnable = + new Runnable { + override def run(): Unit = { + try { + writeNewTokensToHDFS(principal, keytab) + } catch { + case e: Exception => + logWarning("Failed to write out new credentials to HDFS, will try again in an " + + "hour! If this happens too often tasks will fail.", e) + delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) + return + } + cleanupOldFiles() + scheduleRenewal(this) + } + } + // If this is an AM restart, it is possible that the original tokens have expired, which + // means we need to login immediately to get new tokens. + if (getRenewalInterval == 0) writeNewTokensToHDFS(principal, keytab) + // Schedule update of credentials + scheduleRenewal(driverTokenRenewerRunnable) + + } + } + + // Keeps only files that are newer than 30 days, and deletes everything else. At least 5 files + // are kept for safety + private def cleanupOldFiles(): Unit = { + import scala.concurrent.duration._ + try { + val remoteFs = FileSystem.get(hadoopConf) + val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) + hadoopUtil.listFilesSorted( + remoteFs, credentialsPath.getParent, credentialsPath.getName, ".tmp").dropRight(5) + .takeWhile(_.getModificationTime < System.currentTimeMillis() - (30 days).toMillis) + .foreach(x => remoteFs.delete(x.getPath, true)) + } catch { + // Such errors are not fatal, so don't throw. Make sure they are logged though + case e: Exception => + logWarning("Error while attempting to cleanup old tokens. If you are seeing many such " + + "warnings there may be an issue with your HDFS cluster.") + } + } + + private def writeNewTokensToHDFS(principal: String, keytab: String): Unit = { + if (!loggedInViaKeytab) { + // Keytab is copied by YARN to the working directory of the AM, so full path is + // not needed. + logInfo(s"Attempting to login to KDC using principal: $principal") + loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( + principal, keytab) + logInfo("Successfully logged into KDC.") + loggedInViaKeytab = true + // Not exactly sure when HDFS re-logs in, be safe and do it ourselves. + // Periodically check and relogin this keytab. The UGI will take care of not relogging in + // if it is not necessary to relogin. + val reloginRunnable = new Runnable { + override def run(): Unit = { + try { + loggedInUGI.checkTGTAndReloginFromKeytab() + } catch { + case e: Exception => + logError("Error while attempting tp relogin to KDC", e) + } + } + } + delegationTokenRenewer.schedule(reloginRunnable, 6, TimeUnit.HOURS) + } + val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + YarnSparkHadoopUtil.get.obtainTokensForNamenodes(nns, hadoopConf, loggedInUGI.getCredentials) + val remoteFs = FileSystem.get(hadoopConf) + // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM + // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file + // and update the lastCredentialsFileSuffix. + if (lastCredentialsFileSuffix == 0) { + val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) + SparkHadoopUtil.get.listFilesSorted( + remoteFs, credentialsPath.getParent, credentialsPath.getName, ".tmp") + .lastOption.foreach { status => + lastCredentialsFileSuffix = getSuffixForCredentialsPath(status) + } + } + val nextSuffix = lastCredentialsFileSuffix + 1 + val tokenPathStr = + sparkConf.get("spark.yarn.credentials.file") + "-" + nextSuffix + val tokenPath = new Path(tokenPathStr) + val tempTokenPath = new Path(tokenPathStr + ".tmp") + logInfo("Writing out delegation tokens to " + tempTokenPath.toString) + val stream = Option(remoteFs.create(tempTokenPath, true)) + try { + stream.foreach { s => + loggedInUGI.getCredentials.writeTokenStorageToStream(s) + s.hflush() + s.close() + logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") + + remoteFs.rename(tempTokenPath, tokenPath) + logInfo("Delegation token file rename complete.") + } + } finally { + stream.foreach(_.close()) + } + lastCredentialsFileSuffix = nextSuffix + } + + def stop(): Unit = { + delegationTokenRenewer.shutdown() + } + + private def getSuffixForCredentialsPath(credentialsStatus: FileStatus): Int = { + val fileName = credentialsStatus.getPath.getName + fileName.substring(fileName.lastIndexOf("-") + 1).toInt + } + +} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 3d18690cd9cbf..675afc341a5e8 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -78,6 +78,8 @@ private[spark] class ApplicationMaster( // Fields used in cluster mode. private val sparkContextRef = new AtomicReference[SparkContext](null) + private lazy val delegationTokenRenewer = new AMDelegationTokenRenewer(sparkConf, yarnConf) + final def run(): Int = { try { val appAttemptId = client.getAttemptId() @@ -201,6 +203,7 @@ private[spark] class ApplicationMaster( logDebug("shutting down user thread") userClassThread.interrupt() } + if (!inShutdown) delegationTokenRenewer.stop() } } @@ -256,6 +259,9 @@ private[spark] class ApplicationMaster( private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() + // If a principal and keytab have been set, use that to create new credentials for executors + // periodically + delegationTokenRenewer.scheduleLoginFromKeytab() userClassThread = startUserApplication() // This a bit hacky, but we need to wait until the spark.driver.port property has diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index c242b989ef76b..e6fbb54c6ad1d 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -51,19 +51,6 @@ import org.apache.spark.util.Utils */ class YarnSparkHadoopUtil extends SparkHadoopUtil { - @volatile private var loggedInViaKeytab = false - @volatile private var loggedInUGI: UserGroupInformation = null - private var lastCredentialsFileSuffix = 0 - private lazy val delegationTokenRenewer = - Executors.newSingleThreadScheduledExecutor( - Utils.namedThreadFactory("Delegation Token Refresh Thread")) - - // On the executor, this thread wakes up and picks up new tokens from HDFS, if any. - private lazy val executorUpdaterRunnable = - new Runnable { - override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired()) - } - override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) } @@ -103,203 +90,6 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { if (credentials != null) credentials.getSecretKey(new Text(key)) else null } - /* - * The following methods are primarily meant to make sure long-running apps like Spark - * Streaming apps can run without interruption while writing to secure HDFS. The - * scheduleLoginFromKeytab method is called on the driver when the - * CoarseGrainedScheduledBackend starts up. This method wakes up a thread that logs into the KDC - * once 75% of the expiry time of the original delegation tokens used for the container - * has elapsed. It then creates new delegation tokens and writes them to HDFS in a - * pre-specified location - the prefix of which is specified in the sparkConf by - * spark.yarn.credentials.file (so the file(s) would be named c-1, c-2 etc. - each update goes - * to a new file, with a monotonically increasing suffix). After this, the credentials are - * updated once 75% of the new tokens validity has elapsed. - * - * On the executor side, the updateCredentialsIfRequired method is called once 80% of the - * validity of the original tokens has elapsed. At that time the executor finds the - * credentials file with the latest timestamp and checks if it has read those credentials - * before (by keeping track of the suffix of the last file it read). If a new file has - * appeared, it will read the credentials and update the currently running UGI with it. This - * process happens again once 80% of the validity of this has expired. - */ - private[spark] override def scheduleLoginFromKeytab(): Unit = { - sparkConf.getOption("spark.yarn.principal").foreach { principal => - val keytab = sparkConf.get("spark.yarn.keytab") - - def getRenewalInterval = - math.max((0.75 * (getLatestValidity - System.currentTimeMillis())).toLong, 0L) - - def scheduleRenewal(runnable: Runnable) = { - val renewalInterval = getRenewalInterval - logInfo(s"Scheduling login from keytab in $renewalInterval millis.") - delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) - } - - // This thread periodically runs on the driver to update the delegation tokens on HDFS. - val driverTokenRenewerRunnable = - new Runnable { - override def run(): Unit = { - try { - writeNewTokensToHDFS(principal, keytab) - } catch { - case e: Exception => - logWarning("Failed to write out new credentials to HDFS, will try again in an " + - "hour! If this happens too often tasks will fail.", e) - delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) - return - } - scheduleRenewal(this) - } - } - // If this is an AM restart, it is possible that the original tokens have expired, which - // means we need to login immediately to get new tokens. - if (getRenewalInterval == 0) writeNewTokensToHDFS(principal, keytab) - // Schedule update of credentials - scheduleRenewal(driverTokenRenewerRunnable) - } - } - - private def writeNewTokensToHDFS(principal: String, keytab: String): Unit = { - if (!loggedInViaKeytab) { - // Keytab is copied by YARN to the working directory of the AM, so full path is - // not needed. - logInfo(s"Attempting to login to KDC using principal: $principal") - loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( - principal, keytab) - logInfo("Successfully logged into KDC.") - loggedInViaKeytab = true - // Not exactly sure when HDFS re-logs in, be safe and do it ourselves. - // Periodically check and relogin this keytab. The UGI will take care of not relogging in - // if it is not necessary to relogin. - val reloginRunnable = new Runnable { - override def run(): Unit = { - try { - loggedInUGI.checkTGTAndReloginFromKeytab() - } catch { - case e: Exception => - logError("Error while attempting tp relogin to KDC", e) - } - } - } - delegationTokenRenewer.schedule(reloginRunnable, 6, TimeUnit.HOURS) - } - val nns = getNameNodesToAccess(sparkConf) - obtainTokensForNamenodes(nns, conf, loggedInUGI.getCredentials) - val remoteFs = FileSystem.get(conf) - // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM - // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file - // and update the lastCredentialsFileSuffix. - if (lastCredentialsFileSuffix == 0) { - val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) - listCredentialsFilesSorted(remoteFs, credentialsPath) - .lastOption.foreach { status => - lastCredentialsFileSuffix = getSuffixForCredentialsPath(status) - } - } - val nextSuffix = lastCredentialsFileSuffix + 1 - val tokenPathStr = - sparkConf.get("spark.yarn.credentials.file") + "-" + nextSuffix - val tokenPath = new Path(tokenPathStr) - val tempTokenPath = new Path(tokenPathStr + ".tmp") - logInfo("Writing out delegation tokens to " + tempTokenPath.toString) - val stream = Option(remoteFs.create(tempTokenPath, true)) - try { - stream.foreach { s => - loggedInUGI.getCredentials.writeTokenStorageToStream(s) - s.hflush() - s.close() - logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") - remoteFs.rename(tempTokenPath, tokenPath) - logInfo("Delegation token file rename complete.") - } - } finally { - stream.foreach(_.close()) - } - lastCredentialsFileSuffix = nextSuffix - } - - private def listCredentialsFilesSorted( - remoteFs: FileSystem, - credentialsFilePath: Path): Array[FileStatus] = { - val fileStatuses = remoteFs.listStatus(credentialsFilePath.getParent, - new PathFilter { - override def accept(path: Path): Boolean = { - val name = path.getName - name.startsWith(credentialsFilePath.getName) && !name.endsWith(".tmp") - } - }) - Arrays.sort(fileStatuses, new Comparator[FileStatus] { - override def compare(o1: FileStatus, o2: FileStatus): Int = { - Longs.compare(o1.getModificationTime, o2.getModificationTime) - } - }) - fileStatuses - } - - private def getSuffixForCredentialsPath(credentialsStatus: FileStatus): Int = { - val fileName = credentialsStatus.getPath.getName - fileName.substring(fileName.lastIndexOf("-") + 1).toInt - } - override def updateCredentialsIfRequired(): Unit = { - try { - sparkConf.getOption("spark.yarn.credentials.file").foreach { credentialsFile => - val credentialsFilePath = new Path(credentialsFile) - val remoteFs = FileSystem.get(conf) - listCredentialsFilesSorted(remoteFs, credentialsFilePath) - .lastOption.foreach { credentialsStatus => - val suffix = getSuffixForCredentialsPath(credentialsStatus) - if (suffix > lastCredentialsFileSuffix) { - logInfo("Reading new delegation tokens from " + credentialsStatus.getPath) - val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) - lastCredentialsFileSuffix = suffix - UserGroupInformation.getCurrentUser.addCredentials(newCredentials) - val totalValidity = getLatestValidity - credentialsStatus.getModificationTime - val timeToRunRenewal = - credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong - val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() - logInfo("Updated delegation tokens, will check for new tokens in " + - timeFromNowToRenewal + " millis") - delegationTokenRenewer.schedule( - executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) - } else { - // Check every hour to see if new credentials arrived. - logInfo("Updated delegation tokens were expected, but the driver has not updated the " + - "tokens yet, will check again in an hour.") - delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) - } - } - } - } catch { - // Since the file may get deleted while we are reading it, catch the Exception and come - // back in an hour to try again - case e: Exception => - logWarning("Error while trying to update credentials, will try again in 1 hour", e) - delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) - } - } - - private[spark] def getCredentialsFromHDFSFile( - remoteFs: FileSystem, - tokenPath: Path): Credentials = { - val stream = remoteFs.open(tokenPath) - try { - val newCredentials = new Credentials() - newCredentials.readFields(stream) - newCredentials - } finally { - stream.close() - } - } - - private[spark] def getLatestValidity: Long = { - UserGroupInformation.getCurrentUser.getCredentials.getAllTokens - .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) - .map { t => - val identifier = new DelegationTokenIdentifier() - identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) - identifier.getMaxDate - }.foldLeft(0L)(math.max) - } /** * Get the list of namenodes the user may access. */ From bcd11f92ed75bfff5dda1f38d85bd757c07bb122 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 8 Apr 2015 12:58:24 -0700 Subject: [PATCH 26/41] Refactor AM and Executor token update code into separate classes, also send tokens via akka on executor startup. --- .../executor/CoarseGrainedExecutorBackend.scala | 11 ++++++++++- .../cluster/CoarseGrainedClusterMessage.scala | 5 ++++- .../cluster/CoarseGrainedSchedulerBackend.scala | 9 ++++++++- .../deploy/yarn/AMDelegationTokenRenewer.scala | 16 +++++++++++----- .../spark/deploy/yarn/ApplicationMaster.scala | 10 +++++++--- .../org/apache/spark/deploy/yarn/Client.scala | 6 ++---- 6 files changed, 42 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e3b7e37344bf7..4709457be0075 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -17,6 +17,7 @@ package org.apache.spark.executor +import java.io.{ByteArrayInputStream, DataInputStream} import java.net.URL import java.nio.ByteBuffer @@ -26,6 +27,7 @@ import scala.concurrent.Await import akka.actor.{Actor, ActorSelection, Props} import akka.pattern.Patterns import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} +import org.apache.hadoop.security.{UserGroupInformation, Credentials} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState @@ -63,10 +65,17 @@ private[spark] class CoarseGrainedExecutorBackend( } override def receiveWithLogging = { - case RegisteredExecutor => + case RegisteredExecutor(tokens) => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) + tokens.foreach { x => + val inStream = new DataInputStream(new ByteArrayInputStream(x.value.array())) + val creds = new Credentials() + creds.readFields(inStream) + inStream.close() + UserGroupInformation.getCurrentUser.addCredentials(creds) + } case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 9bf74f4be198d..df7bc31ea82b0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -34,7 +34,10 @@ private[spark] object CoarseGrainedClusterMessages { case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) extends CoarseGrainedClusterMessage - case object RegisteredExecutor extends CoarseGrainedClusterMessage + case class RegisteredExecutor(tokens: Option[SerializableBuffer]) + extends CoarseGrainedClusterMessage + + case class NewTokens(tokens: SerializableBuffer) extends CoarseGrainedClusterMessage case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index dae0f50c27f56..9bf737b97542e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -72,6 +72,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Executors we have requested the cluster manager to kill that have not died yet private val executorsPendingToRemove = new HashSet[String] + private var latestTokens: Option[SerializableBuffer] = None + class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { override protected def log = CoarseGrainedSchedulerBackend.this.log private val addressToExecutorId = new HashMap[Address, String] @@ -93,8 +95,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId) } else { logInfo("Registered executor: " + sender + " with ID " + executorId) - sender ! RegisteredExecutor + latestTokens.fold(sender ! RegisteredExecutor(None)) { + x => sender ! RegisteredExecutor(Some(x)) + } addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) @@ -161,6 +165,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste case RetrieveSparkProps => sender ! sparkProperties + + case NewTokens(tokens) => + latestTokens = Some(tokens) } // Make fake resource offers on all executors diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index 54b5369393a82..72c6f1122294f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -16,18 +16,19 @@ */ package org.apache.spark.deploy.yarn +import java.io.{DataOutputStream, ByteArrayOutputStream} +import java.nio.ByteBuffer import java.util.concurrent.{Executors, TimeUnit} -import scala.collection.JavaConversions._ -import scala.concurrent.duration.Duration - +import akka.actor.ActorSelection import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.Utils +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.NewTokens import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.{SerializableBuffer, Utils} /* * The following methods are primarily meant to make sure long-running apps like Spark @@ -58,6 +59,7 @@ class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) private var loggedInViaKeytab = false private var loggedInUGI: UserGroupInformation = null + var driverActor: ActorSelection = null private lazy val hadoopUtil = YarnSparkHadoopUtil.get @@ -175,12 +177,16 @@ class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) val stream = Option(remoteFs.create(tempTokenPath, true)) try { stream.foreach { s => + val baos = new ByteArrayOutputStream() + val dataOutputStream = new DataOutputStream(baos) + loggedInUGI.getCredentials.writeTokenStorageToStream(dataOutputStream) + dataOutputStream.close() loggedInUGI.getCredentials.writeTokenStorageToStream(s) s.hflush() s.close() logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") - remoteFs.rename(tempTokenPath, tokenPath) + driverActor ! NewTokens(new SerializableBuffer(ByteBuffer.wrap(baos.toByteArray))) logInfo("Delegation token file rename complete.") } } finally { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 675afc341a5e8..887c0a113fc39 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -259,9 +259,6 @@ private[spark] class ApplicationMaster( private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() - // If a principal and keytab have been set, use that to create new credentials for executors - // periodically - delegationTokenRenewer.scheduleLoginFromKeytab() userClassThread = startUserApplication() // This a bit hacky, but we need to wait until the spark.driver.port property has @@ -280,6 +277,9 @@ private[spark] class ApplicationMaster( sc.getConf.get("spark.driver.port"), isClusterMode = true) registerAM(sc.ui.map(_.appUIAddress).getOrElse(""), securityMgr) + // If a principal and keytab have been set, use that to create new credentials for executors + // periodically + delegationTokenRenewer.scheduleLoginFromKeytab() userClassThread.join() } } @@ -289,6 +289,9 @@ private[spark] class ApplicationMaster( conf = sparkConf, securityManager = securityMgr)._1 waitForSparkDriver() addAmIpFilter() + // If a principal and keytab have been set, use that to create new credentials for executors + // periodically + delegationTokenRenewer.scheduleLoginFromKeytab() registerAM(sparkConf.get("spark.driver.appUIAddress", ""), securityMgr) // In client mode the actor will stop the reporter thread. @@ -524,6 +527,7 @@ private[spark] class ApplicationMaster( if (!isClusterMode) { context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } + delegationTokenRenewer.driverActor = driver } override def receive = { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 2a27237f9395a..6124f72bfcd02 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -572,15 +572,13 @@ private[spark] class Client( // Generate a file name that can be used for the keytab file, that does not conflict // with any user file. val keytabFileName = f.getName + "-" + UUID.randomUUID().toString - val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(args.principal, args.keytab) - credentials = ugi.getCredentials + UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) loginFromKeytab = true sparkConf.set("spark.yarn.keytab", keytabFileName) sparkConf.set("spark.yarn.principal", args.principal) logInfo("Successfully logged into the KDC.") - } else { - credentials = UserGroupInformation.getCurrentUser.getCredentials } + credentials = UserGroupInformation.getCurrentUser.getCredentials } /** From 7f1bc58affe3c88762c32f9641d0f4671a408b8a Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 9 Apr 2015 13:52:48 -0700 Subject: [PATCH 27/41] Minor fixes, cleanup. --- .../ExecutorDelegationTokenUpdater.scala | 4 +- .../yarn/AMDelegationTokenRenewer.scala | 67 ++++++++++--------- 2 files changed, 36 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala index 8dce094928222..bd7b107725a18 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala @@ -84,9 +84,7 @@ private[spark] class ExecutorDelegationTokenUpdater( } } - private def getCredentialsFromHDFSFile( - remoteFs: FileSystem, - tokenPath: Path): Credentials = { + private def getCredentialsFromHDFSFile(remoteFs: FileSystem, tokenPath: Path): Credentials = { val stream = remoteFs.open(tokenPath) try { val newCredentials = new Credentials() diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index 72c6f1122294f..96db6419dd86f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -58,7 +58,6 @@ class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) Utils.namedThreadFactory("Delegation Token Refresh Thread")) private var loggedInViaKeytab = false - private var loggedInUGI: UserGroupInformation = null var driverActor: ActorSelection = null private lazy val hadoopUtil = YarnSparkHadoopUtil.get @@ -74,9 +73,17 @@ class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) val keytab = sparkConf.get("spark.yarn.keytab") def getRenewalInterval = { + import scala.concurrent.duration._ val credentials = UserGroupInformation.getCurrentUser.getCredentials - math.max((0.75 * (hadoopUtil.getLatestTokenValidity(credentials) - - System.currentTimeMillis())).toLong, 0L) + val interval = (0.75 * (hadoopUtil.getLatestTokenValidity(credentials) - + System.currentTimeMillis())).toLong + // If only 6 hours left, then force a renewal immediately. This is to avoid tokens with + // very less validity being used on AM restart. + if ((interval millis).toHours <= 6) { + 0L + } else { + interval + } } def scheduleRenewal(runnable: Runnable) = { @@ -89,23 +96,32 @@ class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) val driverTokenRenewerRunnable = new Runnable { override def run(): Unit = { + var wroteNewFiles = false try { writeNewTokensToHDFS(principal, keytab) + wroteNewFiles = true + cleanupOldFiles() } catch { case e: Exception => - logWarning("Failed to write out new credentials to HDFS, will try again in an " + - "hour! If this happens too often tasks will fail.", e) - delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) - return + // If the exception was due to some issue deleting files, don't worry about it - + // just try to clean up next time. Else, reschedule for an hour later so new + // tokens get written out. + if (!wroteNewFiles) { + logWarning("Failed to write out new credentials to HDFS, will try again in an " + + "hour! If this happens too often tasks will fail.", e) + delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) + return + } else { + logWarning("Error while attempting to clean up old delegation token files. " + + "Cleanup will be reattempted the next time new tokens are being written.") + } } - cleanupOldFiles() scheduleRenewal(this) } } - // If this is an AM restart, it is possible that the original tokens have expired, which - // means we need to login immediately to get new tokens. - if (getRenewalInterval == 0) writeNewTokensToHDFS(principal, keytab) - // Schedule update of credentials + // Schedule update of credentials. This handles the case of updating the tokens right now + // as well, since the renenwal interval will be 0, and the thread will get scheduled + // immediately. scheduleRenewal(driverTokenRenewerRunnable) } @@ -135,34 +151,20 @@ class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) // Keytab is copied by YARN to the working directory of the AM, so full path is // not needed. logInfo(s"Attempting to login to KDC using principal: $principal") - loggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI( - principal, keytab) + UserGroupInformation.loginUserFromKeytab(principal, keytab) logInfo("Successfully logged into KDC.") loggedInViaKeytab = true - // Not exactly sure when HDFS re-logs in, be safe and do it ourselves. - // Periodically check and relogin this keytab. The UGI will take care of not relogging in - // if it is not necessary to relogin. - val reloginRunnable = new Runnable { - override def run(): Unit = { - try { - loggedInUGI.checkTGTAndReloginFromKeytab() - } catch { - case e: Exception => - logError("Error while attempting tp relogin to KDC", e) - } - } - } - delegationTokenRenewer.schedule(reloginRunnable, 6, TimeUnit.HOURS) } val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) - YarnSparkHadoopUtil.get.obtainTokensForNamenodes(nns, hadoopConf, loggedInUGI.getCredentials) + hadoopUtil.obtainTokensForNamenodes( + nns, hadoopConf, UserGroupInformation.getCurrentUser.getCredentials) val remoteFs = FileSystem.get(hadoopConf) // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file // and update the lastCredentialsFileSuffix. if (lastCredentialsFileSuffix == 0) { val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) - SparkHadoopUtil.get.listFilesSorted( + hadoopUtil.listFilesSorted( remoteFs, credentialsPath.getParent, credentialsPath.getName, ".tmp") .lastOption.foreach { status => lastCredentialsFileSuffix = getSuffixForCredentialsPath(status) @@ -179,9 +181,10 @@ class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) stream.foreach { s => val baos = new ByteArrayOutputStream() val dataOutputStream = new DataOutputStream(baos) - loggedInUGI.getCredentials.writeTokenStorageToStream(dataOutputStream) + val credentials = UserGroupInformation.getCurrentUser.getCredentials + credentials.writeTokenStorageToStream(dataOutputStream) dataOutputStream.close() - loggedInUGI.getCredentials.writeTokenStorageToStream(s) + credentials.writeTokenStorageToStream(s) s.hflush() s.close() logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") From e800c8b0929d2696aa421f98f89256ef9f498959 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 9 Apr 2015 23:06:35 -0700 Subject: [PATCH 28/41] Restore original RegisteredExecutor message, and send new tokens via NewTokens message. --- .../executor/CoarseGrainedExecutorBackend.scala | 16 ++++++++-------- .../cluster/CoarseGrainedClusterMessage.scala | 3 +-- .../cluster/CoarseGrainedSchedulerBackend.scala | 4 ++-- 3 files changed, 11 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e8bcfc0ec9969..f7a1d3347bb96 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -72,17 +72,17 @@ private[spark] class CoarseGrainedExecutorBackend( } override def receive: PartialFunction[Any, Unit] = { - case RegisteredExecutor(tokens) => + case RegisteredExecutor => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) - tokens.foreach { x => - val inStream = new DataInputStream(new ByteArrayInputStream(x.value.array())) - val creds = new Credentials() - creds.readFields(inStream) - inStream.close() - UserGroupInformation.getCurrentUser.addCredentials(creds) - } + + case NewTokens(tokens) => + val inStream = new DataInputStream(new ByteArrayInputStream(tokens.value.array())) + val creds = new Credentials() + creds.readFields(inStream) + inStream.close() + UserGroupInformation.getCurrentUser.addCredentials(creds) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index eaf5b50bfaac3..407a3bba02ff9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -35,8 +35,7 @@ private[spark] object CoarseGrainedClusterMessages { case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) extends CoarseGrainedClusterMessage - case class RegisteredExecutor(tokens: Option[SerializableBuffer]) - extends CoarseGrainedClusterMessage + case object RegisteredExecutor extends CoarseGrainedClusterMessage case class NewTokens(tokens: SerializableBuffer) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index dfc73abbb9f4b..472eb110733f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -124,8 +124,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp context.reply(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) } else { logInfo("Registered executor: " + executorRef + " with ID " + executorId) - context.reply(RegisteredExecutor(latestTokens)) - + context.reply(RegisteredExecutor) + latestTokens.foreach(x => context.reply(NewTokens(x))) addressToExecutorId(executorRef.address) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) From 8a4f268c9a6291264e658c52adb35a075f324b8b Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 15 Apr 2015 17:21:29 -0700 Subject: [PATCH 29/41] Added docs in the security guide. Changed some code to ensure that the renewer objects are created only if required. --- .../ExecutorDelegationTokenUpdater.scala | 67 ++++++----- .../apache/spark/deploy/SparkHadoopUtil.scala | 10 +- .../spark/deploy/SparkSubmitArguments.scala | 5 +- .../CoarseGrainedExecutorBackend.scala | 13 ++- docs/security.md | 1 + .../yarn/AMDelegationTokenRenewer.scala | 109 ++++++++---------- .../spark/deploy/yarn/ApplicationMaster.scala | 16 ++- 7 files changed, 113 insertions(+), 108 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala index bd7b107725a18..f9a7fd94e84f0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala @@ -17,15 +17,13 @@ package org.apache.spark.deploy import java.util.concurrent.{Executors, TimeUnit} -import java.util.{Comparator, Arrays} -import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{PathFilter, FileStatus, Path, FileSystem} +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.util.Utils import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.Utils private[spark] class ExecutorDelegationTokenUpdater( sparkConf: SparkConf, @@ -33,46 +31,47 @@ private[spark] class ExecutorDelegationTokenUpdater( @volatile private var lastCredentialsFileSuffix = 0 - private lazy val delegationTokenRenewer = + private val credentialsFile = sparkConf.get("spark.yarn.credentials.file") + + private val delegationTokenRenewer = Executors.newSingleThreadScheduledExecutor( Utils.namedThreadFactory("Delegation Token Refresh Thread")) // On the executor, this thread wakes up and picks up new tokens from HDFS, if any. - private lazy val executorUpdaterRunnable = + private val executorUpdaterRunnable = new Runnable { override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired()) } def updateCredentialsIfRequired(): Unit = { try { - sparkConf.getOption("spark.yarn.credentials.file").foreach { credentialsFile => - val credentials = UserGroupInformation.getCurrentUser.getCredentials - val credentialsFilePath = new Path(credentialsFile) - val remoteFs = FileSystem.get(hadoopConf) - SparkHadoopUtil.get.listFilesSorted( - remoteFs, credentialsFilePath.getParent, credentialsFilePath.getName, ".tmp") - .lastOption.foreach { credentialsStatus => - val suffix = getSuffixForCredentialsPath(credentialsStatus) - if (suffix > lastCredentialsFileSuffix) { - logInfo("Reading new delegation tokens from " + credentialsStatus.getPath) - val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) - lastCredentialsFileSuffix = suffix - UserGroupInformation.getCurrentUser.addCredentials(newCredentials) - val totalValidity = SparkHadoopUtil.get.getLatestTokenValidity(credentials) - - credentialsStatus.getModificationTime - val timeToRunRenewal = - credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong - val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() - logInfo("Updated delegation tokens, will check for new tokens in " + - timeFromNowToRenewal + " millis") - delegationTokenRenewer.schedule( - executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) - } else { - // Check every hour to see if new credentials arrived. - logInfo("Updated delegation tokens were expected, but the driver has not updated the " + - "tokens yet, will check again in an hour.") - delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) - } + val credentials = UserGroupInformation.getCurrentUser.getCredentials + val credentialsFilePath = new Path(credentialsFile) + val remoteFs = FileSystem.get(hadoopConf) + SparkHadoopUtil.get.listFilesSorted( + remoteFs, credentialsFilePath.getParent, credentialsFilePath.getName, ".tmp") + .lastOption + .foreach { credentialsStatus => + val suffix = getSuffixForCredentialsPath(credentialsStatus) + if (suffix > lastCredentialsFileSuffix) { + logInfo("Reading new delegation tokens from " + credentialsStatus.getPath) + val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) + lastCredentialsFileSuffix = suffix + UserGroupInformation.getCurrentUser.addCredentials(newCredentials) + val totalValidity = SparkHadoopUtil.get.getLatestTokenValidity(credentials) - + credentialsStatus.getModificationTime + val timeToRunRenewal = + credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong + val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() + logInfo("Updated delegation tokens, will check for new tokens in " + + timeFromNowToRenewal + " millis") + delegationTokenRenewer.schedule( + executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) + } else { + // Check every hour to see if new credentials arrived. + logInfo("Updated delegation tokens were expected, but the driver has not updated the " + + "tokens yet, will check again in an hour.") + delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) } } } catch { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 1006c6d0af6f3..64ac9844c1751 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -20,11 +20,11 @@ package org.apache.spark.deploy import java.io.{ByteArrayInputStream, DataInputStream} import java.lang.reflect.Method import java.security.PrivilegedExceptionAction -import java.util.{Comparator, Arrays} +import java.util.{Arrays, Comparator} import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{PathFilter, FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.mapred.JobConf @@ -44,7 +44,7 @@ import scala.collection.JavaConversions._ */ @DeveloperApi class SparkHadoopUtil extends Logging { - val sparkConf = new SparkConf() + private val sparkConf = new SparkConf() val conf: Configuration = newConfiguration(sparkConf) UserGroupInformation.setConfiguration(conf) @@ -209,12 +209,12 @@ class SparkHadoopUtil extends Logging { /** * Lists all the files in a directory with the specified prefix, and does not end with the - * given suffix. + * given suffix. The returned {{FileStatus}} instances are sorted by the modification times of + * the respective files. * @param remoteFs * @param prefix * @return */ - def listFilesSorted( remoteFs: FileSystem, dir: Path, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 148a258ae2a09..7810fb86422b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -501,7 +501,10 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --principal PRINCIPAL Principal to be used to login to KDC, while running on | secure HDFS. | --keytab KEYTAB The full path to the file that contains the keytab for the - | principal specified above. + | principal specified above. This keytab will be copied to + | the node running the Application Master via the Secure + | Distributed Cache, for renewing the login tickets and the + | delegation tokens periodically. """.stripMargin ) SparkSubmit.exitFn() diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index f7a1d3347bb96..7e8398a198bbf 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -174,9 +174,14 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { driverConf.set(key, value) } } - // Periodically update the credentials for this user to ensure HDFS tokens get updated. - val tokenUpdater = new ExecutorDelegationTokenUpdater(driverConf, SparkHadoopUtil.get.conf) - tokenUpdater.updateCredentialsIfRequired() + var tokenUpdaterOption: Option[ExecutorDelegationTokenUpdater] = None + if(driverConf.contains("spark.yarn.credentials.file")) { + // Periodically update the credentials for this user to ensure HDFS tokens get updated. + tokenUpdaterOption = + Some(new ExecutorDelegationTokenUpdater(driverConf, SparkHadoopUtil.get.conf)) + tokenUpdaterOption.get.updateCredentialsIfRequired() + } + val env = SparkEnv.createExecutorEnv( driverConf, executorId, hostname, port, cores, isLocal = false) @@ -192,7 +197,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } env.actorSystem.awaitTermination() - tokenUpdater.stop() + tokenUpdaterOption.foreach(_.stop()) env.rpcEnv.awaitTermination() } } diff --git a/docs/security.md b/docs/security.md index c034ba12ff1fc..098566a8cc549 100644 --- a/docs/security.md +++ b/docs/security.md @@ -31,6 +31,7 @@ SSL must be configured on each node and configured for each component involved i ### YARN mode The key-store can be prepared on the client side and then distributed and used by the executors as the part of the application. It is possible because the user is able to deploy files before the application is started in YARN by using `spark.yarn.dist.files` or `spark.yarn.dist.archives` configuration settings. The responsibility for encryption of transferring these files is on YARN side and has nothing to do with Spark. +For long-running apps like Spark Streaming apps be able to write to HDFS, it is possible to pass a principal and keytab to `spark-submit` via the `--principal` and `--keytab` parameters respectively. The keytab passed in will be copied over to the machine running the Application Master securely via the Hadoop Distributed Cache. The Kerberos login will be periodically renewed using this principal and keytab and the delegation tokens required for HDFS will be generated periodically so the application can continue writing to HDFS. ### Standalone mode The user needs to provide key-stores and configuration options for master and workers. They have to be set by attaching appropriate Java system properties in `SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to use the SSL settings inherited from the worker which spawned that executor. It can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that parameter is set, the settings provided by user on the client side, are not used by the executors. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index 51276be7435d3..51b2ae7775a56 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -16,16 +16,14 @@ */ package org.apache.spark.deploy.yarn -import java.io.{DataOutputStream, ByteArrayOutputStream} +import java.io.{ByteArrayOutputStream, DataOutputStream} import java.nio.ByteBuffer import java.util.concurrent.{Executors, TimeUnit} -import akka.actor.ActorSelection import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.NewTokens import org.apache.spark.{Logging, SparkConf} @@ -50,18 +48,20 @@ import org.apache.spark.util.{SerializableBuffer, Utils} * appeared, it will read the credentials and update the currently running UGI with it. This * process happens again once 80% of the validity of this has expired. */ -class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) extends Logging { +private[yarn] class AMDelegationTokenRenewer( + sparkConf: SparkConf, + hadoopConf: Configuration) extends Logging { private var lastCredentialsFileSuffix = 0 - private lazy val delegationTokenRenewer = + private val delegationTokenRenewer = Executors.newSingleThreadScheduledExecutor( Utils.namedThreadFactory("Delegation Token Refresh Thread")) private var loggedInViaKeytab = false var driverEndPoint: RpcEndpointRef = null - private lazy val hadoopUtil = YarnSparkHadoopUtil.get + private val hadoopUtil = YarnSparkHadoopUtil.get /** * Schedule a login from the keytab and principal set using the --principal and --keytab @@ -70,62 +70,51 @@ class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) * to do the login. This method is a no-op in non-YARN mode. */ private[spark] def scheduleLoginFromKeytab(): Unit = { - sparkConf.getOption("spark.yarn.principal").foreach { principal => - val keytab = sparkConf.get("spark.yarn.keytab") - - def getRenewalInterval: Long = { - import scala.concurrent.duration._ - val credentials = UserGroupInformation.getCurrentUser.getCredentials - val interval = (0.75 * (hadoopUtil.getLatestTokenValidity(credentials) - - System.currentTimeMillis())).toLong - // If only 6 hours left, then force a renewal immediately. This is to avoid tokens with - // very less validity being used on AM restart. - if ((interval millis).toHours <= 6) { - 0L - } else { - interval - } + val principal = sparkConf.get("spark.yarn.principal") + val keytab = sparkConf.get("spark.yarn.keytab") + + def getRenewalInterval: Long = { + import scala.concurrent.duration._ + val credentials = UserGroupInformation.getCurrentUser.getCredentials + val interval = (0.75 * (hadoopUtil.getLatestTokenValidity(credentials) - + System.currentTimeMillis())).toLong + // If only 6 hours left, then force a renewal immediately. This is to avoid tokens with + // very less validity being used on AM restart. + if ((interval millis).toHours <= 6) { + 0L + } else { + interval } + } - def scheduleRenewal(runnable: Runnable): Unit = { - val renewalInterval = getRenewalInterval - logInfo(s"Scheduling login from keytab in $renewalInterval millis.") - delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) - } + def scheduleRenewal(runnable: Runnable): Unit = { + val renewalInterval = getRenewalInterval + logInfo(s"Scheduling login from keytab in $renewalInterval millis.") + delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) + } - // This thread periodically runs on the driver to update the delegation tokens on HDFS. - val driverTokenRenewerRunnable = - new Runnable { - override def run(): Unit = { - var wroteNewFiles = false - try { - writeNewTokensToHDFS(principal, keytab) - wroteNewFiles = true - cleanupOldFiles() - } catch { - case e: Exception => - // If the exception was due to some issue deleting files, don't worry about it - - // just try to clean up next time. Else, reschedule for an hour later so new - // tokens get written out. - if (!wroteNewFiles) { - logWarning("Failed to write out new credentials to HDFS, will try again in an " + - "hour! If this happens too often tasks will fail.", e) - delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) - return - } else { - logWarning("Error while attempting to clean up old delegation token files. " + - "Cleanup will be reattempted the next time new tokens are being written.") - } - } - scheduleRenewal(this) + // This thread periodically runs on the driver to update the delegation tokens on HDFS. + val driverTokenRenewerRunnable = + new Runnable { + override def run(): Unit = { + try { + writeNewTokensToHDFS(principal, keytab) + cleanupOldFiles() + } catch { + case e: Exception => + // Log the error and try to write new tokens back in an hour + logWarning("Failed to write out new credentials to HDFS, will try again in an " + + "hour! If this happens too often tasks will fail.", e) + delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) + return } + scheduleRenewal(this) } - // Schedule update of credentials. This handles the case of updating the tokens right now - // as well, since the renenwal interval will be 0, and the thread will get scheduled - // immediately. - scheduleRenewal(driverTokenRenewerRunnable) - - } + } + // Schedule update of credentials. This handles the case of updating the tokens right now + // as well, since the renenwal interval will be 0, and the thread will get scheduled + // immediately. + scheduleRenewal(driverTokenRenewerRunnable) } // Keeps only files that are newer than 30 days, and deletes everything else. At least 5 files @@ -135,9 +124,11 @@ class AMDelegationTokenRenewer(sparkConf: SparkConf, hadoopConf: Configuration) try { val remoteFs = FileSystem.get(hadoopConf) val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) + val thresholdTime = System.currentTimeMillis() - (30 days).toMillis hadoopUtil.listFilesSorted( - remoteFs, credentialsPath.getParent, credentialsPath.getName, ".tmp").dropRight(5) - .takeWhile(_.getModificationTime < System.currentTimeMillis() - (30 days).toMillis) + remoteFs, credentialsPath.getParent, credentialsPath.getName, ".tmp") + .dropRight(5) + .takeWhile(_.getModificationTime < thresholdTime) .foreach(x => remoteFs.delete(x.getPath, true)) } catch { // Such errors are not fatal, so don't throw. Make sure they are logged though diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index d43914ad711ed..8c8fa602634bc 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -76,7 +76,7 @@ private[spark] class ApplicationMaster( // Fields used in cluster mode. private val sparkContextRef = new AtomicReference[SparkContext](null) - private lazy val delegationTokenRenewer = new AMDelegationTokenRenewer(sparkConf, yarnConf) + private var delegationTokenRenewerOption: Option[AMDelegationTokenRenewer] = None final def run(): Int = { try { @@ -140,6 +140,12 @@ private[spark] class ApplicationMaster( // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) + // If the credentials file config is present, we must periodically renew tokens. So create + // a new AMDelegationTokenRenewer + if(sparkConf.contains("spark.yarn.credentials.file")) { + delegationTokenRenewerOption = Some(new AMDelegationTokenRenewer(sparkConf, yarnConf)) + } + if (isClusterMode) { runDriver(securityMgr) } else { @@ -204,7 +210,7 @@ private[spark] class ApplicationMaster( logDebug("shutting down user thread") userClassThread.interrupt() } - if (!inShutdown) delegationTokenRenewer.stop() + if (!inShutdown) delegationTokenRenewerOption.foreach(_.stop()) } } } @@ -254,7 +260,7 @@ private[spark] class ApplicationMaster( SparkEnv.driverActorSystemName, RpcAddress(host, port.toInt), YarnSchedulerBackend.ENDPOINT_NAME) - delegationTokenRenewer.driverEndPoint = driverEndpoint + delegationTokenRenewerOption.foreach(_.driverEndPoint = driverEndpoint) amEndpoint = rpcEnv.setupEndpoint("YarnAM", new AMEndpoint(rpcEnv, driverEndpoint, isClusterMode)) } @@ -281,7 +287,7 @@ private[spark] class ApplicationMaster( registerAM(sc.ui.map(_.appUIAddress).getOrElse(""), securityMgr) // If a principal and keytab have been set, use that to create new credentials for executors // periodically - delegationTokenRenewer.scheduleLoginFromKeytab() + delegationTokenRenewerOption.foreach(_.scheduleLoginFromKeytab()) userClassThread.join() } } @@ -292,7 +298,7 @@ private[spark] class ApplicationMaster( addAmIpFilter() // If a principal and keytab have been set, use that to create new credentials for executors // periodically - delegationTokenRenewer.scheduleLoginFromKeytab() + delegationTokenRenewerOption.foreach(_.scheduleLoginFromKeytab()) registerAM(sparkConf.get("spark.driver.appUIAddress", ""), securityMgr) // In client mode the actor will stop the reporter thread. From bc083e34108da23e10fbb730b911860540e08d3c Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 20 Apr 2015 22:35:25 -0700 Subject: [PATCH 30/41] Overload RegisteredExecutor to send tokens. Minor doc updates. --- .../CoarseGrainedExecutorBackend.scala | 18 +++++++++--------- .../cluster/CoarseGrainedClusterMessage.scala | 3 ++- .../CoarseGrainedSchedulerBackend.scala | 3 +-- docs/security.md | 3 ++- .../deploy/yarn/AMDelegationTokenRenewer.scala | 15 ++++++++------- .../deploy/yarn/YarnSparkHadoopUtilSuite.scala | 2 +- 6 files changed, 23 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 7e8398a198bbf..c74b70bf22b79 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -55,7 +55,7 @@ private[spark] class CoarseGrainedExecutorBackend( logInfo("Connecting to driver: " + driverUrl) rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => driver = Some(ref) - ref.sendWithReply[RegisteredExecutor.type]( + ref.sendWithReply[RegisteredExecutor]( RegisterExecutor(executorId, self, hostPort, cores, extractLogUrls)) } onComplete { case Success(msg) => Utils.tryLogNonFatalError { @@ -72,17 +72,17 @@ private[spark] class CoarseGrainedExecutorBackend( } override def receive: PartialFunction[Any, Unit] = { - case RegisteredExecutor => + case RegisteredExecutor(tokens) => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) - - case NewTokens(tokens) => - val inStream = new DataInputStream(new ByteArrayInputStream(tokens.value.array())) - val creds = new Credentials() - creds.readFields(inStream) - inStream.close() - UserGroupInformation.getCurrentUser.addCredentials(creds) + tokens.foreach { tokenBuffer => + val inStream = new DataInputStream(new ByteArrayInputStream(tokenBuffer.value.array())) + val creds = new Credentials() + creds.readFields(inStream) + inStream.close() + UserGroupInformation.getCurrentUser.addCredentials(creds) + } case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 407a3bba02ff9..eaf5b50bfaac3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -35,7 +35,8 @@ private[spark] object CoarseGrainedClusterMessages { case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) extends CoarseGrainedClusterMessage - case object RegisteredExecutor extends CoarseGrainedClusterMessage + case class RegisteredExecutor(tokens: Option[SerializableBuffer]) + extends CoarseGrainedClusterMessage case class NewTokens(tokens: SerializableBuffer) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 33f26076533a1..903262356cdbd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -125,8 +125,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp context.reply(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) } else { logInfo("Registered executor: " + executorRef + " with ID " + executorId) - context.reply(RegisteredExecutor) - latestTokens.foreach(x => context.reply(NewTokens(x))) + context.reply(RegisteredExecutor(latestTokens)) addressToExecutorId(executorRef.address) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) diff --git a/docs/security.md b/docs/security.md index 098566a8cc549..d4ffa60e59a33 100644 --- a/docs/security.md +++ b/docs/security.md @@ -31,7 +31,8 @@ SSL must be configured on each node and configured for each component involved i ### YARN mode The key-store can be prepared on the client side and then distributed and used by the executors as the part of the application. It is possible because the user is able to deploy files before the application is started in YARN by using `spark.yarn.dist.files` or `spark.yarn.dist.archives` configuration settings. The responsibility for encryption of transferring these files is on YARN side and has nothing to do with Spark. -For long-running apps like Spark Streaming apps be able to write to HDFS, it is possible to pass a principal and keytab to `spark-submit` via the `--principal` and `--keytab` parameters respectively. The keytab passed in will be copied over to the machine running the Application Master securely via the Hadoop Distributed Cache. The Kerberos login will be periodically renewed using this principal and keytab and the delegation tokens required for HDFS will be generated periodically so the application can continue writing to HDFS. + +For long-running apps like Spark Streaming apps to be able to write to HDFS, it is possible to pass a principal and keytab to `spark-submit` via the `--principal` and `--keytab` parameters respectively. The keytab passed in will be copied over to the machine running the Application Master via the Hadoop Distributed Cache (securely - if YARN is configured with SSL and HDFS encryption is enabled). The Kerberos login will be periodically renewed using this principal and keytab and the delegation tokens required for HDFS will be generated periodically so the application can continue writing to HDFS. ### Standalone mode The user needs to provide key-stores and configuration options for master and workers. They have to be set by attaching appropriate Java system properties in `SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to use the SSL settings inherited from the worker which spawned that executor. It can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that parameter is set, the settings provided by user on the client side, are not used by the executors. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index 51b2ae7775a56..f6452c77b57d1 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -78,13 +78,14 @@ private[yarn] class AMDelegationTokenRenewer( val credentials = UserGroupInformation.getCurrentUser.getCredentials val interval = (0.75 * (hadoopUtil.getLatestTokenValidity(credentials) - System.currentTimeMillis())).toLong - // If only 6 hours left, then force a renewal immediately. This is to avoid tokens with - // very less validity being used on AM restart. - if ((interval millis).toHours <= 6) { - 0L - } else { - interval - } +// // If only 6 hours left, then force a renewal immediately. This is to avoid tokens with +// // very less validity being used on AM restart. +// if ((interval millis).toHours <= 6) { +// 0L +// } else { +// interval +// } + interval } def scheduleRenewal(runnable: Runnable): Unit = { diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 69b9576e81ce2..e10b985c3c236 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -29,7 +29,7 @@ import org.scalatest.{FunSuite, Matchers} import org.apache.hadoop.yarn.api.records.ApplicationAccessType -import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.util.Utils From 42eead40369df6c0a87ace853f5211e271d23ad0 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 27 Apr 2015 17:22:16 -0700 Subject: [PATCH 31/41] Remove RPC part. Refactor and move methods around, use renewal interval rather than max lifetime to create new tokens. --- .../ExecutorDelegationTokenUpdater.scala | 40 ++--- .../apache/spark/deploy/SparkHadoopUtil.scala | 26 +++- .../CoarseGrainedExecutorBackend.scala | 15 +- .../cluster/CoarseGrainedClusterMessage.scala | 5 +- .../CoarseGrainedSchedulerBackend.scala | 4 +- docs/security.md | 2 +- .../yarn/AMDelegationTokenRenewer.scala | 144 +++++++++--------- .../spark/deploy/yarn/ApplicationMaster.scala | 3 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 15 +- 9 files changed, 128 insertions(+), 126 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala index f9a7fd94e84f0..b3cb17252b9ba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala @@ -25,6 +25,8 @@ import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.Utils +import scala.util.control.NonFatal + private[spark] class ExecutorDelegationTokenUpdater( sparkConf: SparkConf, hadoopConf: Configuration) extends Logging { @@ -45,39 +47,41 @@ private[spark] class ExecutorDelegationTokenUpdater( def updateCredentialsIfRequired(): Unit = { try { - val credentials = UserGroupInformation.getCurrentUser.getCredentials val credentialsFilePath = new Path(credentialsFile) val remoteFs = FileSystem.get(hadoopConf) SparkHadoopUtil.get.listFilesSorted( - remoteFs, credentialsFilePath.getParent, credentialsFilePath.getName, ".tmp") - .lastOption - .foreach { credentialsStatus => - val suffix = getSuffixForCredentialsPath(credentialsStatus) + remoteFs, credentialsFilePath.getParent, + credentialsFilePath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) + .lastOption.foreach { credentialsStatus => + val suffix = SparkHadoopUtil.get.getSuffixForCredentialsPath(credentialsStatus.getPath) if (suffix > lastCredentialsFileSuffix) { logInfo("Reading new delegation tokens from " + credentialsStatus.getPath) val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) lastCredentialsFileSuffix = suffix UserGroupInformation.getCurrentUser.addCredentials(newCredentials) - val totalValidity = SparkHadoopUtil.get.getLatestTokenValidity(credentials) - - credentialsStatus.getModificationTime - val timeToRunRenewal = - credentialsStatus.getModificationTime + (0.8 * totalValidity).toLong - val timeFromNowToRenewal = timeToRunRenewal - System.currentTimeMillis() - logInfo("Updated delegation tokens, will check for new tokens in " + - timeFromNowToRenewal + " millis") - delegationTokenRenewer.schedule( - executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) + logInfo("Tokens updated from credentials file.") } else { // Check every hour to see if new credentials arrived. logInfo("Updated delegation tokens were expected, but the driver has not updated the " + "tokens yet, will check again in an hour.") delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) + return } } + val timeFromNowToRenewal = + SparkHadoopUtil.get.getTimeFromNowToRenewal( + 0.8, UserGroupInformation.getCurrentUser.getCredentials) + if (timeFromNowToRenewal <= 0) { + executorUpdaterRunnable.run() + } else { + logInfo(s"Scheduling token refresh from HDFS in $timeFromNowToRenewal millis.") + delegationTokenRenewer.schedule( + executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) + } } catch { // Since the file may get deleted while we are reading it, catch the Exception and come // back in an hour to try again - case e: Exception => + case NonFatal(e) => logWarning("Error while trying to update credentials, will try again in 1 hour", e) delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) } @@ -87,7 +91,7 @@ private[spark] class ExecutorDelegationTokenUpdater( val stream = remoteFs.open(tokenPath) try { val newCredentials = new Credentials() - newCredentials.readFields(stream) + newCredentials.readTokenStorageStream(stream) newCredentials } finally { stream.close() @@ -98,8 +102,4 @@ private[spark] class ExecutorDelegationTokenUpdater( delegationTokenRenewer.shutdown() } - private def getSuffixForCredentialsPath(credentialsStatus: FileStatus): Int = { - val fileName = credentialsStatus.getPath.getName - fileName.substring(fileName.lastIndexOf("-") + 1).toInt - } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 3fdcda3e3e27e..2ede7aa7be0c5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -36,6 +36,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils import scala.collection.JavaConversions._ +import scala.concurrent.duration._ /** * :: DeveloperApi :: @@ -46,6 +47,8 @@ class SparkHadoopUtil extends Logging { private val sparkConf = new SparkConf() val conf: Configuration = newConfiguration(sparkConf) UserGroupInformation.setConfiguration(conf) + private lazy val renewalInterval = + conf.getLong("dfs.namenode.delegation.token.renew-interval", (24 hours).toMillis) /** * Runs the given function with a Hadoop UserGroupInformation as a thread local variable @@ -235,19 +238,28 @@ class SparkHadoopUtil extends Logging { } /** - * Get the latest validity of the HDFS token in the Credentials object. - * @param credentials - * @return + * How much time is remaining (in millis) from now to (fraction * renewal time for the token that + * is valid the latest)? + * This will return -ve (or 0) value if the fraction of validity has already expired. */ - def getLatestTokenValidity(credentials: Credentials): Long = { + def getTimeFromNowToRenewal(fraction: Double, credentials: Credentials): Long = { + val now = System.currentTimeMillis() credentials.getAllTokens.filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) .map { t => val identifier = new DelegationTokenIdentifier() identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) - identifier.getMaxDate + (identifier.getIssueDate + fraction * renewalInterval).toLong - now }.foldLeft(0L)(math.max) } + + private[spark] def getSuffixForCredentialsPath(credentialsPath: Path): Int = { + val fileName = credentialsPath.getName + fileName.substring( + fileName.lastIndexOf(SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM) + 1).toInt + } + + private val HADOOP_CONF_PATTERN = "(\\$\\{hadoopconf-[^\\}\\$\\s]+\\})".r.unanchored /** @@ -298,6 +310,10 @@ object SparkHadoopUtil { } } + val SPARK_YARN_CREDS_TEMP_EXTENSION = ".tmp" + + val SPARK_YARN_CREDS_COUNTER_DELIM = "-" + def get: SparkHadoopUtil = { hadoop } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index c74b70bf22b79..bb9ab32e2cd16 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -55,7 +55,7 @@ private[spark] class CoarseGrainedExecutorBackend( logInfo("Connecting to driver: " + driverUrl) rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => driver = Some(ref) - ref.sendWithReply[RegisteredExecutor]( + ref.sendWithReply[RegisteredExecutor.type]( RegisterExecutor(executorId, self, hostPort, cores, extractLogUrls)) } onComplete { case Success(msg) => Utils.tryLogNonFatalError { @@ -72,17 +72,10 @@ private[spark] class CoarseGrainedExecutorBackend( } override def receive: PartialFunction[Any, Unit] = { - case RegisteredExecutor(tokens) => + case RegisteredExecutor=> logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) - tokens.foreach { tokenBuffer => - val inStream = new DataInputStream(new ByteArrayInputStream(tokenBuffer.value.array())) - val creds = new Credentials() - creds.readFields(inStream) - inStream.close() - UserGroupInformation.getCurrentUser.addCredentials(creds) - } case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -175,7 +168,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } } var tokenUpdaterOption: Option[ExecutorDelegationTokenUpdater] = None - if(driverConf.contains("spark.yarn.credentials.file")) { + if (driverConf.contains("spark.yarn.credentials.file")) { + logInfo("Will periodically update credentials from: " + + driverConf.get("spark.yarn.credentials.file")) // Periodically update the credentials for this user to ensure HDFS tokens get updated. tokenUpdaterOption = Some(new ExecutorDelegationTokenUpdater(driverConf, SparkHadoopUtil.get.conf)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index eaf5b50bfaac3..70364cea62a80 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -35,10 +35,7 @@ private[spark] object CoarseGrainedClusterMessages { case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) extends CoarseGrainedClusterMessage - case class RegisteredExecutor(tokens: Option[SerializableBuffer]) - extends CoarseGrainedClusterMessage - - case class NewTokens(tokens: SerializableBuffer) extends CoarseGrainedClusterMessage + case object RegisteredExecutor extends CoarseGrainedClusterMessage case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 903262356cdbd..8ad15677d924f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -66,7 +66,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Executors we have requested the cluster manager to kill that have not died yet private val executorsPendingToRemove = new HashSet[String] - private var latestTokens: Option[SerializableBuffer] = None class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) extends ThreadSafeRpcEndpoint with Logging { @@ -115,7 +114,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp logWarning(s"Attempted to kill task $taskId for unknown executor $executorId.") } - case NewTokens(tokens) => latestTokens = Option(tokens) } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -125,7 +123,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp context.reply(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) } else { logInfo("Registered executor: " + executorRef + " with ID " + executorId) - context.reply(RegisteredExecutor(latestTokens)) + context.reply(RegisteredExecutor) addressToExecutorId(executorRef.address) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) diff --git a/docs/security.md b/docs/security.md index d4ffa60e59a33..0cb38bdcb3667 100644 --- a/docs/security.md +++ b/docs/security.md @@ -32,7 +32,7 @@ SSL must be configured on each node and configured for each component involved i ### YARN mode The key-store can be prepared on the client side and then distributed and used by the executors as the part of the application. It is possible because the user is able to deploy files before the application is started in YARN by using `spark.yarn.dist.files` or `spark.yarn.dist.archives` configuration settings. The responsibility for encryption of transferring these files is on YARN side and has nothing to do with Spark. -For long-running apps like Spark Streaming apps to be able to write to HDFS, it is possible to pass a principal and keytab to `spark-submit` via the `--principal` and `--keytab` parameters respectively. The keytab passed in will be copied over to the machine running the Application Master via the Hadoop Distributed Cache (securely - if YARN is configured with SSL and HDFS encryption is enabled). The Kerberos login will be periodically renewed using this principal and keytab and the delegation tokens required for HDFS will be generated periodically so the application can continue writing to HDFS. +For long-running apps like Spark Streaming apps to be able to write to HDFS, it is possible to pass a principal and keytab to `spark-submit` via the `--principal` and `--keytab` parameters respectively. The keytab passed in will be copied over to the machine running the Application Master via the Hadoop Distributed Cache (securely - if YARN is configured with SSL and HDFS encryption is enabled). The Kerberos login will be periodically renewed using this principal and keytab and the delegation tokens required for HDFS will be generated periodically so the application can continue writing to HDFS. Please note that the HDFS client configuration file, `hdfs-site.xml` on each executor node must have the value of `dfs.namenode.delegation.token.renew-interval` be the same as it is on the HDFS Namenode for this functionality. ### Standalone mode The user needs to provide key-stores and configuration options for master and workers. They have to be set by attaching appropriate Java system properties in `SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to use the SSL settings inherited from the worker which spawned that executor. It can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that parameter is set, the settings provided by user on the client side, are not used by the executors. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index f6452c77b57d1..cb7e996f555ea 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -16,18 +16,16 @@ */ package org.apache.spark.deploy.yarn -import java.io.{ByteArrayOutputStream, DataOutputStream} -import java.nio.ByteBuffer +import java.security.PrivilegedExceptionAction import java.util.concurrent.{Executors, TimeUnit} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.security.UserGroupInformation +import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.NewTokens import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.{SerializableBuffer, Utils} +import org.apache.spark.util.Utils /* * The following methods are primarily meant to make sure long-running apps like Spark @@ -58,11 +56,11 @@ private[yarn] class AMDelegationTokenRenewer( Executors.newSingleThreadScheduledExecutor( Utils.namedThreadFactory("Delegation Token Refresh Thread")) - private var loggedInViaKeytab = false - var driverEndPoint: RpcEndpointRef = null - private val hadoopUtil = YarnSparkHadoopUtil.get + private val daysToKeepFiles = sparkConf.getInt("spark.yarn.credentials.file.retention.days", 5) + private val numFilesToKeep = sparkConf.getInt("spark.yarn.credentials.file.retention.count", 5) + /** * Schedule a login from the keytab and principal set using the --principal and --keytab * arguments to spark-submit. This login happens only when the credentials of the current user @@ -73,25 +71,22 @@ private[yarn] class AMDelegationTokenRenewer( val principal = sparkConf.get("spark.yarn.principal") val keytab = sparkConf.get("spark.yarn.keytab") - def getRenewalInterval: Long = { - import scala.concurrent.duration._ - val credentials = UserGroupInformation.getCurrentUser.getCredentials - val interval = (0.75 * (hadoopUtil.getLatestTokenValidity(credentials) - - System.currentTimeMillis())).toLong -// // If only 6 hours left, then force a renewal immediately. This is to avoid tokens with -// // very less validity being used on AM restart. -// if ((interval millis).toHours <= 6) { -// 0L -// } else { -// interval -// } - interval - } - + /** + * Schedule the renewal of the tokens. If tokens have already expired, this method will + * synchronously renew them. + * @param runnable + */ def scheduleRenewal(runnable: Runnable): Unit = { - val renewalInterval = getRenewalInterval - logInfo(s"Scheduling login from keytab in $renewalInterval millis.") - delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) + val credentials = UserGroupInformation.getCurrentUser.getCredentials + val renewalInterval = hadoopUtil.getTimeFromNowToRenewal(0.75, credentials) + // Run now! + if (renewalInterval <= 0) { + logInfo("HDFS tokens have expired, creating new tokens now.") + runnable.run() + } else { + logInfo(s"Scheduling login from keytab in $renewalInterval millis.") + delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) + } } // This thread periodically runs on the driver to update the delegation tokens on HDFS. @@ -118,17 +113,18 @@ private[yarn] class AMDelegationTokenRenewer( scheduleRenewal(driverTokenRenewerRunnable) } - // Keeps only files that are newer than 30 days, and deletes everything else. At least 5 files - // are kept for safety + // Keeps only files that are newer than daysToKeepFiles days, and deletes everything else. At + // least numFilesToKeep files are kept for safety private def cleanupOldFiles(): Unit = { import scala.concurrent.duration._ try { val remoteFs = FileSystem.get(hadoopConf) val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) - val thresholdTime = System.currentTimeMillis() - (30 days).toMillis + val thresholdTime = System.currentTimeMillis() - (daysToKeepFiles days).toMillis hadoopUtil.listFilesSorted( - remoteFs, credentialsPath.getParent, credentialsPath.getName, ".tmp") - .dropRight(5) + remoteFs, credentialsPath.getParent, + credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) + .dropRight(numFilesToKeep) .takeWhile(_.getModificationTime < thresholdTime) .foreach(x => remoteFs.delete(x.getPath, true)) } catch { @@ -140,64 +136,68 @@ private[yarn] class AMDelegationTokenRenewer( } private def writeNewTokensToHDFS(principal: String, keytab: String): Unit = { - if (!loggedInViaKeytab) { - // Keytab is copied by YARN to the working directory of the AM, so full path is - // not needed. - logInfo(s"Attempting to login to KDC using principal: $principal") - UserGroupInformation.loginUserFromKeytab(principal, keytab) - logInfo("Successfully logged into KDC.") - loggedInViaKeytab = true - } - val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) - hadoopUtil.obtainTokensForNamenodes( - nns, hadoopConf, UserGroupInformation.getCurrentUser.getCredentials) + // Keytab is copied by YARN to the working directory of the AM, so full path is + // not needed. + + // HACK: + // HDFS will not issue new delegation tokens, if the Credentials object + // passed in already has tokens for that FS even if the tokens are expired (it really only + // checks if there are tokens for the service, and not if they are valid). So the only real + // way to get new tokens is to make sure a different Credentials object is used each time to + // get new tokens and then the new tokens are copied over the the current user's Credentials. + // So: + // - we login as a different user and get the UGI + // - use that UGI to get the tokens (see doAs block below) + // - copy the tokens over to the current user's credentials (this will overwrite the tokens + // in the current user's Credentials object for this FS). + // The login to KDC happens each time new tokens are required, but this is rare enough to not + // have to worry about (like once every day or so). This makes this code clearer than having + // to login and then relogin every time (the HDFS API may not relogin since we don't use this + // UGI directly for HDFS communication. + logInfo(s"Attempting to login to KDC using principal: $principal") + val keytabLoggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + logInfo("Successfully logged into KDC.") + val tempCreds = keytabLoggedInUGI.getCredentials + val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) + val dst = credentialsPath.getParent + keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] { + // Get a copy of the credentials + override def run(): Void = { + val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + dst + hadoopUtil.obtainTokensForNamenodes(nns, hadoopConf, tempCreds, replaceExisting = true) + null + } + }) + // Add the temp credentials back to the original ones. + UserGroupInformation.getCurrentUser.addCredentials(tempCreds) val remoteFs = FileSystem.get(hadoopConf) // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file // and update the lastCredentialsFileSuffix. if (lastCredentialsFileSuffix == 0) { - val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) hadoopUtil.listFilesSorted( - remoteFs, credentialsPath.getParent, credentialsPath.getName, ".tmp") + remoteFs, credentialsPath.getParent, + credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) .lastOption.foreach { status => - lastCredentialsFileSuffix = getSuffixForCredentialsPath(status) + lastCredentialsFileSuffix = hadoopUtil.getSuffixForCredentialsPath(status.getPath) } } val nextSuffix = lastCredentialsFileSuffix + 1 val tokenPathStr = - sparkConf.get("spark.yarn.credentials.file") + "-" + nextSuffix + sparkConf.get("spark.yarn.credentials.file") + + SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM + nextSuffix val tokenPath = new Path(tokenPathStr) - val tempTokenPath = new Path(tokenPathStr + ".tmp") + val tempTokenPath = new Path(tokenPathStr + SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) logInfo("Writing out delegation tokens to " + tempTokenPath.toString) - val stream = Option(remoteFs.create(tempTokenPath, true)) - try { - stream.foreach { s => - val baos = new ByteArrayOutputStream() - val dataOutputStream = new DataOutputStream(baos) - val credentials = UserGroupInformation.getCurrentUser.getCredentials - credentials.writeTokenStorageToStream(dataOutputStream) - dataOutputStream.close() - credentials.writeTokenStorageToStream(s) - s.hflush() - s.close() - logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") - remoteFs.rename(tempTokenPath, tokenPath) - driverEndPoint.send(NewTokens(new SerializableBuffer(ByteBuffer.wrap(baos.toByteArray)))) - logInfo("Delegation token file rename complete.") - } - } finally { - stream.foreach(_.close()) - } + val credentials = UserGroupInformation.getCurrentUser.getCredentials + credentials.writeTokenStorageFile(tempTokenPath, hadoopConf) + logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") + remoteFs.rename(tempTokenPath, tokenPath) + logInfo("Delegation token file rename complete.") lastCredentialsFileSuffix = nextSuffix } def stop(): Unit = { delegationTokenRenewer.shutdown() } - - private def getSuffixForCredentialsPath(credentialsStatus: FileStatus): Int = { - val fileName = credentialsStatus.getPath.getName - fileName.substring(fileName.lastIndexOf("-") + 1).toInt - } - } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 7e2313e240c18..08fb522b1d912 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -142,7 +142,7 @@ private[spark] class ApplicationMaster( // If the credentials file config is present, we must periodically renew tokens. So create // a new AMDelegationTokenRenewer - if(sparkConf.contains("spark.yarn.credentials.file")) { + if (sparkConf.contains("spark.yarn.credentials.file")) { delegationTokenRenewerOption = Some(new AMDelegationTokenRenewer(sparkConf, yarnConf)) } @@ -261,7 +261,6 @@ private[spark] class ApplicationMaster( SparkEnv.driverActorSystemName, RpcAddress(host, port.toInt), YarnSchedulerBackend.ENDPOINT_NAME) - delegationTokenRenewerOption.foreach(_.driverEndPoint = driverEndpoint) amEndpoint = rpcEnv.setupEndpoint("YarnAM", new AMEndpoint(rpcEnv, driverEndpoint, isClusterMode)) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index e6fbb54c6ad1d..6d29070445c6f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -18,21 +18,17 @@ package org.apache.spark.deploy.yarn import java.io._ -import java.util.Arrays -import java.util.Comparator -import java.util.concurrent.{Executors, TimeUnit} import java.util.regex.Matcher import java.util.regex.Pattern -import scala.collection.mutable.HashMap +import org.apache.hadoop.security.token.TokenIdentifier + import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap import scala.util.Try -import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.{Master, JobConf} import org.apache.hadoop.security.Credentials @@ -119,13 +115,14 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { def obtainTokensForNamenodes( paths: Set[Path], conf: Configuration, - creds: Credentials + creds: Credentials, + replaceExisting: Boolean = false ): Unit = { if (UserGroupInformation.isSecurityEnabled()) { val delegTokenRenewer = getTokenRenewer(conf) paths.foreach { dst => val dstFs = dst.getFileSystem(conf) - logDebug("getting token for namenode: " + dst) + logInfo("getting token for namenode: " + dst) dstFs.addDelegationTokens(delegTokenRenewer, creds) } } From 072659e7a71ea809f8dfae5b5d64f8c67949bf05 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 27 Apr 2015 18:03:00 -0700 Subject: [PATCH 32/41] Fix build failure caused by thread factory getting moved to ThreadUtils. --- .../spark/deploy/ExecutorDelegationTokenUpdater.scala | 6 +++--- .../apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala index b3cb17252b9ba..dd810e904bbc5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala @@ -19,11 +19,11 @@ package org.apache.spark.deploy import java.util.concurrent.{Executors, TimeUnit} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} import scala.util.control.NonFatal @@ -37,7 +37,7 @@ private[spark] class ExecutorDelegationTokenUpdater( private val delegationTokenRenewer = Executors.newSingleThreadScheduledExecutor( - Utils.namedThreadFactory("Delegation Token Refresh Thread")) + ThreadUtils.namedThreadFactory("Delegation Token Refresh Thread")) // On the executor, this thread wakes up and picks up new tokens from HDFS, if any. private val executorUpdaterRunnable = diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index cb7e996f555ea..aa0c9398d01b7 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -20,12 +20,12 @@ import java.security.PrivilegedExceptionAction import java.util.concurrent.{Executors, TimeUnit} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.Utils +import org.apache.spark.util.ThreadUtils /* * The following methods are primarily meant to make sure long-running apps like Spark @@ -54,7 +54,7 @@ private[yarn] class AMDelegationTokenRenewer( private val delegationTokenRenewer = Executors.newSingleThreadScheduledExecutor( - Utils.namedThreadFactory("Delegation Token Refresh Thread")) + ThreadUtils.namedThreadFactory("Delegation Token Refresh Thread")) private val hadoopUtil = YarnSparkHadoopUtil.get From 6963bbc3736125010a7ff61d4945c9a84ae5fb4c Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 28 Apr 2015 00:39:41 -0700 Subject: [PATCH 33/41] Schedule renewal in AM before starting user class. Else, a restarted AM cannot access HDFS if the user class tries to. --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 11e036f239bb8..6c148f32c9838 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -131,6 +131,9 @@ private[spark] class ApplicationMaster( // a new AMDelegationTokenRenewer if (sparkConf.contains("spark.yarn.credentials.file")) { delegationTokenRenewerOption = Some(new AMDelegationTokenRenewer(sparkConf, yarnConf)) + // If a principal and keytab have been set, use that to create new credentials for executors + // periodically + delegationTokenRenewerOption.foreach(_.scheduleLoginFromKeytab()) } if (isClusterMode) { @@ -272,9 +275,6 @@ private[spark] class ApplicationMaster( sc.getConf.get("spark.driver.port"), isClusterMode = true) registerAM(sc.ui.map(_.appUIAddress).getOrElse(""), securityMgr) - // If a principal and keytab have been set, use that to create new credentials for executors - // periodically - delegationTokenRenewerOption.foreach(_.scheduleLoginFromKeytab()) userClassThread.join() } } From 09fe2244f82b5037772c40693378b2def43b12bb Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 28 Apr 2015 18:31:56 -0700 Subject: [PATCH 34/41] Use token.renew to get token's renewal interval rather than using hdfs-site.xml --- .../ExecutorDelegationTokenUpdater.scala | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 13 +++++---- .../CoarseGrainedExecutorBackend.scala | 6 +---- docs/security.md | 2 +- .../yarn/AMDelegationTokenRenewer.scala | 13 +++++---- .../spark/deploy/yarn/ApplicationMaster.scala | 3 --- .../org/apache/spark/deploy/yarn/Client.scala | 27 +++++++++++++++++-- .../deploy/yarn/YarnSparkHadoopUtil.scala | 7 ++--- 8 files changed, 42 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala index dd810e904bbc5..80363aa3a1e0b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala @@ -70,7 +70,7 @@ private[spark] class ExecutorDelegationTokenUpdater( } val timeFromNowToRenewal = SparkHadoopUtil.get.getTimeFromNowToRenewal( - 0.8, UserGroupInformation.getCurrentUser.getCredentials) + sparkConf, 0.8, UserGroupInformation.getCurrentUser.getCredentials) if (timeFromNowToRenewal <= 0) { executorUpdaterRunnable.run() } else { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 2ede7aa7be0c5..ef30938a60e02 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.JobContext import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark._ +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -47,8 +47,6 @@ class SparkHadoopUtil extends Logging { private val sparkConf = new SparkConf() val conf: Configuration = newConfiguration(sparkConf) UserGroupInformation.setConfiguration(conf) - private lazy val renewalInterval = - conf.getLong("dfs.namenode.delegation.token.renew-interval", (24 hours).toMillis) /** * Runs the given function with a Hadoop UserGroupInformation as a thread local variable @@ -213,9 +211,6 @@ class SparkHadoopUtil extends Logging { * Lists all the files in a directory with the specified prefix, and does not end with the * given suffix. The returned {{FileStatus}} instances are sorted by the modification times of * the respective files. - * @param remoteFs - * @param prefix - * @return */ def listFilesSorted( remoteFs: FileSystem, @@ -242,8 +237,12 @@ class SparkHadoopUtil extends Logging { * is valid the latest)? * This will return -ve (or 0) value if the fraction of validity has already expired. */ - def getTimeFromNowToRenewal(fraction: Double, credentials: Credentials): Long = { + def getTimeFromNowToRenewal( + sparkConf: SparkConf, + fraction: Double, + credentials: Credentials): Long = { val now = System.currentTimeMillis() + val renewalInterval = sparkConf.getLong("spark.yarn.renewal.interval", (24 hours).toMillis) credentials.getAllTokens.filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) .map { t => val identifier = new DelegationTokenIdentifier() diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 68444d5434450..b9b6295ecca2d 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -17,15 +17,12 @@ package org.apache.spark.executor -import java.io.{ByteArrayInputStream, DataInputStream} import java.net.URL import java.nio.ByteBuffer import scala.collection.mutable import scala.util.{Failure, Success} -import org.apache.hadoop.security.{Credentials, UserGroupInformation} - import org.apache.spark.rpc._ import org.apache.spark._ import org.apache.spark.TaskState.TaskState @@ -77,7 +74,7 @@ private[spark] class CoarseGrainedExecutorBackend( } override def receive: PartialFunction[Any, Unit] = { - case RegisteredExecutor=> + case RegisteredExecutor => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) @@ -195,7 +192,6 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { workerUrl.foreach { url => env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } - env.actorSystem.awaitTermination() tokenUpdaterOption.foreach(_.stop()) env.rpcEnv.awaitTermination() } diff --git a/docs/security.md b/docs/security.md index 0cb38bdcb3667..d4ffa60e59a33 100644 --- a/docs/security.md +++ b/docs/security.md @@ -32,7 +32,7 @@ SSL must be configured on each node and configured for each component involved i ### YARN mode The key-store can be prepared on the client side and then distributed and used by the executors as the part of the application. It is possible because the user is able to deploy files before the application is started in YARN by using `spark.yarn.dist.files` or `spark.yarn.dist.archives` configuration settings. The responsibility for encryption of transferring these files is on YARN side and has nothing to do with Spark. -For long-running apps like Spark Streaming apps to be able to write to HDFS, it is possible to pass a principal and keytab to `spark-submit` via the `--principal` and `--keytab` parameters respectively. The keytab passed in will be copied over to the machine running the Application Master via the Hadoop Distributed Cache (securely - if YARN is configured with SSL and HDFS encryption is enabled). The Kerberos login will be periodically renewed using this principal and keytab and the delegation tokens required for HDFS will be generated periodically so the application can continue writing to HDFS. Please note that the HDFS client configuration file, `hdfs-site.xml` on each executor node must have the value of `dfs.namenode.delegation.token.renew-interval` be the same as it is on the HDFS Namenode for this functionality. +For long-running apps like Spark Streaming apps to be able to write to HDFS, it is possible to pass a principal and keytab to `spark-submit` via the `--principal` and `--keytab` parameters respectively. The keytab passed in will be copied over to the machine running the Application Master via the Hadoop Distributed Cache (securely - if YARN is configured with SSL and HDFS encryption is enabled). The Kerberos login will be periodically renewed using this principal and keytab and the delegation tokens required for HDFS will be generated periodically so the application can continue writing to HDFS. ### Standalone mode The user needs to provide key-stores and configuration options for master and workers. They have to be set by attaching appropriate Java system properties in `SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to use the SSL settings inherited from the worker which spawned that executor. It can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that parameter is set, the settings provided by user on the client side, are not used by the executors. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index aa0c9398d01b7..619f10b62a416 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -32,12 +32,12 @@ import org.apache.spark.util.ThreadUtils * Streaming apps can run without interruption while writing to secure HDFS. The * scheduleLoginFromKeytab method is called on the driver when the * CoarseGrainedScheduledBackend starts up. This method wakes up a thread that logs into the KDC - * once 75% of the expiry time of the original delegation tokens used for the container + * once 75% of the renewal interval of the original delegation tokens used for the container * has elapsed. It then creates new delegation tokens and writes them to HDFS in a * pre-specified location - the prefix of which is specified in the sparkConf by * spark.yarn.credentials.file (so the file(s) would be named c-1, c-2 etc. - each update goes * to a new file, with a monotonically increasing suffix). After this, the credentials are - * updated once 75% of the new tokens validity has elapsed. + * updated once 75% of the new tokens renewal interval has elapsed. * * On the executor side, the updateCredentialsIfRequired method is called once 80% of the * validity of the original tokens has elapsed. At that time the executor finds the @@ -72,13 +72,12 @@ private[yarn] class AMDelegationTokenRenewer( val keytab = sparkConf.get("spark.yarn.keytab") /** - * Schedule the renewal of the tokens. If tokens have already expired, this method will - * synchronously renew them. - * @param runnable + * Schedule re-login and creation of new tokens. If tokens have already expired, this method + * will synchronously create new ones. */ def scheduleRenewal(runnable: Runnable): Unit = { val credentials = UserGroupInformation.getCurrentUser.getCredentials - val renewalInterval = hadoopUtil.getTimeFromNowToRenewal(0.75, credentials) + val renewalInterval = hadoopUtil.getTimeFromNowToRenewal(sparkConf, 0.75, credentials) // Run now! if (renewalInterval <= 0) { logInfo("HDFS tokens have expired, creating new tokens now.") @@ -164,7 +163,7 @@ private[yarn] class AMDelegationTokenRenewer( // Get a copy of the credentials override def run(): Void = { val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + dst - hadoopUtil.obtainTokensForNamenodes(nns, hadoopConf, tempCreds, replaceExisting = true) + hadoopUtil.obtainTokensForNamenodes(nns, hadoopConf, tempCreds) null } }) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 6c148f32c9838..82f2b7e1dd288 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -283,9 +283,6 @@ private[spark] class ApplicationMaster( rpcEnv = RpcEnv.create("sparkYarnAM", Utils.localHostName, 0, sparkConf, securityMgr) waitForSparkDriver() addAmIpFilter() - // If a principal and keytab have been set, use that to create new credentials for executors - // periodically - delegationTokenRenewerOption.foreach(_.scheduleLoginFromKeytab()) registerAM(sparkConf.get("spark.driver.appUIAddress", ""), securityMgr) // In client mode the actor will stop the reporter thread. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 4af51645f2e4e..9e4de1eb2d6c0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -17,9 +17,10 @@ package org.apache.spark.deploy.yarn -import java.io.{File, FileOutputStream} +import java.io.{ByteArrayInputStream, DataInputStream, File, FileOutputStream} import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer +import java.security.PrivilegedExceptionAction import java.util.UUID import java.util.zip.{ZipEntry, ZipOutputStream} @@ -402,6 +403,27 @@ private[spark] class Client( } } + /** + * Get the renewal interval for tokens. + */ + private def getTokenRenewalInterval(stagingDirPath: Path): Long = { + // We cannot use the tokens generated above since those have renewer yarn. Trying to renew + // those will fail with an access control issue. So create new tokens with the logged in + // user as renewer. + val creds = new Credentials() + YarnSparkHadoopUtil.get.obtainTokensForNamenodes(Set(stagingDirPath), hadoopConf, creds, + Some(sparkConf.get("spark.yarn.principal"))) + val t = creds.getAllTokens + .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) + .head + val newExpiration = t.renew(hadoopConf) + val identifier = new DelegationTokenIdentifier() + identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) + val interval = newExpiration - identifier.getIssueDate + logInfo(s"Renewal Interval set to $interval") + interval + } + /** * Set up the environment for launching our ApplicationMaster container. */ @@ -420,8 +442,9 @@ private[spark] class Client( sparkConf.set( "spark.yarn.credentials.file", new Path(stagingDirPath, credentialsFile).toString) logInfo(s"Credentials file set to: $credentialsFile") + val renewalInterval = getTokenRenewalInterval(stagingDirPath) + sparkConf.set("spark.yarn.renewal.interval", renewalInterval.toString) } - // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 6d29070445c6f..a5c454e2e5c5e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -21,9 +21,6 @@ import java.io._ import java.util.regex.Matcher import java.util.regex.Pattern -import org.apache.hadoop.security.token.TokenIdentifier - -import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap import scala.util.Try @@ -116,10 +113,10 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { paths: Set[Path], conf: Configuration, creds: Credentials, - replaceExisting: Boolean = false + renewer: Option[String] = None ): Unit = { if (UserGroupInformation.isSecurityEnabled()) { - val delegTokenRenewer = getTokenRenewer(conf) + val delegTokenRenewer = renewer.getOrElse(getTokenRenewer(conf)) paths.foreach { dst => val dstFs = dst.getFileSystem(conf) logInfo("getting token for namenode: " + dst) From 611923a74d009ee6f1a8d57672c034c64fa17685 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 28 Apr 2015 20:44:10 -0700 Subject: [PATCH 35/41] Make sure the namenodes are listed correctly for creating tokens. --- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 9e4de1eb2d6c0..bb8a496e2d5d0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -411,8 +411,9 @@ private[spark] class Client( // those will fail with an access control issue. So create new tokens with the logged in // user as renewer. val creds = new Credentials() - YarnSparkHadoopUtil.get.obtainTokensForNamenodes(Set(stagingDirPath), hadoopConf, creds, - Some(sparkConf.get("spark.yarn.principal"))) + val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + stagingDirPath + YarnSparkHadoopUtil.get.obtainTokensForNamenodes( + nns, hadoopConf, creds, Some(sparkConf.get("spark.yarn.principal"))) val t = creds.getAllTokens .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) .head From 36eb8a956c357388e4fdf5858cb4f27236f26a9e Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 29 Apr 2015 23:55:14 -0700 Subject: [PATCH 36/41] Change the renewal interval config param. Fix a bunch of comments. --- .../main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 5 ++++- .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 2 +- .../apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala | 5 +++-- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index ef30938a60e02..00194fba84ff7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -242,7 +242,10 @@ class SparkHadoopUtil extends Logging { fraction: Double, credentials: Credentials): Long = { val now = System.currentTimeMillis() - val renewalInterval = sparkConf.getLong("spark.yarn.renewal.interval", (24 hours).toMillis) + + val renewalInterval = + sparkConf.getLong("spark.yarn.token.renewal.interval", (24 hours).toMillis) + credentials.getAllTokens.filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) .map { t => val identifier = new DelegationTokenIdentifier() diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index b9b6295ecca2d..0728ae8f0f562 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -192,8 +192,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { workerUrl.foreach { url => env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } - tokenUpdaterOption.foreach(_.stop()) env.rpcEnv.awaitTermination() + tokenUpdaterOption.foreach(_.stop()) } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index 619f10b62a416..9ff02046de805 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -64,8 +64,9 @@ private[yarn] class AMDelegationTokenRenewer( /** * Schedule a login from the keytab and principal set using the --principal and --keytab * arguments to spark-submit. This login happens only when the credentials of the current user - * are about to expire. This method reads SPARK_PRINCIPAL and SPARK_KEYTAB from the environment - * to do the login. This method is a no-op in non-YARN mode. + * are about to expire. This method reads spark.yarn.principal and spark.yarn.keytab from + * SparkConf to do the login. This method is a no-op in non-YARN mode. + * */ private[spark] def scheduleLoginFromKeytab(): Unit = { val principal = sparkConf.get("spark.yarn.principal") diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index bb8a496e2d5d0..21abef0e02d40 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -444,7 +444,7 @@ private[spark] class Client( "spark.yarn.credentials.file", new Path(stagingDirPath, credentialsFile).toString) logInfo(s"Credentials file set to: $credentialsFile") val renewalInterval = getTokenRenewalInterval(stagingDirPath) - sparkConf.set("spark.yarn.renewal.interval", renewalInterval.toString) + sparkConf.set("spark.yarn.token.renewal.interval", renewalInterval.toString) } // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) From e851f702c63ac4e83bca87e3937aa68daa45dd99 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 30 Apr 2015 17:33:47 -0700 Subject: [PATCH 37/41] Move the ExecutorDelegationTokenRenewer to yarn module. Use reflection to use it. --- .../CoarseGrainedExecutorBackend.scala | 22 ++++++++++++++----- .../ExecutorDelegationTokenUpdater.scala | 6 ++--- 2 files changed, 19 insertions(+), 9 deletions(-) rename {core/src/main/scala/org/apache/spark/deploy => yarn/src/main/scala/org/apache/spark/deploy/yarn}/ExecutorDelegationTokenUpdater.scala (98%) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 0728ae8f0f562..d4ed0a089e3f0 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -20,13 +20,15 @@ package org.apache.spark.executor import java.net.URL import java.nio.ByteBuffer +import org.apache.hadoop.conf.Configuration + import scala.collection.mutable import scala.util.{Failure, Success} import org.apache.spark.rpc._ import org.apache.spark._ import org.apache.spark.TaskState.TaskState -import org.apache.spark.deploy.{ExecutorDelegationTokenUpdater, SparkHadoopUtil} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -168,14 +170,20 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { driverConf.set(key, value) } } - var tokenUpdaterOption: Option[ExecutorDelegationTokenUpdater] = None + // Delegation Token Updater is not supported in Hadoop 1, so use reflection. + // Can't use Option[ExecutorDelegationTokenUpdater] because it is built only in YARN + // profile, so use Option[Any] since even the stop method call will be via reflection. + var tokenUpdaterOption: Option[Any] = None + val tokenUpdaterClass = + Class.forName("org.apache.spark.deploy.yarn.ExecutorDelegationTokenUpdater") if (driverConf.contains("spark.yarn.credentials.file")) { logInfo("Will periodically update credentials from: " + driverConf.get("spark.yarn.credentials.file")) // Periodically update the credentials for this user to ensure HDFS tokens get updated. - tokenUpdaterOption = - Some(new ExecutorDelegationTokenUpdater(driverConf, SparkHadoopUtil.get.conf)) - tokenUpdaterOption.get.updateCredentialsIfRequired() + val constructor = + tokenUpdaterClass.getDeclaredConstructor(classOf[SparkConf], classOf[Configuration]) + tokenUpdaterOption = Some(constructor.newInstance(driverConf, SparkHadoopUtil.get.conf)) + tokenUpdaterClass.getMethod("updateCredentialsIfRequired").invoke(tokenUpdaterOption.get) } val env = SparkEnv.createExecutorEnv( @@ -193,7 +201,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } env.rpcEnv.awaitTermination() - tokenUpdaterOption.foreach(_.stop()) + if (tokenUpdaterOption.isDefined) { + tokenUpdaterClass.getMethod("stop").invoke(tokenUpdaterOption.get) + } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala index 80363aa3a1e0b..e75d9ede5eee7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDelegationTokenUpdater.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala @@ -14,16 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy +package org.apache.spark.deploy.yarn import java.util.concurrent.{Executors, TimeUnit} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.{Credentials, UserGroupInformation} - -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.{Logging, SparkConf} import scala.util.control.NonFatal From 7bff6e9c3cb3184ca4b544d0c96f7d9fed85512d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 30 Apr 2015 17:37:20 -0700 Subject: [PATCH 38/41] Make sure all required classes are present in the jar. Fix import order. --- .../executor/CoarseGrainedExecutorBackend.scala | 14 +++++++++----- .../yarn/ExecutorDelegationTokenUpdater.scala | 3 ++- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index d4ed0a089e3f0..d3d40d6fd5e63 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -174,16 +174,20 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { // Can't use Option[ExecutorDelegationTokenUpdater] because it is built only in YARN // profile, so use Option[Any] since even the stop method call will be via reflection. var tokenUpdaterOption: Option[Any] = None - val tokenUpdaterClass = - Class.forName("org.apache.spark.deploy.yarn.ExecutorDelegationTokenUpdater") + var tokenUpdaterClass: Option[Class[_]] = None if (driverConf.contains("spark.yarn.credentials.file")) { logInfo("Will periodically update credentials from: " + driverConf.get("spark.yarn.credentials.file")) + + tokenUpdaterClass = + Some(Class.forName("org.apache.spark.deploy.yarn.ExecutorDelegationTokenUpdater")) + // Periodically update the credentials for this user to ensure HDFS tokens get updated. val constructor = - tokenUpdaterClass.getDeclaredConstructor(classOf[SparkConf], classOf[Configuration]) + tokenUpdaterClass.get.getDeclaredConstructor(classOf[SparkConf], classOf[Configuration]) tokenUpdaterOption = Some(constructor.newInstance(driverConf, SparkHadoopUtil.get.conf)) - tokenUpdaterClass.getMethod("updateCredentialsIfRequired").invoke(tokenUpdaterOption.get) + tokenUpdaterClass.get.getMethod("updateCredentialsIfRequired") + .invoke(tokenUpdaterOption.get) } val env = SparkEnv.createExecutorEnv( @@ -202,7 +206,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } env.rpcEnv.awaitTermination() if (tokenUpdaterOption.isDefined) { - tokenUpdaterClass.getMethod("stop").invoke(tokenUpdaterOption.get) + tokenUpdaterClass.get.getMethod("stop").invoke(tokenUpdaterOption.get) } } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala index e75d9ede5eee7..229c2c4d5eb36 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala @@ -21,9 +21,10 @@ import java.util.concurrent.{Executors, TimeUnit} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.{Credentials, UserGroupInformation} + import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.{ThreadUtils, Utils} import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.{ThreadUtils, Utils} import scala.util.control.NonFatal From b5e7a728340d44ae96e3adf9817aa3a58365c466 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 30 Apr 2015 20:48:31 -0700 Subject: [PATCH 39/41] Remove reflection, use a method in SparkHadoopUtil to update the token renewer. --- .../apache/spark/deploy/SparkHadoopUtil.scala | 11 ++++++++++ .../CoarseGrainedExecutorBackend.scala | 20 ++----------------- .../deploy/yarn/YarnSparkHadoopUtil.scala | 11 ++++++++++ 3 files changed, 24 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 00194fba84ff7..cac754b51c76e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -292,6 +292,17 @@ class SparkHadoopUtil extends Logging { } } } + + /** + * Start a thread to periodically update the current user's credentials with new delegation + * tokens so that writes to HDFS do not fail. + */ + private[spark] def startExecutorDelegationTokenRenewer(conf: SparkConf){} + + /** + * Stop the thread that does the delegation token updates. + */ + private[spark] def stopExecutorDelegationTokenRenewer() {} } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index d3d40d6fd5e63..0b18ee33042f8 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -170,24 +170,10 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { driverConf.set(key, value) } } - // Delegation Token Updater is not supported in Hadoop 1, so use reflection. - // Can't use Option[ExecutorDelegationTokenUpdater] because it is built only in YARN - // profile, so use Option[Any] since even the stop method call will be via reflection. - var tokenUpdaterOption: Option[Any] = None - var tokenUpdaterClass: Option[Class[_]] = None if (driverConf.contains("spark.yarn.credentials.file")) { logInfo("Will periodically update credentials from: " + driverConf.get("spark.yarn.credentials.file")) - - tokenUpdaterClass = - Some(Class.forName("org.apache.spark.deploy.yarn.ExecutorDelegationTokenUpdater")) - - // Periodically update the credentials for this user to ensure HDFS tokens get updated. - val constructor = - tokenUpdaterClass.get.getDeclaredConstructor(classOf[SparkConf], classOf[Configuration]) - tokenUpdaterOption = Some(constructor.newInstance(driverConf, SparkHadoopUtil.get.conf)) - tokenUpdaterClass.get.getMethod("updateCredentialsIfRequired") - .invoke(tokenUpdaterOption.get) + SparkHadoopUtil.get.startExecutorDelegationTokenRenewer(driverConf) } val env = SparkEnv.createExecutorEnv( @@ -205,9 +191,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } env.rpcEnv.awaitTermination() - if (tokenUpdaterOption.isDefined) { - tokenUpdaterClass.get.getMethod("stop").invoke(tokenUpdaterOption.get) - } + SparkHadoopUtil.get.stopExecutorDelegationTokenRenewer() } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index a5c454e2e5c5e..badfc39592d63 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -44,6 +44,8 @@ import org.apache.spark.util.Utils */ class YarnSparkHadoopUtil extends SparkHadoopUtil { + private var tokenRenewer: Option[ExecutorDelegationTokenUpdater] = None + override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) } @@ -125,6 +127,15 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } } + private[spark] override def startExecutorDelegationTokenRenewer(sparkConf: SparkConf): Unit = { + tokenRenewer = Some(new ExecutorDelegationTokenUpdater(sparkConf, conf)) + tokenRenewer.get.updateCredentialsIfRequired() + } + + private[spark] override def stopExecutorDelegationTokenRenewer(): Unit ={ + tokenRenewer.foreach(_.stop()) + } + } object YarnSparkHadoopUtil { From 4d04301f04566b7e38f9ba4bc3bf5d591d30c224 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 30 Apr 2015 20:53:46 -0700 Subject: [PATCH 40/41] Minor formatting fixes. --- .../main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index cac754b51c76e..29eb9f5f64dcc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -297,7 +297,7 @@ class SparkHadoopUtil extends Logging { * Start a thread to periodically update the current user's credentials with new delegation * tokens so that writes to HDFS do not fail. */ - private[spark] def startExecutorDelegationTokenRenewer(conf: SparkConf){} + private[spark] def startExecutorDelegationTokenRenewer(conf: SparkConf) {} /** * Stop the thread that does the delegation token updates. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index badfc39592d63..12cb8d065be75 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -132,7 +132,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { tokenRenewer.get.updateCredentialsIfRequired() } - private[spark] override def stopExecutorDelegationTokenRenewer(): Unit ={ + private[spark] override def stopExecutorDelegationTokenRenewer(): Unit = { tokenRenewer.foreach(_.stop()) } From 3c86bbac8e0116575e5c632c91e77e4d2687dfc1 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 1 May 2015 10:46:09 -0700 Subject: [PATCH 41/41] Import fixes. Import postfixOps explicitly. --- .../main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 1 + .../apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala | 2 ++ .../org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala | 4 ++-- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 29eb9f5f64dcc..b563034457a91 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -37,6 +37,7 @@ import org.apache.spark.util.Utils import scala.collection.JavaConversions._ import scala.concurrent.duration._ +import scala.language.postfixOps /** * :: DeveloperApi :: diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index 9ff02046de805..8999f8059c5e3 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.yarn import java.security.PrivilegedExceptionAction import java.util.concurrent.{Executors, TimeUnit} +import scala.language.postfixOps + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.UserGroupInformation diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 12cb8d065be75..ba91872107d0c 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import java.io._ +import java.io.File import java.util.regex.Matcher import java.util.regex.Pattern @@ -25,7 +25,7 @@ import scala.collection.mutable.HashMap import scala.util.Try import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.Path import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.{Master, JobConf} import org.apache.hadoop.security.Credentials