Browse Source

HADOOP-10607. Create API to separate credential/password storage from
applications. (Larry McCay via omalley)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1603491 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 11 years ago
parent
commit
c79728478c
18 changed files with 1656 additions and 34 deletions
  1. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 2 0
      hadoop-common-project/hadoop-common/src/main/bin/hadoop
  3. 76 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  4. 3 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
  5. 0 27
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
  6. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  7. 52 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ProviderUtils.java
  8. 124 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProvider.java
  9. 76 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProviderFactory.java
  10. 423 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
  11. 290 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java
  12. 123 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java
  13. 17 0
      hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.alias.CredentialProviderFactory
  14. 5 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProvider.java
  15. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
  16. 173 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java
  17. 51 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProvider.java
  18. 234 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -149,6 +149,9 @@ Trunk (Unreleased)
 
 
     HADOOP-10485. Remove dead classes in hadoop-streaming. (wheat9)
     HADOOP-10485. Remove dead classes in hadoop-streaming. (wheat9)
 
 
+    HADOOP-10607. Create API to separate credential/password storage from
+    applications. (Larry McCay via omalley)
+
   BUG FIXES
   BUG FIXES
 
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
     HADOOP-9451. Fault single-layer config if node group topology is enabled.

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

@@ -116,6 +116,8 @@ case $COMMAND in
     elif [ "$COMMAND" = "archive" ] ; then
     elif [ "$COMMAND" = "archive" ] ; then
       CLASS=org.apache.hadoop.tools.HadoopArchives
       CLASS=org.apache.hadoop.tools.HadoopArchives
       CLASSPATH=${CLASSPATH}:${TOOL_PATH}
       CLASSPATH=${CLASSPATH}:${TOOL_PATH}
+    elif [ "$COMMAND" = "credential" ] ; then
+      CLASS=org.apache.hadoop.security.alias.CredentialShell
     elif [[ "$COMMAND" = -*  ]] ; then
     elif [[ "$COMMAND" = -*  ]] ; then
         # class and package names cannot begin with a -
         # class and package names cannot begin with a -
         echo "Error: No command named \`$COMMAND' was found. Perhaps you meant \`hadoop ${COMMAND#-}'"
         echo "Error: No command named \`$COMMAND' was found. Perhaps you meant \`hadoop ${COMMAND#-}'"

+ 76 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -78,6 +78,9 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProvider.CredentialEntry;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -1767,6 +1770,79 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     set(name, StringUtils.arrayToString(values));
     set(name, StringUtils.arrayToString(values));
   }
   }
 
 
+  /**
+   * Get the value for a known password configuration element.
+   * In order to enable the elimination of clear text passwords in config,
+   * this method attempts to resolve the property name as an alias through
+   * the CredentialProvider API and conditionally fallsback to config.
+   * @param name property name
+   * @return password
+   */
+  public char[] getPassword(String name) throws IOException {
+    char[] pass = null;
+
+    pass = getPasswordFromCredenitalProviders(name);
+
+    if (pass == null) {
+      pass = getPasswordFromConfig(name);
+    }
+
+    return pass;
+  }
+
+  /**
+   * Try and resolve the provided element name as a credential provider
+   * alias.
+   * @param name alias of the provisioned credential
+   * @return password or null if not found
+   * @throws IOException
+   */
+  protected char[] getPasswordFromCredenitalProviders(String name)
+      throws IOException {
+    char[] pass = null;
+    try {
+      List<CredentialProvider> providers =
+          CredentialProviderFactory.getProviders(this);
+
+      if (providers != null) {
+        for (CredentialProvider provider : providers) {
+          try {
+            CredentialEntry entry = provider.getCredentialEntry(name);
+            if (entry != null) {
+              pass = entry.getCredential();
+              break;
+            }
+          }
+          catch (IOException ioe) {
+            throw new IOException("Can't get key " + name + " from key provider" +
+            		"of type: " + provider.getClass().getName() + ".", ioe);
+          }
+        }
+      }
+    }
+    catch (IOException ioe) {
+      throw new IOException("Configuration problem with provider path.", ioe);
+    }
+
+    return pass;
+  }
+
+  /**
+   * Fallback to clear text passwords in configuration.
+   * @param name
+   * @return clear text password or null
+   */
+  protected char[] getPasswordFromConfig(String name) {
+    char[] pass = null;
+    if (getBoolean(CredentialProvider.CLEAR_TEXT_FALLBACK, true)) {
+      String passStr = get(name);
+      if (passStr != null) {
+        pass = passStr.toCharArray();
+      }
+    }
+    return pass;
+  }
+
   /**
   /**
    * Get the socket address for <code>name</code> property as a
    * Get the socket address for <code>name</code> property as a
    * <code>InetSocketAddress</code>.
    * <code>InetSocketAddress</code>.

+ 3 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.ProviderUtils;
+
 import javax.crypto.spec.SecretKeySpec;
 import javax.crypto.spec.SecretKeySpec;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
@@ -101,7 +103,7 @@ public class JavaKeyStoreProvider extends KeyProvider {
 
 
   private JavaKeyStoreProvider(URI uri, Configuration conf) throws IOException {
   private JavaKeyStoreProvider(URI uri, Configuration conf) throws IOException {
     this.uri = uri;
     this.uri = uri;
-    path = unnestUri(uri);
+    path = ProviderUtils.unnestUri(uri);
     fs = path.getFileSystem(conf);
     fs = path.getFileSystem(conf);
     // Get the password file from the conf, if not present from the user's
     // Get the password file from the conf, if not present from the user's
     // environment var
     // environment var

+ 0 - 27
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java

@@ -488,33 +488,6 @@ public abstract class KeyProvider {
     return name + "@" + version;
     return name + "@" + version;
   }
   }
 
 
-  /**
-   * Convert a nested URI to decode the underlying path. The translation takes
-   * the authority and parses it into the underlying scheme and authority.
-   * For example, "myscheme://hdfs@nn/my/path" is converted to
-   * "hdfs://nn/my/path".
-   * @param nestedUri the URI from the nested URI
-   * @return the unnested path
-   */
-  public static Path unnestUri(URI nestedUri) {
-    String[] parts = nestedUri.getAuthority().split("@", 2);
-    StringBuilder result = new StringBuilder(parts[0]);
-    result.append("://");
-    if (parts.length == 2) {
-      result.append(parts[1]);
-    }
-    result.append(nestedUri.getPath());
-    if (nestedUri.getQuery() != null) {
-      result.append("?");
-      result.append(nestedUri.getQuery());
-    }
-    if (nestedUri.getFragment() != null) {
-      result.append("#");
-      result.append(nestedUri.getFragment());
-    }
-    return new Path(result.toString());
-  }
-
   /**
   /**
    * Find the provider with the given key.
    * Find the provider with the given key.
    * @param providerList the list of providers
    * @param providerList the list of providers

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
 import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
@@ -147,7 +148,7 @@ public class KMSClientProvider extends KeyProvider {
   }
   }
 
 
   public KMSClientProvider(URI uri, Configuration conf) throws IOException {
   public KMSClientProvider(URI uri, Configuration conf) throws IOException {
-    Path path = unnestUri(uri);
+    Path path = ProviderUtils.unnestUri(uri);
     URL url = path.toUri().toURL();
     URL url = path.toUri().toURL();
     kmsUrl = createServiceURL(url);
     kmsUrl = createServiceURL(url);
     if ("https".equalsIgnoreCase(url.getProtocol())) {
     if ("https".equalsIgnoreCase(url.getProtocol())) {

+ 52 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ProviderUtils.java

@@ -0,0 +1,52 @@
+/**
+ * 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;
+
+import java.net.URI;
+
+import org.apache.hadoop.fs.Path;
+
+public class ProviderUtils {
+  /**
+   * Convert a nested URI to decode the underlying path. The translation takes
+   * the authority and parses it into the underlying scheme and authority.
+   * For example, "myscheme://hdfs@nn/my/path" is converted to
+   * "hdfs://nn/my/path".
+   * @param nestedUri the URI from the nested URI
+   * @return the unnested path
+   */
+  public static Path unnestUri(URI nestedUri) {
+    String[] parts = nestedUri.getAuthority().split("@", 2);
+    StringBuilder result = new StringBuilder(parts[0]);
+    result.append("://");
+    if (parts.length == 2) {
+      result.append(parts[1]);
+    }
+    result.append(nestedUri.getPath());
+    if (nestedUri.getQuery() != null) {
+      result.append("?");
+      result.append(nestedUri.getQuery());
+    }
+    if (nestedUri.getFragment() != null) {
+      result.append("#");
+      result.append(nestedUri.getFragment());
+    }
+    return new Path(result.toString());
+  }
+}

