-
Notifications
You must be signed in to change notification settings - Fork 29k
[Spark-21842][Mesos] Support Kerberos ticket renewal and creation in Mesos #19272
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 21 commits
44a6098
781c5a7
18d2c6c
43ab547
f136eaa
488f72a
f5925fd
3f22efe
e522150
837157d
c95f80b
e8bbc9e
1596d80
864ab7e
7e0590a
f93c551
b2fbcf2
4558cea
5f254e5
8df7e37
45b46ed
18d77ff
049e4b5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -140,12 +140,23 @@ class SparkHadoopUtil extends Logging { | |
| if (!new File(keytabFilename).exists()) { | ||
| throw new SparkException(s"Keytab file: ${keytabFilename} does not exist") | ||
| } else { | ||
| logInfo("Attempting to login to Kerberos" + | ||
| s" using principal: ${principalName} and keytab: ${keytabFilename}") | ||
| logInfo("Attempting to login to Kerberos " + | ||
| s"using principal: ${principalName} and keytab: ${keytabFilename}") | ||
| UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Add or overwrite current user's credentials with serialized delegation tokens, | ||
| * also confirms correct hadoop configuration is set. | ||
| */ | ||
| def addDelegationTokens(tokens: Array[Byte], sparkConf: SparkConf) { | ||
|
||
| UserGroupInformation.setConfiguration(newConfiguration(sparkConf)) | ||
| val creds = deserialize(tokens) | ||
| logInfo(s"Adding/updating delegation tokens ${dumpTokens(creds)}") | ||
| addCurrentUserCredentials(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 | ||
|
|
@@ -439,6 +450,8 @@ object SparkHadoopUtil { | |
| case e: Exception => throw new SparkException("Unable to load YARN support", e) | ||
| } | ||
|
|
||
| val TICKET_CACHE_ENVVAR = "KRB5CCNAME" | ||
|
|
||
| val SPARK_YARN_CREDS_TEMP_EXTENSION = ".tmp" | ||
|
|
||
| val SPARK_YARN_CREDS_COUNTER_DELIM = "-" | ||
|
|
@@ -462,6 +475,19 @@ object SparkHadoopUtil { | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Given an expiration date (e.g. for Hadoop Delegation Tokens) return a the date | ||
| * when a given fraction of the duration until the expiration date has passed. | ||
| * Formula: current time + (fraction * (time until expiration)) | ||
| * @param expirationDate Drop-dead expiration date | ||
| * @param fraction fraction of the time until expiration return | ||
| * @return Date when the fraction of the time until expiration has passed | ||
| */ | ||
| def getDateOfNextUpdate(expirationDate: Long, fraction: Double): Long = { | ||
|
||
| val ct = System.currentTimeMillis | ||
| (ct + (fraction * (expirationDate - ct))).toLong | ||
| } | ||
|
|
||
| /** | ||
| * Returns a Configuration object with Spark configuration applied on top. Unlike | ||
| * the instance method, this will always return a Configuration instance, and not a | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -123,6 +123,10 @@ private[spark] class CoarseGrainedExecutorBackend( | |
| executor.stop() | ||
| } | ||
| }.start() | ||
|
|
||
| case UpdateDelegationTokens(tokenBytes) => | ||
| logInfo(s"Received tokens of ${tokenBytes.length} bytes") | ||
| SparkHadoopUtil.get.addDelegationTokens(tokenBytes, env.conf) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can you add a |
||
| } | ||
|
|
||
| override def onDisconnected(remoteAddress: RpcAddress): Unit = { | ||
|
|
@@ -219,9 +223,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | |
| SparkHadoopUtil.get.startCredentialUpdater(driverConf) | ||
| } | ||
|
|
||
| cfg.hadoopDelegationCreds.foreach { hadoopCreds => | ||
| val creds = SparkHadoopUtil.get.deserialize(hadoopCreds) | ||
| SparkHadoopUtil.get.addCurrentUserCredentials(creds) | ||
| cfg.hadoopDelegationCreds.foreach { tokens => | ||
| SparkHadoopUtil.get.addDelegationTokens(tokens, driverConf) | ||
| } | ||
|
|
||
| val env = SparkEnv.createExecutorEnv( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,11 +24,7 @@ import javax.annotation.concurrent.GuardedBy | |
| import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} | ||
| import scala.concurrent.Future | ||
|
|
||
| import org.apache.hadoop.security.UserGroupInformation | ||
|
|
||
| import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.deploy.security.HadoopDelegationTokenManager | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.rpc._ | ||
| import org.apache.spark.scheduler._ | ||
|
|
@@ -99,12 +95,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| // The num of current max ExecutorId used to re-register appMaster | ||
| @volatile protected var currentExecutorIdCounter = 0 | ||
|
|
||
| // hadoop token manager used by some sub-classes (e.g. Mesos) | ||
| def hadoopDelegationTokenManager: Option[HadoopDelegationTokenManager] = None | ||
|
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No longer needed because resource-manager backends (may) implement their own |
||
|
|
||
| // Hadoop delegation tokens to be sent to the executors. | ||
| val hadoopDelegationCreds: Option[Array[Byte]] = getHadoopDelegationCreds() | ||
|
|
||
| class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) | ||
| extends ThreadSafeRpcEndpoint with Logging { | ||
|
|
||
|
|
@@ -159,6 +149,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| scheduler.getExecutorsAliveOnHost(host).foreach { exec => | ||
| killExecutors(exec.toSeq, replace = true, force = true) | ||
| } | ||
|
|
||
| case UpdateDelegationTokens(newDelegationTokens) => | ||
| executorDataMap.values.foreach { ed => | ||
| ed.executorEndpoint.send(UpdateDelegationTokens(newDelegationTokens)) | ||
| } | ||
| } | ||
|
|
||
| override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { | ||
|
|
@@ -236,7 +231,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| val reply = SparkAppConfig( | ||
| sparkProperties, | ||
| SparkEnv.get.securityManager.getIOEncryptionKey(), | ||
| hadoopDelegationCreds) | ||
| fetchHadoopDelegationTokens()) | ||
| context.reply(reply) | ||
| } | ||
|
|
||
|
|
@@ -686,18 +681,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| true | ||
| } | ||
|
|
||
| protected def getHadoopDelegationCreds(): Option[Array[Byte]] = { | ||
|
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This method was only called once, and would discard the renewal time information limiting it's utility. |
||
| if (UserGroupInformation.isSecurityEnabled && hadoopDelegationTokenManager.isDefined) { | ||
| hadoopDelegationTokenManager.map { manager => | ||
| val creds = UserGroupInformation.getCurrentUser.getCredentials | ||
| val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) | ||
| manager.obtainDelegationTokens(hadoopConf, creds) | ||
| SparkHadoopUtil.get.serialize(creds) | ||
| } | ||
| } else { | ||
| None | ||
| } | ||
| } | ||
| protected def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { None } | ||
| } | ||
|
|
||
| private[spark] object CoarseGrainedSchedulerBackend { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,7 +17,7 @@ | |
|
|
||
| package org.apache.spark.scheduler.cluster.mesos | ||
|
|
||
| import org.apache.spark.{SparkContext, SparkException} | ||
| import org.apache.spark.SparkContext | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Change not needed?
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| import org.apache.spark.internal.config._ | ||
| import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add a comment about what this method is doing and why it's needed. (YARN never sets the authentication method, so it'd be good to know why Mesos needs to do it.)