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/20 23:06:32 UTC

[1/3] git commit: ACCUMULO-3068 Add create/flush methods to CredentialProvider, configure ITs with CP too.

Repository: accumulo
Updated Branches:
  refs/heads/1.6.1-SNAPSHOT 64e5c6ac8 -> 1fe275c22
  refs/heads/master fbe6ff36a -> 5734ab133


ACCUMULO-3068 Add create/flush methods to CredentialProvider, configure ITs with CP too.

Add some more reflection to work create and flush CredentialProviders. Changes
MAC to add support for using a CredentialProvider for Sensitive properties.


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

Branch: refs/heads/1.6.1-SNAPSHOT
Commit: 1fe275c22d2c18a20e6fd6e19c023150f93a40b0
Parents: 64e5c6a
Author: Josh Elser <el...@apache.org>
Authored: Wed Aug 20 16:44:09 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Aug 20 16:44:09 2014 -0400

----------------------------------------------------------------------
 .../conf/CredentialProviderFactoryShim.java     | 186 +++++++++++++++----
 .../conf/CredentialProviderFactoryShimTest.java |  24 ++-
 .../impl/MiniAccumuloConfigImpl.java            |  67 ++++++-
 test/pom.xml                                    |   2 +
 .../accumulo/test/functional/AbstractMacIT.java |   3 +
 5 files changed, 240 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/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 513b8cb..91dcc54 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,32 +37,36 @@ import org.slf4j.LoggerFactory;
  */
 public class CredentialProviderFactoryShim {
   private static final Logger log = LoggerFactory.getLogger(CredentialProviderFactoryShim.class);
-  
+
   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_PROVIDER_CREATE_CREDENTIAL_ENTRY_METHOD_NAME = "createCredentialEntry";
+  public static final String HADOOP_CRED_PROVIDER_FLUSH_METHOD_NAME = "flush";
+
   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 Method createCredentialEntryMethod = null;
+  private static Method flushMethod = 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.
    *
    * @return True if the CredentialProvider classes/methods are available, false otherwise.
    */
-  protected static synchronized boolean isHadoopCredentialProviderAvailable() {
+  public static synchronized boolean isHadoopCredentialProviderAvailable() {
     // If we already found the class
     if (null != hadoopClassesAvailable) {
       // Make sure everything is initialized as expected
@@ -74,9 +78,9 @@ public class CredentialProviderFactoryShim {
         return false;
       }
     }
-    
+
     hadoopClassesAvailable = false;
-    
+
     // Load Hadoop CredentialProviderFactory
     Class<?> hadoopCredProviderFactoryClz = null;
     try {
@@ -85,7 +89,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 +100,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 +111,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 +120,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 +131,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 +142,29 @@ 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 CredentialProvider.createCredentialEntry(String, char[])
+    try {
+      createCredentialEntryMethod = hadoopCredProviderClz.getMethod(HADOOP_CRED_PROVIDER_CREATE_CREDENTIAL_ENTRY_METHOD_NAME, String.class, char[].class);
+    } catch (SecurityException e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_CREATE_CREDENTIAL_ENTRY_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    } catch (NoSuchMethodException e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_CREATE_CREDENTIAL_ENTRY_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    }
+
+    // Load Hadoop CredentialProvider.flush()
+    try {
+      flushMethod = hadoopCredProviderClz.getMethod(HADOOP_CRED_PROVIDER_FLUSH_METHOD_NAME);
+    } catch (SecurityException e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_FLUSH_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    } catch (NoSuchMethodException e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_FLUSH_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    }
+
     // Load Hadoop CredentialEntry
     Class<?> hadoopCredentialEntryClz = null;
     try {
@@ -147,7 +173,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 +184,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 +213,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 +222,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 for {} from {}", alias, providerObj, e);
@@ -231,26 +257,26 @@ public class CredentialProviderFactoryShim {
 
     // If we didn't find it, this isn't an error, it just wasn't set in the CredentialProvider
     log.trace("Could not extract credential for {} from providers", alias);
-    
+
     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);
@@ -259,7 +285,7 @@ public class CredentialProviderFactoryShim {
               continue;
             }
           }
-          
+
         } catch (IllegalArgumentException e) {
           log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, providerObj, e);
           continue;
@@ -272,10 +298,10 @@ public class CredentialProviderFactoryShim {
         }
       }
     }
-    
+
     return aliases;
   }
