You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by go...@apache.org on 2016/03/16 21:58:24 UTC

[33/50] incubator-slider git commit: SLIDER-1077 tail end: filter out timeline token from renewer list; move getTokenExpirtyTime into credential utils & include better error message

SLIDER-1077 tail end: filter out timeline token from renewer list; move getTokenExpirtyTime into credential utils & include better error message


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/c9fc7f64
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/c9fc7f64
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/c9fc7f64

Branch: refs/heads/feature/SLIDER-906_docker_support
Commit: c9fc7f646ce15ddaf1306be1245d08197d95ff22
Parents: 9be27e6
Author: Steve Loughran <st...@apache.org>
Authored: Thu Feb 4 19:49:09 2016 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Feb 4 19:49:09 2016 +0000

----------------------------------------------------------------------
 .../slider/core/launch/CredentialUtils.java       | 16 ++++++++++++++++
 .../slider/server/appmaster/SliderAppMaster.java  |  9 +++------
 .../server/appmaster/actions/QueueExecutor.java   |  2 +-
 .../security/FsDelegationTokenManager.java        | 18 ++++++------------
 4 files changed, 26 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/c9fc7f64/slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java b/slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java
index 012a737..255890b 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/CredentialUtils.java
@@ -349,6 +349,22 @@ public final class CredentialUtils {
     return buffer.toString();
   }
 
+  /**
+   * Get the expiry time of a token.
+   * @param token token to examine
+   * @return the time in milliseconds after which the token is invalid.
+   * @throws IOException
+   */
+  public static long getTokenExpiryTime(Token token) throws IOException {
+    TokenIdentifier identifier = token.decodeIdentifier();
+    Preconditions.checkState(identifier instanceof AbstractDelegationTokenIdentifier,
+        "Token %s of type: %s has an identifier which cannot be examined: %s",
+        token, token.getClass(), identifier);
+    AbstractDelegationTokenIdentifier id =
+        (AbstractDelegationTokenIdentifier) identifier;
+    return id.getMaxDate();
+  }
+
   private static class TokenComparator
       implements Comparator<Token<? extends TokenIdentifier>>, Serializable {
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/c9fc7f64/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 24c32bb..3f609b1 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -80,6 +80,7 @@ import org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes;
 import org.apache.hadoop.registry.server.integration.RMRegistryOperationsService;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.WebAppException;
 import org.apache.hadoop.yarn.webapp.WebApps;
@@ -294,11 +295,6 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   private ApplicationAttemptId appAttemptID;
 
   /**
-   * Security info client to AM key returned after registration
-   */
-  private ByteBuffer clientToAMKey;
-
-  /**
    * App ACLs
    */
   protected Map<ApplicationAccessType, String> applicationACLs;
@@ -1133,8 +1129,9 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   private void processAMCredentials(SecurityConfiguration securityConfig)
       throws IOException {
 
-    List<Text> filteredTokens = new ArrayList<>(2);
+    List<Text> filteredTokens = new ArrayList<>(3);
     filteredTokens.add(AMRMTokenIdentifier.KIND_NAME);
+    filteredTokens.add(TimelineDelegationTokenIdentifier.KIND_NAME);
 
     boolean keytabProvided = securityConfig.isKeytabProvided();
     log.info("Slider AM Security Mode: {}", keytabProvided ? "KEYTAB" : "TOKEN");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/c9fc7f64/slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java
index fd4e37b..d0fc2cf 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/actions/QueueExecutor.java
@@ -51,7 +51,7 @@ public class QueueExecutor implements Runnable {
   }
 
   @VisibleForTesting
-  QueueExecutor(QueueService actionQueues) {
+  public QueueExecutor(QueueService actionQueues) {
     Preconditions.checkNotNull(actionQueues);
     this.appMaster = null;
     this.appState = null;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/c9fc7f64/slider-core/src/main/java/org/apache/slider/server/services/security/FsDelegationTokenManager.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/security/FsDelegationTokenManager.java b/slider-core/src/main/java/org/apache/slider/server/services/security/FsDelegationTokenManager.java
index 63f66a3..617fe3c 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/security/FsDelegationTokenManager.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/security/FsDelegationTokenManager.java
@@ -23,10 +23,10 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.hadoop.util.Time;
 import org.apache.slider.common.SliderXmlConfKeys;
 import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.launch.CredentialUtils;
 import org.apache.slider.server.appmaster.SliderAppMaster;
 import org.apache.slider.server.appmaster.actions.AsyncAction;
 import org.apache.slider.server.appmaster.actions.QueueAccess;
@@ -114,7 +114,7 @@ public class FsDelegationTokenManager {
     if (renewAction != null) {
       renewAction.getToken().cancel(configuration);
     }
-    log.info("Renewing action {} removed and HDFS delegation token renewal "
+    log.info("Renewing action {} removed and delegation token renewal "
              + "cancelled", getRenewingActionName());
   }
 
@@ -156,9 +156,9 @@ public class FsDelegationTokenManager {
                      remoteUser.getShortUserName());
             Token token = fs.getDelegationToken(
                 remoteUser.getShortUserName());
-            tokenExpiryTime = getTokenExpiryTime(token);
-            log.info("Initial delegation token obtained with expiry time of {}", getPrintableExpirationTime(
-                tokenExpiryTime));
+            tokenExpiryTime = CredentialUtils.getTokenExpiryTime(token);
+            log.info("Initial delegation token obtained with expiry time of {}",
+                getPrintableExpirationTime(tokenExpiryTime));
             return token;
           }
         });
@@ -166,12 +166,6 @@ public class FsDelegationTokenManager {
       log.info("Initial request returned delegation token {}", token);
     }
 
-    private long getTokenExpiryTime(Token token) throws IOException {
-      AbstractDelegationTokenIdentifier id =
-          (AbstractDelegationTokenIdentifier)token.decodeIdentifier();
-      return id.getMaxDate();
-    }
-
     protected FileSystem getFileSystem()
         throws IOException, InterruptedException {
       // return non-cache FS reference
@@ -242,7 +236,7 @@ public class FsDelegationTokenManager {
         token = findMatchingToken(service, tokens);
         currentUser.addToken(token.getService(), token);
 
-        tokenExpiryTime = getTokenExpiryTime(token);
+        tokenExpiryTime = CredentialUtils.getTokenExpiryTime(token);
 
         log.info("Expired HDFS delegation token replaced and added as credential"
                  + " to current user.  Token expires at {}",