소스 검색

HADOOP-12563. Updated utility (dtutil) to create/modify token files. Contributed by Matthew Paduano

Ravi Prakash 9 년 전
부모
커밋
4838b735f0
18개의 변경된 파일1600개의 추가작업 그리고 77개의 파일을 삭제
  1. 4 0
      hadoop-common-project/hadoop-common/src/main/bin/hadoop
  2. 124 45
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java
  3. 41 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFetcher.java
  4. 271 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java
  5. 326 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java
  6. 63 31
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java
  7. 114 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/CommandShell.java
  8. 11 0
      hadoop-common-project/hadoop-common/src/main/proto/Security.proto
  9. 19 0
      hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
  10. 41 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtFetcher.java
  11. 264 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java
  12. 128 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java
  13. 14 0
      hadoop-common-project/hadoop-common/src/test/resources/META-INF/services/org.apache.hadoop.security.token.DtFetcher
  14. 82 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsDtFetcher.java
  15. 39 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/SWebHdfsDtFetcher.java
  16. 39 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/WebHdfsDtFetcher.java
  17. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
  18. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.DtFetcher

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/bin/hadoop

@@ -33,6 +33,7 @@ function hadoop_usage
   hadoop_add_subcommand "daemonlog" "get/set the log level for each daemon"
   hadoop_add_subcommand "distch" "distributed metadata changer"
   hadoop_add_subcommand "distcp" "copy file or directories recursively"
+  hadoop_add_subcommand "dtutil" "operations related to delegation tokens"
   hadoop_add_subcommand "envvars" "display computed Hadoop environment variables"
   hadoop_add_subcommand "fs" "run a generic filesystem user client"
   hadoop_add_subcommand "jar <jar>" "run a jar file. NOTE: please use \"yarn jar\" to launch YARN applications, not this command."
@@ -139,6 +140,9 @@ case ${COMMAND} in
     CLASS=org.apache.hadoop.tools.DistCp
     hadoop_add_to_classpath_tools hadoop-distcp
   ;;
+  dtutil)
+    CLASS=org.apache.hadoop.security.token.DtUtilShell
+  ;;
   envvars)
     echo "JAVA_HOME='${JAVA_HOME}'"
     echo "HADOOP_COMMON_HOME='${HADOOP_COMMON_HOME}'"

+ 124 - 45
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.security;
 
+import com.google.protobuf.ByteString;
+
 import java.io.BufferedInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
@@ -25,6 +27,7 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
@@ -47,9 +50,11 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.proto.SecurityProtos.CredentialsKVProto;
+import org.apache.hadoop.security.proto.SecurityProtos.CredentialsProto;
 
 /**
- * A class that provides the facilities of reading and writing 
+ * A class that provides the facilities of reading and writing
  * secret keys and Tokens.
  */
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@@ -58,34 +63,34 @@ public class Credentials implements Writable {
   private static final Log LOG = LogFactory.getLog(Credentials.class);
 
   private  Map<Text, byte[]> secretKeysMap = new HashMap<Text, byte[]>();
-  private  Map<Text, Token<? extends TokenIdentifier>> tokenMap = 
-    new HashMap<Text, Token<? extends TokenIdentifier>>(); 
+  private  Map<Text, Token<? extends TokenIdentifier>> tokenMap =
+      new HashMap<Text, Token<? extends TokenIdentifier>>();
 
   /**
-   * Create an empty credentials instance
+   * Create an empty credentials instance.
    */
   public Credentials() {
   }
-  
+
   /**
-   * Create a copy of the given credentials
+   * Create a copy of the given credentials.
    * @param credentials to copy
    */
   public Credentials(Credentials credentials) {
     this.addAll(credentials);
   }
-  
+
   /**
-   * Returns the Token object for the alias
+   * Returns the Token object for the alias.
    * @param alias the alias for the Token
    * @return token for this alias
    */
   public Token<? extends TokenIdentifier> getToken(Text alias) {
     return tokenMap.get(alias);
   }
-  
+
   /**
-   * Add a token in the storage (in memory)
+   * Add a token in the storage (in memory).
    * @param alias the alias for the key
    * @param t the token object
    */
@@ -96,14 +101,14 @@ public class Credentials implements Writable {
       LOG.warn("Null token ignored for " + alias);
     }
   }
-  
+
   /**
-   * Return all the tokens in the in-memory map
+   * Return all the tokens in the in-memory map.
    */
   public Collection<Token<? extends TokenIdentifier>> getAllTokens() {
     return tokenMap.values();
   }
-  
+
   /**
    * @return number of Tokens in the in-memory map
    */
@@ -112,23 +117,23 @@ public class Credentials implements Writable {
   }
 
   /**
-   * Returns the key bytes for the alias
+   * Returns the key bytes for the alias.
    * @param alias the alias for the key
    * @return key for this alias
    */
   public byte[] getSecretKey(Text alias) {
     return secretKeysMap.get(alias);
   }
-  
+
   /**
    * @return number of keys in the in-memory map
    */
   public int numberOfSecretKeys() {
     return secretKeysMap.size();
   }
-  
+
   /**
-   * Set the key for an alias
+   * Set the key for an alias.
    * @param alias the alias for the key
    * @param key the key bytes
    */
