You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/08/05 04:32:03 UTC

[2/4] git commit: ACCUMULO-3045 Create a CredentialProviderToken

ACCUMULO-3045 Create a CredentialProviderToken

PasswordToken where the password is retrieved from
a Hadoop CredentialProvider. Extends the PasswordToken
due to the reliance on it in the ZK-based authenticator
and authorizor interfaces.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/1ec33f1a
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/1ec33f1a
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/1ec33f1a

Branch: refs/heads/master
Commit: 1ec33f1a52c9166fab96b815fa89724b05737939
Parents: e2dc550
Author: Josh Elser <el...@apache.org>
Authored: Mon Aug 4 21:14:09 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Aug 4 21:14:09 2014 -0400

----------------------------------------------------------------------
 .../tokens/CredentialProviderToken.java         | 103 +++++++++++++++
 .../client/security/tokens/PasswordToken.java   |   8 +-
 .../conf/CredentialProviderFactoryShim.java     | 106 ++++++++-------
 .../tokens/CredentialProviderTokenTest.java     | 129 +++++++++++++++++++
 .../conf/CredentialProviderFactoryShimTest.java |   8 ++
 core/src/test/resources/passwords.jceks         | Bin 0 -> 963 bytes
 6 files changed, 306 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1ec33f1a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/CredentialProviderToken.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/CredentialProviderToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/CredentialProviderToken.java
new file mode 100644
index 0000000..21e883b
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/CredentialProviderToken.java
@@ -0,0 +1,103 @@
+/*
+ * 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.accumulo.core.client.security.tokens;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.CharBuffer;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import jline.internal.Preconditions;
+
+import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * An {@link AuthenticationToken} backed by a Hadoop CredentialProvider.
+ */
+public class CredentialProviderToken extends PasswordToken {
+  public static final String NAME_PROPERTY = "name", CREDENTIAL_PROVIDERS_PROPERTY = "credentialProviders";
+
+  public CredentialProviderToken() {
+    super();
+  }
+
+  public CredentialProviderToken(String name, String credentialProviders) throws IOException {
+    Preconditions.checkNotNull(name);
+    Preconditions.checkNotNull(credentialProviders);
+
+    setWithCredentialProviders(name, credentialProviders);
+  }
+
+  protected void setWithCredentialProviders(String name, String credentialProviders) throws IOException {
+    final Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credentialProviders);
+
+    char[] password = CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, name);
+
+    if (null == password) {
+      throw new IOException("No password could be extracted from CredentialProvider(s) with " + name);
+    }
+
+    setPassword(CharBuffer.wrap(password));    
+  }
+  
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+  }
+  
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+  }
+  
+  @Override
+  public void init(Properties properties) {
+    char[] nameCharArray = properties.get(NAME_PROPERTY), credentialProvidersCharArray = properties.get(CREDENTIAL_PROVIDERS_PROPERTY);
+    if (null != nameCharArray && null != credentialProvidersCharArray) {
+      try {
+        this.setWithCredentialProviders(new String(nameCharArray), new String(credentialProvidersCharArray));
+      } catch (IOException e) {
+        throw new IllegalArgumentException("Could not extract password from CredentialProvider", e);
+      }
+
+      return;
+    }
+
+    throw new IllegalArgumentException("Expected " + NAME_PROPERTY + " and " + CREDENTIAL_PROVIDERS_PROPERTY + " properties.");
+  }
+  
+  @Override
+  public Set<TokenProperty> getProperties() {
+    LinkedHashSet<TokenProperty> properties = new LinkedHashSet<TokenProperty>();
+    // Neither name or CPs are sensitive
+    properties.add(new TokenProperty(NAME_PROPERTY, "Alias to extract from CredentialProvider", false));
+    properties.add(new TokenProperty(CREDENTIAL_PROVIDERS_PROPERTY, "Comma separated list of URLs defining CredentialProvider(s)", false));
+    return properties;
+  }
+  
+  @Override
+  public CredentialProviderToken clone() {
+    CredentialProviderToken clone = new CredentialProviderToken();
+    clone.setPassword(this.getPassword());
+    return clone;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1ec33f1a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
index 1474435..bd9feba 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java
@@ -124,8 +124,12 @@ public class PasswordToken implements AuthenticationToken {
       throw new RuntimeException(e);
     }
   }
