You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/03/31 10:18:29 UTC

[GitHub] [spark] gaborgsomogyi commented on a change in pull request #32009: [SPARK-34914][CORE] Local scheduler backend support update token

gaborgsomogyi commented on a change in pull request #32009:
URL: https://github.com/apache/spark/pull/32009#discussion_r604759649



##########
File path: core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
##########
@@ -178,4 +188,23 @@ private[spark] class LocalSchedulerBackend(
     }
   }
 
+  private def updateDelegationTokens(): Unit = {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      val delegationTokenManager =
+        new HadoopDelegationTokenManager(conf, scheduler.sc.hadoopConfiguration, localEndpoint)
+      val ugi = UserGroupInformation.getCurrentUser()
+      val tokens = if (delegationTokenManager.renewalEnabled) {
+        delegationTokenManager.start()
+      } else {
+        val creds = ugi.getCredentials()
+        delegationTokenManager.obtainDelegationTokens(creds)
+        if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) {
+          SparkHadoopUtil.get.serialize(creds)
+        } else {
+          null
+        }
+      }
+      SparkHadoopUtil.get.addDelegationTokens(tokens, conf)

Review comment:
       `tokens` can be null, right?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
##########
@@ -76,6 +81,9 @@ private[spark] class LocalEndpoint(
 
     case KillTask(taskId, interruptThread, reason) =>
       executor.killTask(taskId, interruptThread, reason)
+
+    case UpdateDelegationTokens(tokens) =>

Review comment:
       Just thinking out loud. Because in local version of Spark the executor, backend, and master all run in the same JVM it's not necessary to send the tokens to the `executorEndpoint` so it's fine as it.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
##########
@@ -178,4 +188,23 @@ private[spark] class LocalSchedulerBackend(
     }
   }
 
+  private def updateDelegationTokens(): Unit = {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      val delegationTokenManager =

Review comment:
       Since we don't store the reference of the `HadoopDelegationTokenManager` how does this renew the tokens? Or renewal is not considered?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
##########
@@ -76,6 +81,9 @@ private[spark] class LocalEndpoint(
 
     case KillTask(taskId, interruptThread, reason) =>
       executor.killTask(taskId, interruptThread, reason)
+
+    case UpdateDelegationTokens(tokens) =>
+      SparkHadoopUtil.get.addDelegationTokens(tokens, scheduler.conf)

Review comment:
       It would be good to add some info like in `CoarseGrainedExecutorBackend` to increase debug possibilities:
   ```
         logInfo(s"Received tokens of ${tokenBytes.length} bytes")
   ```
   

##########
File path: core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
##########
@@ -178,4 +188,23 @@ private[spark] class LocalSchedulerBackend(
     }
   }
 
+  private def updateDelegationTokens(): Unit = {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      val delegationTokenManager =
+        new HadoopDelegationTokenManager(conf, scheduler.sc.hadoopConfiguration, localEndpoint)
+      val ugi = UserGroupInformation.getCurrentUser()
+      val tokens = if (delegationTokenManager.renewalEnabled) {
+        delegationTokenManager.start()

Review comment:
       `delegationTokenManager` is never stopper.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org