-  
+
   /**
    * Create a Hadoop {@link Configuration} with the appropriate members to access CredentialProviders
    *
@@ -289,7 +315,7 @@ public class CredentialProviderFactoryShim {
     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.
    *
@@ -304,15 +330,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.
    *
@@ -324,12 +350,92 @@ 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();
   }
+
+  /**
+   * Create a CredentialEntry using the configured Providers. If multiple CredentialProviders are configured, the first will be used.
+   *
+   * @param conf
+   *          Configuration for the CredentialProvider
+   * @param name
+   *          CredentialEntry name (alias)
+   * @param credential
+   *          The credential
+   * @throws IOException
+   */
+  public static void createEntry(Configuration conf, String name, char[] credential) throws IOException {
+    Preconditions.checkNotNull(conf);
+    Preconditions.checkNotNull(name);
+    Preconditions.checkNotNull(credential);
+
+    if (!isHadoopCredentialProviderAvailable()) {
+      log.warn("Hadoop CredentialProvider is not available");
+      return;
+    }
+
+    List<Object> providers = getCredentialProviders(conf);
+    if (null == providers) {
+      throw new IOException("Could not fetch any CredentialProviders, is the implementation available?");
+    }
+
+    if (1 != providers.size()) {
+      log.warn("Found more than one CredentialProvider. Using first provider found");
+    }
+
+    Object provider = providers.get(0);
+    createEntryInProvider(provider, name, credential);
+  }
+
+  /**
+   * Create a CredentialEntry with the give name and credential in the credentialProvider. The credentialProvider argument must be an instance of Hadoop
+   * CredentialProvider.
+   *
+   * @param credentialProvider
+   *          Instance of CredentialProvider
+   * @param name
+   *          CredentialEntry name (alias)
+   * @param credential
+   *          The credential to store
+   * @throws IOException
+   */
+  public static void createEntryInProvider(Object credentialProvider, String name, char[] credential) throws IOException {
+    Preconditions.checkNotNull(credentialProvider);
+    Preconditions.checkNotNull(name);
+    Preconditions.checkNotNull(credential);
+
+    if (!isHadoopCredentialProviderAvailable()) {
+      log.warn("Hadoop CredentialProvider is not available");
+      return;
+    }
+
+    try {
+      createCredentialEntryMethod.invoke(credentialProvider, name, credential);
+    } catch (IllegalArgumentException e) {
+      log.warn("Failed to invoke createCredentialEntry method on CredentialProvider", e);
+      return;
+    } catch (IllegalAccessException e) {
+      log.warn("Failed to invoke createCredentialEntry method", e);
+      return;
+    } catch (InvocationTargetException e) {
+      log.warn("Failed to invoke createCredentialEntry method", e);
+      return;
+    }
+
+    try {
+      flushMethod.invoke(credentialProvider);
+    } catch (IllegalArgumentException e) {
+      log.warn("Failed to invoke flush method on CredentialProvider", e);
+    } catch (IllegalAccessException e) {
+      log.warn("Failed to invoke flush method on CredentialProvider", e);
+    } catch (InvocationTargetException e) {
+      log.warn("Failed to invoke flush method on CredentialProvider", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/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 100e297..5016750 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
@@ -117,7 +117,6 @@ public class CredentialProviderFactoryShimTest {
 
   @Test
   public void testEmptyAndPopulatedKeyStores() throws IOException {
-
     if (!isCredentialProviderAvailable) {
       return;
     }
@@ -154,4 +153,27 @@ public class CredentialProviderFactoryShimTest {
     Assert.assertNotNull(actualConf);
     Assert.assertEquals(path, actualConf.get(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH));
   }
+
+  @Test
+  public void createKeystoreProvider() throws Exception {
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    File targetDir = new File(System.getProperty("user.dir") + "/target");
+    File keystoreFile = new File(targetDir, "create.jks");
+    if (keystoreFile.exists()) {
+      keystoreFile.delete();
+    }
+
+    String providerUrl = "jceks://file" + keystoreFile.getAbsolutePath();
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, providerUrl);
+
+    String alias = "foo";
+    char[] credential = "bar".toCharArray();
+    CredentialProviderFactoryShim.createEntry(conf, alias, credential);
+
+    Assert.assertArrayEquals(credential, CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, alias));
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
index 337eda0..4878967 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -19,13 +19,18 @@ package org.apache.accumulo.minicluster.impl;
 import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.accumulo.cluster.AccumuloConfig;
+import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.server.util.PortUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Logger;
 
 /**
  * Holds configuration for {@link MiniAccumuloClusterImpl}. Required configurations must be passed to constructor(s) and all other configurations are optional.
@@ -33,7 +38,7 @@ import org.apache.accumulo.server.util.PortUtils;
  * @since 1.6.0
  */
 public class MiniAccumuloConfigImpl implements AccumuloConfig {
-
+  private static final Logger log = Logger.getLogger(MiniAccumuloConfigImpl.class);
   private static final String DEFAULT_INSTANCE_SECRET = "DONTTELL";
 
   private File dir = null;
@@ -65,6 +70,8 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
 
   private boolean useMiniDFS = false;
 
+  private boolean useCredentialProvider = false;
+
   private String[] classpathItems = null;
 
   private String[] nativePathItems = null;
@@ -126,6 +133,10 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
       mergePropWithRandomPort(Property.MONITOR_PORT.getKey());
       mergePropWithRandomPort(Property.GC_PORT.getKey());
 
+      if (isUseCredentialProvider()) {
+        updateConfigForCredentialProvider();
+      }
+
       // zookeeper port should be set explicitly in this class, not just on the site config
       if (zooKeeperPort == 0)
         zooKeeperPort = PortUtils.getRandomFreePort();
@@ -135,6 +146,46 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
     return this;
   }
 
+  private void updateConfigForCredentialProvider() {
+    String cpPaths = siteConfig.get(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
+    if (null != cpPaths && !Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getDefaultValue().equals(cpPaths)) {
+      // Already configured
+      return;
+    }
+
+    if (!CredentialProviderFactoryShim.isHadoopCredentialProviderAvailable()) {
+      throw new RuntimeException("Cannot use CredentialProvider when implementation is not available. Be sure to use >=Hadoop-2.6.0");
+    }
+
+    File keystoreFile = new File(getConfDir(), "credential-provider.jks");
+    String keystoreUri = "jceks://file" + keystoreFile.getAbsolutePath();
+    Configuration conf = CredentialProviderFactoryShim.getConfiguration(keystoreUri);
+
+    // Set the URI on the siteCfg
+    siteConfig.put(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), keystoreUri);
+
+    Iterator<Entry<String,String>> entries = siteConfig.entrySet().iterator();
+    while (entries.hasNext()) {
+      Entry<String,String> entry = entries.next();
+
+      // Not a @Sensitive Property, ignore it
+      if (!Property.isSensitive(entry.getKey())) {
+        continue;
+      }
+
+      // Add the @Sensitive Property to the CredentialProvider
+      try {
+        CredentialProviderFactoryShim.createEntry(conf, entry.getKey(), entry.getValue().toCharArray());
+      } catch (IOException e) {
+        log.warn("Attempted to add " + entry.getKey() + " to CredentialProvider but failed", e);
+        continue;
+      }
+
+      // Only remove it from the siteCfg if we succeeded in adding it to the CredentialProvider
+      entries.remove();
+    }
+  }
+
   @SuppressWarnings("deprecation")
   private void setInstanceLocation() {
     mergeProp(Property.INSTANCE_DFS_URI.getKey(), "file:///");
@@ -496,6 +547,20 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
     this.siteConfig.put(p.getKey(), value);
   }
 
+  /**
+   * @return the useCredentialProvider
+   */
+  public boolean isUseCredentialProvider() {
+    return useCredentialProvider;
+  }
+
+  /**
+   * @param useCredentialProvider the useCredentialProvider to set
+   */
+  public void setUseCredentialProvider(boolean useCredentialProvider) {
+    this.useCredentialProvider = useCredentialProvider;
+  }
+
   @Override
   public MiniAccumuloClusterImpl build() throws IOException {
     return new MiniAccumuloClusterImpl(this);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index 7c932d6..daf4fbd 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -201,6 +201,7 @@
           <configuration>
             <systemPropertyVariables>
               <timeout.factor>${timeout.factor}</timeout.factor>
+              <org.apache.accumulo.test.functional.useCredProviderForIT>${useCredProviderForIT}</org.apache.accumulo.test.functional.useCredProviderForIT>
               <org.apache.accumulo.test.functional.useSslForIT>${useSslForIT}</org.apache.accumulo.test.functional.useSslForIT>
             </systemPropertyVariables>
           </configuration>
@@ -210,6 +211,7 @@
           <artifactId>maven-surefire-plugin</artifactId>
           <configuration>
             <systemPropertyVariables>
+              <org.apache.accumulo.test.functional.useCredProviderForIT>${useCredProviderForIT}</org.apache.accumulo.test.functional.useCredProviderForIT>
               <org.apache.accumulo.test.functional.useSslForIT>${useSslForIT}</org.apache.accumulo.test.functional.useSslForIT>
             </systemPropertyVariables>
           </configuration>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java b/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
index 4734558..415682b 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
@@ -85,6 +85,9 @@ public abstract class AbstractMacIT {
     if ("true".equals(System.getProperty("org.apache.accumulo.test.functional.useSslForIT"))) {
       configureForSsl(cfg, folder);
     }
+    if ("true".equals(System.getProperty("org.apache.accumulo.test.functional.useCredProviderForIT"))) {
+      cfg.setUseCredentialProvider(true);
+    }
   }
 
   protected static void configureForSsl(MiniAccumuloConfigImpl cfg, File folder) {


[3/3] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/master
Commit: 5734ab1335dca44c25a090073026f83d7d84679e
Parents: fbe6ff3 1fe275c
Author: Josh Elser <el...@apache.org>
Authored: Wed Aug 20 17:05:12 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Aug 20 17:05:12 2014 -0400

----------------------------------------------------------------------
 .../conf/CredentialProviderFactoryShim.java     | 186 +++++++++++++++----
 .../conf/CredentialProviderFactoryShimTest.java |  24 ++-
 .../impl/MiniAccumuloConfigImpl.java            |  67 ++++++-
 test/pom.xml                                    |   2 +
 .../accumulo/test/functional/AbstractMacIT.java |   3 +
 5 files changed, 240 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5734ab13/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
----------------------------------------------------------------------
diff --cc minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
index 2d1b2d7,4878967..c585ecc
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@@ -125,9 -132,11 +132,13 @@@ public class MiniAccumuloConfigImpl imp
        mergePropWithRandomPort(Property.TSERV_CLIENTPORT.getKey());
        mergePropWithRandomPort(Property.MONITOR_PORT.getKey());
        mergePropWithRandomPort(Property.GC_PORT.getKey());
 +      mergePropWithRandomPort(Property.REPLICATION_RECEIPT_SERVICE_PORT.getKey());
 +      mergePropWithRandomPort(Property.MASTER_REPLICATION_COORDINATOR_PORT.getKey());
  
+       if (isUseCredentialProvider()) {
+         updateConfigForCredentialProvider();
+       }
+ 
        // zookeeper port should be set explicitly in this class, not just on the site config
        if (zooKeeperPort == 0)
          zooKeeperPort = PortUtils.getRandomFreePort();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5734ab13/test/pom.xml
----------------------------------------------------------------------


[2/3] git commit: ACCUMULO-3068 Add create/flush methods to CredentialProvider, configure ITs with CP too.

Posted by el...@apache.org.
ACCUMULO-3068 Add create/flush methods to CredentialProvider, configure ITs with CP too.

Add some more reflection to work create and flush CredentialProviders. Changes
MAC to add support for using a CredentialProvider for Sensitive properties.


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

Branch: refs/heads/master
Commit: 1fe275c22d2c18a20e6fd6e19c023150f93a40b0
Parents: 64e5c6a
Author: Josh Elser <el...@apache.org>
Authored: Wed Aug 20 16:44:09 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Aug 20 16:44:09 2014 -0400

----------------------------------------------------------------------
 .../conf/CredentialProviderFactoryShim.java     | 186 +++++++++++++++----
 .../conf/CredentialProviderFactoryShimTest.java |  24 ++-
 .../impl/MiniAccumuloConfigImpl.java            |  67 ++++++-
 test/pom.xml                                    |   2 +
 .../accumulo/test/functional/AbstractMacIT.java |   3 +
 5 files changed, 240 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/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 513b8cb..91dcc54 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,32 +37,36 @@ import org.slf4j.LoggerFactory;
  */
 public class CredentialProviderFactoryShim {
   private static final Logger log = LoggerFactory.getLogger(CredentialProviderFactoryShim.class);
-  
+
   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_PROVIDER_CREATE_CREDENTIAL_ENTRY_METHOD_NAME = "createCredentialEntry";
+  public static final String HADOOP_CRED_PROVIDER_FLUSH_METHOD_NAME = "flush";
+
   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 Method createCredentialEntryMethod = null;
+  private static Method flushMethod = 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.
    *
    * @return True if the CredentialProvider classes/methods are available, false otherwise.
    */
-  protected static synchronized boolean isHadoopCredentialProviderAvailable() {
+  public static synchronized boolean isHadoopCredentialProviderAvailable() {
     // If we already found the class
     if (null != hadoopClassesAvailable) {
       // Make sure everything is initialized as expected
@@ -74,9 +78,9 @@ public class CredentialProviderFactoryShim {
         return false;
       }
     }
-    
+
     hadoopClassesAvailable = false;
-    
+
     // Load Hadoop CredentialProviderFactory
     Class<?> hadoopCredProviderFactoryClz = null;
     try {
@@ -85,7 +89,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 +100,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 +111,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 +120,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 +131,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 +142,29 @@ 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 CredentialProvider.createCredentialEntry(String, char[])
+    try {
+      createCredentialEntryMethod = hadoopCredProviderClz.getMethod(HADOOP_CRED_PROVIDER_CREATE_CREDENTIAL_ENTRY_METHOD_NAME, String.class, char[].class);
+    } catch (SecurityException e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_CREATE_CREDENTIAL_ENTRY_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    } catch (NoSuchMethodException e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_CREATE_CREDENTIAL_ENTRY_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    }
+
+    // Load Hadoop CredentialProvider.flush()
+    try {
+      flushMethod = hadoopCredProviderClz.getMethod(HADOOP_CRED_PROVIDER_FLUSH_METHOD_NAME);
+    } catch (SecurityException e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_FLUSH_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    } catch (NoSuchMethodException e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_FLUSH_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    }
+
     // Load Hadoop CredentialEntry
     Class<?> hadoopCredentialEntryClz = null;
     try {
@@ -147,7 +173,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 +184,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 +213,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 +222,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 for {} from {}", alias, providerObj, e);
@@ -231,26 +257,26 @@ public class CredentialProviderFactoryShim {
 
     // If we didn't find it, this isn't an error, it just wasn't set in the CredentialProvider
     log.trace("Could not extract credential for {} from providers", alias);
-    
+
     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);
@@ -259,7 +285,7 @@ public class CredentialProviderFactoryShim {
               continue;
             }
           }
-          
+
         } catch (IllegalArgumentException e) {
           log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, providerObj, e);
           continue;
@@ -272,10 +298,10 @@ public class CredentialProviderFactoryShim {
         }
       }
     }
-    
+
     return aliases;
   }
-  
+
   /**
    * Create a Hadoop {@link Configuration} with the appropriate members to access CredentialProviders
    *
@@ -289,7 +315,7 @@ public class CredentialProviderFactoryShim {
     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.
    *
@@ -304,15 +330,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.
    *
@@ -324,12 +350,92 @@ 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();
   }
+
+  /**
+   * Create a CredentialEntry using the configured Providers. If multiple CredentialProviders are configured, the first will be used.
+   *
+   * @param conf
+   *          Configuration for the CredentialProvider
+   * @param name
+   *          CredentialEntry name (alias)
+   * @param credential
+   *          The credential
+   * @throws IOException
+   */
+  public static void createEntry(Configuration conf, String name, char[] credential) throws IOException {
+    Preconditions.checkNotNull(conf);
+    Preconditions.checkNotNull(name);
+    Preconditions.checkNotNull(credential);
+
+    if (!isHadoopCredentialProviderAvailable()) {
+      log.warn("Hadoop CredentialProvider is not available");
+      return;
+    }
+
+    List<Object> providers = getCredentialProviders(conf);
+    if (null == providers) {
+      throw new IOException("Could not fetch any CredentialProviders, is the implementation available?");
+    }
+
+    if (1 != providers.size()) {
+      log.warn("Found more than one CredentialProvider. Using first provider found");
+    }
+
+    Object provider = providers.get(0);
+    createEntryInProvider(provider, name, credential);
+  }
+
+  /**
+   * Create a CredentialEntry with the give name and credential in the credentialProvider. The credentialProvider argument must be an instance of Hadoop
+   * CredentialProvider.
+   *
+   * @param credentialProvider
+   *          Instance of CredentialProvider
+   * @param name
+   *          CredentialEntry name (alias)
+   * @param credential
+   *          The credential to store
+   * @throws IOException
+   */
+  public static void createEntryInProvider(Object credentialProvider, String name, char[] credential) throws IOException {
+    Preconditions.checkNotNull(credentialProvider);
+    Preconditions.checkNotNull(name);
+    Preconditions.checkNotNull(credential);
+
+    if (!isHadoopCredentialProviderAvailable()) {
+      log.warn("Hadoop CredentialProvider is not available");
+      return;
+    }
+
+    try {
+      createCredentialEntryMethod.invoke(credentialProvider, name, credential);
+    } catch (IllegalArgumentException e) {
+      log.warn("Failed to invoke createCredentialEntry method on CredentialProvider", e);
+      return;
+    } catch (IllegalAccessException e) {
+      log.warn("Failed to invoke createCredentialEntry method", e);
+      return;
+    } catch (InvocationTargetException e) {
+      log.warn("Failed to invoke createCredentialEntry method", e);
+      return;
+    }
+
+    try {
+      flushMethod.invoke(credentialProvider);
+    } catch (IllegalArgumentException e) {
+      log.warn("Failed to invoke flush method on CredentialProvider", e);
+    } catch (IllegalAccessException e) {
+      log.warn("Failed to invoke flush method on CredentialProvider", e);
+    } catch (InvocationTargetException e) {
+      log.warn("Failed to invoke flush method on CredentialProvider", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/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 100e297..5016750 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
@@ -117,7 +117,6 @@ public class CredentialProviderFactoryShimTest {
 
   @Test
   public void testEmptyAndPopulatedKeyStores() throws IOException {
-
     if (!isCredentialProviderAvailable) {
       return;
     }
@@ -154,4 +153,27 @@ public class CredentialProviderFactoryShimTest {
     Assert.assertNotNull(actualConf);
     Assert.assertEquals(path, actualConf.get(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH));
   }
+
+  @Test
+  public void createKeystoreProvider() throws Exception {
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    File targetDir = new File(System.getProperty("user.dir") + "/target");
+    File keystoreFile = new File(targetDir, "create.jks");
+    if (keystoreFile.exists()) {
+      keystoreFile.delete();
+    }
+
+    String providerUrl = "jceks://file" + keystoreFile.getAbsolutePath();
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, providerUrl);
+
+    String alias = "foo";
+    char[] credential = "bar".toCharArray();
+    CredentialProviderFactoryShim.createEntry(conf, alias, credential);
+
+    Assert.assertArrayEquals(credential, CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, alias));
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
index 337eda0..4878967 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloConfigImpl.java
@@ -19,13 +19,18 @@ package org.apache.accumulo.minicluster.impl;
 import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.accumulo.cluster.AccumuloConfig;
+import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.server.util.PortUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Logger;
 
 /**
  * Holds configuration for {@link MiniAccumuloClusterImpl}. Required configurations must be passed to constructor(s) and all other configurations are optional.
@@ -33,7 +38,7 @@ import org.apache.accumulo.server.util.PortUtils;
  * @since 1.6.0
  */
 public class MiniAccumuloConfigImpl implements AccumuloConfig {
-
+  private static final Logger log = Logger.getLogger(MiniAccumuloConfigImpl.class);
   private static final String DEFAULT_INSTANCE_SECRET = "DONTTELL";
 
   private File dir = null;
@@ -65,6 +70,8 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
 
   private boolean useMiniDFS = false;
 
+  private boolean useCredentialProvider = false;
+
   private String[] classpathItems = null;
 
   private String[] nativePathItems = null;
@@ -126,6 +133,10 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
       mergePropWithRandomPort(Property.MONITOR_PORT.getKey());
       mergePropWithRandomPort(Property.GC_PORT.getKey());
 
+      if (isUseCredentialProvider()) {
+        updateConfigForCredentialProvider();
+      }
+
       // zookeeper port should be set explicitly in this class, not just on the site config
       if (zooKeeperPort == 0)
         zooKeeperPort = PortUtils.getRandomFreePort();
@@ -135,6 +146,46 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
     return this;
   }
 
+  private void updateConfigForCredentialProvider() {
+    String cpPaths = siteConfig.get(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
+    if (null != cpPaths && !Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getDefaultValue().equals(cpPaths)) {
+      // Already configured
+      return;
+    }
+
+    if (!CredentialProviderFactoryShim.isHadoopCredentialProviderAvailable()) {
+      throw new RuntimeException("Cannot use CredentialProvider when implementation is not available. Be sure to use >=Hadoop-2.6.0");
+    }
+
+    File keystoreFile = new File(getConfDir(), "credential-provider.jks");
+    String keystoreUri = "jceks://file" + keystoreFile.getAbsolutePath();
+    Configuration conf = CredentialProviderFactoryShim.getConfiguration(keystoreUri);
+
+    // Set the URI on the siteCfg
+    siteConfig.put(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), keystoreUri);
+
+    Iterator<Entry<String,String>> entries = siteConfig.entrySet().iterator();
+    while (entries.hasNext()) {
+      Entry<String,String> entry = entries.next();
+
+      // Not a @Sensitive Property, ignore it
+      if (!Property.isSensitive(entry.getKey())) {
+        continue;
+      }
+
+      // Add the @Sensitive Property to the CredentialProvider
+      try {
+        CredentialProviderFactoryShim.createEntry(conf, entry.getKey(), entry.getValue().toCharArray());
+      } catch (IOException e) {
+        log.warn("Attempted to add " + entry.getKey() + " to CredentialProvider but failed", e);
+        continue;
+      }
+
+      // Only remove it from the siteCfg if we succeeded in adding it to the CredentialProvider
+      entries.remove();
+    }
+  }
+
   @SuppressWarnings("deprecation")
   private void setInstanceLocation() {
     mergeProp(Property.INSTANCE_DFS_URI.getKey(), "file:///");
@@ -496,6 +547,20 @@ public class MiniAccumuloConfigImpl implements AccumuloConfig {
     this.siteConfig.put(p.getKey(), value);
   }
 
+  /**
+   * @return the useCredentialProvider
+   */
+  public boolean isUseCredentialProvider() {
+    return useCredentialProvider;
+  }
+
+  /**
+   * @param useCredentialProvider the useCredentialProvider to set
+   */
+  public void setUseCredentialProvider(boolean useCredentialProvider) {
+    this.useCredentialProvider = useCredentialProvider;
+  }
+
   @Override
   public MiniAccumuloClusterImpl build() throws IOException {
     return new MiniAccumuloClusterImpl(this);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index 7c932d6..daf4fbd 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -201,6 +201,7 @@
           <configuration>
             <systemPropertyVariables>
               <timeout.factor>${timeout.factor}</timeout.factor>
+              <org.apache.accumulo.test.functional.useCredProviderForIT>${useCredProviderForIT}</org.apache.accumulo.test.functional.useCredProviderForIT>
               <org.apache.accumulo.test.functional.useSslForIT>${useSslForIT}</org.apache.accumulo.test.functional.useSslForIT>
             </systemPropertyVariables>
           </configuration>
@@ -210,6 +211,7 @@
           <artifactId>maven-surefire-plugin</artifactId>
           <configuration>
             <systemPropertyVariables>
+              <org.apache.accumulo.test.functional.useCredProviderForIT>${useCredProviderForIT}</org.apache.accumulo.test.functional.useCredProviderForIT>
               <org.apache.accumulo.test.functional.useSslForIT>${useSslForIT}</org.apache.accumulo.test.functional.useSslForIT>
             </systemPropertyVariables>
           </configuration>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1fe275c2/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java b/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
index 4734558..415682b 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/AbstractMacIT.java
@@ -85,6 +85,9 @@ public abstract class AbstractMacIT {
     if ("true".equals(System.getProperty("org.apache.accumulo.test.functional.useSslForIT"))) {
       configureForSsl(cfg, folder);
     }
+    if ("true".equals(System.getProperty("org.apache.accumulo.test.functional.useCredProviderForIT"))) {
+      cfg.setUseCredentialProvider(true);
+    }
   }
 
   protected static void configureForSsl(MiniAccumuloConfigImpl cfg, File folder) {