You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2019/06/28 05:10:49 UTC

[accumulo] branch master updated: Clean up Hadoop CredentialProvider utility code (#1238)

This is an automated email from the ASF dual-hosted git repository.

ctubbsii pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
     new 2a99334  Clean up Hadoop CredentialProvider utility code (#1238)
2a99334 is described below

commit 2a9933458617f3d8f2a76c11d60ca6c7f1e80e24
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Fri Jun 28 01:10:44 2019 -0400

    Clean up Hadoop CredentialProvider utility code (#1238)
    
    * Stop using reflection to load CredentialProvider code from Hadoop,
    since we now expect Hadoop3 or later (these were added in 2.6.0)
    * Rename the utility class
    * Simplify some lookup code with streams
---
 .../security/tokens/CredentialProviderToken.java   |   6 +-
 .../core/clientImpl/ClientConfConverter.java       |  19 +-
 .../core/conf/CredentialProviderFactoryShim.java   | 442 ---------------------
 .../core/conf/HadoopCredentialProvider.java        | 159 ++++++++
 .../accumulo/core/conf/SiteConfiguration.java      |   8 +-
 .../tokens/CredentialProviderTokenTest.java        |  38 --
 .../core/clientImpl/ClientContextTest.java         |  30 +-
 ...Test.java => HadoopCredentialProviderTest.java} |  67 +---
 .../accumulo/core/conf/SiteConfigurationTest.java  |  16 -
 .../miniclusterImpl/MiniAccumuloConfigImpl.java    |  14 +-
 10 files changed, 204 insertions(+), 595 deletions(-)

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
index 65f82ca..0547aa4 100644
--- 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
@@ -23,7 +23,7 @@ import java.nio.CharBuffer;
 import java.util.LinkedHashSet;
 import java.util.Set;
 
-import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
+import org.apache.accumulo.core.conf.HadoopCredentialProvider;
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -51,9 +51,9 @@ public class CredentialProviderToken extends PasswordToken {
     this.name = name;
     this.credentialProviders = credentialProviders;
     final Configuration conf = new Configuration();
-    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credentialProviders);
+    HadoopCredentialProvider.setPath(conf, credentialProviders);
 
-    char[] password = CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, name);
+    char[] password = HadoopCredentialProvider.getValue(conf, name);
 
     if (password == null) {
       throw new IOException(
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
index 0550c72..ca012cb 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientConfConverter.java
@@ -25,8 +25,8 @@ import java.util.function.Predicate;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.conf.HadoopCredentialProvider;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.rpc.SaslConnectionParams;
 import org.apache.hadoop.security.authentication.util.KerberosName;
@@ -165,8 +165,7 @@ public class ClientConfConverter {
         if (property.isSensitive()) {
           org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration();
           if (hadoopConf != null) {
-            char[] value =
-                CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key);
+            char[] value = HadoopCredentialProvider.getValue(hadoopConf, key);
             if (value != null) {
               log.trace("Loaded sensitive value for {} from CredentialProvider", key);
               return new String(value);
@@ -177,9 +176,9 @@ public class ClientConfConverter {
           }
         }
 
-        if (config.containsKey(key))
+        if (config.containsKey(key)) {
           return config.getString(key);
-        else {
+        } else {
           // Reconstitute the server kerberos property from the client config
           if (property == Property.GENERAL_KERBEROS_PRINCIPAL) {
             if (config.containsKey(
@@ -203,8 +202,9 @@ public class ClientConfConverter {
         Iterator<String> keyIter = config.getKeys();
         while (keyIter.hasNext()) {
           String key = keyIter.next();
-          if (filter.test(key))
+          if (filter.test(key)) {
             props.put(key, config.getString(key));
+          }
         }
 
         // Two client props that don't exist on the server config. Client doesn't need to know about
@@ -226,13 +226,12 @@ public class ClientConfConverter {
         // Attempt to load sensitive properties from a CredentialProvider, if configured
         org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration();
         if (hadoopConf != null) {
-          for (String key : CredentialProviderFactoryShim.getKeys(hadoopConf)) {
+          for (String key : HadoopCredentialProvider.getKeys(hadoopConf)) {
             if (!Property.isValidPropertyKey(key) || !Property.isSensitive(key)) {
               continue;
             }
             if (filter.test(key)) {
-              char[] value =
-                  CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key);
+              char[] value = HadoopCredentialProvider.getValue(hadoopConf, key);
               if (value != null) {
                 props.put(key, new String(value));
               }
@@ -246,7 +245,7 @@ public class ClientConfConverter {
             config.getString(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
         if (credProviderPaths != null && !credProviderPaths.isEmpty()) {
           org.apache.hadoop.conf.Configuration hConf = new org.apache.hadoop.conf.Configuration();
-          hConf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credProviderPaths);
+          HadoopCredentialProvider.setPath(hConf, credProviderPaths);
           return hConf;
         }
 
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
deleted file mode 100644
index a029a2c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
+++ /dev/null
@@ -1,442 +0,0 @@
-/*
- * 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.conf;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.IOException;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.hadoop.conf.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Shim around Hadoop: tries to use the CredentialProviderFactory provided by hadoop-common, falling
- * back to a copy inside accumulo-core.
- * <p>
- * The CredentialProvider classes only exist in 2.6.0, so, to use them, we have to do a bunch of
- * reflection. This will also help us to continue to support [2.2.0,2.6.0) when 2.6.0 is officially
- * released.
- */
-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;
-
-  // access to cachedProviders should be synchronized when necessary (for example see
-  // getCredentialProviders)
-  private static final ConcurrentHashMap<String,List<Object>> cachedProviders =
-      new ConcurrentHashMap<>();
-
-  /**
-   * 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.
-   */
-  public static synchronized boolean isHadoopCredentialProviderAvailable() {
-    // If we already found the class
-    if (hadoopClassesAvailable != null) {
-      // Make sure everything is initialized as expected
-      // Otherwise we failed to load it
-      return hadoopClassesAvailable && getProvidersMethod != null
-          && hadoopCredProviderFactory != null && getCredentialEntryMethod != null
-          && getCredentialMethod != null;
-    }
-
-    hadoopClassesAvailable = false;
-
-    // Load Hadoop CredentialProviderFactory
-    Class<?> hadoopCredProviderFactoryClz = null;
-    try {
-      hadoopCredProviderFactoryClz = Class.forName(HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME);
-    } catch (ClassNotFoundException e) {
-      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);
-    } catch (SecurityException | NoSuchMethodException e) {
-      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.getDeclaredConstructor().newInstance();
-    } catch (ReflectiveOperationException e) {
-      log.trace("Could not instantiate class {}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, e);
-      return false;
-    }
-
-    // Load Hadoop CredentialProvider
-    Class<?> hadoopCredProviderClz = null;
-    try {
-      hadoopCredProviderClz = Class.forName(HADOOP_CRED_PROVIDER_CLASS_NAME);
-    } catch (ClassNotFoundException e) {
-      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);
-    } catch (SecurityException | NoSuchMethodException e) {
-      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);
-    } catch (SecurityException | NoSuchMethodException e) {
-      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 | 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 | 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 {
-      hadoopCredentialEntryClz = Class.forName(HADOOP_CRED_ENTRY_CLASS_NAME);
-    } catch (ClassNotFoundException e) {
-      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);
-    } catch (SecurityException | NoSuchMethodException e) {
-      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.
-   *
-   * @param conf
-   *          Configuration with Property#GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS defined
-   * @return The List of CredentialProviders, or null if they could not be loaded
-   */
-  @SuppressWarnings("unchecked")
-  protected static List<Object> getCredentialProviders(Configuration conf) {
-    String path = conf.get(CREDENTIAL_PROVIDER_PATH);
-    if (path == null || path.isEmpty()) {
-      return null;
-    }
-
-    List<Object> providersList = cachedProviders.get(path);
-    if (providersList != null) {
-      return providersList;
-    }
-
-    // Call CredentialProviderFactory.getProviders(Configuration)
-    Object providersObj = null;
-    try {
-      providersObj = getProvidersMethod.invoke(hadoopCredProviderFactory, conf);
-    } catch (IllegalArgumentException | InvocationTargetException | IllegalAccessException e) {
-      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 {
-      providersList = (List<Object>) providersObj;
-      List<Object> previousValue = cachedProviders.putIfAbsent(path, providersList);
-      if (previousValue != null) {
-        return previousValue;
-      } else {
-        return providersList;
-      }
-    } catch (ClassCastException e) {
-      log.error("Expected a List from {} method",
-          HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME, e);
-      return null;
-    }
-  }
-
-  protected static char[] getFromHadoopCredentialProvider(Configuration conf, String alias) {
-    List<Object> providerObjList = getCredentialProviders(conf);
-
-    if (providerObjList == null) {
-      return null;
-    }
-
-    for (Object providerObj : providerObjList) {
-      try {
-        // Invoke CredentialProvider.getCredentialEntry(String)
-        Object credEntryObj = getCredentialEntryMethod.invoke(providerObj, alias);
-
-        if (credEntryObj == null) {
-          continue;
-        }
-
-        // Then, CredentialEntry.getCredential()
-        Object credential = getCredentialMethod.invoke(credEntryObj);
-
-        return (char[]) credential;
-      } catch (IllegalArgumentException | InvocationTargetException | IllegalAccessException e) {
-        log.warn("Failed to get credential for {} from {}", alias, providerObj, e);
-        continue;
-      }
-    }
-
-    // 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 (providerObjList == null) {
-      log.debug("Failed to get CredProviders");
-      return Collections.emptyList();
-    }
-
-    ArrayList<String> aliases = new ArrayList<>();
-    for (Object providerObj : providerObjList) {
-      if (providerObj != null) {
-        Object aliasesObj;
-        try {
-          aliasesObj = getAliasesMethod.invoke(providerObj);
-
-          if (aliasesObj != null && aliasesObj instanceof List) {
-            try {
-              aliases.addAll((List<String>) aliasesObj);
-            } catch (ClassCastException e) {
-              log.warn("Could not cast aliases ({}) from {} to a List<String>", aliasesObj,
-                  providerObj, e);
-              continue;
-            }
-          }
-
-        } catch (IllegalArgumentException | InvocationTargetException | IllegalAccessException e) {
-          log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME,
-              providerObj, e);
-          continue;
-        }
-      }
-    }
-
-    return aliases;
-  }
-
-  /**
-   * Adds the Credential Provider configuration elements to the provided {@link Configuration}.
-   *
-   * @param conf
-   *          Existing Hadoop Configuration
-   * @param credentialProviders
-   *          Comma-separated list of CredentialProvider URLs
-   */
-  public static Configuration getConfiguration(Configuration conf, String credentialProviders) {
-    requireNonNull(conf);
-    requireNonNull(credentialProviders);
-    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.
-   *
-   * @param conf
-   *          Configuration for CredentialProvider
-   * @param alias
-   *          Name of CredentialEntry key
-   * @return The credential if found, null otherwise
-   */
-  public static char[] getValueFromCredentialProvider(Configuration conf, String alias) {
-    requireNonNull(conf);
-    requireNonNull(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.
-   *
-   * @param conf
-   *          Configuration for the CredentialProvider
-   * @return A list of aliases. An empty list if no CredentialProviders are configured, or the
-   *         providers are empty.
-   */
-  public static List<String> getKeys(Configuration conf) {
-    requireNonNull(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
-   */
-  public static void createEntry(Configuration conf, String name, char[] credential)
-      throws IOException {
-    requireNonNull(conf);
-    requireNonNull(name);
-    requireNonNull(credential);
-
-    if (!isHadoopCredentialProviderAvailable()) {
-      log.warn("Hadoop CredentialProvider is not available");
-      return;
-    }
-
-    List<Object> providers = getCredentialProviders(conf);
-    if (providers == null) {
-      throw new IOException(
-          "Could not fetch any CredentialProviders, is the implementation available?");
-    }
-
-    if (providers.size() != 1) {
-      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
-   */
-  public static void createEntryInProvider(Object credentialProvider, String name,
-      char[] credential) {
-    requireNonNull(credentialProvider);
-    requireNonNull(name);
-    requireNonNull(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 | InvocationTargetException e) {
-      log.warn("Failed to invoke createCredentialEntry method", e);
-      return;
-    }
-
-    try {
-      flushMethod.invoke(credentialProvider);
-    } catch (IllegalArgumentException | InvocationTargetException | IllegalAccessException e) {
-      log.warn("Failed to invoke flush method on CredentialProvider", e);
-    }
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/HadoopCredentialProvider.java b/core/src/main/java/org/apache/accumulo/core/conf/HadoopCredentialProvider.java
new file mode 100644
index 0000000..58f4bfc
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/conf/HadoopCredentialProvider.java
@@ -0,0 +1,159 @@
+/*
+ * 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.conf;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Shim around Hadoop's CredentialProviderFactory provided by hadoop-common.
+ */
+public class HadoopCredentialProvider {
+  private static final Logger log = LoggerFactory.getLogger(HadoopCredentialProvider.class);
+
+  private static final String CREDENTIAL_PROVIDER_PATH = "hadoop.security.credential.provider.path";
+
+  // access to cachedProviders should be synchronized when necessary
+  private static final ConcurrentHashMap<String,List<CredentialProvider>> cachedProviders =
+      new ConcurrentHashMap<>();
+
+  /**
+   * Set the Hadoop Credential Provider path in the provided Hadoop Configuration.
+   *
+   * @param conf
+   *          the Hadoop Configuration object
+   * @param path
+   *          the credential provider paths to set
+   */
+  public static void setPath(Configuration conf, String path) {
+    conf.set(CREDENTIAL_PROVIDER_PATH, path);
+  }
+
+  /**
+   * Fetch/cache the configured providers.
+   *
+   * @return The List of CredentialProviders, or null if they could not be loaded
+   */
+  private static List<CredentialProvider> getProviders(Configuration conf) {
+    String path = conf.get(CREDENTIAL_PROVIDER_PATH);
+    if (path == null || path.isEmpty()) {
+      log.debug("Failed to get CredentialProviders; no provider path specified");
+      return null;
+    }
+    final List<CredentialProvider> providers;
+    try {
+      providers = CredentialProviderFactory.getProviders(conf);
+    } catch (IOException e) {
+      log.warn("Exception invoking CredentialProviderFactory.getProviders(conf)", e);
+      return null;
+    }
+    return cachedProviders.computeIfAbsent(path, p -> providers);
+  }
+
+  /**
+   * Attempt to extract the password from any configured CredentialProviders for the given alias. If
+   * no providers or credential is found, null is returned.
+   *
+   * @param conf
+   *          Configuration for CredentialProvider
+   * @param alias
+   *          Name of CredentialEntry key
+   * @return The credential if found, null otherwise
+   */
+  public static char[] getValue(Configuration conf, String alias) {
+    requireNonNull(alias);
+    List<CredentialProvider> providerList = getProviders(requireNonNull(conf));
+    return providerList == null ? null : providerList.stream().map(provider -> {
+      try {
+        return provider.getCredentialEntry(alias);
+      } catch (IOException e) {
+        log.warn("Failed to call getCredentialEntry(alias) for provider {}", provider, e);
+        return null;
+      }
+    }).filter(Objects::nonNull).map(entry -> entry.getCredential()).findFirst().orElseGet(() -> {
+      // 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;
+    });
+  }
+
+  /**
+   * Attempt to extract all aliases from any configured CredentialProviders.
+   *
+   * @param conf
+   *          Configuration for the CredentialProvider
+   * @return A list of aliases. An empty list if no CredentialProviders are configured, or the
+   *         providers are empty.
+   */
+  public static List<String> getKeys(Configuration conf) {
+    List<CredentialProvider> providerList = getProviders(requireNonNull(conf));
+    return providerList == null ? Collections.emptyList()
+        : providerList.stream().flatMap(provider -> {
+          List<String> aliases = null;
+          try {
+            aliases = provider.getAliases();
+          } catch (IOException e) {
+            log.warn("Problem getting aliases from provider {}", provider, e);
+          }
+          return aliases == null ? Stream.empty() : aliases.stream();
+        }).collect(Collectors.toList());
+  }
+
+  /**
+   * 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
+   */
+  public static void createEntry(Configuration conf, String name, char[] credential)
+      throws IOException {
+    requireNonNull(conf);
+    requireNonNull(name);
+    requireNonNull(credential);
+
+    List<CredentialProvider> providers = getProviders(conf);
+    if (providers == null || providers.isEmpty()) {
+      throw new IOException("Could not fetch any CredentialProviders");
+    }
+
+    CredentialProvider provider = providers.get(0);
+    if (providers.size() != 1) {
+      log.warn("Found more than one CredentialProvider. Using first provider found ({})", provider);
+    }
+    provider.createCredentialEntry(name, credential);
+    provider.flush();
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
index d735622..04ffb53 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
@@ -110,11 +110,10 @@ public class SiteConfiguration extends AccumuloConfiguration {
     String credProvider = result.get(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
     if (credProvider != null) {
       org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration();
-      hadoopConf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credProvider);
+      HadoopCredentialProvider.setPath(hadoopConf, credProvider);
       for (Property property : Property.values()) {
         if (property.isSensitive()) {
-          char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf,
-              property.getKey());
+          char[] value = HadoopCredentialProvider.getValue(hadoopConf, property.getKey());
           if (value != null) {
             result.put(property.getKey(), new String(value));
           }
@@ -198,8 +197,9 @@ public class SiteConfiguration extends AccumuloConfiguration {
       parent.getProperties(props, filter);
     }
     config.keySet().forEach(k -> {
-      if (filter.test(k))
+      if (filter.test(k)) {
         props.put(k, config.get(k));
+      }
     });
   }
 }
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
index 5423fb2..4b3b0fd 100644
--- 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
@@ -20,14 +20,11 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.IOException;
 import java.net.URL;
 
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Properties;
-import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -35,8 +32,6 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 public class CredentialProviderTokenTest {
 
-  private static boolean isCredentialProviderAvailable = false;
-
   // Keystore contains: {'root.password':'password', 'bob.password':'bob'}
   private static String keystorePath;
 
@@ -44,13 +39,6 @@ public class CredentialProviderTokenTest {
       justification = "keystoreUrl location isn't provided by user input")
   @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");
     assertNotNull(keystoreUrl);
     keystorePath = "jceks://file/" + new File(keystoreUrl.getFile()).getAbsolutePath();
@@ -58,10 +46,6 @@ public class CredentialProviderTokenTest {
 
   @Test
   public void testPasswordsFromCredentialProvider() throws Exception {
-    if (!isCredentialProviderAvailable) {
-      return;
-    }
-
     CredentialProviderToken token = new CredentialProviderToken("root.password", keystorePath);
     assertEquals("root.password", token.getName());
     assertEquals(keystorePath, token.getCredentialProviders());
@@ -73,10 +57,6 @@ public class CredentialProviderTokenTest {
 
   @Test
   public void testEqualityAfterInit() throws Exception {
-    if (!isCredentialProviderAvailable) {
-      return;
-    }
-
     CredentialProviderToken token = new CredentialProviderToken("root.password", keystorePath);
 
     CredentialProviderToken uninitializedToken = new CredentialProviderToken();
@@ -89,25 +69,7 @@ public class CredentialProviderTokenTest {
   }
 
   @Test
-  public void testMissingClassesThrowsException() {
-    if (isCredentialProviderAvailable) {
-      return;
-    }
-
-    try {
-      new CredentialProviderToken("root.password", keystorePath);
-      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();
 
diff --git a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientContextTest.java b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientContextTest.java
index d76fbe6..fc33613 100644
--- a/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientContextTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/clientImpl/ClientContextTest.java
@@ -27,7 +27,6 @@ import java.util.Map;
 import java.util.Properties;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
 import org.apache.accumulo.core.conf.Property;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -36,7 +35,6 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 public class ClientContextTest {
 
-  private static boolean isCredentialProviderAvailable = false;
   private static final String keystoreName = "/site-cfg.jceks";
 
   // site-cfg.jceks={'ignored.property'=>'ignored', 'instance.secret'=>'mysecret',
@@ -47,20 +45,9 @@ public class ClientContextTest {
       justification = "provided keystoreUrl path isn't user provided")
   @BeforeClass
   public static void setUpBeforeClass() {
-    try {
-      Class.forName(CredentialProviderFactoryShim.HADOOP_CRED_PROVIDER_CLASS_NAME);
-      isCredentialProviderAvailable = true;
-    } catch (Exception e) {
-      isCredentialProviderAvailable = false;
-    }
-
-    if (isCredentialProviderAvailable) {
-      URL keystoreUrl = ClientContextTest.class.getResource(keystoreName);
-
-      assertNotNull("Could not find " + keystoreName, keystoreUrl);
-
-      keystore = new File(keystoreUrl.getFile());
-    }
+    URL keystoreUrl = ClientContextTest.class.getResource(keystoreName);
+    assertNotNull("Could not find " + keystoreName, keystoreUrl);
+    keystore = new File(keystoreUrl.getFile());
   }
 
   protected String getKeyStoreUrl(File absoluteFilePath) {
@@ -69,10 +56,6 @@ public class ClientContextTest {
 
   @Test
   public void loadSensitivePropertyFromCredentialProvider() {
-    if (!isCredentialProviderAvailable) {
-      return;
-    }
-
     String absPath = getKeyStoreUrl(keystore);
     Properties props = new Properties();
     props.setProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), absPath);
@@ -82,9 +65,6 @@ public class ClientContextTest {
 
   @Test
   public void defaultValueForSensitiveProperty() {
-    if (!isCredentialProviderAvailable) {
-      return;
-    }
     Properties props = new Properties();
     AccumuloConfiguration accClientConf = ClientConfConverter.toAccumuloConf(props);
     assertEquals(Property.INSTANCE_SECRET.getDefaultValue(),
@@ -93,10 +73,6 @@ public class ClientContextTest {
 
   @Test
   public void sensitivePropertiesIncludedInProperties() {
-    if (!isCredentialProviderAvailable) {
-      return;
-    }
-
     String absPath = getKeyStoreUrl(keystore);
     Properties clientProps = new Properties();
     clientProps.setProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), absPath);
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java b/core/src/test/java/org/apache/accumulo/core/conf/HadoopCredentialProviderTest.java
similarity index 70%
rename from core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java
rename to core/src/test/java/org/apache/accumulo/core/conf/HadoopCredentialProviderTest.java
index b230f3e..c2c9890 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/HadoopCredentialProviderTest.java
@@ -20,7 +20,6 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertSame;
 
 import java.io.File;
 import java.net.URL;
@@ -34,7 +33,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -43,11 +41,10 @@ import org.slf4j.LoggerFactory;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "paths not set by user input")
-public class CredentialProviderFactoryShimTest {
+public class HadoopCredentialProviderTest {
 
   private static final Configuration hadoopConf = new Configuration();
-  private static final Logger log =
-      LoggerFactory.getLogger(CredentialProviderFactoryShimTest.class);
+  private static final Logger log = LoggerFactory.getLogger(HadoopCredentialProviderTest.class);
 
   private static final String populatedKeyStoreName = "/accumulo.jceks",
       emptyKeyStoreName = "/empty.jceks";
@@ -55,16 +52,9 @@ public class CredentialProviderFactoryShimTest {
 
   @BeforeClass
   public static void checkCredentialProviderAvailable() {
-    try {
-      Class.forName(CredentialProviderFactoryShim.HADOOP_CRED_PROVIDER_CLASS_NAME);
-    } catch (Exception e) {
-      // If we can't load the credential provider class, don't run the tests
-      Assume.assumeNoException(e);
-    }
-
     URL populatedKeyStoreUrl =
-        CredentialProviderFactoryShimTest.class.getResource(populatedKeyStoreName),
-        emptyKeyStoreUrl = CredentialProviderFactoryShimTest.class.getResource(emptyKeyStoreName);
+        HadoopCredentialProviderTest.class.getResource(populatedKeyStoreName),
+        emptyKeyStoreUrl = HadoopCredentialProviderTest.class.getResource(emptyKeyStoreName);
 
     assertNotNull("Could not find " + populatedKeyStoreName, populatedKeyStoreUrl);
     assertNotNull("Could not find " + emptyKeyStoreName, emptyKeyStoreUrl);
@@ -79,22 +69,21 @@ public class CredentialProviderFactoryShimTest {
 
   @Test(expected = NullPointerException.class)
   public void testNullConfigOnGetValue() {
-    CredentialProviderFactoryShim.getValueFromCredentialProvider(null, "alias");
+    HadoopCredentialProvider.getValue(null, "alias");
   }
 
   @Test(expected = NullPointerException.class)
   public void testNullAliasOnGetValue() {
-    CredentialProviderFactoryShim.getValueFromCredentialProvider(new Configuration(false), null);
+    HadoopCredentialProvider.getValue(new Configuration(false), null);
   }
 
   protected void checkCredentialProviders(Configuration conf, Map<String,String> expectation) {
-    List<String> keys = CredentialProviderFactoryShim.getKeys(conf);
+    List<String> keys = HadoopCredentialProvider.getKeys(conf);
     assertNotNull(keys);
 
     assertEquals(expectation.keySet(), new HashSet<>(keys));
     for (String expectedKey : keys) {
-      char[] value =
-          CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, expectedKey);
+      char[] value = HadoopCredentialProvider.getValue(conf, expectedKey);
       assertNotNull(value);
       assertEquals(expectation.get(expectedKey), new String(value));
     }
@@ -104,7 +93,7 @@ public class CredentialProviderFactoryShimTest {
   public void testExtractFromProvider() {
     String absPath = getKeyStoreUrl(populatedKeyStore);
     Configuration conf = new Configuration();
-    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, absPath);
+    HadoopCredentialProvider.setPath(conf, absPath);
     Map<String,String> expectations = new HashMap<>();
     expectations.put("key1", "value1");
     expectations.put("key2", "value2");
@@ -116,7 +105,7 @@ public class CredentialProviderFactoryShimTest {
   public void testEmptyKeyStoreParses() {
     String absPath = getKeyStoreUrl(emptyKeyStore);
     Configuration conf = new Configuration();
-    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, absPath);
+    HadoopCredentialProvider.setPath(conf, absPath);
 
     checkCredentialProviders(conf, new HashMap<>());
   }
@@ -126,8 +115,7 @@ public class CredentialProviderFactoryShimTest {
     String populatedAbsPath = getKeyStoreUrl(populatedKeyStore),
         emptyAbsPath = getKeyStoreUrl(emptyKeyStore);
     Configuration conf = new Configuration();
-    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH,
-        populatedAbsPath + "," + emptyAbsPath);
+    HadoopCredentialProvider.setPath(conf, populatedAbsPath + "," + emptyAbsPath);
     Map<String,String> expectations = new HashMap<>();
     expectations.put("key1", "value1");
     expectations.put("key2", "value2");
@@ -138,21 +126,21 @@ public class CredentialProviderFactoryShimTest {
   @Test
   public void testNonExistentClassesDoesntFail() {
     Configuration conf = new Configuration();
-    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, "jceks://file/foo/bar.jceks");
-    List<String> keys = CredentialProviderFactoryShim.getKeys(conf);
+    HadoopCredentialProvider.setPath(conf, "jceks://file/foo/bar.jceks");
+    List<String> keys = HadoopCredentialProvider.getKeys(conf);
     assertNotNull(keys);
     assertEquals(Collections.emptyList(), keys);
 
-    assertNull(CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, "key1"));
+    assertNull(HadoopCredentialProvider.getValue(conf, "key1"));
   }
 
   @Test
   public void testConfigurationCreation() {
     final String path = "jceks://file/tmp/foo.jks";
-    final Configuration actualConf =
-        CredentialProviderFactoryShim.getConfiguration(hadoopConf, path);
+    final Configuration actualConf = hadoopConf;
+    HadoopCredentialProvider.setPath(actualConf, path);
     assertNotNull(actualConf);
-    assertEquals(path, actualConf.get(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH));
+    assertEquals(path, actualConf.get("hadoop.security.credential.provider.path"));
   }
 
   @Test
@@ -167,14 +155,13 @@ public class CredentialProviderFactoryShimTest {
 
     String providerUrl = "jceks://file" + keystoreFile.getAbsolutePath();
     Configuration conf = new Configuration();
-    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, providerUrl);
+    HadoopCredentialProvider.setPath(conf, providerUrl);
 
     String alias = "foo";
     char[] credential = "bar".toCharArray();
-    CredentialProviderFactoryShim.createEntry(conf, alias, credential);
+    HadoopCredentialProvider.createEntry(conf, alias, credential);
 
-    assertArrayEquals(credential,
-        CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, alias));
+    assertArrayEquals(credential, HadoopCredentialProvider.getValue(conf, alias));
   }
 
   @Test
@@ -197,8 +184,8 @@ public class CredentialProviderFactoryShimTest {
       // Put the populated keystore in hdfs
       dfs.copyFromLocalFile(new Path(populatedKeyStore.toURI()), destPath);
 
-      Configuration cpConf = CredentialProviderFactoryShim.getConfiguration(dfsConfiguration,
-          "jceks://hdfs/accumulo.jceks");
+      Configuration cpConf = dfsConfiguration;
+      HadoopCredentialProvider.setPath(cpConf, "jceks://hdfs/accumulo.jceks");
 
       // The values in the keystore
       Map<String,String> expectations = new HashMap<>();
@@ -211,14 +198,4 @@ public class CredentialProviderFactoryShimTest {
     }
   }
 
-  @Test
-  public void existingConfigurationReturned() {
-    Configuration conf = new Configuration(false);
-    conf.set("foo", "bar");
-    Configuration conf2 =
-        CredentialProviderFactoryShim.getConfiguration(conf, "jceks:///file/accumulo.jceks");
-    // Same object
-    assertSame(conf, conf2);
-    assertEquals("bar", conf.get("foo"));
-  }
 }
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/SiteConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/conf/SiteConfigurationTest.java
index ada7c55..41a9a46 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/SiteConfigurationTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/SiteConfigurationTest.java
@@ -25,7 +25,6 @@ import java.net.URL;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableMap;
@@ -33,27 +32,12 @@ import com.google.common.collect.ImmutableMap;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 public class SiteConfigurationTest {
-  private static boolean isCredentialProviderAvailable;
-
-  @BeforeClass
-  public static void checkCredentialProviderAvailable() {
-    try {
-      Class.forName(CredentialProviderFactoryShim.HADOOP_CRED_PROVIDER_CLASS_NAME);
-      isCredentialProviderAvailable = true;
-    } catch (Exception e) {
-      isCredentialProviderAvailable = false;
-    }
-  }
 
   @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN",
       justification = "path to keystore not provided by user input")
   @Test
   public void testOnlySensitivePropertiesExtractedFromCredentialProvider()
       throws SecurityException {
-    if (!isCredentialProviderAvailable) {
-      return;
-    }
-
     // site-cfg.jceks={'ignored.property'=>'ignored', 'instance.secret'=>'mysecret',
     // 'general.rpc.timeout'=>'timeout'}
     URL keystore = SiteConfigurationTest.class.getResource("/site-cfg.jceks");
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
index aa2872e..7b8c9e1 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
@@ -26,7 +26,7 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.CredentialProviderFactoryShim;
+import org.apache.accumulo.core.conf.HadoopCredentialProvider;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.minicluster.MemoryUnit;
@@ -199,15 +199,10 @@ public class MiniAccumuloConfigImpl {
       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(getHadoopConfiguration(), keystoreUri);
+    Configuration conf = getHadoopConfiguration();
+    HadoopCredentialProvider.setPath(conf, keystoreUri);
 
     // Set the URI on the siteCfg
     siteConfig.put(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), keystoreUri);
@@ -223,8 +218,7 @@ public class MiniAccumuloConfigImpl {
 
       // Add the @Sensitive Property to the CredentialProvider
       try {
-        CredentialProviderFactoryShim.createEntry(conf, entry.getKey(),
-            entry.getValue().toCharArray());
+        HadoopCredentialProvider.createEntry(conf, entry.getKey(), entry.getValue().toCharArray());
       } catch (IOException e) {
         log.warn("Attempted to add " + entry.getKey() + " to CredentialProvider but failed", e);
         continue;