@@ -21,22 +21,19 @@ import java.io.File
2121import java .net .URI
2222import java .security .PrivilegedExceptionAction
2323import java .util .ServiceLoader
24- import java .util .concurrent .{ScheduledExecutorService , TimeUnit }
25-
26- import scala .collection .mutable
24+ import java .util .concurrent .TimeUnit
2725
2826import org .apache .hadoop .conf .Configuration
2927import org .apache .hadoop .security .{Credentials , UserGroupInformation }
3028
3129import org .apache .spark .SparkConf
3230import org .apache .spark .deploy .SparkHadoopUtil
33- import org .apache .spark .internal .Logging
3431import org .apache .spark .internal .config ._
3532import org .apache .spark .rpc .RpcEndpointRef
3633import org .apache .spark .scheduler .cluster .CoarseGrainedClusterMessages .UpdateDelegationTokens
3734import org .apache .spark .security .HadoopDelegationTokenProvider
3835import org .apache .spark .ui .UIUtils
39- import org .apache .spark .util .{ ThreadUtils , Utils }
36+ import org .apache .spark .util .Utils
4037
4138/**
4239 * Manager for delegation tokens in a Spark application.
@@ -61,9 +58,11 @@ import org.apache.spark.util.{ThreadUtils, Utils}
6158 * generated.
6259 */
6360private [spark] class HadoopDelegationTokenManager (
64- protected val sparkConf : SparkConf ,
65- protected val hadoopConf : Configuration ,
66- protected val schedulerRef : RpcEndpointRef ) extends Logging {
61+ override protected val sparkConf : SparkConf ,
62+ override protected val hadoopConf : Configuration ,
63+ override protected val schedulerRef : RpcEndpointRef )
64+ extends ServiceCredentialsManager [HadoopDelegationTokenProvider ](
65+ sparkConf, hadoopConf, schedulerRef) {
6766
6867 private val principal = sparkConf.get(PRINCIPAL ).orNull
6968
@@ -74,11 +73,15 @@ private[spark] class HadoopDelegationTokenManager(
7473 require((principal == null ) == (keytab == null ),
7574 " Both principal and keytab must be defined, or neither." )
7675
77- private val delegationTokenProviders = loadProviders()
78- logDebug(" Using the following builtin delegation token providers: " +
79- s " ${delegationTokenProviders.keys.mkString(" , " )}. " )
76+ val delegationTokenProviders : Map [String , HadoopDelegationTokenProvider ] = credentialsProviders
77+ .asInstanceOf [Map [String , HadoopDelegationTokenProvider ]]
78+
79+ def credentialsType : String = " Delegation token"
8080
81- private var renewalExecutor : ScheduledExecutorService = _
81+ def credentialsConfig : ServiceCredentialsConfig = HadoopDelegationTokenManager
82+
83+ def getProviderLoader : ServiceLoader [HadoopDelegationTokenProvider ] =
84+ ServiceLoader .load(classOf [HadoopDelegationTokenProvider ], Utils .getContextOrSparkClassLoader)
8285
8386 /** @return Whether delegation token renewal is enabled. */
8487 def renewalEnabled : Boolean = sparkConf.get(KERBEROS_RENEWAL_CREDENTIALS ) match {
@@ -97,14 +100,15 @@ private[spark] class HadoopDelegationTokenManager(
97100 *
98101 * @return New set of delegation tokens created for the configured principal.
99102 */
100- def start (): Array [Byte ] = {
101- require(renewalEnabled, " Token renewal must be enabled to start the renewer." )
102- require(schedulerRef != null , " Token renewal requires a scheduler endpoint." )
103- renewalExecutor =
104- ThreadUtils .newDaemonSingleThreadScheduledExecutor(" Credential Renewal Thread" )
103+ override def start (): Array [Byte ] = {
104+ super .start()
105+ }
105106
106- val ugi = UserGroupInformation .getCurrentUser()
107- if (ugi.isFromKeytab()) {
107+ def updateCredentialsGrantingTicket (): Unit = {
108+ require(renewalExecutor != null ,
109+ " Renewal executor must be initialized before updating TGT." )
110+ val ugi = UserGroupInformation .getCurrentUser
111+ if (ugi.isFromKeytab) {
108112 // In Hadoop 2.x, renewal of the keytab-based login seems to be automatic, but in Hadoop 3.x,
109113 // it is configurable (see hadoop.kerberos.keytab.login.autorenewal.enabled, added in
110114 // HADOOP-9567). This task will make sure that the user stays logged in regardless of that
@@ -119,14 +123,10 @@ private[spark] class HadoopDelegationTokenManager(
119123 renewalExecutor.scheduleAtFixedRate(tgtRenewalTask, tgtRenewalPeriod, tgtRenewalPeriod,
120124 TimeUnit .SECONDS )
121125 }
122-
123- updateTokensTask()
124126 }
125127
126- def stop (): Unit = {
127- if (renewalExecutor != null ) {
128- renewalExecutor.shutdownNow()
129- }
128+ def updateCredentialsTask (): Array [Byte ] = {
129+ updateTokensTask()
130130 }
131131
132132 /**
@@ -171,23 +171,6 @@ private[spark] class HadoopDelegationTokenManager(
171171 (creds, nextRenewal)
172172 }
173173
174- // Visible for testing.
175- def isProviderLoaded (serviceName : String ): Boolean = {
176- delegationTokenProviders.contains(serviceName)
177- }
178-
179- private def scheduleRenewal (delay : Long ): Unit = {
180- val _delay = math.max(0 , delay)
181- logInfo(s " Scheduling renewal in ${UIUtils .formatDuration(_delay)}. " )
182-
183- val renewalTask = new Runnable () {
184- override def run (): Unit = {
185- updateTokensTask()
186- }
187- }
188- renewalExecutor.schedule(renewalTask, _delay, TimeUnit .MILLISECONDS )
189- }
190-
191174 /**
192175 * Periodic task to login to the KDC and create new delegation tokens. Re-schedules itself
193176 * to fetch the next set of tokens when needed.
@@ -224,14 +207,7 @@ private[spark] class HadoopDelegationTokenManager(
224207 ugi.doAs(new PrivilegedExceptionAction [Credentials ]() {
225208 override def run (): Credentials = {
226209 val (creds, nextRenewal) = obtainDelegationTokens()
227-
228- // Calculate the time when new credentials should be created, based on the configured
229- // ratio.
230- val now = System .currentTimeMillis
231- val ratio = sparkConf.get(CREDENTIALS_RENEWAL_INTERVAL_RATIO )
232- val delay = (ratio * (nextRenewal - now)).toLong
233- logInfo(s " Calculated delay on renewal is $delay, based on next renewal $nextRenewal " +
234- s " and the ratio $ratio, and current time $now" )
210+ val delay = calculateNextRenewalInterval(nextRenewal)
235211 scheduleRenewal(delay)
236212 creds
237213 }
@@ -255,57 +231,12 @@ private[spark] class HadoopDelegationTokenManager(
255231 UserGroupInformation .getCurrentUser()
256232 }
257233 }
258-
259- private def loadProviders (): Map [String , HadoopDelegationTokenProvider ] = {
260- val loader = ServiceLoader .load(classOf [HadoopDelegationTokenProvider ],
261- Utils .getContextOrSparkClassLoader)
262- val providers = mutable.ArrayBuffer [HadoopDelegationTokenProvider ]()
263-
264- val iterator = loader.iterator
265- while (iterator.hasNext) {
266- try {
267- providers += iterator.next
268- } catch {
269- case t : Throwable =>
270- logDebug(s " Failed to load built in provider. " , t)
271- }
272- }
273-
274- // Filter out providers for which spark.security.credentials.{service}.enabled is false.
275- providers
276- .filter { p => HadoopDelegationTokenManager .isServiceEnabled(sparkConf, p.serviceName) }
277- .map { p => (p.serviceName, p) }
278- .toMap
279- }
280234}
281235
282- private [spark] object HadoopDelegationTokenManager extends Logging {
283- private val providerEnabledConfig = " spark.security.credentials.%s.enabled"
236+ private [spark] object HadoopDelegationTokenManager extends ServiceCredentialsConfig {
237+ override def providerEnabledConfig : String = " spark.security.credentials.%s.enabled"
284238
285- private val deprecatedProviderEnabledConfigs = List (
239+ override def deprecatedProviderEnabledConfigs : List [ String ] = List (
286240 " spark.yarn.security.tokens.%s.enabled" ,
287241 " spark.yarn.security.credentials.%s.enabled" )
288-
289- def isServiceEnabled (sparkConf : SparkConf , serviceName : String ): Boolean = {
290- val key = providerEnabledConfig.format(serviceName)
291-
292- deprecatedProviderEnabledConfigs.foreach { pattern =>
293- val deprecatedKey = pattern.format(serviceName)
294- if (sparkConf.contains(deprecatedKey)) {
295- logWarning(s " ${deprecatedKey} is deprecated. Please use ${key} instead. " )
296- }
297- }
298-
299- val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern =>
300- sparkConf
301- .getOption(pattern.format(serviceName))
302- .map(_.toBoolean)
303- .getOrElse(true )
304- }
305-
306- sparkConf
307- .getOption(key)
308- .map(_.toBoolean)
309- .getOrElse(isEnabledDeprecated)
310- }
311242}
0 commit comments