-  
-  private void setPassword(CharBuffer charBuffer) {
+
+  protected void setPassword(byte[] password) {
+    this.password = Arrays.copyOf(password, password.length);
+  }
+
+  protected void setPassword(CharBuffer charBuffer) {
     // encode() kicks back a C-string, which is not compatible with the old passwording system
     ByteBuffer bb = Constants.UTF8.encode(charBuffer);
     // create array using byter buffer length

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1ec33f1a/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java b/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
index 1bf3ce6..a1fd8c7 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
@@ -37,26 +37,26 @@ import org.slf4j.LoggerFactory;
  */
 public class CredentialProviderFactoryShim {
   private static final Logger log = LoggerFactory.getLogger(CredentialProviderFactoryShim.class);
-
-  protected static final String HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME = "org.apache.hadoop.security.alias.JavaKeyStoreProvider$Factory";
-  protected static final String HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME = "getProviders";
-
-  protected static final String HADOOP_CRED_PROVIDER_CLASS_NAME = "org.apache.hadoop.security.alias.CredentialProvider";
-  protected static final String HADOOP_CRED_PROVIDER_GET_CREDENTIAL_ENTRY_METHOD_NAME = "getCredentialEntry";
-  protected static final String HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME = "getAliases";
-
-  protected static final String HADOOP_CRED_ENTRY_CLASS_NAME = "org.apache.hadoop.security.alias.CredentialProvider$CredentialEntry";
-  protected static final String HADOOP_CRED_ENTRY_GET_CREDENTIAL_METHOD_NAME = "getCredential";
-
+  
+  public static final String HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME = "org.apache.hadoop.security.alias.JavaKeyStoreProvider$Factory";
+  public static final String HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME = "getProviders";
+  
+  public static final String HADOOP_CRED_PROVIDER_CLASS_NAME = "org.apache.hadoop.security.alias.CredentialProvider";
+  public static final String HADOOP_CRED_PROVIDER_GET_CREDENTIAL_ENTRY_METHOD_NAME = "getCredentialEntry";
+  public static final String HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME = "getAliases";
+  
+  public static final String HADOOP_CRED_ENTRY_CLASS_NAME = "org.apache.hadoop.security.alias.CredentialProvider$CredentialEntry";
+  public static final String HADOOP_CRED_ENTRY_GET_CREDENTIAL_METHOD_NAME = "getCredential";
+  
   public static final String CREDENTIAL_PROVIDER_PATH = "hadoop.security.credential.provider.path";
-
+  
   private static Object hadoopCredProviderFactory = null;
   private static Method getProvidersMethod = null;
   private static Method getAliasesMethod = null;
   private static Method getCredentialEntryMethod = null;
   private static Method getCredentialMethod = null;
   private static Boolean hadoopClassesAvailable = null;
-
+  
   /**
    * Determine if we can load the necessary CredentialProvider classes. Only loaded the first time, so subsequent invocations of this method should return fast.
    *
@@ -74,9 +74,9 @@ public class CredentialProviderFactoryShim {
         return false;
       }
     }
-
+    
     hadoopClassesAvailable = false;
-
+    
     // Load Hadoop CredentialProviderFactory
     Class<?> hadoopCredProviderFactoryClz = null;
     try {
@@ -85,7 +85,7 @@ public class CredentialProviderFactoryShim {
       log.trace("Could not load class {}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, e);
       return false;
     }
-
+    
     // Load Hadoop CredentialProviderFactory.getProviders(Configuration)
     try {
       getProvidersMethod = hadoopCredProviderFactoryClz.getMethod(HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME, Configuration.class);
@@ -96,7 +96,7 @@ public class CredentialProviderFactoryShim {
       log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME, HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, e);
       return false;
     }
-
+    
     // Instantiate Hadoop CredentialProviderFactory
     try {
       hadoopCredProviderFactory = hadoopCredProviderFactoryClz.newInstance();
@@ -107,7 +107,7 @@ public class CredentialProviderFactoryShim {
       log.trace("Could not instantiate class {}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, e);
       return false;
     }
-
+    
     // Load Hadoop CredentialProvider
     Class<?> hadoopCredProviderClz = null;
     try {
@@ -116,7 +116,7 @@ public class CredentialProviderFactoryShim {
       log.trace("Could not load class {}", HADOOP_CRED_PROVIDER_CLASS_NAME, e);
       return false;
     }
-
+    
     // Load Hadoop CredentialProvider.getCredentialEntry(String)
     try {
       getCredentialEntryMethod = hadoopCredProviderClz.getMethod(HADOOP_CRED_PROVIDER_GET_CREDENTIAL_ENTRY_METHOD_NAME, String.class);
@@ -127,7 +127,7 @@ public class CredentialProviderFactoryShim {
       log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_GET_CREDENTIAL_ENTRY_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
       return false;
     }
-
+    
     // Load Hadoop CredentialProvider.getAliases()
     try {
       getAliasesMethod = hadoopCredProviderClz.getMethod(HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME);
@@ -138,7 +138,7 @@ public class CredentialProviderFactoryShim {
       log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
       return false;
     }
-
+    
     // Load Hadoop CredentialEntry
     Class<?> hadoopCredentialEntryClz = null;
     try {
@@ -147,7 +147,7 @@ public class CredentialProviderFactoryShim {
       log.trace("Could not load class {}", HADOOP_CRED_ENTRY_CLASS_NAME);
       return false;
     }
-
+    
     // Load Hadoop CredentialEntry.getCredential()
     try {
       getCredentialMethod = hadoopCredentialEntryClz.getMethod(HADOOP_CRED_ENTRY_GET_CREDENTIAL_METHOD_NAME);
@@ -158,12 +158,12 @@ public class CredentialProviderFactoryShim {
       log.trace("Could not find {} method on {}", HADOOP_CRED_ENTRY_GET_CREDENTIAL_METHOD_NAME, HADOOP_CRED_ENTRY_CLASS_NAME, e);
       return false;
     }
-
+    
     hadoopClassesAvailable = true;
-
+    
     return true;
   }
-
+  
   /**
    * Wrapper to fetch the configured {@code List<CredentialProvider>}s.
    *
@@ -187,7 +187,7 @@ public class CredentialProviderFactoryShim {
       log.warn("Could not invoke {}.{}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME, e);
       return null;
     }
-
+    
     // Cast the Object to List<Object> (actually List<CredentialProvider>)
     try {
       return (List<Object>) providersObj;
@@ -196,26 +196,26 @@ public class CredentialProviderFactoryShim {
       return null;
     }
   }
-
+  
   protected static char[] getFromHadoopCredentialProvider(Configuration conf, String alias) {
     List<Object> providerObjList = getCredentialProviders(conf);
-
+    
     if (null == providerObjList) {
       return null;
     }
-
+    
     for (Object providerObj : providerObjList) {
       try {
         // Invoke CredentialProvider.getCredentialEntry(String)
         Object credEntryObj = getCredentialEntryMethod.invoke(providerObj, alias);
-
+        
         if (null == credEntryObj) {
           continue;
         }
-
+        
         // Then, CredentialEntry.getCredential()
         Object credential = getCredentialMethod.invoke(credEntryObj);
-
+        
         return (char[]) credential;
       } catch (IllegalArgumentException e) {
         log.warn("Failed to get credential from {}", providerObj, e);
@@ -228,28 +228,28 @@ public class CredentialProviderFactoryShim {
         continue;
       }
     }
-
+    
     log.warn("Could not extract credential from providers");
-
+    
     return null;
   }
-
+  
   @SuppressWarnings("unchecked")
   protected static List<String> getAliasesFromHadoopCredentialProvider(Configuration conf) {
     List<Object> providerObjList = getCredentialProviders(conf);
-
+    
     if (null == providerObjList) {
       log.debug("Failed to get CredProviders");
       return Collections.emptyList();
     }
-
+    
     ArrayList<String> aliases = new ArrayList<String>();
     for (Object providerObj : providerObjList) {
       if (null != providerObj) {
         Object aliasesObj;
         try {
           aliasesObj = getAliasesMethod.invoke(providerObj);
-
+          
           if (null != aliasesObj && aliasesObj instanceof List) {
             try {
               aliases.addAll((List<String>) aliasesObj);
@@ -258,7 +258,7 @@ public class CredentialProviderFactoryShim {
               continue;
             }
           }
-
+          
         } catch (IllegalArgumentException e) {
           log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, providerObj, e);
           continue;
@@ -271,10 +271,24 @@ public class CredentialProviderFactoryShim {
         }
       }
     }
-
+    
     return aliases;
   }
-
+  
+  /**
+   * Create a Hadoop {@link Configuration} with the appropriate members to access CredentialProviders
+   *
+   * @param credentialProviders
+   *          Comma-separated list of CredentialProvider URLs
+   * @return Configuration to be used for CredentialProvider
+   */
+  public static Configuration getConfiguration(String credentialProviders) {
+    Preconditions.checkNotNull(credentialProviders);
+    final Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credentialProviders);
+    return conf;
+  }
+  
   /**
    * Attempt to extract the password from any configured CredentialsProviders for the given alias. If no providers or credential is found, null is returned.
    *
@@ -289,15 +303,15 @@ public class CredentialProviderFactoryShim {
   public static char[] getValueFromCredentialProvider(Configuration conf, String alias) throws IOException {
     Preconditions.checkNotNull(conf);
     Preconditions.checkNotNull(alias);
-
+    
     if (isHadoopCredentialProviderAvailable()) {
       log.trace("Hadoop CredentialProvider is available, attempting to extract value for {}", alias);
       return getFromHadoopCredentialProvider(conf, alias);
     }
-
+    
     return null;
   }
-
+  
   /**
    * Attempt to extract all aliases from any configured CredentialsProviders.
    *
@@ -309,12 +323,12 @@ public class CredentialProviderFactoryShim {
    */
   public static List<String> getKeys(Configuration conf) throws IOException {
     Preconditions.checkNotNull(conf);
-
+    
     if (isHadoopCredentialProviderAvailable()) {
       log.trace("Hadoop CredentialProvider is available, attempting to extract all aliases");
       return getAliasesFromHadoopCredentialProvider(conf);
     }
-
+    
     return Collections.emptyList();
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1ec33f1a/core/src/test/java/org/apache/accumulo/core/client/security/tokens/CredentialProviderTokenTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/security/tokens/CredentialProviderTokenTest.java b/core/src/test/java/org/apache/accumulo/core/client/security/tokens/CredentialProviderTokenTest.java
new file mode 100644
index 0000000..17d2297
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/security/tokens/CredentialProviderTokenTest.java
@@ -0,0 +1,129 @@
+/*
+ * 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.accumulo.core.client.security.tokens;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Properties;
+import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class CredentialProviderTokenTest {
+
+  private static boolean isCredentialProviderAvailable = false;
+
+  // Keystore contains: {'root.password':'password', 'bob.password':'bob'}
+  private static String keystorePath;
+
+  @BeforeClass
+  public static void setup() {
+    try {
+      Class.forName(CredentialProviderFactoryShim.HADOOP_CRED_PROVIDER_CLASS_NAME);
+      isCredentialProviderAvailable = true;
+    } catch (Exception e) {
+      isCredentialProviderAvailable = false;
+    }
+
+    URL keystoreUrl = CredentialProviderTokenTest.class.getResource("/passwords.jceks");
+    Assert.assertNotNull(keystoreUrl);
+    keystorePath = "jceks://file/" + new File(keystoreUrl.getFile()).getAbsolutePath();
+  }
+
+  @Test
+  public void testPasswordsFromCredentialProvider() throws Exception {
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    CredentialProviderToken token = new CredentialProviderToken("root.password", keystorePath);
+    Assert.assertArrayEquals("password".getBytes(Constants.UTF8), token.getPassword());
+
+    token = new CredentialProviderToken("bob.password", keystorePath);
+    Assert.assertArrayEquals("bob".getBytes(Constants.UTF8), token.getPassword());
+  }
+
+  @Test
+  public void testEqualityAfterInit() throws Exception {
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    CredentialProviderToken token = new CredentialProviderToken("root.password", keystorePath);
+
+    CredentialProviderToken uninitializedToken = new CredentialProviderToken();
+    Properties props = new Properties();
+    props.put(CredentialProviderToken.NAME_PROPERTY, "root.password");
+    props.put(CredentialProviderToken.CREDENTIAL_PROVIDERS_PROPERTY, keystorePath);
+    uninitializedToken.init(props);
+
+    Assert.assertArrayEquals(token.getPassword(), uninitializedToken.getPassword());
+  }
+
+  @Test
+  public void testMissingClassesThrowsException() throws Exception {
+    if (isCredentialProviderAvailable) {
+      return;
+    }
+
+    try {
+      new CredentialProviderToken("root.password", keystorePath);
+      Assert.fail("Should fail to create CredentialProviderToken when classes are not available");
+    } catch (IOException e) {
+      // pass
+    }
+  }
+
+  @Test
+  public void cloneReturnsCorrectObject() throws Exception {
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    CredentialProviderToken token = new CredentialProviderToken("root.password", keystorePath);
+    CredentialProviderToken clone = token.clone();
+
+    Assert.assertEquals(token, clone);
+    Assert.assertArrayEquals(token.getPassword(), clone.getPassword());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void missingProperties() throws Exception {
+    CredentialProviderToken token = new CredentialProviderToken();
+    token.init(new Properties());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void missingNameProperty() throws Exception {
+    CredentialProviderToken token = new CredentialProviderToken();
+    Properties props = new Properties();
+    props.put(CredentialProviderToken.NAME_PROPERTY, "root.password");
+    token.init(props);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void missingProviderProperty() throws Exception {
+    CredentialProviderToken token = new CredentialProviderToken();
+    Properties props = new Properties();
+    props.put(CredentialProviderToken.CREDENTIAL_PROVIDERS_PROPERTY, keystorePath);
+    token.init(props);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1ec33f1a/core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java b/core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java
index 42497a8..100e297 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java
@@ -146,4 +146,12 @@ public class CredentialProviderFactoryShimTest {
 
     Assert.assertNull(CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, "key1"));
   }
+
+  @Test
+  public void testConfigurationCreation() throws IOException {
+    final String path = "jceks://file/tmp/foo.jks";
+    final Configuration actualConf = CredentialProviderFactoryShim.getConfiguration(path);
+    Assert.assertNotNull(actualConf);
+    Assert.assertEquals(path, actualConf.get(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH));
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1ec33f1a/core/src/test/resources/passwords.jceks
----------------------------------------------------------------------
diff --git a/core/src/test/resources/passwords.jceks b/core/src/test/resources/passwords.jceks
new file mode 100755
index 0000000..9bbb6bc
Binary files /dev/null and b/core/src/test/resources/passwords.jceks differ