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/01 19:31:45 UTC

[1/3] git commit: ACCUMULO-2464 Reflect around CredentialProvider* to pull sensitive configuration values not from plaintext-sources

Repository: accumulo
Updated Branches:
  refs/heads/1.6.1-SNAPSHOT 3a93bad37 -> fb450ffb2
  refs/heads/master 8abe4df6c -> db72ba358


ACCUMULO-2464 Reflect around CredentialProvider* to pull sensitive configuration values not from plaintext-sources

The CredentialProvider classes are a new introduction in 2.6.0 designed to provide a common
interface, via Hadoop Common, that downstream processes can leverage which are "more secure"
than storing them in a plaintext-readable Configuration file (like the common practice of
Hadoop Configuration "site" XML files).

CredentialProviderFactoryShim uses reflection to:

1) Find if these new classes are available
2) Instantiate them, and the Methods we need to invoke
3) And, invoke them with the given information

This provides us with a backwards and forwards compatibile class that allows early
adopters to use 2.6.0 features, and, when >=2.6.0 is the "norm", allow users to continue
to use <2.6.0 transparently.


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

Branch: refs/heads/1.6.1-SNAPSHOT
Commit: fb450ffb2dd310624b91d2850f9bc416ac84d05b
Parents: 3a93bad
Author: Josh Elser <el...@apache.org>
Authored: Thu Jul 31 19:00:52 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri Aug 1 12:04:31 2014 -0400

----------------------------------------------------------------------
 .../conf/CredentialProviderFactoryShim.java     | 320 +++++++++++++++++++
 .../org/apache/accumulo/core/conf/Property.java |   1 +
 .../accumulo/core/conf/SiteConfiguration.java   |  53 ++-
 .../conf/CredentialProviderFactoryShimTest.java | 149 +++++++++
 core/src/test/resources/accumulo.jceks          | Bin 0 -> 946 bytes
 core/src/test/resources/empty.jceks             | Bin 0 -> 32 bytes
 6 files changed, 522 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb450ffb/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