+ 124 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProvider.java

@@ -0,0 +1,124 @@
+/**
+ * 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.alias;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A provider of credentials or password for Hadoop applications. Provides an
+ * abstraction to separate credential storage from users of them. It
+ * is intended to support getting or storing passwords in a variety of ways,
+ * including third party bindings.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class CredentialProvider {
+  public static final String CLEAR_TEXT_FALLBACK 
+    = "hadoop.security.credential.clear-text-fallback";
+
+  /**
+   * The combination of both the alias and the actual credential value.
+   */
+  public static class CredentialEntry {
+    private final String alias;
+    private final char[] credential;
+
+    protected CredentialEntry(String alias,
+                         char[] credential) {
+      this.alias = alias;
+      this.credential = credential;
+    }
+
+    public String getAlias() {
+      return alias;
+    }
+
+    public char[] getCredential() {
+      return credential;
+    }
+
+    public String toString() {
+      StringBuilder buf = new StringBuilder();
+      buf.append("alias(");
+      buf.append(alias);
+      buf.append(")=");
+      if (credential == null) {
+        buf.append("null");
+      } else {
+        for(char c: credential) {
+          buf.append(c);
+        }
+      }
+      return buf.toString();
+    }
+  }
+
+  /**
+   * Indicates whether this provider represents a store
+   * that is intended for transient use - such as the UserProvider
+   * is. These providers are generally used to provide job access to
+   * passwords rather than for long term storage.
+   * @return true if transient, false otherwise
+   */
+  public boolean isTransient() {
+    return false;
+  }
+
+  /**
+   * Ensures that any changes to the credentials are written to persistent store.
+   * @throws IOException
+   */
+  public abstract void flush() throws IOException;
+
+  /**
+   * Get the credential entry for a specific alias.
+   * @param alias the name of a specific credential
+   * @return the credentialEntry
+   * @throws IOException
+   */
+  public abstract CredentialEntry getCredentialEntry(String alias) 
+      throws IOException;
+
+  /**
+   * Get the aliases for all credentials.
+   * @return the list of alias names
+   * @throws IOException
+   */
+  public abstract List<String> getAliases() throws IOException;
+
+  /**
+   * Create a new credential. The given alias must not already exist.
+   * @param name the alias of the credential
+   * @param credential the credential value for the alias.
+   * @throws IOException
+   */
+  public abstract CredentialEntry createCredentialEntry(String name, 
+      char[] credential) throws IOException;
+
+  /**
+   * Delete the given credential.
+   * @param name the alias of the credential to delete
+   * @throws IOException
+   */
+  public abstract void deleteCredentialEntry(String name) throws IOException;
+}

+ 76 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProviderFactory.java