@@ -145,7 +150,7 @@ public class Credentials implements Writable {
   }
 
   /**
-   * Return all the secret key entries in the in-memory map
+   * Return all the secret key entries in the in-memory map.
    */
   public List<Text> getAllSecretKeys() {
     List<Text> list = new java.util.ArrayList<Text>();
@@ -155,13 +160,13 @@ public class Credentials implements Writable {
   }
 
   /**
-   * Convenience method for reading a token storage file, and loading the Tokens
-   * therein in the passed UGI
+   * Convenience method for reading a token storage file and loading its Tokens.
    * @param filename
    * @param conf
    * @throws IOException
    */
-  public static Credentials readTokenStorageFile(Path filename, Configuration conf)
+  public static Credentials readTokenStorageFile(Path filename,
+                                                 Configuration conf)
   throws IOException {
     FSDataInputStream in = null;
     Credentials credentials = new Credentials();
@@ -178,13 +183,13 @@ public class Credentials implements Writable {
   }
 
   /**
-   * Convenience method for reading a token storage file, and loading the Tokens
-   * therein in the passed UGI
+   * Convenience method for reading a token storage file and loading its Tokens.
    * @param filename
    * @param conf
    * @throws IOException
    */
-  public static Credentials readTokenStorageFile(File filename, Configuration conf)
+  public static Credentials readTokenStorageFile(File filename,
+                                                 Configuration conf)
       throws IOException {
     DataInputStream in = null;
     Credentials credentials = new Credentials();
@@ -199,10 +204,9 @@ public class Credentials implements Writable {
       IOUtils.cleanup(LOG, in);
     }
   }
-  
+
   /**
-   * Convenience method for reading a token storage file directly from a 
-   * datainputstream
+   * Convenience method for reading a token from a DataInputStream.
    */
   public void readTokenStorageStream(DataInputStream in) throws IOException {
     byte[] magic = new byte[TOKEN_STORAGE_MAGIC.length];
@@ -211,25 +215,36 @@ public class Credentials implements Writable {
       throw new IOException("Bad header found in token storage.");
     }
     byte version = in.readByte();
-    if (version != TOKEN_STORAGE_VERSION) {
-      throw new IOException("Unknown version " + version + 
+    if (version != TOKEN_STORAGE_VERSION &&
+        version != OLD_TOKEN_STORAGE_VERSION) {
+      throw new IOException("Unknown version " + version +
                             " in token storage.");
     }
-    readFields(in);
+    if (version == OLD_TOKEN_STORAGE_VERSION) {
+      readFields(in);
+    } else if (version == TOKEN_STORAGE_VERSION) {
+      readProtos(in);
+    }
   }
-  
+
   private static final byte[] TOKEN_STORAGE_MAGIC =
       "HDTS".getBytes(Charsets.UTF_8);
-  private static final byte TOKEN_STORAGE_VERSION = 0;
-  
+  private static final byte TOKEN_STORAGE_VERSION = 1;
+
+  /**
+   *  For backward compatibility.
+   */
+  private static final byte OLD_TOKEN_STORAGE_VERSION = 0;
+
+
   public void writeTokenStorageToStream(DataOutputStream os)
-    throws IOException {
+      throws IOException {
     os.write(TOKEN_STORAGE_MAGIC);
     os.write(TOKEN_STORAGE_VERSION);
-    write(os);
+    writeProto(os);
   }
 
-  public void writeTokenStorageFile(Path filename, 
+  public void writeTokenStorageFile(Path filename,
                                     Configuration conf) throws IOException {
     FSDataOutputStream os = filename.getFileSystem(conf).create(filename);
     writeTokenStorageToStream(os);
@@ -237,7 +252,29 @@ public class Credentials implements Writable {
   }
 
   /**
-   * Stores all the keys to DataOutput
+   *  For backward compatibility.
+   */
+  public void writeLegacyTokenStorageLocalFile(File f) throws IOException {
+    writeLegacyOutputStream(new DataOutputStream(new FileOutputStream(f)));
+  }
+
+  /**
+   *  For backward compatibility.
+   */
+  public void writeLegacyTokenStorageFile(Path filename, Configuration conf)
+      throws IOException {
+    writeLegacyOutputStream(filename.getFileSystem(conf).create(filename));
+  }
+
+  private void writeLegacyOutputStream(DataOutputStream os) throws IOException {
+    os.write(TOKEN_STORAGE_MAGIC);
+    os.write(OLD_TOKEN_STORAGE_VERSION);
+    write(os);
+    os.close();
+  }
+
+  /**
+   * Stores all the keys to DataOutput.
    * @param out
    * @throws IOException
    */
@@ -245,12 +282,12 @@ public class Credentials implements Writable {
   public void write(DataOutput out) throws IOException {
     // write out tokens first
     WritableUtils.writeVInt(out, tokenMap.size());
-    for(Map.Entry<Text, 
-        Token<? extends TokenIdentifier>> e: tokenMap.entrySet()) {
+    for(Map.Entry<Text,
+            Token<? extends TokenIdentifier>> e: tokenMap.entrySet()) {
       e.getKey().write(out);
       e.getValue().write(out);
     }
-    
+
     // now write out secret keys
     WritableUtils.writeVInt(out, secretKeysMap.size());
     for(Map.Entry<Text, byte[]> e : secretKeysMap.entrySet()) {
@@ -259,9 +296,51 @@ public class Credentials implements Writable {
       out.write(e.getValue());
     }
   }
-  
+
+  /**
+   * Write contents of this instance as CredentialsProto message to DataOutput.
+   * @param out
+   * @throws IOException
+   */
+  public void writeProto(DataOutput out) throws IOException {
+    CredentialsProto.Builder storage = CredentialsProto.newBuilder();
+    for (Map.Entry<Text, Token<? extends TokenIdentifier>> e :
+                                                         tokenMap.entrySet()) {
+      CredentialsKVProto.Builder kv = CredentialsKVProto.newBuilder().
+          setAliasBytes(ByteString.copyFrom(
+              e.getKey().getBytes(), 0, e.getKey().getLength())).
+          setToken(e.getValue().toTokenProto());
+      storage.addTokens(kv.build());
+    }
+
+    for(Map.Entry<Text, byte[]> e : secretKeysMap.entrySet()) {
+      CredentialsKVProto.Builder kv = CredentialsKVProto.newBuilder().
+          setAliasBytes(ByteString.copyFrom(
+              e.getKey().getBytes(), 0, e.getKey().getLength())).
+          setSecret(ByteString.copyFrom(e.getValue()));
+      storage.addSecrets(kv.build());
+    }
+    storage.build().writeTo((DataOutputStream)out);
+  }
+
   /**
-   * Loads all the keys
+   * Populates keys/values from proto buffer storage.
+   * @param in - stream ready to read a serialized proto buffer message
+   */
+  public void readProtos(DataInput in) throws IOException {
+    CredentialsProto storage = CredentialsProto.parseFrom((DataInputStream)in);
+    for (CredentialsKVProto kv : storage.getTokensList()) {
+      addToken(new Text(kv.getAliasBytes().toByteArray()),
+               (Token<? extends TokenIdentifier>) new Token(kv.getToken()));
+    }
+    for (CredentialsKVProto kv : storage.getSecretsList()) {
+      addSecretKey(new Text(kv.getAliasBytes().toByteArray()),
+                   kv.getSecret().toByteArray());
+    }
+  }
+
+  /**
+   * Loads all the keys.
    * @param in
    * @throws IOException
    */
@@ -269,7 +348,7 @@ public class Credentials implements Writable {
   public void readFields(DataInput in) throws IOException {
     secretKeysMap.clear();
     tokenMap.clear();
-    
+
     int size = WritableUtils.readVInt(in);
     for(int i=0; i<size; i++) {
       Text alias = new Text();
@@ -278,7 +357,7 @@ public class Credentials implements Writable {
       t.readFields(in);
       tokenMap.put(alias, t);
     }
-    
+
     size = WritableUtils.readVInt(in);
     for(int i=0; i<size; i++) {
       Text alias = new Text();
@@ -289,7 +368,7 @@ public class Credentials implements Writable {
       secretKeysMap.put(alias, value);
     }
   }
- 
+
   /**
    * Copy all of the credentials from one credential object into another.
    * Existing secrets and tokens are overwritten.

+ 41 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFetcher.java

@@ -0,0 +1,41 @@
+/**
+ * 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.hadoop.security.token;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+
+/**
+ *  DtFetcher is an interface which permits the abstraction and separation of
+ *  delegation token fetch implementaions across different packages and
+ *  compilation units.  Resolution of fetcher impl will be done at runtime.
+ */
+public interface DtFetcher {
+  /** Return a key used to identify the object/service implementation. */
+  Text getServiceName();
+
+  /** Used to allow the service API to indicate whether a token is required. */
+  boolean isTokenRequired();
+
+  /** Add any number of delegation tokens to Credentials object and return
+   *  a token instance that is appropriate for aliasing, or null if none. */
+  Token<?> addDelegationTokens(Configuration conf, Credentials creds,
+                               String renewer, String url) throws Exception;
+}

+ 271 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java

@@ -0,0 +1,271 @@
+/**
+ * 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.hadoop.security.token;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.DateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.ServiceLoader;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+
+/**
+ * DtFileOperations is a collection of delegation token file operations.
+ */
+public final class DtFileOperations {
+  private static final Log LOG = LogFactory.getLog(DtFileOperations.class);
+
+  /** No public constructor as per checkstyle. */
+  private DtFileOperations() { }
+
+  /**
+   * Use FORMAT_* as arguments to format parameters.
+   * FORMAT_PB is for protobuf output.
+   */
+  public static final String FORMAT_PB = "protobuf";
+
+  /**
+   * Use FORMAT_* as arguments to format parameters.
+   * FORMAT_JAVA is a legacy option for java serialization output.
+   */
+  public static final String FORMAT_JAVA = "java";
+
+  private static final String NA_STRING = "-NA-";
+  private static final String PREFIX_HTTP = "http://";
+  private static final String PREFIX_HTTPS = "https://";
+
+  /** Let the DtFetcher code add the appropriate prefix if HTTP/S is used. */
+  private static String stripPrefix(String u) {
+    return u.replaceFirst(PREFIX_HTTP, "").replaceFirst(PREFIX_HTTPS, "");
+  }
+
+  /** Match token service field to alias text.  True if alias is null. */
+  private static boolean matchAlias(Token<?> token, Text alias) {
+    return alias == null || token.getService().equals(alias);
+  }
+
+  /** Match fetcher's service name to the service text and/or url prefix. */
+  private static boolean matchService(
+      DtFetcher fetcher, Text service, String url) {
+    Text sName = fetcher.getServiceName();
+    return (service == null && url.startsWith(sName.toString() + "://")) ||
+           (service != null && service.equals(sName));
+  }
+
+  /** Format a long integer type into a date string. */
+  private static String formatDate(long date) {
+    DateFormat df = DateFormat.getDateTimeInstance(
+        DateFormat.SHORT, DateFormat.SHORT);
+    return df.format(new Date(date));
+  }
+
+  /** Add the service prefix for a local filesystem. */
+  private static Path fileToPath(File f) {
+    return new Path("file:" + f.getAbsolutePath());
+  }
+
+  /** Write out a Credentials object as a local file.
+   *  @param f a local File object.
+   *  @param format a string equal to FORMAT_PB or FORMAT_JAVA.
+   *  @param creds the Credentials object to be written out.
+   *  @param conf a Configuration object passed along.
+   *  @throws IOException
+   */
+  public static void doFormattedWrite(
+      File f, String format, Credentials creds, Configuration conf)
+      throws IOException {
+    if (format == null || format.equals(FORMAT_PB)) {
+      creds.writeTokenStorageFile(fileToPath(f), conf);
+    } else { // if (format != null && format.equals(FORMAT_JAVA)) {
+      creds.writeLegacyTokenStorageLocalFile(f);
+    }
+  }
+
+  /** Print out a Credentials file from the local filesystem.
+   *  @param tokenFile a local File object.
+   *  @param alias print only tokens matching alias (null matches all).
+   *  @param conf Configuration object passed along.
+   *  @param out print to this stream.
+   *  @throws IOException
+   */
+  public static void printTokenFile(
+      File tokenFile, Text alias, Configuration conf, PrintStream out)
+      throws IOException {
+    out.println("File: " + tokenFile.getPath());
+    Credentials creds = Credentials.readTokenStorageFile(tokenFile, conf);
+    printCredentials(creds, alias, out);
+  }
+
+  /** Print out a Credentials object.
+   *  @param creds the Credentials object to be printed out.
+   *  @param alias print only tokens matching alias (null matches all).
+   *  @param out print to this stream.
+   *  @throws IOException
+   */
+  public static void printCredentials(
+      Credentials creds, Text alias, PrintStream out)
+      throws IOException {
+    boolean tokenHeader = true;
+    String fmt = "%-24s %-20s %-15s %-12s %s%n";
+    for (Token<?> token : creds.getAllTokens()) {
+      if (matchAlias(token, alias)) {
+        if (tokenHeader) {
+          out.printf(fmt, "Token kind", "Service", "Renewer", "Exp date",
+                     "URL enc token");
+          out.println(StringUtils.repeat("-", 80));
+          tokenHeader = false;
+        }
+        AbstractDelegationTokenIdentifier id =
+            (AbstractDelegationTokenIdentifier) token.decodeIdentifier();
+        out.printf(fmt, token.getKind(), token.getService(),
+                   (id != null) ? id.getRenewer() : NA_STRING,
+                   (id != null) ? formatDate(id.getMaxDate()) : NA_STRING,
+                   token.encodeToUrlString());
+      }
+    }
+  }
+
+  /** Fetch a token from a service and save to file in the local filesystem.
+   *  @param tokenFile a local File object to hold the output.
+   *  @param fileFormat a string equal to FORMAT_PB or FORMAT_JAVA, for output
+   *  @param alias overwrite service field of fetched token with this text.
+   *  @param service use a DtFetcher implementation matching this service text.
+   *  @param url pass this URL to fetcher after stripping any http/s prefix.
+   *  @param renewer pass this renewer to the fetcher.
+   *  @param conf Configuration object passed along.
+   *  @throws IOException
+   */
+  public static void getTokenFile(File tokenFile, String fileFormat,
+      Text alias, Text service, String url, String renewer, Configuration conf)
+      throws Exception {
+    Token<?> token = null;
+    Credentials creds = tokenFile.exists() ?
+        Credentials.readTokenStorageFile(tokenFile, conf) : new Credentials();
+    ServiceLoader<DtFetcher> loader = ServiceLoader.load(DtFetcher.class);
+    for (DtFetcher fetcher : loader) {
+      if (matchService(fetcher, service, url)) {
+        if (!fetcher.isTokenRequired()) {
+          String message = "DtFetcher for service '" + service +
+              "' does not require a token.  Check your configuration.  " +
+              "Note: security may be disabled or there may be two DtFetcher " +
+              "providers for the same service designation.";
+          LOG.error(message);
+          throw new IllegalArgumentException(message);
+        }
+        token = fetcher.addDelegationTokens(conf, creds, renewer,
+                                            stripPrefix(url));
+      }
+    }
+    if (alias != null) {
+      if (token == null) {
+        String message = "DtFetcher for service '" + service + "'" +
+            " does not allow aliasing.  Cannot apply alias '" + alias + "'." +
+            "  Drop alias flag to get token for this service.";
+        LOG.error(message);
+        throw new IOException(message);
+      }
+      Token<?> aliasedToken = token.copyToken();
+      aliasedToken.setService(alias);
+      creds.addToken(alias, aliasedToken);
+      LOG.info("Add token with service " + alias);
+    }
+    doFormattedWrite(tokenFile, fileFormat, creds, conf);
+  }
+
+  /** Append tokens from list of files in local filesystem, saving to last file.
+   *  @param tokenFiles list of local File objects.  Last file holds the output.
+   *  @param fileFormat a string equal to FORMAT_PB or FORMAT_JAVA, for output
+   *  @param conf Configuration object passed along.
+   *  @throws IOException
+   */
+  public static void appendTokenFiles(
+      ArrayList<File> tokenFiles, String fileFormat, Configuration conf)
+      throws IOException {
+    Credentials newCreds = new Credentials();
+    File lastTokenFile = null;
+    for (File tokenFile : tokenFiles) {
+      lastTokenFile = tokenFile;
+      Credentials creds = Credentials.readTokenStorageFile(tokenFile, conf);
+      for (Token<?> token : creds.getAllTokens()) {
+        newCreds.addToken(token.getService(), token);
+      }
+    }
+    doFormattedWrite(lastTokenFile, fileFormat, newCreds, conf);
+  }
+
+  /** Remove a token from a file in the local filesystem, matching alias.
+   *  @param cancel cancel token as well as remove from file.
+   *  @param tokenFile a local File object.
+   *  @param fileFormat a string equal to FORMAT_PB or FORMAT_JAVA, for output
+   *  @param alias remove only tokens matching alias; null matches all.
+   *  @param conf Configuration object passed along.
+   *  @throws IOException
+   *  @throws InterruptedException
+   */
+  public static void removeTokenFromFile(boolean cancel,
+      File tokenFile, String fileFormat, Text alias, Configuration conf)
+      throws IOException, InterruptedException {
+    Credentials newCreds = new Credentials();
+    Credentials creds = Credentials.readTokenStorageFile(tokenFile, conf);
+    for (Token<?> token : creds.getAllTokens()) {
+      if (matchAlias(token, alias)) {
+        if (token.isManaged() && cancel) {
+          token.cancel(conf);
+          LOG.info("Canceled " + token.getKind() + ":" + token.getService());
+        }
+      } else {
+        newCreds.addToken(token.getService(), token);
+      }
+    }
+    doFormattedWrite(tokenFile, fileFormat, newCreds, conf);
+  }
+
+  /** Renew a token from a file in the local filesystem, matching alias.
+   *  @param tokenFile a local File object.
+   *  @param fileFormat a string equal to FORMAT_PB or FORMAT_JAVA, for output
+   *  @param alias renew only tokens matching alias; null matches all.
+   *  @param conf Configuration object passed along.
+   *  @throws IOException
+   *  @throws InterruptedException
+   */
+  public static void renewTokenFile(
+      File tokenFile, String fileFormat, Text alias, Configuration conf)
+      throws IOException, InterruptedException {
+    Credentials creds = Credentials.readTokenStorageFile(tokenFile, conf);
+    for (Token<?> token : creds.getAllTokens()) {
+      if (token.isManaged() && matchAlias(token, alias)) {
+        long result = token.renew(conf);
+        LOG.info("Renewed" + token.getKind() + ":" + token.getService() +
+                 " until " + formatDate(result));
+      }
+    }
+    doFormattedWrite(tokenFile, fileFormat, creds, conf);
+  }
+}

+ 326 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java

@@ -0,0 +1,326 @@
+/**
+ * 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.hadoop.security.token;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.tools.CommandShell;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ *  DtUtilShell is a set of command line token file management operations.
+ */
+public class DtUtilShell extends CommandShell {
+  private static final Log LOG = LogFactory.getLog(DtUtilShell.class);
+
+  private static final String FORMAT_SUBSTRING = "[-format (" +
+      DtFileOperations.FORMAT_JAVA + "|" +
+      DtFileOperations.FORMAT_PB + ")]";
+  public static final String DT_USAGE = "hadoop dtutil " +
+      "[-keytab <keytab_file> -principal <principal_name>] " +
+      "subcommand (help|print|get|append|cancel|remove|renew) " +
+         FORMAT_SUBSTRING + " [-alias <alias>] filename...";
+
+  // command line options
+  private static final String HELP = "help";
+  private static final String KEYTAB = "-keytab";
+  private static final String PRINCIPAL = "-principal";
+  private static final String PRINT = "print";
+  private static final String GET = "get";
+  private static final String APPEND = "append";
+  private static final String CANCEL = "cancel";
+  private static final String REMOVE = "remove";
+  private static final String RENEW = "renew";
+  private static final String RENEWER = "-renewer";
+  private static final String SERVICE = "-service";
+  private static final String ALIAS = "-alias";
+  private static final String FORMAT = "-format";
+
+  // configuration state from args, conf
+  private String keytab = null;
+  private String principal = null;
+  private Text alias = null;
+  private Text service = null;
+  private String renewer = null;
+  private String format = DtFileOperations.FORMAT_PB;
+  private ArrayList<File> tokenFiles = null;
+  private File firstFile = null;
+
+  /**
+   * Parse arguments looking for Kerberos keytab/principal.
+   * If both are found: remove both from the argument list and attempt login.
+   * If only one of the two is found: remove it from argument list, log warning
+   * and do not attempt login.
+   * If neither is found: return original args array, doing nothing.
+   * Return the pruned args array if either flag is present.
+   */
+  private String[] maybeDoLoginFromKeytabAndPrincipal(String[] args)
+      throws IOException{
+    ArrayList<String> savedArgs = new ArrayList<String>(args.length);
+    for (int i = 0; i < args.length; i++) {
+      String current = args[i];
+      if (current.equals(PRINCIPAL)) {
+        principal = args[++i];
+      } else if (current.equals(KEYTAB)) {
+        keytab = args[++i];
+      } else {
+        savedArgs.add(current);
+      }
+    }
+    int newSize = savedArgs.size();
+    if (newSize != args.length) {
+      if (principal != null && keytab != null) {
+        UserGroupInformation.loginUserFromKeytab(principal, keytab);
+      } else {
+        LOG.warn("-principal and -keytab not both specified!  " +
+                 "Kerberos login not attempted.");
+      }
+      return savedArgs.toArray(new String[newSize]);
+    }
+    return args;
+  }
+
+  /**
+   * Parse the command line arguments and initialize subcommand.
+   * Also will attempt to perform Kerberos login if both -principal and -keytab
+   * flags are passed in args array.
+   * @param args
+   * @return 0 if the argument(s) were recognized, 1 otherwise
+   * @throws Exception
+   */
+  @Override
+  protected int init(String[] args) throws Exception {
+    if (0 == args.length) {
+      return 1;
+    }
+    tokenFiles = new ArrayList<File>();
+    args = maybeDoLoginFromKeytabAndPrincipal(args);
+    for (int i = 0; i < args.length; i++) {
+      if (i == 0) {
+        String command = args[0];
+        if (command.equals(HELP)) {
+          return 1;
+        } else if (command.equals(PRINT)) {
+          setSubCommand(new Print());
+        } else if (command.equals(GET)) {
+          setSubCommand(new Get(args[++i]));
+        } else if (command.equals(APPEND)) {
+          setSubCommand(new Append());
+        } else if (command.equals(CANCEL)) {
+          setSubCommand(new Remove(true));
+        } else if (command.equals(REMOVE)) {
+          setSubCommand(new Remove(false));
+        } else if (command.equals(RENEW)) {
+          setSubCommand(new Renew());
+        }
+      } else if (args[i].equals(ALIAS)) {
+        alias = new Text(args[++i]);
+      } else if (args[i].equals(SERVICE)) {
+        service = new Text(args[++i]);
+      } else if (args[i].equals(RENEWER)) {
+        renewer = args[++i];
+      } else if (args[i].equals(FORMAT)) {
+        format = args[++i];
+        if (!format.equals(DtFileOperations.FORMAT_JAVA) &&
+            !format.equals(DtFileOperations.FORMAT_PB)) {
+          LOG.error("-format must be '" + DtFileOperations.FORMAT_JAVA +
+                    "' or '" + DtFileOperations.FORMAT_PB + "' not '" +
+                    format + "'");
+          return 1;
+        }
+      } else {
+        for (; i < args.length; i++) {
+          File f = new File(args[i]);
+          if (f.exists()) {
+            tokenFiles.add(f);
+          }
+          if (firstFile == null) {
+            firstFile = f;
+          }
+        }
+        if (tokenFiles.size() == 0 && firstFile == null) {
+          LOG.error("Must provide a filename to all commands.");
+          return 1;
+        }
+      }
+    }
+    return 0;
+  }
+
+  @Override
+  public String getCommandUsage() {
+    return String.format("%n%s%n   %s%n   %s%n   %s%n   %s%n   %s%n   %s%n%n",
+                  DT_USAGE, (new Print()).getUsage(), (new Get()).getUsage(),
+                  (new Append()).getUsage(), (new Remove(true)).getUsage(),
+                  (new Remove(false)).getUsage(), (new Renew()).getUsage());
+  }
+
+  private class Print extends SubCommand {
+    public static final String PRINT_USAGE =
+        "dtutil print [-alias <alias>] filename...";
+
+    @Override
+    public void execute() throws Exception {
+      for (File tokenFile : tokenFiles) {
+        DtFileOperations.printTokenFile(tokenFile, alias, getConf(), getOut());
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return PRINT_USAGE;
+    }
+  }
+
+  private class Get extends SubCommand {
+    public static final String GET_USAGE = "dtutil get URL " +
+        "[-service <scheme>] " + FORMAT_SUBSTRING +
+        "[-alias <alias>] [-renewer <renewer>] filename";
+    private static final String PREFIX_HTTP = "http://";
+    private static final String PREFIX_HTTPS = "https://";
+
+    private String url = null;
+
+    public Get() { }
+
+    public Get(String arg) {
+      url = arg;
+    }
+
+    public boolean isGenericUrl() {
+      return url.startsWith(PREFIX_HTTP) || url.startsWith(PREFIX_HTTPS);
+    }
+
+    public boolean validate() {
+      if (service != null && !isGenericUrl()) {
+        LOG.error("Only provide -service with http/https URL.");
+        return false;
+      }
+      if (service == null && isGenericUrl()) {
+        LOG.error("Must provide -service with http/https URL.");
+        return false;
+      }
+      if (url.indexOf("://") == -1) {
+        LOG.error("URL does not contain a service specification: " + url);
+        return false;
+      }
+      return true;
+    }
+
+    @Override
+    public void execute() throws Exception {
+      DtFileOperations.getTokenFile(
+          firstFile, format, alias, service, url, renewer, getConf());
+    }
+
+    @Override
+    public String getUsage() {
+      return GET_USAGE;
+    }
+  }
+
+  private class Append extends SubCommand {
+    public static final String APPEND_USAGE =
+        "dtutil append " + FORMAT_SUBSTRING + "filename...";
+
+    @Override
+    public void execute() throws Exception {
+      DtFileOperations.appendTokenFiles(tokenFiles, format, getConf());
+    }
+
+    @Override
+    public String getUsage() {
+      return APPEND_USAGE;
+    }
+  }
+
+  private class Remove extends SubCommand {
+    public static final String REMOVE_USAGE =
+        "dtutil remove -alias <alias> " + FORMAT_SUBSTRING + " filename...";
+    public static final String CANCEL_USAGE =
+        "dtutil cancel -alias <alias> " + FORMAT_SUBSTRING + " filename...";
+    private boolean cancel = false;
+
+    public Remove(boolean arg) {
+      cancel = arg;
+    }
+
+    @Override
+    public boolean validate() {
+      if (alias == null) {
+        LOG.error("-alias flag is not optional for remove or cancel");
+        return false;
+      }
+      return true;
+    }
+
+    @Override
+    public void execute() throws Exception {
+      for (File tokenFile : tokenFiles) {
+        DtFileOperations.removeTokenFromFile(
+            cancel, tokenFile, format, alias, getConf());
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      if (cancel) {
+        return CANCEL_USAGE;
+      }
+      return REMOVE_USAGE;
+    }
+  }
+
+  private class Renew extends SubCommand {
+    public static final String RENEW_USAGE =
+        "dtutil renew -alias <alias> filename...";
+
+    @Override
+    public boolean validate() {
+      if (alias == null) {
+        LOG.error("-alias flag is not optional for renew");
+        return false;
+      }
+      return true;
+    }
+
+    @Override
+    public void execute() throws Exception {
+      for (File tokenFile : tokenFiles) {
+        DtFileOperations.renewTokenFile(tokenFile, format, alias, getConf());
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return RENEW_USAGE;
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    System.exit(ToolRunner.run(new Configuration(), new DtUtilShell(), args));
+  }
+}

+ 63 - 31
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.security.token;
 
 import com.google.common.collect.Maps;
+import com.google.protobuf.ByteString;
 import com.google.common.primitives.Bytes;
 
 import org.apache.commons.codec.binary.Base64;
@@ -28,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.*;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.util.ReflectionUtils;
 
 import java.io.*;
@@ -43,15 +45,15 @@ import java.util.UUID;
 @InterfaceStability.Evolving
 public class Token<T extends TokenIdentifier> implements Writable {
   public static final Log LOG = LogFactory.getLog(Token.class);
-  
+
   private static Map<Text, Class<? extends TokenIdentifier>> tokenKindMap;
-  
+
   private byte[] identifier;
   private byte[] password;
   private Text kind;
   private Text service;
   private TokenRenewer renewer;
-  
+
   /**
    * Construct a token given a token identifier and a secret manager for the
    * type of the token identifier.
@@ -64,7 +66,7 @@ public class Token<T extends TokenIdentifier> implements Writable {
     kind = id.getKind();
     service = new Text();
   }
- 
+
   /**
    * Construct a token from the components.
    * @param identifier the token identifier
@@ -80,7 +82,7 @@ public class Token<T extends TokenIdentifier> implements Writable {
   }
 
   /**
-   * Default constructor
+   * Default constructor.
    */
   public Token() {
     identifier = new byte[0];
@@ -100,14 +102,44 @@ public class Token<T extends TokenIdentifier> implements Writable {
     this.service = other.service;
   }
 
+  public Token<T> copyToken() {
+    return new Token<T>(this);
+  }
+
   /**
-   * Get the token identifier's byte representation
+   * Construct a Token from a TokenProto.
+   * @param tokenPB the TokenProto object
+   */
+  public Token(TokenProto tokenPB) {
+    this.identifier = tokenPB.getIdentifier().toByteArray();
+    this.password = tokenPB.getPassword().toByteArray();
+    this.kind = new Text(tokenPB.getKindBytes().toByteArray());
+    this.service = new Text(tokenPB.getServiceBytes().toByteArray());
+  }
+
+  /**
+   * Construct a TokenProto from this Token instance.
+   * @return a new TokenProto object holding copies of data in this instance
+   */
+  public TokenProto toTokenProto() {
+    return TokenProto.newBuilder().
+        setIdentifier(ByteString.copyFrom(this.getIdentifier())).
+        setPassword(ByteString.copyFrom(this.getPassword())).
+        setKindBytes(ByteString.copyFrom(
+            this.getKind().getBytes(), 0, this.getKind().getLength())).
+        setServiceBytes(ByteString.copyFrom(
+            this.getService().getBytes(), 0, this.getService().getLength())).
+        build();
+  }
+
+  /**
+   * Get the token identifier's byte representation.
    * @return the token identifier's byte representation
    */
   public byte[] getIdentifier() {
     return identifier;
   }
-  
+
   private static Class<? extends TokenIdentifier>
       getClassForIdentifier(Text kind) {
     Class<? extends TokenIdentifier> cls = null;
@@ -126,12 +158,12 @@ public class Token<T extends TokenIdentifier> implements Writable {
     }
     return cls;
   }
-  
+
   /**
    * Get the token identifier object, or null if it could not be constructed
    * (because the class could not be loaded, for example).
    * @return the token identifier, or null
-   * @throws IOException 
+   * @throws IOException
    */
   @SuppressWarnings("unchecked")
   public T decodeIdentifier() throws IOException {
@@ -141,22 +173,22 @@ public class Token<T extends TokenIdentifier> implements Writable {
     }
     TokenIdentifier tokenIdentifier = ReflectionUtils.newInstance(cls, null);
     ByteArrayInputStream buf = new ByteArrayInputStream(identifier);
-    DataInputStream in = new DataInputStream(buf);  
+    DataInputStream in = new DataInputStream(buf);
     tokenIdentifier.readFields(in);
     in.close();
     return (T) tokenIdentifier;
   }
-  
+
   /**
-   * Get the token password/secret
+   * Get the token password/secret.
    * @return the token password/secret
    */
   public byte[] getPassword() {
     return password;
   }
-  
+
   /**
-   * Get the token kind
+   * Get the token kind.
    * @return the kind of the token
    */
   public synchronized Text getKind() {
@@ -175,15 +207,15 @@ public class Token<T extends TokenIdentifier> implements Writable {
   }
 
   /**
-   * Get the service on which the token is supposed to be used
+   * Get the service on which the token is supposed to be used.
    * @return the service name
    */
   public Text getService() {
     return service;
   }
-  
+
   /**
-   * Set the service on which the token is supposed to be used
+   * Set the service on which the token is supposed to be used.
    * @param newService the service name
    */
   public void setService(Text newService) {
@@ -244,14 +276,14 @@ public class Token<T extends TokenIdentifier> implements Writable {
     System.arraycopy(buf.getData(), 0, raw, 0, buf.getLength());
     return encoder.encodeToString(raw);
   }
-  
+
   /**
-   * Modify the writable to the value from the newValue
+   * Modify the writable to the value from the newValue.
    * @param obj the object to read into
    * @param newValue the string with the url-safe base64 encoded bytes
    * @throws IOException
    */
-  private static void decodeWritable(Writable obj, 
+  private static void decodeWritable(Writable obj,
                                      String newValue) throws IOException {
     Base64 decoder = new Base64(0, null, true);
     DataInputBuffer buf = new DataInputBuffer();
@@ -261,14 +293,14 @@ public class Token<T extends TokenIdentifier> implements Writable {
   }
 
   /**
-   * Encode this token as a url safe string
+   * Encode this token as a url safe string.
    * @return the encoded string
    * @throws IOException
    */
   public String encodeToUrlString() throws IOException {
     return encodeWritable(this);
   }
-  
+
   /**
    * Decode the given url safe string into this token.
    * @param newValue the encoded string
@@ -277,7 +309,7 @@ public class Token<T extends TokenIdentifier> implements Writable {
   public void decodeFromUrlString(String newValue) throws IOException {
     decodeWritable(this, newValue);
   }
-  
+
   @SuppressWarnings("unchecked")
   @Override
   public boolean equals(Object right) {
@@ -293,12 +325,12 @@ public class Token<T extends TokenIdentifier> implements Writable {
              service.equals(r.service);
     }
   }
-  
+
   @Override
   public int hashCode() {
     return WritableComparator.hashBytes(identifier, identifier.length);
   }
-  
+
   private static void addBinaryBuffer(StringBuilder buffer, byte[] bytes) {
     for (int idx = 0; idx < bytes.length; idx++) {
       // if not the first, put a blank separator in
@@ -313,7 +345,7 @@ public class Token<T extends TokenIdentifier> implements Writable {
       buffer.append(num);
     }
   }
-  
+
   private void identifierToString(StringBuilder buffer) {
     T id = null;
     try {
@@ -375,7 +407,7 @@ public class Token<T extends TokenIdentifier> implements Writable {
   }
 
   /**
-   * Renew this delegation token
+   * Renew this delegation token.
    * @return the new expiration time
    * @throws IOException
    * @throws InterruptedException
@@ -384,9 +416,9 @@ public class Token<T extends TokenIdentifier> implements Writable {
                     ) throws IOException, InterruptedException {
     return getRenewer().renew(this, conf);
   }
-  
+
   /**
-   * Cancel this delegation token
+   * Cancel this delegation token.
    * @throws IOException
    * @throws InterruptedException
    */
@@ -394,7 +426,7 @@ public class Token<T extends TokenIdentifier> implements Writable {
                      ) throws IOException, InterruptedException {
     getRenewer().cancel(this, conf);
   }
-  
+
   /**
    * A trivial renewer for token kinds that aren't managed. Sub-classes need
    * to implement getKind for their token kind.
@@ -402,7 +434,7 @@ public class Token<T extends TokenIdentifier> implements Writable {
   @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
   @InterfaceStability.Evolving
   public static class TrivialRenewer extends TokenRenewer {
-    
+
     // define the kind for this renewer
     protected Text getKind() {
       return null;

+ 114 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/CommandShell.java

@@ -0,0 +1,114 @@
+/**
+ * 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.hadoop.tools;
+
+import java.io.PrintStream;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.Tool;
+
+/**
+ * This program is a CLI utility base class utilizing hadoop Tool class.
+ */
+public abstract class CommandShell extends Configured implements Tool {
+
+  private PrintStream out = System.out;
+  private PrintStream err = System.err;
+
+  /** The subcommand instance for this shell command, if any. */
+  private SubCommand subcommand = null;
+
+  /**
+   * Return usage string for the command including any summary of subcommands.
+   */
+  public abstract String getCommandUsage();
+
+  public void setSubCommand(SubCommand cmd) {
+    subcommand = cmd;
+  }
+
+  public void setOut(PrintStream p) {
+    out = p;
+  }
+
+  public PrintStream getOut() {
+    return out;
+  }
+
+  public void setErr(PrintStream p) {
+    err = p;
+  }
+
+  public PrintStream getErr() {
+    return err;
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    int exitCode = 0;
+    try {
+      exitCode = init(args);
+      if (exitCode != 0) {
+        printShellUsage();
+        return exitCode;
+      }
+      if (subcommand.validate()) {
+        subcommand.execute();
+      } else {
+        printShellUsage();
+        exitCode = 1;
+      }
+    } catch (Exception e) {
+      printShellUsage();
+      e.printStackTrace(err);
+      return 1;
+    }
+    return exitCode;
+  }
+
+  /**
+   * Parse the command line arguments and initialize subcommand instance.
+   * @param args
+   * @return 0 if the argument(s) were recognized, 1 otherwise
+   */
+  protected abstract int init(String[] args) throws Exception;
+
+  private void printShellUsage() {
+    if (subcommand != null) {
+      out.println(subcommand.getUsage());
+    } else {
+      out.println(getCommandUsage());
+    }
+    out.flush();
+  }
+
+  /**
+   * Base class for any subcommands of this shell command.
+   */
+  protected abstract class SubCommand {
+
+    public boolean validate() {
+      return true;
+    }
+
+    public abstract void execute() throws Exception;
+
+    public abstract String getUsage();
+  }
+}

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/proto/Security.proto

@@ -38,6 +38,17 @@ message TokenProto {
   required string service = 4;
 }
 
+message CredentialsKVProto {
+  required string alias = 1;
+  optional hadoop.common.TokenProto token = 2;
+  optional bytes secret = 3;
+}
+
+message CredentialsProto {
+  repeated hadoop.common.CredentialsKVProto tokens = 1;
+  repeated hadoop.common.CredentialsKVProto secrets = 2;
+}
+
 message GetDelegationTokenRequestProto {
   required string renewer = 1;
 }

+ 19 - 0
hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md

@@ -134,6 +134,25 @@ Change the ownership and permissions on many files at once.
 
 Copy file or directories recursively. More information can be found at [Hadoop DistCp Guide](../../hadoop-distcp/DistCp.html).
 
+### `dtutil`
+
+Usage: `hadoop dtutil [-keytab` *keytab_file* `-principal` *principal_name* `]` *subcommand* `[-format (java|protobuf)] [-alias` *alias* `] [-renewer` *renewer* `]` *filename...*
+
+Utility to fetch and manage hadoop delegation tokens inside credentials files.  It is intended to replace the simpler command `fetchdt`.  There are multiple subcommands, each with their own flags and options.
+
+For every subcommand that writes out a file, the `-format` option will specify the internal format to use.  `java` is the legacy format that matches `fetchdt`.  The default is `protobuf`.
+
+For every subcommand that connects to a service, convenience flags are provided to specify the kerberos principal name and keytab file to use for auth.
+
+| SUBCOMMAND | Description |
+|:---- |:---- |
+| `print` <br/>&nbsp;&nbsp; `[-alias` *alias* `]` <br/>&nbsp;&nbsp; *filename* `[` *filename2* `...]` | Print out the fields in the tokens contained in *filename* (and *filename2* ...). <br/> If *alias* is specified, print only tokens matching *alias*.  Otherwise, print all tokens. |
+| `get` *URL* <br/>&nbsp;&nbsp; `[-service` *scheme* `]` <br/>&nbsp;&nbsp; `[-format (java|protobuf)]` <br/>&nbsp;&nbsp; `[-alias` *alias* `]` <br/>&nbsp;&nbsp; `[-renewer` *renewer* `]` <br/>&nbsp;&nbsp; *filename* | Fetch a token from service at *URL* and place it in *filename*. <br/> *URL* is required and must immediately follow `get`.<br/> *URL* is the service URL, e.g. *hdfs:&#47;&#47;localhost:9000*. <br/> *alias* will overwrite the service field in the token. <br/> It is intended for hosts that have external and internal names, e.g. *firewall.com:14000*. <br/> *filename* should come last and is the name of the token file. <br/>  It will be created if it does not exist.  Otherwise, token(s) are added to existing file. <br/> The `-service` flag should only be used with a URL which starts with `http` or `https`. <br/> The following are equivalent: *hdfs:&#47;&#47;localhost:9000/* vs. *http:&#47;&#47;localhost:9000* `-service` *hdfs* |
+| `append` <br/>&nbsp;&nbsp; `[-format (java|protobuf)]` <br/>&nbsp;&nbsp; *filename* *filename2* `[` *filename3* `...]` | Append the contents of the first N filenames onto the last filename. <br/>  When tokens with common service fields are present in multiple files, earlier files' tokens are overwritten. <br/>  That is, tokens present in the last file are always preserved. |
+| `remove -alias` *alias* <br/>&nbsp;&nbsp; `[-format (java|protobuf)]` <br/>&nbsp;&nbsp; *filename* `[` *filename2* `...]` | From each file specified, remove the tokens matching *alias* and write out each file using specified format. <br/>  *alias* must be specified. |
+| `cancel -alias` *alias* <br/>&nbsp;&nbsp; `[-format (java|protobuf)]` <br/>&nbsp;&nbsp;  *filename* `[` *filename2* `...]` | Just like `remove`, except the tokens are also cancelled using the service specified in the token object. <br/> *alias* must be specified. |
+| `renew -alias` *alias* <br/>&nbsp;&nbsp; `[-format (java|protobuf)]` <br/>&nbsp;&nbsp;  *filename* `[` *filename2* `...]` | For each file specified, renew the tokens matching *alias* and write out each file using specified format. <br/> *alias* must be specified. |
+
 ### `fs`
 
 This command is documented in the [File System Shell Guide](./FileSystemShell.html). It is a synonym for `hdfs dfs` when HDFS is in use.

+ 41 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtFetcher.java

@@ -0,0 +1,41 @@
+/**
+ * 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.hadoop.security.token;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.DtFetcher;
+
+public class TestDtFetcher implements DtFetcher {
+  public Text getServiceName() {
+    return TestDtUtilShell.SERVICE_GET;
+  }
+
+  public boolean isTokenRequired() {
+    return true;
+  }
+
+  public Token<?> addDelegationTokens(Configuration conf,
+      Credentials creds, String renewer, String url) throws Exception {
+    creds.addToken(TestDtUtilShell.MOCK_TOKEN.getService(),
+                   TestDtUtilShell.MOCK_TOKEN);
+    return TestDtUtilShell.MOCK_TOKEN;
+  }
+}

+ 264 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/TestDtUtilShell.java

@@ -0,0 +1,264 @@
+/**
+ * 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.hadoop.security.token;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.FileInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.DtFetcher;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.mockito.Mockito;
+
+public class TestDtUtilShell {
+  private static byte[] IDENTIFIER = {
+      0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72};
+  private static byte[] PASSWORD = {
+      0x70, 0x61, 0x73, 0x73, 0x77, 0x6f, 0x72, 0x64};
+  private static Text KIND = new Text("testTokenKind");
+  private static Text SERVICE = new Text("testTokenService");
+  private static Text SERVICE2 = new Text("ecivreSnekoTtset");
+  private static Configuration defaultConf = new Configuration();
+  private static FileSystem localFs = null;
+  private final String alias = "proxy_ip:1234";
+  private final String renewer = "yarn";
+  private final String getUrl = SERVICE_GET.toString() + "://localhost:9000/";
+  private final String getUrl2 = "http://localhost:9000/";
+  public static Text SERVICE_GET = new Text("testTokenServiceGet");
+  public static Text KIND_GET = new Text("testTokenKindGet");
+  public static Token<?> MOCK_TOKEN =
+      new Token(IDENTIFIER, PASSWORD, KIND_GET, SERVICE_GET);
+  static {
+    try {
+      defaultConf.set("fs.defaultFS", "file:///");
+      localFs = FileSystem.getLocal(defaultConf);
+    } catch (IOException e) {
+      throw new RuntimeException("init failure", e);
+    }
+  }
+
+  private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private final Path workDir = new Path(
+             System.getProperty("test.build.data", "/tmp"), "TestDtUtilShell");
+  private final Path tokenFile = new Path(workDir, "testPrintTokenFile");
+  private final Path tokenFile2 = new Path(workDir, "testPrintTokenFile2");
+  private final Path tokenLegacyFile = new Path(workDir, "testPrintTokenFile3");
+  private final Path tokenFileGet = new Path(workDir, "testGetTokenFile");
+  private final String tokenFilename = tokenFile.toString();
+  private final String tokenFilename2 = tokenFile2.toString();
+  private final String tokenFilenameGet = tokenFileGet.toString();
+  private String[] args = null;
+  private DtUtilShell dt = null;
+  private int rc = 0;
+
+  @Before
+  public void setup() throws Exception {
+    localFs.mkdirs(localFs.makeQualified(workDir));
+    makeTokenFile(tokenFile, false, null);
+    makeTokenFile(tokenFile2, false, SERVICE2);
+    makeTokenFile(tokenLegacyFile, true, null);
+    dt = new DtUtilShell();
+    dt.setConf(new Configuration());
+    dt.setOut(new PrintStream(outContent));
+    outContent.reset();
+    rc = 0;
+  }
+
+  @After
+  public void teardown() throws Exception {
+    localFs.delete(localFs.makeQualified(workDir), true);
+  }
+
+  public void makeTokenFile(Path tokenPath, boolean legacy, Text service)
+        throws IOException {
+    if (service == null) {
+      service = SERVICE;
+    }
+    Credentials creds = new Credentials();
+    Token<? extends TokenIdentifier> tok = (Token<? extends TokenIdentifier>)
+        new Token(IDENTIFIER, PASSWORD, KIND, service);
+    creds.addToken(tok.getService(), tok);
+    if (legacy) {
+      creds.writeLegacyTokenStorageLocalFile(new File(tokenPath.toString()));
+    } else {
+      creds.writeTokenStorageFile(tokenPath, defaultConf);
+    }
+  }
+
+  @Test
+  public void testPrint() throws Exception {
+    args = new String[] {"print", tokenFilename};
+    rc = dt.run(args);
+    assertEquals("test simple print exit code", 0, rc);
+    assertTrue("test simple print output kind:\n" + outContent.toString(),
+               outContent.toString().contains(KIND.toString()));
+    assertTrue("test simple print output service:\n" + outContent.toString(),
+               outContent.toString().contains(SERVICE.toString()));
+
+    outContent.reset();
+    args = new String[] {"print", tokenLegacyFile.toString()};
+    rc = dt.run(args);
+    assertEquals("test legacy print exit code", 0, rc);
+    assertTrue("test simple print output kind:\n" + outContent.toString(),
+               outContent.toString().contains(KIND.toString()));
+    assertTrue("test simple print output service:\n" + outContent.toString(),
+               outContent.toString().contains(SERVICE.toString()));
+
+    outContent.reset();
+    args = new String[] {
+        "print", "-alias", SERVICE.toString(), tokenFilename};
+    rc = dt.run(args);
+    assertEquals("test alias print exit code", 0, rc);
+    assertTrue("test simple print output kind:\n" + outContent.toString(),
+               outContent.toString().contains(KIND.toString()));
+    assertTrue("test simple print output service:\n" + outContent.toString(),
+               outContent.toString().contains(SERVICE.toString()));
+
+    outContent.reset();
+    args = new String[] {
+        "print", "-alias", "not-a-serivce", tokenFilename};
+    rc = dt.run(args);
+    assertEquals("test no alias print exit code", 0, rc);
+    assertFalse("test no alias print output kind:\n" + outContent.toString(),
+                outContent.toString().contains(KIND.toString()));
+    assertFalse("test no alias print output service:\n" + outContent.toString(),
+                outContent.toString().contains(SERVICE.toString()));
+  }
+
+  @Test
+  public void testAppend() throws Exception {
+    args = new String[] {"append", tokenFilename, tokenFilename2};
+    rc = dt.run(args);
+    assertEquals("test simple append exit code", 0, rc);
+    args = new String[] {"print", tokenFilename2};
+    rc = dt.run(args);
+    assertEquals("test simple append print exit code", 0, rc);
+    assertTrue("test simple append output kind:\n" + outContent.toString(),
+               outContent.toString().contains(KIND.toString()));
+    assertTrue("test simple append output service:\n" + outContent.toString(),
+               outContent.toString().contains(SERVICE.toString()));
+    assertTrue("test simple append output service:\n" + outContent.toString(),
+               outContent.toString().contains(SERVICE2.toString()));
+  }
+
+  @Test
+  public void testRemove() throws Exception {
+    args = new String[] {"remove", "-alias", SERVICE.toString(), tokenFilename};
+    rc = dt.run(args);
+    assertEquals("test simple remove exit code", 0, rc);
+    args = new String[] {"print", tokenFilename};
+    rc = dt.run(args);
+    assertEquals("test simple remove print exit code", 0, rc);
+    assertFalse("test simple remove output kind:\n" + outContent.toString(),
+                outContent.toString().contains(KIND.toString()));
+    assertFalse("test simple remove output service:\n" + outContent.toString(),
+                outContent.toString().contains(SERVICE.toString()));
+  }
+
+  @Test
+  public void testGet() throws Exception {
+    args = new String[] {"get", getUrl, tokenFilenameGet};
+    rc = dt.run(args);
+    assertEquals("test mocked get exit code", 0, rc);
+    args = new String[] {"print", tokenFilenameGet};
+    rc = dt.run(args);
+    String oc = outContent.toString();
+    assertEquals("test print after get exit code", 0, rc);
+    assertTrue("test print after get output kind:\n" + oc,
+               oc.contains(KIND_GET.toString()));
+    assertTrue("test print after get output service:\n" + oc,
+               oc.contains(SERVICE_GET.toString()));
+  }
+
+  @Test
+  public void testGetWithServiceFlag() throws Exception {
+    args = new String[] {"get", getUrl2, "-service", SERVICE_GET.toString(),
+                         tokenFilenameGet};
+    rc = dt.run(args);
+    assertEquals("test mocked get with service flag exit code", 0, rc);
+    args = new String[] {"print", tokenFilenameGet};
+    rc = dt.run(args);
+    String oc = outContent.toString();
+    assertEquals("test print after get with service flag exit code", 0, rc);
+    assertTrue("test print after get with service flag output kind:\n" + oc,
+               oc.contains(KIND_GET.toString()));
+    assertTrue("test print after get with service flag output service:\n" + oc,
+               oc.contains(SERVICE_GET.toString()));
+  }
+
+  @Test
+  public void testGetWithAliasFlag() throws Exception {
+    args = new String[] {"get", getUrl, "-alias", alias, tokenFilenameGet};
+    rc = dt.run(args);
+    assertEquals("test mocked get with alias flag exit code", 0, rc);
+    args = new String[] {"print", tokenFilenameGet};
+    rc = dt.run(args);
+    String oc = outContent.toString();
+    assertEquals("test print after get with alias flag exit code", 0, rc);
+    assertTrue("test print after get with alias flag output kind:\n" + oc,
+               oc.contains(KIND_GET.toString()));
+    assertTrue("test print after get with alias flag output alias:\n" + oc,
+               oc.contains(alias));
+    assertFalse("test print after get with alias flag output old service:\n" +
+                oc, oc.contains(SERVICE_GET.toString()));
+  }
+
+  @Test
+  public void testFormatJavaFlag() throws Exception {
+    args = new String[] {"get", getUrl, "-format", "java", tokenFilenameGet};
+    rc = dt.run(args);
+    assertEquals("test mocked get with java format flag exit code", 0, rc);
+    Credentials creds = new Credentials();
+    Credentials spyCreds = Mockito.spy(creds);
+    DataInputStream in = new DataInputStream(
+        new FileInputStream(tokenFilenameGet));
+    spyCreds.readTokenStorageStream(in);
+    Mockito.verify(spyCreds).readFields(in);
+  }
+
+  @Test
+  public void testFormatProtoFlag() throws Exception {
+    args = new String[] {
+        "get", getUrl, "-format", "protobuf", tokenFilenameGet};
+    rc = dt.run(args);
+    assertEquals("test mocked get with protobuf format flag exit code", 0, rc);
+    Credentials creds = new Credentials();
+    Credentials spyCreds = Mockito.spy(creds);
+    DataInputStream in = new DataInputStream(
+        new FileInputStream(tokenFilenameGet));
+    spyCreds.readTokenStorageStream(in);
+    Mockito.verify(spyCreds).readProtos(in);
+  }
+}

+ 128 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java

@@ -0,0 +1,128 @@
+/**
+ * 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.hadoop.tools;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.CommandShell;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCommandShell {
+
+  public class Example extends CommandShell {
+    public static final String EXAMPLE = "example";
+    public static final String HELLO = "hello";
+    public static final String HELLO_MSG = "hello is running";
+    public static final String GOODBYE = "goodbye";
+    public static final String GOODBYE_MSG = "goodbye is running";
+
+    public String[] savedArgs = null;
+
+    @Override
+    protected int init(String[] args) throws Exception {
+      String command = args[0];
+      if (command.equals(HELLO)) {
+        setSubCommand(new Hello());
+      } else if (command.equals(GOODBYE)) {
+        setSubCommand(new Goodbye());
+      } else{
+        return 1;
+      }
+      savedArgs = args;
+      return 0;
+    }
+
+    public String getCommandUsage() {
+      return EXAMPLE;
+    }
+
+    public class Hello extends SubCommand {
+      public static final String HELLO_USAGE = EXAMPLE + " hello";
+      @Override
+      public boolean validate() {
+        return savedArgs.length == 1;
+      }
+      @Override
+      public void execute() throws Exception {
+        System.out.println(HELLO_MSG);
+      }
+      @Override
+      public String getUsage() {
+        return HELLO_USAGE;
+      }
+    }
+
+    public class Goodbye extends SubCommand {
+      public static final String GOODBYE_USAGE = EXAMPLE + " goodbye";
+      @Override
+      public void execute() throws Exception {
+        System.out.println(GOODBYE_MSG);
+      }
+      @Override
+      public String getUsage() {
+        return GOODBYE_USAGE;
+      }
+    }
+  }
+
+  private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+
+  private String outMsg(String message) {
+    return "OUT:\n" + outContent.toString() + "\n" + message;
+  }
+
+  @Before
+  public void setup() throws Exception {
+    System.setOut(new PrintStream(outContent));
+  }
+
+  @Test
+  public void testCommandShellExample() throws Exception {
+    Example ex = new Example();
+    ex.setConf(new Configuration());
+    int rc = 0;
+
+    outContent.reset();
+    String[] args1 = {"hello"};
+    rc = ex.run(args1);
+    assertEquals(outMsg("test exit code - normal hello"), 0, rc);
+    assertTrue(outMsg("test normal hello message"),
+               outContent.toString().contains(Example.HELLO_MSG));
+
+    outContent.reset();
+    String[] args2 = {"hello", "x"};
+    rc = ex.run(args2);
+    assertEquals(outMsg("test exit code - bad hello"), 1, rc);
+    assertTrue(outMsg("test bad hello message"),
+               outContent.toString().contains(Example.Hello.HELLO_USAGE));
+
+    outContent.reset();
+    String[] args3 = {"goodbye"};
+    rc = ex.run(args3);
+    assertEquals(outMsg("test exit code - normal goodbye"), 0, rc);
+    assertTrue(outMsg("test normal goodbye message"),
+               outContent.toString().contains(Example.GOODBYE_MSG));
+  }
+}

+ 14 - 0
hadoop-common-project/hadoop-common/src/test/resources/META-INF/services/org.apache.hadoop.security.token.DtFetcher

@@ -0,0 +1,14 @@
+#
+#   Licensed 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.
+#
+org.apache.hadoop.security.token.TestDtFetcher

+ 82 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsDtFetcher.java

@@ -0,0 +1,82 @@
+/**
+ * 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.hadoop.hdfs;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.DtFetcher;
+import org.apache.hadoop.security.token.Token;
+
+
+/**
+ *  DtFetcher is an interface which permits the abstraction and separation of
+ *  delegation token fetch implementaions across different packages and
+ *  compilation units.  Resolution of fetcher impl will be done at runtime.
+ */
+public class HdfsDtFetcher implements DtFetcher {
+  private static final Log LOG = LogFactory.getLog(HdfsDtFetcher.class);
+
+  private static final String SERVICE_NAME = HdfsConstants.HDFS_URI_SCHEME;
+
+  private static final String FETCH_FAILED = "Fetch of delegation token failed";
+
+  /**
+   * Returns the service name for HDFS, which is also a valid URL prefix.
+   */
+  public Text getServiceName() {
+    return new Text(SERVICE_NAME);
+  }
+
+  public boolean isTokenRequired() {
+    return UserGroupInformation.isSecurityEnabled();
+  }
+
+  /**
+   *  Returns Token object via FileSystem, null if bad argument.
+   *  @param conf - a Configuration object used with FileSystem.get()
+   *  @param creds - a Credentials object to which token(s) will be added
+   *  @param renewer  - the renewer to send with the token request
+   *  @param url  - the URL to which the request is sent
+   *  @return a Token, or null if fetch fails.
+   */
+  public Token<?> addDelegationTokens(Configuration conf, Credentials creds,
+                                  String renewer, String url) throws Exception {
+    if (!url.startsWith(getServiceName().toString())) {
+      url = getServiceName().toString() + "://" + url;
+    }
+    FileSystem fs = FileSystem.get(URI.create(url), conf);
+    Token<?> token = fs.getDelegationToken(renewer);
+    if (token == null) {
+      LOG.error(FETCH_FAILED);
+      throw new IOException(FETCH_FAILED);
+    }
+    creds.addToken(token.getService(), token);
+    return token;
+  }
+}

+ 39 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/SWebHdfsDtFetcher.java

@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hdfs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.io.Text;
+
+/**
+ *  DtFetcher for SWebHdfsFileSystem using the base class HdfsDtFetcher impl.
+ */
+public class SWebHdfsDtFetcher extends HdfsDtFetcher {
+  private static final Log LOG = LogFactory.getLog(SWebHdfsDtFetcher.class);
+
+  private static final String SERVICE_NAME = WebHdfsConstants.SWEBHDFS_SCHEME;
+
+  @Override
+  public Text getServiceName() {
+    return new Text(SERVICE_NAME);
+  }
+}

+ 39 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/WebHdfsDtFetcher.java

@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hdfs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
+import org.apache.hadoop.io.Text;
+
+/**
+ *  DtFetcher for WebHdfsFileSystem using the base class HdfsDtFetcher impl.
+ */
+public class WebHdfsDtFetcher extends HdfsDtFetcher {
+  private static final Log LOG = LogFactory.getLog(WebHdfsDtFetcher.class);
+
+  private static final String SERVICE_NAME = WebHdfsConstants.WEBHDFS_SCHEME;
+
+  @Override
+  public Text getServiceName() {
+    return new Text(SERVICE_NAME);
+  }
+}

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -179,7 +179,8 @@ public class DelegationTokenFetcher {
     if (null != token) {
       Credentials cred = new Credentials();
       cred.addToken(token.getService(), token);
-      cred.writeTokenStorageFile(tokenFile, conf);
+      // dtutil is replacing this tool; preserve legacy functionality
+      cred.writeLegacyTokenStorageFile(tokenFile, conf);
 
       if (LOG.isDebugEnabled()) {
         LOG.debug("Fetched token " + fs.getUri() + " for " +

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.DtFetcher

@@ -0,0 +1,18 @@
+# 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.
+
+org.apache.hadoop.hdfs.HdfsDtFetcher
+org.apache.hadoop.hdfs.WebHdfsDtFetcher
+org.apache.hadoop.hdfs.SWebHdfsDtFetcher