new file mode 100644
index 0000000..1bf3ce6
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
@@ -0,0 +1,320 @@
+/*
+ * 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 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 jline.internal.Preconditions;
+
+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);
+
+  protected static final String HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME = "org.apache.hadoop.security.alias.JavaKeyStoreProvider$Factory";
+  protected static final String HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME = "getProviders";
+
+  protected static final String HADOOP_CRED_PROVIDER_CLASS_NAME = "org.apache.hadoop.security.alias.CredentialProvider";
+  protected static final String HADOOP_CRED_PROVIDER_GET_CREDENTIAL_ENTRY_METHOD_NAME = "getCredentialEntry";
+  protected static final String HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME = "getAliases";
+
+  protected static final String HADOOP_CRED_ENTRY_CLASS_NAME = "org.apache.hadoop.security.alias.CredentialProvider$CredentialEntry";
+  protected static final String HADOOP_CRED_ENTRY_GET_CREDENTIAL_METHOD_NAME = "getCredential";
+
+  public static final String CREDENTIAL_PROVIDER_PATH = "hadoop.security.credential.provider.path";
+
+  private static Object hadoopCredProviderFactory = null;
+  private static Method getProvidersMethod = null;
+  private static Method getAliasesMethod = null;
+  private static Method getCredentialEntryMethod = null;
+  private static Method getCredentialMethod = null;
+  private static Boolean hadoopClassesAvailable = null;
+
+  /**
+   * Determine if we can load the necessary CredentialProvider classes. Only loaded the first time, so subsequent invocations of this method should return fast.
+   *
+   * @return True if the CredentialProvider classes/methods are available, false otherwise.
+   */
+  protected static synchronized boolean isHadoopCredentialProviderAvailable() {
+    // If we already found the class
+    if (null != hadoopClassesAvailable) {
+      // Make sure everything is initialized as expected
+      if (hadoopClassesAvailable && null != getProvidersMethod && null != hadoopCredProviderFactory && null != getCredentialEntryMethod
+          && null != getCredentialMethod) {
+        return true;
+      } else {
+        // Otherwise we failed to load it
+        return false;
+      }
+    }
+
+    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 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;
+    } catch (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.newInstance();
+    } catch (InstantiationException e) {
+      log.trace("Could not instantiate class {}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, e);
+      return false;
+    } catch (IllegalAccessException 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 e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_GET_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_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 e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    } catch (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 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 e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_ENTRY_GET_CREDENTIAL_METHOD_NAME, HADOOP_CRED_ENTRY_CLASS_NAME, e);
+      return false;
+    } catch (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) {
+    // Call CredentialProviderFactory.getProviders(Configuration)
+    Object providersObj = null;
+    try {
+      providersObj = getProvidersMethod.invoke(hadoopCredProviderFactory, conf);
+    } catch (IllegalArgumentException e) {
+      log.warn("Could not invoke {}.{}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME, e);
+      return null;
+    } catch (IllegalAccessException e) {
+      log.warn("Could not invoke {}.{}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME, e);
+      return null;
+    } catch (InvocationTargetException 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 {
+      return (List<Object>) providersObj;
+    } 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 (null == providerObjList) {
+      return null;
+    }
+
+    for (Object providerObj : providerObjList) {
+      try {
+        // Invoke CredentialProvider.getCredentialEntry(String)
+        Object credEntryObj = getCredentialEntryMethod.invoke(providerObj, alias);
+
+        if (null == credEntryObj) {
+          continue;
+        }
+
+        // Then, CredentialEntry.getCredential()
+        Object credential = getCredentialMethod.invoke(credEntryObj);
+
+        return (char[]) credential;
+      } catch (IllegalArgumentException e) {
+        log.warn("Failed to get credential from {}", providerObj, e);
+        continue;
+      } catch (IllegalAccessException e) {
+        log.warn("Failed to get credential from {}", providerObj, e);
+        continue;
+      } catch (InvocationTargetException e) {
+        log.warn("Failed to get credential from {}", providerObj, e);
+        continue;
+      }
+    }
+
+    log.warn("Could not extract credential from providers");
+
+    return null;
+  }
+
+  @SuppressWarnings("unchecked")
+  protected static List<String> getAliasesFromHadoopCredentialProvider(Configuration conf) {
+    List<Object> providerObjList = getCredentialProviders(conf);
+
+    if (null == providerObjList) {
+      log.debug("Failed to get CredProviders");
+      return Collections.emptyList();
+    }
+
+    ArrayList<String> aliases = new ArrayList<String>();
+    for (Object providerObj : providerObjList) {
+      if (null != providerObj) {
+        Object aliasesObj;
+        try {
+          aliasesObj = getAliasesMethod.invoke(providerObj);
+
+          if (null != aliasesObj && aliasesObj instanceof List) {
+            try {
+              aliases.addAll((List<String>) aliasesObj);
+            } catch (ClassCastException e) {
+              log.warn("Could not cast aliases ({}) from {} to a List<String>", aliasesObj, providerObj, e);
+              continue;
+            }
+          }
+
+        } catch (IllegalArgumentException e) {
+          log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, providerObj, e);
+          continue;
+        } catch (IllegalAccessException e) {
+          log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, providerObj, e);
+          continue;
+        } catch (InvocationTargetException e) {
+          log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, providerObj, e);
+          continue;
+        }
+      }
+    }
+
+    return aliases;
+  }
+
+  /**
+   * 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
+   * @throws IOException
+   *           On errors reading a CredentialProvider
+   */
+  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.
+   *
+   * @param conf
+   *          Configuration for the CredentialProvider
+   * @return A list of aliases. An empty list if no CredentialProviders are configured, or the providers are empty.
+   * @throws IOException
+   *           On errors reading a CredentialProvider
+   */
+  public static List<String> getKeys(Configuration conf) throws IOException {
+    Preconditions.checkNotNull(conf);
+
+    if (isHadoopCredentialProviderAvailable()) {
+      log.trace("Hadoop CredentialProvider is available, attempting to extract all aliases");
+      return getAliasesFromHadoopCredentialProvider(conf);
+    }
+
+    return Collections.emptyList();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb450ffb/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 8c9a326..3a794a4 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -157,6 +157,7 @@ public enum Property {
   @Experimental
   GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.RandomVolumeChooser", PropertyType.CLASSNAME,
       "The class that will be used to select which volume will be used to create new files."),
+  GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS("general.security.credential.provider.paths", "", PropertyType.STRING, "Comma-separated list of paths to CredentialProviders"),
 
   // properties that are specific to master server behavior
   MASTER_PREFIX("master.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the master server"),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb450ffb/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
----------------------------------------------------------------------
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 997df64..a5e14af 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
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.conf;
 
+import java.io.IOException;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -64,7 +65,26 @@ public class SiteConfiguration extends AccumuloConfiguration {
   @Override
   public String get(Property property) {
     String key = property.getKey();
-    
+
+    // If the property is sensitive, see if CredentialProvider was configured.
+    if (property.isSensitive()) {
+      Configuration hadoopConf = getHadoopConfiguration();
+      if (null != hadoopConf) {
+        // Try to find the sensitive value from the CredentialProvider
+        try {
+          char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key);
+          if (null != value) {
+            if (log.isTraceEnabled()) {
+              log.trace("Loaded " + key + "=" + new String(value) + " from CredentialProvider");
+            }
+            return new String(value);
+          }
+        } catch (IOException e) {
+          log.warn("Failed to extract sensitive property (" + key + ") from Hadoop CredentialProvider, falling back to accumulo-site.xml", e);
+        }
+      }
+    }
+
     String value = getXmlConfig().get(key);
     
     if (value == null || !property.getType().isValidFormat(value)) {
@@ -79,11 +99,42 @@ public class SiteConfiguration extends AccumuloConfiguration {
   public void getProperties(Map<String,String> props, PropertyFilter filter) {
     parent.getProperties(props, filter);
 
+    Configuration hadoopConf = getHadoopConfiguration();
+    if (null != hadoopConf) {
+      try {
+        for (String key : CredentialProviderFactoryShim.getKeys(hadoopConf)) {
+          if (filter.accept(key)) {
+            char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key);
+            if (null != value) {
+              props.put(key, new String(value));
+            }
+          }
+        }
+      } catch (IOException e) {
+        log.warn("Failed to extract sensitive properties from Hadoop CredentialProvider, falling back to accumulo-site.xml", e);
+      }
+    }
+
     for (Entry<String,String> entry : getXmlConfig())
       if (filter.accept(entry.getKey()))
         props.put(entry.getKey(), entry.getValue());
   }
 
+  protected Configuration getHadoopConfiguration() {
+    String credProviderPathsKey = Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey();
+    String credProviderPathsValue = getXmlConfig().get(credProviderPathsKey);
+
+    if (null != credProviderPathsValue) {
+      // We have configuration for a CredentialProvider
+      // Try to pull the sensitive password from there
+      Configuration conf = new Configuration();
+      conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credProviderPathsValue);
+      return conf;
+    }
+
+    return null;
+  }
+
   /**
    * method here to support testing, do not call
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb450ffb/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
new file mode 100644
index 0000000..42497a8
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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 java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class CredentialProviderFactoryShimTest {
+
+  private static boolean isCredentialProviderAvailable = false;
+  private static final String populatedKeyStoreName = "/accumulo.jceks", emptyKeyStoreName = "/empty.jceks";
+  private static File emptyKeyStore, populatedKeyStore;
+
+  @BeforeClass
+  public static void checkCredentialProviderAvailable() {
+    try {
+      Class.forName(CredentialProviderFactoryShim.HADOOP_CRED_PROVIDER_CLASS_NAME);
+      isCredentialProviderAvailable = true;
+    } catch (Exception e) {
+      isCredentialProviderAvailable = false;
+    }
+
+    if (isCredentialProviderAvailable) {
+      URL populatedKeyStoreUrl = CredentialProviderFactoryShimTest.class.getResource(populatedKeyStoreName),
+          emptyKeyStoreUrl = CredentialProviderFactoryShimTest.class.getResource(emptyKeyStoreName); 
+
+      Assert.assertNotNull("Could not find " + populatedKeyStoreName, populatedKeyStoreUrl);
+      Assert.assertNotNull("Could not find " + emptyKeyStoreName, emptyKeyStoreUrl);
+
+      populatedKeyStore = new File(populatedKeyStoreUrl.getFile());
+      emptyKeyStore = new File(emptyKeyStoreUrl.getFile());
+    }
+  }
+
+  protected String getKeyStoreUrl(File absoluteFilePath) {
+    return "jceks://file" + absoluteFilePath.getAbsolutePath();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullConfigOnGetValue() throws IOException {
+    CredentialProviderFactoryShim.getValueFromCredentialProvider(null, "alias");
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullAliasOnGetValue() throws IOException {
+    CredentialProviderFactoryShim.getValueFromCredentialProvider(new Configuration(false), null);
+  }
+
+  protected void checkCredentialProviders(Configuration conf, Map<String,String> expectation) throws IOException {
+    List<String> keys = CredentialProviderFactoryShim.getKeys(conf);
+    Assert.assertNotNull(keys);
+    
+    Assert.assertEquals(expectation.keySet(), new HashSet<String>(keys));
+    for (String expectedKey : keys) {
+      char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, expectedKey);
+      Assert.assertNotNull(value);
+      Assert.assertEquals(expectation.get(expectedKey), new String(value));
+    }
+  }
+
+  @Test
+  public void testExtractFromProvider() throws IOException {
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    String absPath = getKeyStoreUrl(populatedKeyStore);
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, absPath);
+    Map<String,String> expectations = new HashMap<String,String>();
+    expectations.put("key1", "value1");
+    expectations.put("key2", "value2");
+
+    checkCredentialProviders(conf, expectations);
+  }
+
+  @Test
+  public void testEmptyKeyStoreParses() throws IOException {
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    String absPath = getKeyStoreUrl(emptyKeyStore);
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, absPath);
+
+    checkCredentialProviders(conf, new HashMap<String,String>());
+  }
+
+  @Test
+  public void testEmptyAndPopulatedKeyStores() throws IOException {
+
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    String populatedAbsPath = getKeyStoreUrl(populatedKeyStore), emptyAbsPath = getKeyStoreUrl(emptyKeyStore);
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, populatedAbsPath + "," + emptyAbsPath);
+    Map<String,String> expectations = new HashMap<String,String>();
+    expectations.put("key1", "value1");
+    expectations.put("key2", "value2");
+
+    checkCredentialProviders(conf, expectations);
+  }
+
+  @Test
+  public void testNonExistentClassesDoesntFail() throws IOException {
+    if (isCredentialProviderAvailable) {
+      return;
+    }
+
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, "jceks://file/foo/bar.jceks");
+    List<String> keys = CredentialProviderFactoryShim.getKeys(conf);
+    Assert.assertNotNull(keys);
+    Assert.assertEquals(Collections.emptyList(), keys);
+
+    Assert.assertNull(CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, "key1"));
+  }
+}

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

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


[2/3] git commit: ACCUMULO-2464 Reflect around CredentialProvider* to pull sensitive configuration values not from plaintext-sources

Posted by el...@apache.org.
ACCUMULO-2464 Reflect around CredentialProvider* to pull sensitive configuration values not from plaintext-sources

The CredentialProvider classes are a new introduction in 2.6.0 designed to provide a common
interface, via Hadoop Common, that downstream processes can leverage which are "more secure"
than storing them in a plaintext-readable Configuration file (like the common practice of
Hadoop Configuration "site" XML files).

CredentialProviderFactoryShim uses reflection to:

1) Find if these new classes are available
2) Instantiate them, and the Methods we need to invoke
3) And, invoke them with the given information

This provides us with a backwards and forwards compatibile class that allows early
adopters to use 2.6.0 features, and, when >=2.6.0 is the "norm", allow users to continue
to use <2.6.0 transparently.


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

Branch: refs/heads/master
Commit: fb450ffb2dd310624b91d2850f9bc416ac84d05b
Parents: 3a93bad
Author: Josh Elser <el...@apache.org>
Authored: Thu Jul 31 19:00:52 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri Aug 1 12:04:31 2014 -0400

----------------------------------------------------------------------
 .../conf/CredentialProviderFactoryShim.java     | 320 +++++++++++++++++++
 .../org/apache/accumulo/core/conf/Property.java |   1 +
 .../accumulo/core/conf/SiteConfiguration.java   |  53 ++-
 .../conf/CredentialProviderFactoryShimTest.java | 149 +++++++++
 core/src/test/resources/accumulo.jceks          | Bin 0 -> 946 bytes
 core/src/test/resources/empty.jceks             | Bin 0 -> 32 bytes
 6 files changed, 522 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb450ffb/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
new file mode 100644
index 0000000..1bf3ce6
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
@@ -0,0 +1,320 @@
+/*
+ * 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 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 jline.internal.Preconditions;
+
+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);
+
+  protected static final String HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME = "org.apache.hadoop.security.alias.JavaKeyStoreProvider$Factory";
+  protected static final String HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME = "getProviders";
+
+  protected static final String HADOOP_CRED_PROVIDER_CLASS_NAME = "org.apache.hadoop.security.alias.CredentialProvider";
+  protected static final String HADOOP_CRED_PROVIDER_GET_CREDENTIAL_ENTRY_METHOD_NAME = "getCredentialEntry";
+  protected static final String HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME = "getAliases";
+
+  protected static final String HADOOP_CRED_ENTRY_CLASS_NAME = "org.apache.hadoop.security.alias.CredentialProvider$CredentialEntry";
+  protected static final String HADOOP_CRED_ENTRY_GET_CREDENTIAL_METHOD_NAME = "getCredential";
+
+  public static final String CREDENTIAL_PROVIDER_PATH = "hadoop.security.credential.provider.path";
+
+  private static Object hadoopCredProviderFactory = null;
+  private static Method getProvidersMethod = null;
+  private static Method getAliasesMethod = null;
+  private static Method getCredentialEntryMethod = null;
+  private static Method getCredentialMethod = null;
+  private static Boolean hadoopClassesAvailable = null;
+
+  /**
+   * Determine if we can load the necessary CredentialProvider classes. Only loaded the first time, so subsequent invocations of this method should return fast.
+   *
+   * @return True if the CredentialProvider classes/methods are available, false otherwise.
+   */
+  protected static synchronized boolean isHadoopCredentialProviderAvailable() {
+    // If we already found the class
+    if (null != hadoopClassesAvailable) {
+      // Make sure everything is initialized as expected
+      if (hadoopClassesAvailable && null != getProvidersMethod && null != hadoopCredProviderFactory && null != getCredentialEntryMethod
+          && null != getCredentialMethod) {
+        return true;
+      } else {
+        // Otherwise we failed to load it
+        return false;
+      }
+    }
+
+    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 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;
+    } catch (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.newInstance();
+    } catch (InstantiationException e) {
+      log.trace("Could not instantiate class {}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, e);
+      return false;
+    } catch (IllegalAccessException 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 e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_GET_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_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 e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, HADOOP_CRED_PROVIDER_CLASS_NAME, e);
+      return false;
+    } catch (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 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 e) {
+      log.trace("Could not find {} method on {}", HADOOP_CRED_ENTRY_GET_CREDENTIAL_METHOD_NAME, HADOOP_CRED_ENTRY_CLASS_NAME, e);
+      return false;
+    } catch (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) {
+    // Call CredentialProviderFactory.getProviders(Configuration)
+    Object providersObj = null;
+    try {
+      providersObj = getProvidersMethod.invoke(hadoopCredProviderFactory, conf);
+    } catch (IllegalArgumentException e) {
+      log.warn("Could not invoke {}.{}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME, e);
+      return null;
+    } catch (IllegalAccessException e) {
+      log.warn("Could not invoke {}.{}", HADOOP_CRED_PROVIDER_FACTORY_CLASS_NAME, HADOOP_CRED_PROVIDER_FACTORY_GET_PROVIDERS_METHOD_NAME, e);
+      return null;
+    } catch (InvocationTargetException 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 {
+      return (List<Object>) providersObj;
+    } 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 (null == providerObjList) {
+      return null;
+    }
+
+    for (Object providerObj : providerObjList) {
+      try {
+        // Invoke CredentialProvider.getCredentialEntry(String)
+        Object credEntryObj = getCredentialEntryMethod.invoke(providerObj, alias);
+
+        if (null == credEntryObj) {
+          continue;
+        }
+
+        // Then, CredentialEntry.getCredential()
+        Object credential = getCredentialMethod.invoke(credEntryObj);
+
+        return (char[]) credential;
+      } catch (IllegalArgumentException e) {
+        log.warn("Failed to get credential from {}", providerObj, e);
+        continue;
+      } catch (IllegalAccessException e) {
+        log.warn("Failed to get credential from {}", providerObj, e);
+        continue;
+      } catch (InvocationTargetException e) {
+        log.warn("Failed to get credential from {}", providerObj, e);
+        continue;
+      }
+    }
+
+    log.warn("Could not extract credential from providers");
+
+    return null;
+  }
+
+  @SuppressWarnings("unchecked")
+  protected static List<String> getAliasesFromHadoopCredentialProvider(Configuration conf) {
+    List<Object> providerObjList = getCredentialProviders(conf);
+
+    if (null == providerObjList) {
+      log.debug("Failed to get CredProviders");
+      return Collections.emptyList();
+    }
+
+    ArrayList<String> aliases = new ArrayList<String>();
+    for (Object providerObj : providerObjList) {
+      if (null != providerObj) {
+        Object aliasesObj;
+        try {
+          aliasesObj = getAliasesMethod.invoke(providerObj);
+
+          if (null != aliasesObj && aliasesObj instanceof List) {
+            try {
+              aliases.addAll((List<String>) aliasesObj);
+            } catch (ClassCastException e) {
+              log.warn("Could not cast aliases ({}) from {} to a List<String>", aliasesObj, providerObj, e);
+              continue;
+            }
+          }
+
+        } catch (IllegalArgumentException e) {
+          log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, providerObj, e);
+          continue;
+        } catch (IllegalAccessException e) {
+          log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, providerObj, e);
+          continue;
+        } catch (InvocationTargetException e) {
+          log.warn("Failed to invoke {} on {}", HADOOP_CRED_PROVIDER_GET_ALIASES_METHOD_NAME, providerObj, e);
+          continue;
+        }
+      }
+    }
+
+    return aliases;
+  }
+
+  /**
+   * 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
+   * @throws IOException
+   *           On errors reading a CredentialProvider
+   */
+  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.
+   *
+   * @param conf
+   *          Configuration for the CredentialProvider
+   * @return A list of aliases. An empty list if no CredentialProviders are configured, or the providers are empty.
+   * @throws IOException
+   *           On errors reading a CredentialProvider
+   */
+  public static List<String> getKeys(Configuration conf) throws IOException {
+    Preconditions.checkNotNull(conf);
+
+    if (isHadoopCredentialProviderAvailable()) {
+      log.trace("Hadoop CredentialProvider is available, attempting to extract all aliases");
+      return getAliasesFromHadoopCredentialProvider(conf);
+    }
+
+    return Collections.emptyList();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb450ffb/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 8c9a326..3a794a4 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -157,6 +157,7 @@ public enum Property {
   @Experimental
   GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.RandomVolumeChooser", PropertyType.CLASSNAME,
       "The class that will be used to select which volume will be used to create new files."),
+  GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS("general.security.credential.provider.paths", "", PropertyType.STRING, "Comma-separated list of paths to CredentialProviders"),
 
   // properties that are specific to master server behavior
   MASTER_PREFIX("master.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the master server"),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb450ffb/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
----------------------------------------------------------------------
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 997df64..a5e14af 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
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.conf;
 
+import java.io.IOException;
 import java.util.Map;
 import java.util.Map.Entry;
 
@@ -64,7 +65,26 @@ public class SiteConfiguration extends AccumuloConfiguration {
   @Override
   public String get(Property property) {
     String key = property.getKey();
-    
+
+    // If the property is sensitive, see if CredentialProvider was configured.
+    if (property.isSensitive()) {
+      Configuration hadoopConf = getHadoopConfiguration();
+      if (null != hadoopConf) {
+        // Try to find the sensitive value from the CredentialProvider
+        try {
+          char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key);
+          if (null != value) {
+            if (log.isTraceEnabled()) {
+              log.trace("Loaded " + key + "=" + new String(value) + " from CredentialProvider");
+            }
+            return new String(value);
+          }
+        } catch (IOException e) {
+          log.warn("Failed to extract sensitive property (" + key + ") from Hadoop CredentialProvider, falling back to accumulo-site.xml", e);
+        }
+      }
+    }
+
     String value = getXmlConfig().get(key);
     
     if (value == null || !property.getType().isValidFormat(value)) {
@@ -79,11 +99,42 @@ public class SiteConfiguration extends AccumuloConfiguration {
   public void getProperties(Map<String,String> props, PropertyFilter filter) {
     parent.getProperties(props, filter);
 
+    Configuration hadoopConf = getHadoopConfiguration();
+    if (null != hadoopConf) {
+      try {
+        for (String key : CredentialProviderFactoryShim.getKeys(hadoopConf)) {
+          if (filter.accept(key)) {
+            char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key);
+            if (null != value) {
+              props.put(key, new String(value));
+            }
+          }
+        }
+      } catch (IOException e) {
+        log.warn("Failed to extract sensitive properties from Hadoop CredentialProvider, falling back to accumulo-site.xml", e);
+      }
+    }
+
     for (Entry<String,String> entry : getXmlConfig())
       if (filter.accept(entry.getKey()))
         props.put(entry.getKey(), entry.getValue());
   }
 
+  protected Configuration getHadoopConfiguration() {
+    String credProviderPathsKey = Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey();
+    String credProviderPathsValue = getXmlConfig().get(credProviderPathsKey);
+
+    if (null != credProviderPathsValue) {
+      // We have configuration for a CredentialProvider
+      // Try to pull the sensitive password from there
+      Configuration conf = new Configuration();
+      conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credProviderPathsValue);
+      return conf;
+    }
+
+    return null;
+  }
+
   /**
    * method here to support testing, do not call
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb450ffb/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
new file mode 100644
index 0000000..42497a8
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShimTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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 java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class CredentialProviderFactoryShimTest {
+
+  private static boolean isCredentialProviderAvailable = false;
+  private static final String populatedKeyStoreName = "/accumulo.jceks", emptyKeyStoreName = "/empty.jceks";
+  private static File emptyKeyStore, populatedKeyStore;
+
+  @BeforeClass
+  public static void checkCredentialProviderAvailable() {
+    try {
+      Class.forName(CredentialProviderFactoryShim.HADOOP_CRED_PROVIDER_CLASS_NAME);
+      isCredentialProviderAvailable = true;
+    } catch (Exception e) {
+      isCredentialProviderAvailable = false;
+    }
+
+    if (isCredentialProviderAvailable) {
+      URL populatedKeyStoreUrl = CredentialProviderFactoryShimTest.class.getResource(populatedKeyStoreName),
+          emptyKeyStoreUrl = CredentialProviderFactoryShimTest.class.getResource(emptyKeyStoreName); 
+
+      Assert.assertNotNull("Could not find " + populatedKeyStoreName, populatedKeyStoreUrl);
+      Assert.assertNotNull("Could not find " + emptyKeyStoreName, emptyKeyStoreUrl);
+
+      populatedKeyStore = new File(populatedKeyStoreUrl.getFile());
+      emptyKeyStore = new File(emptyKeyStoreUrl.getFile());
+    }
+  }
+
+  protected String getKeyStoreUrl(File absoluteFilePath) {
+    return "jceks://file" + absoluteFilePath.getAbsolutePath();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullConfigOnGetValue() throws IOException {
+    CredentialProviderFactoryShim.getValueFromCredentialProvider(null, "alias");
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullAliasOnGetValue() throws IOException {
+    CredentialProviderFactoryShim.getValueFromCredentialProvider(new Configuration(false), null);
+  }
+
+  protected void checkCredentialProviders(Configuration conf, Map<String,String> expectation) throws IOException {
+    List<String> keys = CredentialProviderFactoryShim.getKeys(conf);
+    Assert.assertNotNull(keys);
+    
+    Assert.assertEquals(expectation.keySet(), new HashSet<String>(keys));
+    for (String expectedKey : keys) {
+      char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, expectedKey);
+      Assert.assertNotNull(value);
+      Assert.assertEquals(expectation.get(expectedKey), new String(value));
+    }
+  }
+
+  @Test
+  public void testExtractFromProvider() throws IOException {
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    String absPath = getKeyStoreUrl(populatedKeyStore);
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, absPath);
+    Map<String,String> expectations = new HashMap<String,String>();
+    expectations.put("key1", "value1");
+    expectations.put("key2", "value2");
+
+    checkCredentialProviders(conf, expectations);
+  }
+
+  @Test
+  public void testEmptyKeyStoreParses() throws IOException {
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    String absPath = getKeyStoreUrl(emptyKeyStore);
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, absPath);
+
+    checkCredentialProviders(conf, new HashMap<String,String>());
+  }
+
+  @Test
+  public void testEmptyAndPopulatedKeyStores() throws IOException {
+
+    if (!isCredentialProviderAvailable) {
+      return;
+    }
+
+    String populatedAbsPath = getKeyStoreUrl(populatedKeyStore), emptyAbsPath = getKeyStoreUrl(emptyKeyStore);
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, populatedAbsPath + "," + emptyAbsPath);
+    Map<String,String> expectations = new HashMap<String,String>();
+    expectations.put("key1", "value1");
+    expectations.put("key2", "value2");
+
+    checkCredentialProviders(conf, expectations);
+  }
+
+  @Test
+  public void testNonExistentClassesDoesntFail() throws IOException {
+    if (isCredentialProviderAvailable) {
+      return;
+    }
+
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, "jceks://file/foo/bar.jceks");
+    List<String> keys = CredentialProviderFactoryShim.getKeys(conf);
+    Assert.assertNotNull(keys);
+    Assert.assertEquals(Collections.emptyList(), keys);
+
+    Assert.assertNull(CredentialProviderFactoryShim.getValueFromCredentialProvider(conf, "key1"));
+  }
+}

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

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


[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/db72ba35
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/db72ba35
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/db72ba35

Branch: refs/heads/master
Commit: db72ba3588d022138447de457fd719c17f6cca96
Parents: 8abe4df6 fb450ff
Author: Josh Elser <el...@apache.org>
Authored: Fri Aug 1 13:16:31 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri Aug 1 13:16:31 2014 -0400

----------------------------------------------------------------------
 .../conf/CredentialProviderFactoryShim.java     | 320 +++++++++++++++++++
 .../org/apache/accumulo/core/conf/Property.java |   1 +
 .../accumulo/core/conf/SiteConfiguration.java   |  53 ++-
 .../conf/CredentialProviderFactoryShimTest.java | 149 +++++++++
 core/src/test/resources/accumulo.jceks          | Bin 0 -> 946 bytes
 core/src/test/resources/empty.jceks             | Bin 0 -> 32 bytes
 6 files changed, 522 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/db72ba35/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/db72ba35/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
index 3696c0d,a5e14af..8114ea8
--- a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
@@@ -103,9 -120,23 +139,24 @@@ public class SiteConfiguration extends 
          props.put(entry.getKey(), entry.getValue());
    }
  
+   protected Configuration getHadoopConfiguration() {
+     String credProviderPathsKey = Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey();
+     String credProviderPathsValue = getXmlConfig().get(credProviderPathsKey);
+ 
+     if (null != credProviderPathsValue) {
+       // We have configuration for a CredentialProvider
+       // Try to pull the sensitive password from there
+       Configuration conf = new Configuration();
+       conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credProviderPathsValue);
+       return conf;
+     }
+ 
+     return null;
+   }
+ 
    /**
 -   * method here to support testing, do not call
 +   * Clears the configuration properties in this configuration (but not the
 +   * parent). This method supports testing and should not be called.
     */
    synchronized public static void clearInstance() {
      instance = null;