@@ -0,0 +1,76 @@
+/**
+ * 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.alias;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ServiceLoader;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A factory to create a list of CredentialProvider based on the path given in a
+ * Configuration. It uses a service loader interface to find the available
+ * CredentialProviders and create them based on the list of URIs.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class CredentialProviderFactory {
+  public static final String CREDENTIAL_PROVIDER_PATH =
+      "hadoop.security.credential.provider.path";
+
+  public abstract CredentialProvider createProvider(URI providerName,
+                                             Configuration conf
+                                             ) throws IOException;
+
+  private static final ServiceLoader<CredentialProviderFactory> serviceLoader =
+      ServiceLoader.load(CredentialProviderFactory.class);
+
+  public static List<CredentialProvider> getProviders(Configuration conf
+                                               ) throws IOException {
+    List<CredentialProvider> result = new ArrayList<CredentialProvider>();
+    for(String path: conf.getStringCollection(CREDENTIAL_PROVIDER_PATH)) {
+      try {
+        URI uri = new URI(path);
+        boolean found = false;
+        for(CredentialProviderFactory factory: serviceLoader) {
+          CredentialProvider kp = factory.createProvider(uri, conf);
+          if (kp != null) {
+            result.add(kp);
+            found = true;
+            break;
+          }
+        }
+        if (!found) {
+          throw new IOException("No CredentialProviderFactory for " + uri + " in " +
+              CREDENTIAL_PROVIDER_PATH);
+        }
+      } catch (URISyntaxException error) {
+        throw new IOException("Bad configuration of " + CREDENTIAL_PROVIDER_PATH +
+            " at " + path, error);
+      }
+    }
+    return result;
+  }
+}

+ 423 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java

@@ -0,0 +1,423 @@
+/**
+ * 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.alias;
+
+import java.io.Console;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.security.InvalidParameterException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * This program is the CLI utility for the CredentialProvider facilities in 
+ * Hadoop.
+ */
+public class CredentialShell extends Configured implements Tool {
+  final static private String USAGE_PREFIX = "Usage: hadoop credential " +
+  		"[generic options]\n";
+  final static private String COMMANDS =
+      "   [--help]\n" +
+      "   [" + CreateCommand.USAGE + "]\n" +
+      "   [" + DeleteCommand.USAGE + "]\n" +
+      "   [" + ListCommand.USAGE + "]\n";
+
+  private boolean interactive = false;
+  private Command command = null;
+
+  /** allows stdout to be captured if necessary */
+  public PrintStream out = System.out;
+  /** allows stderr to be captured if necessary */
+  public PrintStream err = System.err;
+
+  private boolean userSuppliedProvider = false;
+  private String value = null;
+  private PasswordReader passwordReader;
+
+  @Override
+  public int run(String[] args) throws Exception {
+    int exitCode = 0;
+    try {
+      exitCode = init(args);
+      if (exitCode != 0) {
+        return exitCode;
+      }
+      if (command.validate()) {
+          command.execute();
+      } else {
+        exitCode = -1;
+      }
+    } catch (Exception e) {
+      e.printStackTrace(err);
+      return -1;
+    }
+    return exitCode;
+  }
+
+  /**
+   * Parse the command line arguments and initialize the data
+   * <pre>
+   * % hadoop alias create alias [--provider providerPath]
+   * % hadoop alias list [-provider providerPath]
+   * % hadoop alias delete alias [--provider providerPath] [-i]
+   * </pre>
+   * @param args
+   * @return
+   * @throws IOException
+   */
+  private int init(String[] args) throws IOException {
+    for (int i = 0; i < args.length; i++) { // parse command line
+      if (args[i].equals("create")) {
+        String alias = args[++i];
+        command = new CreateCommand(alias);
+        if (alias.equals("--help")) {
+          printCredShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("delete")) {
+        String alias = args[++i];
+        command = new DeleteCommand(alias);
+        if (alias.equals("--help")) {
+          printCredShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("list")) {
+        command = new ListCommand();
+      } else if (args[i].equals("--provider")) {
+        userSuppliedProvider = true;
+        getConf().set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, 
+            args[++i]);
+      } else if (args[i].equals("-i") || (args[i].equals("--interactive"))) {
+        interactive = true;
+      } else if (args[i].equals("-v") || (args[i].equals("--value"))) {
+        value = args[++i];
+      } else if (args[i].equals("--help")) {
+        printCredShellUsage();
+        return -1;
+      } else {
+        printCredShellUsage();
+        ToolRunner.printGenericCommandUsage(System.err);
+        return -1;
+      }
+    }
+    return 0;
+  }
+
+  private void printCredShellUsage() {
+    out.println(USAGE_PREFIX + COMMANDS);
+    if (command != null) {
+      out.println(command.getUsage());
+    }
+    else {
+      out.println("=========================================================" +
+      		"======");
+      out.println(CreateCommand.USAGE + ":\n\n" + CreateCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(DeleteCommand.USAGE + ":\n\n" + DeleteCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(ListCommand.USAGE + ":\n\n" + ListCommand.DESC);
+    }
+  }
+
+  private abstract class Command {
+    protected CredentialProvider provider = null;
+
+    public boolean validate() {
+      return true;
+    }
+
+    protected CredentialProvider getCredentialProvider() {
+      CredentialProvider provider = null;
+      List<CredentialProvider> providers;
+      try {
+        providers = CredentialProviderFactory.getProviders(getConf());
+        if (userSuppliedProvider) {
+          provider = providers.get(0);
+        }
+        else {
+          for (CredentialProvider p : providers) {
+            if (!p.isTransient()) {
+              provider = p;
+              break;
+            }
+          }
+        }
+      } catch (IOException e) {
+        e.printStackTrace(err);
+      }
+      return provider;
+    }
+
+    protected void printProviderWritten() {
+        out.println(provider.getClass().getName() + " has been updated.");
+    }
+
+    protected void warnIfTransientProvider() {
+      if (provider.isTransient()) {
+        out.println("WARNING: you are modifying a transient provider.");
+      }
+    }
+
+    public abstract void execute() throws Exception;
+
+    public abstract String getUsage();
+  }
+
+  private class ListCommand extends Command {
+    public static final String USAGE = "list <alias> [--provider] [--help]";
+    public static final String DESC =
+        "The list subcommand displays the aliases contained within \n" +
+        "a particular provider - as configured in core-site.xml or " +
+        "indicated\nthrough the --provider argument.";
+
+    public boolean validate() {
+      boolean rc = true;
+      provider = getCredentialProvider();
+      if (provider == null) {
+        out.println("There are no non-transient CredentialProviders configured.\n"
+            + "Consider using the --provider option to indicate the provider\n"
+            + "to use. If you want to list a transient provider then you\n"
+            + "you MUST use the --provider argument.");
+        rc = false;
+      }
+      return rc;
+    }
+
+    public void execute() throws IOException {
+      List<String> aliases;
+      try {
+        aliases = provider.getAliases();
+        out.println("Listing aliases for CredentialProvider: " + provider.toString());
+        for (String alias : aliases) {
+          out.println(alias);
+        }
+      } catch (IOException e) {
+        out.println("Cannot list aliases for CredentialProvider: " + provider.toString()
+            + ": " + e.getMessage());
+        throw e;
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+
+  private class DeleteCommand extends Command {
+    public static final String USAGE = "delete <alias> [--provider] [--help]";
+    public static final String DESC =
+        "The delete subcommand deletes the credenital\n" +
+        "specified as the <alias> argument from within the provider\n" +
+        "indicated through the --provider argument";
+
+    String alias = null;
+    boolean cont = true;
+
+    public DeleteCommand(String alias) {
+      this.alias = alias;
+    }
+
+    @Override
+    public boolean validate() {
+      provider = getCredentialProvider();
+      if (provider == null) {
+        out.println("There are no valid CredentialProviders configured.\n"
+            + "Nothing will be deleted.\n"
+            + "Consider using the --provider option to indicate the provider"
+            + " to use.");
+        return false;
+      }
+      if (alias == null) {
+        out.println("There is no alias specified. Please provide the" +
+            "mandatory <alias>. See the usage description with --help.");
+        return false;
+      }
+      if (interactive) {
+        try {
+          cont = ToolRunner
+              .confirmPrompt("You are about to DELETE the credential: " + 
+                  alias + " from CredentialProvider " + provider.toString() +
+                  ". Continue?:");
+          if (!cont) {
+            out.println("Nothing has been be deleted.");
+          }
+          return cont;
+        } catch (IOException e) {
+          out.println(alias + " will not be deleted.");
+          e.printStackTrace(err);
+        }
+      }
+      return true;
+    }
+
+    public void execute() throws IOException {
+      warnIfTransientProvider();
+      out.println("Deleting credential: " + alias + " from CredentialProvider: "
+          + provider.toString());
+      if (cont) {
+        try {
+          provider.deleteCredentialEntry(alias);
+          out.println(alias + " has been successfully deleted.");
+          provider.flush();
+          printProviderWritten();
+        } catch (IOException e) {
+          out.println(alias + "has NOT been deleted.");
+          throw e;
+        }
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+
+  private class CreateCommand extends Command {
+    public static final String USAGE = "create <alias> [--provider] [--help]";
+    public static final String DESC =
+        "The create subcommand creates a new credential for the name specified\n" +
+        "as the <alias> argument within the provider indicated through\n" +
+        "the --provider argument.";
+
+    String alias = null;
+
+    public CreateCommand(String alias) {
+      this.alias = alias;
+    }
+
+    public boolean validate() {
+      boolean rc = true;
+      provider = getCredentialProvider();
+      if (provider == null) {
+        out.println("There are no valid CredentialProviders configured." +
+        		"\nCredential will not be created.\n"
+            + "Consider using the --provider option to indicate the provider" +
+            " to use.");
+        rc = false;
+      }
+      if (alias == null) {
+        out.println("There is no alias specified. Please provide the" +
+        		"mandatory <alias>. See the usage description with --help.");
+        rc = false;
+      }
+      return rc;
+    }
+
+    public void execute() throws IOException, NoSuchAlgorithmException {
+      warnIfTransientProvider();
+      try {
+        char[] credential = null;
+        if (value != null) {
+          // testing only
+          credential = value.toCharArray();
+        }
+        else {
+           credential = promptForCredential();
+        }
+        provider.createCredentialEntry(alias, credential);
+        out.println(alias + " has been successfully created.");
+        provider.flush();
+        printProviderWritten();
+      } catch (InvalidParameterException e) {
+        out.println(alias + " has NOT been created. " + e.getMessage());
+        throw e;
+      } catch (IOException e) {
+        out.println(alias + " has NOT been created. " + e.getMessage());
+        throw e;
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+  
+  protected char[] promptForCredential() throws IOException {
+    PasswordReader c = getPasswordReader();
+    if (c == null) {
+      throw new IOException("No console available for prompting user.");
+    }
+    
+    char[] cred = null;
+
+    boolean noMatch;
+    do {
+      char[] newPassword1 = c.readPassword("Enter password: ");
+      char[] newPassword2 = c.readPassword("Enter password again: ");
+      noMatch = !Arrays.equals(newPassword1, newPassword2);
+      if (noMatch) {
+        Arrays.fill(newPassword1, ' ');
+        c.format("Passwords don't match. Try again.%n");
+      } else {
+        cred = newPassword1;
+      }
+      Arrays.fill(newPassword2, ' ');
+    } while (noMatch);
+    return cred;
+  }
+  
+  public PasswordReader getPasswordReader() {
+    if (passwordReader == null) {
+      passwordReader = new PasswordReader();
+    }
+    return passwordReader;
+  }
+  
+  public void setPasswordReader(PasswordReader reader) {
+    passwordReader = reader;
+  }
+  
+  // to facilitate testing since Console is a final class...
+  public static class PasswordReader {
+    public char[] readPassword(String prompt) {
+      Console console = System.console();
+      char[] pass = console.readPassword(prompt);
+      return pass;
+    }
+
+    public void format(String message) {
+      Console console = System.console();
+      console.format(message);
+    }
+  }
+  
+  
+  /**
+   * Main program.
+   *
+   * @param args
+   *          Command line arguments
+   * @throws Exception
+   */
+  public static void main(String[] args) throws Exception {
+    int res = ToolRunner.run(new Configuration(), new CredentialShell(), args);
+    System.exit(res);
+  }
+}

+ 290 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java

@@ -0,0 +1,290 @@
+/**
+ * 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.alias;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.ProviderUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URL;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * CredentialProvider based on Java's KeyStore file format. The file may be 
+ * stored in any Hadoop FileSystem using the following name mangling:
+ *  jceks://hdfs@nn1.example.com/my/creds.jceks -> hdfs://nn1.example.com/my/creds.jceks
+ *  jceks://file/home/larry/creds.jceks -> file:///home/larry/creds.jceks
+ *
+ * The password for the keystore is taken from the HADOOP_CREDSTORE_PASSWORD
+ * environment variable with a default of 'none'.
+ *
+ * It is expected that for access to credential protected resource to copy the 
+ * creds from the original provider into the job's Credentials object, which is
+ * accessed via the UserProvider. Therefore, this provider won't be directly 
+ * used by MapReduce tasks.
+ */
+@InterfaceAudience.Private
+public class JavaKeyStoreProvider extends CredentialProvider {
+  public static final String SCHEME_NAME = "jceks";
+  public static final String CREDENTIAL_PASSWORD_NAME =
+      "HADOOP_CREDSTORE_PASSWORD";
+  public static final String KEYSTORE_PASSWORD_FILE_KEY =
+      "hadoop.security.credstore.java-keystore-provider.password-file";
+  public static final String KEYSTORE_PASSWORD_DEFAULT = "none";
+
+  private final URI uri;
+  private final Path path;
+  private final FileSystem fs;
+  private final FsPermission permissions;
+  private final KeyStore keyStore;
+  private char[] password = null;
+  private boolean changed = false;
+  private Lock readLock;
+  private Lock writeLock;
+
+  private final Map<String, CredentialEntry> cache = new HashMap<String, CredentialEntry>();
+
+  private JavaKeyStoreProvider(URI uri, Configuration conf) throws IOException {
+    this.uri = uri;
+    path = ProviderUtils.unnestUri(uri);
+    fs = path.getFileSystem(conf);
+    // Get the password from the user's environment
+    if (System.getenv().containsKey(CREDENTIAL_PASSWORD_NAME)) {
+      password = System.getenv(CREDENTIAL_PASSWORD_NAME).toCharArray();
+    }
+    // if not in ENV get check for file
+    if (password == null) {
+      String pwFile = conf.get(KEYSTORE_PASSWORD_FILE_KEY);
+      if (pwFile != null) {
+        ClassLoader cl = Thread.currentThread().getContextClassLoader();
+        URL pwdFile = cl.getResource(pwFile);
+        if (pwdFile != null) {
+          InputStream is = pwdFile.openStream();
+          try {
+            password = IOUtils.toCharArray(is);
+          } finally {
+            is.close();
+          }
+        }
+      }
+    }
+    if (password == null) {
+      password = KEYSTORE_PASSWORD_DEFAULT.toCharArray();
+    }
+    try {
+      keyStore = KeyStore.getInstance(SCHEME_NAME);
+      if (fs.exists(path)) {
+        // save off permissions in case we need to
+        // rewrite the keystore in flush()
+        FileStatus s = fs.getFileStatus(path);
+        permissions = s.getPermission();
+
+        keyStore.load(fs.open(path), password);
+      } else {
+        permissions = new FsPermission("700");
+        // required to create an empty keystore. *sigh*
+        keyStore.load(null, password);
+      }
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't create keystore", e);
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException("Can't load keystore " + path, e);
+    } catch (CertificateException e) {
+      throw new IOException("Can't load keystore " + path, e);
+    }
+    ReadWriteLock lock = new ReentrantReadWriteLock(true);
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+  }
+
+  @Override
+  public CredentialEntry getCredentialEntry(String alias) throws IOException {
+    readLock.lock();
+    try {
+      SecretKeySpec key = null;
+      try {
+        if (cache.containsKey(alias)) {
+          return cache.get(alias);
+        }
+        if (!keyStore.containsAlias(alias)) {
+          return null;
+        }
+        key = (SecretKeySpec) keyStore.getKey(alias, password);
+      } catch (KeyStoreException e) {
+        throw new IOException("Can't get credential " + alias + " from " +
+                              path, e);
+      } catch (NoSuchAlgorithmException e) {
+        throw new IOException("Can't get algorithm for credential " + alias + " from " +
+                              path, e);
+      } catch (UnrecoverableKeyException e) {
+        throw new IOException("Can't recover credential " + alias + " from " + path, e);
+      }
+      return new CredentialEntry(alias, bytesToChars(key.getEncoded()));
+    } 
+    finally {
+      readLock.unlock();
+    }
+  }
+  
+  public static char[] bytesToChars(byte[] bytes) {
+    String pass = new String(bytes);
+    return pass.toCharArray();
+  }
+
+  @Override
+  public List<String> getAliases() throws IOException {
+    readLock.lock();
+    try {
+      ArrayList<String> list = new ArrayList<String>();
+      String alias = null;
+      try {
+        Enumeration<String> e = keyStore.aliases();
+        while (e.hasMoreElements()) {
+           alias = e.nextElement();
+           list.add(alias);
+        }
+      } catch (KeyStoreException e) {
+        throw new IOException("Can't get alias " + alias + " from " + path, e);
+      }
+      return list;
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public CredentialEntry createCredentialEntry(String alias, char[] credential)
+      throws IOException {
+    try {
+      if (keyStore.containsAlias(alias) || cache.containsKey(alias)) {
+        throw new IOException("Credential " + alias + " already exists in " + this);
+      }
+    } catch (KeyStoreException e) {
+      throw new IOException("Problem looking up credential " + alias + " in " + this,
+          e);
+    }
+    return innerSetCredential(alias, credential);
+  }
+
+  @Override
+  public void deleteCredentialEntry(String name) throws IOException {
+    writeLock.lock();
+    try {
+      try {
+        if (keyStore.containsAlias(name)) {
+          keyStore.deleteEntry(name);
+        }
+        else {
+          throw new IOException("Credential " + name + " does not exist in " + this);
+        }
+      } catch (KeyStoreException e) {
+        throw new IOException("Problem removing " + name + " from " +
+            this, e);
+      }
+      cache.remove(name);
+      changed = true;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  CredentialEntry innerSetCredential(String alias, char[] material)
+      throws IOException {
+    try {
+      keyStore.setKeyEntry(alias, new SecretKeySpec(
+          new String(material).getBytes("UTF-8"), "AES"),
+          password, null);
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't store credential " + alias + " in " + this,
+          e);
+    }
+    changed = true;
+    return new CredentialEntry(alias, material);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    writeLock.lock();
+    try {
+      if (!changed) {
+        return;
+      }
+      // write out the keystore
+      FSDataOutputStream out = FileSystem.create(fs, path, permissions);
+      try {
+        keyStore.store(out, password);
+      } catch (KeyStoreException e) {
+        throw new IOException("Can't store keystore " + this, e);
+      } catch (NoSuchAlgorithmException e) {
+        throw new IOException("No such algorithm storing keystore " + this, e);
+      } catch (CertificateException e) {
+        throw new IOException("Certificate exception storing keystore " + this,
+            e);
+      }
+      out.close();
+      changed = false;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return uri.toString();
+  }
+
+  /**
+   * The factory to create JksProviders, which is used by the ServiceLoader.
+   */
+  public static class Factory extends CredentialProviderFactory {
+    @Override
+    public CredentialProvider createProvider(URI providerName,
+                                      Configuration conf) throws IOException {
+      if (SCHEME_NAME.equals(providerName.getScheme())) {
+        return new JavaKeyStoreProvider(providerName, conf);
+      }
+      return null;
+    }
+  }
+}

+ 123 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java

@@ -0,0 +1,123 @@
+/**
+ * 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.alias;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * A CredentialProvider for UGIs. It uses the credentials object associated
+ * with the current user to find credentials. This provider is created using a
+ * URI of "user:///".
+ */
+@InterfaceAudience.Private
+public class UserProvider extends CredentialProvider {
+  public static final String SCHEME_NAME = "user";
+  private final UserGroupInformation user;
+  private final Credentials credentials;
+  private final Map<String, CredentialEntry> cache = new HashMap<String, 
+      CredentialEntry>();
+
+  private UserProvider() throws IOException {
+    user = UserGroupInformation.getCurrentUser();
+    credentials = user.getCredentials();
+  }
+
+  @Override
+  public boolean isTransient() {
+    return true;
+  }
+
+  @Override
+  public CredentialEntry getCredentialEntry(String alias) {
+    byte[] bytes = credentials.getSecretKey(new Text(alias));
+    if (bytes == null) {
+      return null;
+    }
+    return new CredentialEntry(alias, new String(bytes).toCharArray());
+  }
+
+  @Override
+  public CredentialEntry createCredentialEntry(String name, char[] credential) 
+      throws IOException {
+    Text nameT = new Text(name);
+    if (credentials.getSecretKey(nameT) != null) {
+      throw new IOException("Credential " + name + 
+          " already exists in " + this);
+    }
+    credentials.addSecretKey(new Text(name), 
+        new String(credential).getBytes("UTF-8"));
+    return new CredentialEntry(name, credential);
+  }
+
+  @Override
+  public void deleteCredentialEntry(String name) throws IOException {
+    byte[] cred = credentials.getSecretKey(new Text(name));
+    if (cred != null) {
+      credentials.removeSecretKey(new Text(name));
+    }
+    else {
+      throw new IOException("Credential " + name + 
+          " does not exist in " + this);
+    }
+    cache.remove(name);
+  }
+
+  @Override
+  public String toString() {
+    return SCHEME_NAME + ":///";
+  }
+
+  @Override
+  public void flush() {
+    user.addCredentials(credentials);
+  }
+
+  public static class Factory extends CredentialProviderFactory {
+
+    @Override
+    public CredentialProvider createProvider(URI providerName,
+                                      Configuration conf) throws IOException {
+      if (SCHEME_NAME.equals(providerName.getScheme())) {
+        return new UserProvider();
+      }
+      return null;
+    }
+  }
+
+  @Override
+  public List<String> getAliases() throws IOException {
+    List<String> list = new ArrayList<String>();
+    List<Text> aliases = credentials.getAllSecretKeys();
+    for (Text key : aliases) {
+      list.add(key.toString());
+    }
+    return list;
+  }
+}

+ 17 - 0
hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.alias.CredentialProviderFactory

@@ -0,0 +1,17 @@
+# 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.security.alias.JavaKeyStoreProvider$Factory
+org.apache.hadoop.security.alias.UserProvider$Factory

+ 5 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProvider.java

@@ -21,6 +21,7 @@ import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.ProviderUtils;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import java.io.IOException;
 import java.io.IOException;
@@ -133,13 +134,13 @@ public class TestKeyProvider {
   @Test
   @Test
   public void testUnnestUri() throws Exception {
   public void testUnnestUri() throws Exception {
     assertEquals(new Path("hdfs://nn.example.com/my/path"),
     assertEquals(new Path("hdfs://nn.example.com/my/path"),
-        KeyProvider.unnestUri(new URI("myscheme://hdfs@nn.example.com/my/path")));
+        ProviderUtils.unnestUri(new URI("myscheme://hdfs@nn.example.com/my/path")));
     assertEquals(new Path("hdfs://nn/my/path?foo=bar&baz=bat#yyy"),
     assertEquals(new Path("hdfs://nn/my/path?foo=bar&baz=bat#yyy"),
-        KeyProvider.unnestUri(new URI("myscheme://hdfs@nn/my/path?foo=bar&baz=bat#yyy")));
+        ProviderUtils.unnestUri(new URI("myscheme://hdfs@nn/my/path?foo=bar&baz=bat#yyy")));
     assertEquals(new Path("inner://hdfs@nn1.example.com/my/path"),
     assertEquals(new Path("inner://hdfs@nn1.example.com/my/path"),
-        KeyProvider.unnestUri(new URI("outer://inner@hdfs@nn1.example.com/my/path")));
+        ProviderUtils.unnestUri(new URI("outer://inner@hdfs@nn1.example.com/my/path")));
     assertEquals(new Path("user:///"),
     assertEquals(new Path("user:///"),
-        KeyProvider.unnestUri(new URI("outer://user/")));
+        ProviderUtils.unnestUri(new URI("outer://user/")));
   }
   }
 
 
   private static class MyKeyProvider extends KeyProvider {
   private static class MyKeyProvider extends KeyProvider {

+ 2 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
@@ -213,7 +214,7 @@ public class TestKeyProviderFactory {
     file.delete();
     file.delete();
     conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, ourUrl);
     conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, ourUrl);
     checkSpecificProvider(conf, ourUrl);
     checkSpecificProvider(conf, ourUrl);
-    Path path = KeyProvider.unnestUri(new URI(ourUrl));
+    Path path = ProviderUtils.unnestUri(new URI(ourUrl));
     FileSystem fs = path.getFileSystem(conf);
     FileSystem fs = path.getFileSystem(conf);
     FileStatus s = fs.getFileStatus(path);
     FileStatus s = fs.getFileStatus(path);
     assertTrue(s.getPermission().toString().equals("rwx------"));
     assertTrue(s.getPermission().toString().equals("rwx------"));

+ 173 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java

@@ -0,0 +1,173 @@
+/**
+ * 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.alias;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.alias.CredentialShell.PasswordReader;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCredShell {
+  private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private final ByteArrayOutputStream errContent = new ByteArrayOutputStream();
+  private static final File tmpDir =
+      new File(System.getProperty("test.build.data", "/tmp"), "creds");
+  
+  @Before
+  public void setup() throws Exception {
+    System.setOut(new PrintStream(outContent));
+    System.setErr(new PrintStream(errContent));
+  }
+  
+  @Test
+  public void testCredentialSuccessfulLifecycle() throws Exception {
+    outContent.reset();
+    String[] args1 = {"create", "credential1", "--value", "p@ssw0rd", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    int rc = 0;
+    CredentialShell cs = new CredentialShell();
+    cs.setConf(new Configuration());
+    rc = cs.run(args1);
+    assertEquals(outContent.toString(), 0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+    		"created."));
+
+    outContent.reset();
+    String[] args2 = {"list", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    rc = cs.run(args2);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("credential1"));
+
+    outContent.reset();
+    String[] args4 = {"delete", "credential1", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    rc = cs.run(args4);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+    		"deleted."));
+
+    outContent.reset();
+    String[] args5 = {"list", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    rc = cs.run(args5);
+    assertEquals(0, rc);
+    assertFalse(outContent.toString(), outContent.toString().contains("credential1"));
+  }
+
+  @Test
+  public void testInvalidProvider() throws Exception {
+    String[] args1 = {"create", "credential1", "--value", "p@ssw0rd", "--provider", 
+      "sdff://file/tmp/credstore.jceks"};
+    
+    int rc = 0;
+    CredentialShell cs = new CredentialShell();
+    cs.setConf(new Configuration());
+    rc = cs.run(args1);
+    assertEquals(-1, rc);
+    assertTrue(outContent.toString().contains("There are no valid " +
+    		"CredentialProviders configured."));
+  }
+
+  @Test
+  public void testTransientProviderWarning() throws Exception {
+    String[] args1 = {"create", "credential1", "--value", "p@ssw0rd", "--provider", 
+      "user:///"};
+    
+    int rc = 0;
+    CredentialShell cs = new CredentialShell();
+    cs.setConf(new Configuration());
+    rc = cs.run(args1);
+    assertEquals(outContent.toString(), 0, rc);
+    assertTrue(outContent.toString().contains("WARNING: you are modifying a " +
+    		"transient provider."));
+
+    String[] args2 = {"delete", "credential1", "--provider", "user:///"};
+    rc = cs.run(args2);
+    assertEquals(outContent.toString(), 0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+        "deleted."));
+  }
+  
+  @Test
+  public void testTransientProviderOnlyConfig() throws Exception {
+    String[] args1 = {"create", "credential1"};
+    
+    int rc = 0;
+    CredentialShell cs = new CredentialShell();
+    Configuration config = new Configuration();
+    config.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, "user:///");
+    cs.setConf(config);
+    rc = cs.run(args1);
+    assertEquals(-1, rc);
+    assertTrue(outContent.toString().contains("There are no valid " +
+    		"CredentialProviders configured."));
+  }
+  
+  @Test
+  public void testPromptForCredential() throws Exception {
+    String[] args1 = {"create", "credential1", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    ArrayList<String> passwords = new ArrayList<String>();
+    passwords.add("p@ssw0rd");
+    passwords.add("p@ssw0rd");
+    int rc = 0;
+    CredentialShell shell = new CredentialShell();
+    shell.setConf(new Configuration());
+    shell.setPasswordReader(new MockPasswordReader(passwords));
+    rc = shell.run(args1);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+        "created."));
+
+    String[] args2 = {"delete", "credential1", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    rc = shell.run(args2);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+        "deleted."));
+  }
+  
+  public class MockPasswordReader extends CredentialShell.PasswordReader {
+    List<String> passwords = null;
+    
+    public MockPasswordReader(List<String> passwds) {
+      passwords = passwds;
+    }
+
+    @Override
+    public char[] readPassword(String prompt) {
+      if (passwords.size() == 0) return null;
+      String pass = passwords.remove(0);
+      return pass.toCharArray();
+    }
+
+    @Override
+    public void format(String message) {
+      System.out.println(message);
+    }
+  }
+}

+ 51 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProvider.java

@@ -0,0 +1,51 @@
+/**
+ * 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.alias;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.ProviderUtils;
+import org.junit.Test;
+
+import java.net.URI;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestCredentialProvider {
+
+  @Test
+  public void testCredentialEntry() throws Exception {
+    char[] key1 = new char[]{1,2,3,4};
+    CredentialProvider.CredentialEntry obj = 
+        new CredentialProvider.CredentialEntry("cred1", key1);
+    assertEquals("cred1", obj.getAlias());
+    assertArrayEquals(new char[]{1,2,3,4}, obj.getCredential());
+  }
+
+  @Test
+  public void testUnnestUri() throws Exception {
+    assertEquals(new Path("hdfs://nn.example.com/my/path"),
+        ProviderUtils.unnestUri(new URI("myscheme://hdfs@nn.example.com/my/path")));
+    assertEquals(new Path("hdfs://nn/my/path?foo=bar&baz=bat#yyy"),
+        ProviderUtils.unnestUri(new URI("myscheme://hdfs@nn/my/path?foo=bar&baz=bat#yyy")));
+    assertEquals(new Path("inner://hdfs@nn1.example.com/my/path"),
+        ProviderUtils.unnestUri(new URI("outer://inner@hdfs@nn1.example.com/my/path")));
+    assertEquals(new Path("user:///"),
+        ProviderUtils.unnestUri(new URI("outer://user/")));
+  }
+}

+ 234 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java

@@ -0,0 +1,234 @@
+/**
+ * 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.alias;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestCredentialProviderFactory {
+  
+  private static char[] chars = { 'a', 'b', 'c', 'd', 'e', 'f', 'g',
+  'h', 'j', 'k', 'm', 'n', 'p', 'q', 'r', 's', 't', 'u', 'v', 'w',
+  'x', 'y', 'z', 'A', 'B', 'C', 'D', 'E', 'F', 'G', 'H', 'J', 'K',
+  'M', 'N', 'P', 'Q', 'R', 'S', 'T', 'U', 'V', 'W', 'X', 'Y', 'Z',
+  '2', '3', '4', '5', '6', '7', '8', '9',};
+
+  private static final File tmpDir =
+      new File(System.getProperty("test.build.data", "/tmp"), "creds");
+
+  @Test
+  public void testFactory() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+        UserProvider.SCHEME_NAME + ":///," +
+            JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks");
+    List<CredentialProvider> providers = 
+        CredentialProviderFactory.getProviders(conf);
+    assertEquals(2, providers.size());
+    assertEquals(UserProvider.class, providers.get(0).getClass());
+    assertEquals(JavaKeyStoreProvider.class, providers.get(1).getClass());
+    assertEquals(UserProvider.SCHEME_NAME +
+        ":///", providers.get(0).toString());
+    assertEquals(JavaKeyStoreProvider.SCHEME_NAME +
+        "://file" + tmpDir + "/test.jks",
+        providers.get(1).toString());
+  }
+
+  @Test
+  public void testFactoryErrors() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, "unknown:///");
+    try {
+      List<CredentialProvider> providers = 
+          CredentialProviderFactory.getProviders(conf);
+      assertTrue("should throw!", false);
+    } catch (IOException e) {
+      assertEquals("No CredentialProviderFactory for unknown:/// in " +
+          CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+          e.getMessage());
+    }
+  }
+
+  @Test
+  public void testUriErrors() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, "unkn@own:/x/y");
+    try {
+      List<CredentialProvider> providers = 
+          CredentialProviderFactory.getProviders(conf);
+      assertTrue("should throw!", false);
+    } catch (IOException e) {
+      assertEquals("Bad configuration of " +
+          CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH +
+          " at unkn@own:/x/y", e.getMessage());
+    }
+  }
+
+  private static char[] generatePassword(int length) {
+    StringBuffer sb = new StringBuffer();
+    Random r = new Random();
+    for (int i = 0; i < length; i++) {
+      sb.append(chars[r.nextInt(chars.length)]);
+    }
+    return sb.toString().toCharArray();
+  }
+  
+  static void checkSpecificProvider(Configuration conf,
+                                   String ourUrl) throws Exception {
+    CredentialProvider provider = 
+        CredentialProviderFactory.getProviders(conf).get(0);
+    char[] passwd = generatePassword(16);
+
+    // ensure that we get nulls when the key isn't there
+    assertEquals(null, provider.getCredentialEntry("no-such-key"));
+    assertEquals(null, provider.getCredentialEntry("key"));
+    // create a new key
+    try {
+      provider.createCredentialEntry("pass", passwd);
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+    // make sure we get back the right key
+    assertArrayEquals(passwd, provider.getCredentialEntry("pass").getCredential());
+    // try recreating pass
+    try {
+      provider.createCredentialEntry("pass", passwd);
+      assertTrue("should throw", false);
+    } catch (IOException e) {
+      assertEquals("Credential pass already exists in " + ourUrl, e.getMessage());
+    }
+    provider.deleteCredentialEntry("pass");
+    try {
+      provider.deleteCredentialEntry("pass");
+      assertTrue("should throw", false);
+    } catch (IOException e) {
+      assertEquals("Credential pass does not exist in " + ourUrl, e.getMessage());
+    }
+    char[] passTwo = new char[]{'1', '2', '3'};
+    provider.createCredentialEntry("pass", passwd);
+    provider.createCredentialEntry("pass2", passTwo);
+    assertArrayEquals(passTwo,
+        provider.getCredentialEntry("pass2").getCredential());
+
+    // write them to disk so that configuration.getPassword will find them
+    provider.flush();
+
+    // configuration.getPassword should get this from provider
+    assertArrayEquals(passTwo, conf.getPassword("pass2"));
+
+    // configuration.getPassword should get this from config
+    conf.set("onetwothree", "123");
+    assertArrayEquals(passTwo, conf.getPassword("onetwothree"));
+
+    // configuration.getPassword should NOT get this from config since
+    // we are disabling the fallback to clear text config
+    conf.set(CredentialProvider.CLEAR_TEXT_FALLBACK, "false");
+    assertArrayEquals(null, conf.getPassword("onetwothree"));
+
+    // get a new instance of the provider to ensure it was saved correctly
+    provider = CredentialProviderFactory.getProviders(conf).get(0);
+    assertTrue(provider != null);
+    assertArrayEquals(new char[]{'1', '2', '3'},
+        provider.getCredentialEntry("pass2").getCredential());
+    assertArrayEquals(passwd, provider.getCredentialEntry("pass").getCredential());
+
+    List<String> creds = provider.getAliases();
+    assertTrue("Credentials should have been returned.", creds.size() == 2);
+    assertTrue("Returned Credentials should have included pass.", creds.contains("pass"));
+    assertTrue("Returned Credentials should have included pass2.", creds.contains("pass2"));
+  }
+
+  @Test
+  public void testUserProvider() throws Exception {
+    Configuration conf = new Configuration();
+    final String ourUrl = UserProvider.SCHEME_NAME + ":///";
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+    checkSpecificProvider(conf, ourUrl);
+    // see if the credentials are actually in the UGI
+    Credentials credentials =
+        UserGroupInformation.getCurrentUser().getCredentials();
+    assertArrayEquals(new byte[]{'1', '2', '3'},
+        credentials.getSecretKey(new Text("pass2")));
+  }
+
+  @Test
+  public void testJksProvider() throws Exception {
+    Configuration conf = new Configuration();
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks";
+
+    File file = new File(tmpDir, "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+    checkSpecificProvider(conf, ourUrl);
+    Path path = ProviderUtils.unnestUri(new URI(ourUrl));
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus s = fs.getFileStatus(path);
+    assertTrue(s.getPermission().toString().equals("rwx------"));
+    assertTrue(file + " should exist", file.isFile());
+
+    // check permission retention after explicit change
+    fs.setPermission(path, new FsPermission("777"));
+    checkPermissionRetention(conf, ourUrl, path);
+  }
+
+  public void checkPermissionRetention(Configuration conf, String ourUrl, 
+      Path path) throws Exception {
+    CredentialProvider provider = CredentialProviderFactory.getProviders(conf).get(0);
+    // let's add a new credential and flush and check that permissions are still set to 777
+    char[] cred = new char[32];
+    for(int i =0; i < cred.length; ++i) {
+      cred[i] = (char) i;
+    }
+    // create a new key
+    try {
+      provider.createCredentialEntry("key5", cred);
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+    provider.flush();
+    // get a new instance of the provider to ensure it was saved correctly
+    provider = CredentialProviderFactory.getProviders(conf).get(0);
+    assertArrayEquals(cred, provider.getCredentialEntry("key5").getCredential());
+
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus s = fs.getFileStatus(path);
+    assertTrue("Permissions should have been retained from the preexisting " +
+    		"keystore.", s.getPermission().toString().equals("rwxrwxrwx"));
+  }
+}