You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mw...@apache.org on 2018/12/19 19:52:00 UTC

[accumulo] branch master updated: Refactored SiteConfiguration (#841)

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

mwalch 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 90dfad3  Refactored SiteConfiguration (#841)
90dfad3 is described below

commit 90dfad3b341a5098113ea8fdce28a0bc4511d067
Author: Mike Walch <mw...@apache.org>
AuthorDate: Wed Dec 19 14:51:56 2018 -0500

    Refactored SiteConfiguration (#841)
    
    * Constructor now creates ImmutableMap
    * Improved unit test
---
 .../core/clientImpl/ClientConfConverter.java       |  21 +-
 .../core/conf/CredentialProviderFactoryShim.java   |   7 +-
 .../org/apache/accumulo/core/conf/Property.java    |  12 --
 .../accumulo/core/conf/SiteConfiguration.java      | 217 +++++----------------
 .../accumulo/core/conf/SiteConfigurationTest.java  |  51 ++---
 core/src/test/resources/accumulo2.properties       |  23 +++
 6 files changed, 113 insertions(+), 218 deletions(-)

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 bb55d09..55372f3 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
@@ -166,19 +166,14 @@ public class ClientConfConverter {
         if (property.isSensitive()) {
           org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration();
           if (hadoopConf != null) {
-            try {
-              char[] value = CredentialProviderFactoryShim
-                  .getValueFromCredentialProvider(hadoopConf, key);
-              if (value != null) {
-                log.trace("Loaded sensitive value for {} from CredentialProvider", key);
-                return new String(value);
-              } else {
-                log.trace("Tried to load sensitive value for {} from CredentialProvider, "
-                    + "but none was found", key);
-              }
-            } catch (IOException e) {
-              log.warn("Failed to extract sensitive property ({}) from Hadoop CredentialProvider,"
-                  + " falling back to base AccumuloConfiguration", key, e);
+            char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf,
+                key);
+            if (value != null) {
+              log.trace("Loaded sensitive value for {} from CredentialProvider", key);
+              return new String(value);
+            } else {
+              log.trace("Tried to load sensitive value for {} from CredentialProvider, "
+                  + "but none was found", key);
             }
           }
         }
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java b/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
index c8ef29e..33e9ab6 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/CredentialProviderFactoryShim.java
@@ -349,20 +349,15 @@ public class CredentialProviderFactoryShim {
    * @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 {
+  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;
   }
 
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 e20b7ef..dc759a3 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
@@ -1186,18 +1186,6 @@ public enum Property {
             || key.startsWith(TABLE_SCAN_DISPATCHER_OPTS.getKey())));
   }
 
-  /**
-   * Properties we check the value of within the TabletServer request handling or maintenance
-   * processing loops.
-   */
-  public static final EnumSet<Property> HOT_PATH_PROPERTIES = EnumSet.of(
-      Property.TSERV_CLIENT_TIMEOUT, Property.TSERV_TOTAL_MUTATION_QUEUE_MAX,
-      Property.GC_TRASH_IGNORE, Property.TSERV_MAJC_DELAY, Property.TABLE_MINC_LOGS_MAX,
-      Property.TSERV_MAJC_MAXCONCURRENT, Property.REPLICATION_WORKER_THREADS,
-      Property.TABLE_DURABILITY, Property.INSTANCE_ZK_TIMEOUT, Property.TABLE_CLASSPATH,
-      Property.MASTER_METADATA_SUSPENDABLE, Property.TABLE_FAILURES_IGNORE,
-      Property.TABLE_SCAN_MAXMEM, Property.INSTANCE_CRYPTO_SERVICE, Property.TABLE_DELETE_BEHAVIOR);
-
   private static final EnumSet<Property> fixedProperties = EnumSet.of(Property.TSERV_CLIENTPORT,
       Property.TSERV_NATIVEMAP_ENABLED, Property.TSERV_SCAN_MAX_OPENFILES,
       Property.MASTER_CLIENTPORT, Property.GC_PORT);
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 df46dc2..96e6d85 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
@@ -23,20 +23,19 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
-import java.util.function.Function;
 import java.util.function.Predicate;
-import java.util.stream.Collectors;
-import java.util.stream.StreamSupport;
 
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.commons.configuration.CompositeConfiguration;
-import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.ImmutableMap;
+
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 /**
@@ -49,24 +48,24 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  * property is not defined, it defaults to "accumulo.properties" and will look on classpath for
  * file.
  * <p>
- * This class is a singleton.
- * <p>
  * <b>Note</b>: Client code should not use this class, and it may be deprecated in the future.
  */
 public class SiteConfiguration extends AccumuloConfiguration {
+
   private static final Logger log = LoggerFactory.getLogger(SiteConfiguration.class);
 
   private static final AccumuloConfiguration parent = DefaultConfiguration.getInstance();
 
-  private CompositeConfiguration internalConfig;
-
-  private final Map<String,String> overrides;
-  private final Map<String,String> staticConfigs;
+  private final ImmutableMap<String,String> config;
 
   public SiteConfiguration() {
     this(getAccumuloPropsLocation());
   }
 
+  public SiteConfiguration(Map<String,String> overrides) {
+    this(getAccumuloPropsLocation(), overrides);
+  }
+
   public SiteConfiguration(File accumuloPropsFile) {
     this(accumuloPropsFile, Collections.emptyMap());
   }
@@ -79,37 +78,52 @@ public class SiteConfiguration extends AccumuloConfiguration {
     this(accumuloPropsLocation, Collections.emptyMap());
   }
 
-  @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD",
-      justification = "location of props is specified by an admin")
   public SiteConfiguration(URL accumuloPropsLocation, Map<String,String> overrides) {
-    this.overrides = overrides;
+    config = createMap(accumuloPropsLocation, overrides);
+  }
 
-    init();
-    PropertiesConfiguration config = new PropertiesConfiguration();
+  @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD",
+      justification = "location of props is specified by an admin")
+  private static ImmutableMap<String,String> createMap(URL accumuloPropsLocation,
+      Map<String,String> overrides) {
+    CompositeConfiguration config = new CompositeConfiguration();
+    config.setThrowExceptionOnMissing(false);
     config.setDelimiterParsingDisabled(true);
+    PropertiesConfiguration propsConfig = new PropertiesConfiguration();
+    propsConfig.setDelimiterParsingDisabled(true);
     if (accumuloPropsLocation != null) {
       try {
-        config.load(accumuloPropsLocation.openStream());
+        propsConfig.load(accumuloPropsLocation.openStream());
       } catch (IOException | ConfigurationException e) {
         throw new IllegalArgumentException(e);
       }
     }
-    internalConfig.addConfiguration(config);
+    config.addConfiguration(propsConfig);
+
+    // Add all properties in config file
+    Map<String,String> result = new HashMap<>();
+    config.getKeys().forEachRemaining(key -> result.put(key, config.getString(key)));
 
-    Map<String,String> temp = StreamSupport
-        .stream(((Iterable<String>) internalConfig::getKeys).spliterator(), false)
-        .collect(Collectors.toMap(Function.identity(), internalConfig::getString));
+    // Add all overrides
+    overrides.forEach(result::put);
 
-    /*
-     * If any of the configs used in hot codepaths are unset here, set a null so that we'll default
-     * to the parent config without contending for the Hadoop Configuration object
-     */
-    for (Property hotConfig : Property.HOT_PATH_PROPERTIES) {
-      if (!(temp.containsKey(hotConfig.getKey()))) {
-        temp.put(hotConfig.getKey(), null);
+    // Add sensitive properties from credential provider (if set)
+    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(
+          CachedConfiguration.getInstance());
+      hadoopConf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credProvider);
+      for (Property property : Property.values()) {
+        if (property.isSensitive()) {
+          char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf,
+              property.getKey());
+          if (value != null) {
+            result.put(property.getKey(), new String(value));
+          }
+        }
       }
     }
-    staticConfigs = Collections.unmodifiableMap(temp);
+    return ImmutableMap.copyOf(result);
   }
 
   private static URL toURL(File f) {
@@ -157,80 +171,22 @@ public class SiteConfiguration extends AccumuloConfiguration {
     }
   }
 
-  private void init() {
-    internalConfig = new CompositeConfiguration();
-    internalConfig.setThrowExceptionOnMissing(false);
-    internalConfig.setDelimiterParsingDisabled(true);
-  }
-
-  private synchronized Configuration getConfiguration() {
-    if (internalConfig == null) {
-      init();
-    }
-    return internalConfig;
-  }
-
   @Override
   public String get(Property property) {
-    if (overrides.containsKey(property.getKey())) {
-      return overrides.get(property.getKey());
-    }
-
-    String key = property.getKey();
-    // If the property is sensitive, see if CredentialProvider was configured.
-    if (property.isSensitive()) {
-      String hadoopVal = getSensitiveFromHadoop(property);
-      if (hadoopVal != null) {
-        return hadoopVal;
-      }
-    }
-
-    /*
-     * Check the available-on-load configs and fall-back to the possibly-update Configuration
-     * object.
-     */
-    String value = staticConfigs.containsKey(key) ? staticConfigs.get(key)
-        : getConfiguration().getString(key);
-
+    String value = config.get(property.getKey());
     if (value == null || !property.getType().isValidFormat(value)) {
-      if (value != null)
-        log.error("Using default value for {} due to improperly formatted {}: {}", key,
-            property.getType(), value);
+      if (value != null) {
+        log.error("Using default value for {} due to improperly formatted {}: {}",
+            property.getKey(), property.getType(), value);
+      }
       value = parent.get(property);
     }
-
     return value;
   }
 
-  private String getSensitiveFromHadoop(Property property) {
-    org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration();
-    if (hadoopConf != null) {
-      // Try to find the sensitive value from the CredentialProvider
-      try {
-        char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf,
-            property.getKey());
-        if (value != null) {
-          return new String(value);
-        }
-      } catch (IOException e) {
-        log.warn("Failed to extract sensitive property (" + property.getKey()
-            + ") from Hadoop CredentialProvider, falling back to accumulo.properties", e);
-      }
-    }
-    return null;
-  }
-
   @Override
   public boolean isPropertySet(Property prop, boolean cacheAndWatch) {
-    if (prop.isSensitive()) {
-      String hadoopVal = getSensitiveFromHadoop(prop);
-      if (hadoopVal != null) {
-        return true;
-      }
-    }
-    return overrides.containsKey(prop.getKey()) || staticConfigs.containsKey(prop.getKey())
-        || getConfiguration().containsKey(prop.getKey())
-        || parent.isPropertySet(prop, cacheAndWatch);
+    return config.containsKey(prop.getKey()) || parent.isPropertySet(prop, cacheAndWatch);
   }
 
   @Override
@@ -243,78 +199,9 @@ public class SiteConfiguration extends AccumuloConfiguration {
     if (useDefaults) {
       parent.getProperties(props, filter);
     }
-
-    StreamSupport.stream(((Iterable<String>) getConfiguration()::getKeys).spliterator(), false)
-        .filter(filter).forEach(k -> props.put(k, getConfiguration().getString(k)));
-
-    // CredentialProvider should take precedence over site
-    org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration();
-    if (hadoopConf != null) {
-      try {
-        for (String key : CredentialProviderFactoryShim.getKeys(hadoopConf)) {
-          if (!Property.isValidPropertyKey(key) || !Property.isSensitive(key)) {
-            continue;
-          }
-
-          if (filter.test(key)) {
-            char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf,
-                key);
-            if (value != null) {
-              props.put(key, new String(value));
-            }
-          }
-        }
-      } catch (IOException e) {
-        log.warn("Failed to extract sensitive properties from Hadoop"
-            + " CredentialProvider, falling back to accumulo.properties", e);
-      }
-    }
-    if (overrides != null) {
-      for (Map.Entry<String,String> entry : overrides.entrySet()) {
-        if (filter.test(entry.getKey())) {
-          props.put(entry.getKey(), entry.getValue());
-        }
-      }
-    }
-  }
-
-  protected org.apache.hadoop.conf.Configuration getHadoopConfiguration() {
-    String credProviderPathsKey = Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey();
-    String credProviderPathsValue = getConfiguration().getString(credProviderPathsKey);
-
-    if (credProviderPathsValue != null) {
-      // We have configuration for a CredentialProvider
-      // Try to pull the sensitive password from there
-      org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(
-          CachedConfiguration.getInstance());
-      conf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credProviderPathsValue);
-      return conf;
-    }
-
-    return null;
-  }
-
-  /**
-   * Sets a property. This method supports testing and should not be called.
-   *
-   * @param property
-   *          property to set
-   * @param value
-   *          property value
-   */
-  public void set(Property property, String value) {
-    set(property.getKey(), value);
-  }
-
-  /**
-   * Sets a property. This method supports testing and should not be called.
-   *
-   * @param key
-   *          key of property to set
-   * @param value
-   *          property value
-   */
-  public void set(String key, String value) {
-    getConfiguration().setProperty(key, value);
+    config.keySet().forEach(k -> {
+      if (filter.test(k))
+        props.put(k, config.get(k));
+    });
   }
 }
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 c017b37..ee412f5 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
@@ -24,10 +24,7 @@ import java.io.File;
 import java.net.URL;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.function.Predicate;
 
-import org.apache.hadoop.conf.Configuration;
-import org.easymock.EasyMock;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -57,33 +54,43 @@ public class SiteConfigurationTest {
       return;
     }
 
-    SiteConfiguration siteCfg = EasyMock.createMockBuilder(SiteConfiguration.class)
-        .addMockedMethod("getHadoopConfiguration").createMock();
-
-    siteCfg.set(Property.INSTANCE_SECRET, "ignored");
-
     // site-cfg.jceks={'ignored.property'=>'ignored', 'instance.secret'=>'mysecret',
     // 'general.rpc.timeout'=>'timeout'}
     URL keystore = SiteConfigurationTest.class.getResource("/site-cfg.jceks");
     assertNotNull(keystore);
-    String keystorePath = new File(keystore.getFile()).getAbsolutePath();
-
-    Configuration hadoopConf = new Configuration();
-    hadoopConf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH,
-        "jceks://file" + keystorePath);
+    String credProvPath = "jceks://file" + new File(keystore.getFile()).getAbsolutePath();
 
-    EasyMock.expect(siteCfg.getHadoopConfiguration()).andReturn(hadoopConf).once();
+    SiteConfiguration config = new SiteConfiguration(ImmutableMap
+        .of(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), credProvPath));
 
-    EasyMock.replay(siteCfg);
+    assertEquals("mysecret", config.get(Property.INSTANCE_SECRET));
+    assertNull(config.get("ignored.property"));
+    assertEquals(Property.GENERAL_RPC_TIMEOUT.getDefaultValue(),
+        config.get(Property.GENERAL_RPC_TIMEOUT.getKey()));
+  }
 
-    Map<String,String> props = new HashMap<>();
-    Predicate<String> all = x -> true;
-    siteCfg.getProperties(props, all);
+  @Test
+  public void testDefault() {
+    SiteConfiguration conf = new SiteConfiguration();
+    assertEquals("localhost:2181", conf.get(Property.INSTANCE_ZK_HOST));
+    assertEquals("DEFAULT", conf.get(Property.INSTANCE_SECRET));
+    assertEquals("", conf.get(Property.INSTANCE_VOLUMES));
+    assertEquals("120s", conf.get(Property.GENERAL_RPC_TIMEOUT));
+    assertEquals("1g", conf.get(Property.TSERV_WALOG_MAX_SIZE));
+    assertEquals("org.apache.accumulo.core.cryptoImpl.NoCryptoService",
+        conf.get(Property.INSTANCE_CRYPTO_SERVICE));
+  }
 
-    assertEquals("mysecret", props.get(Property.INSTANCE_SECRET.getKey()));
-    assertNull(props.get("ignored.property"));
-    assertEquals(Property.GENERAL_RPC_TIMEOUT.getDefaultValue(),
-        props.get(Property.GENERAL_RPC_TIMEOUT.getKey()));
+  @Test
+  public void testFile() {
+    URL propsUrl = getClass().getClassLoader().getResource("accumulo2.properties");
+    SiteConfiguration conf = new SiteConfiguration(propsUrl);
+    assertEquals("myhost123:2181", conf.get(Property.INSTANCE_ZK_HOST));
+    assertEquals("mysecret", conf.get(Property.INSTANCE_SECRET));
+    assertEquals("hdfs://localhost:8020/accumulo123", conf.get(Property.INSTANCE_VOLUMES));
+    assertEquals("123s", conf.get(Property.GENERAL_RPC_TIMEOUT));
+    assertEquals("256M", conf.get(Property.TSERV_WALOG_MAX_SIZE));
+    assertEquals("DummyService", conf.get(Property.INSTANCE_CRYPTO_SERVICE));
   }
 
   @Test
diff --git a/core/src/test/resources/accumulo2.properties b/core/src/test/resources/accumulo2.properties
new file mode 100644
index 0000000..71de2bd
--- /dev/null
+++ b/core/src/test/resources/accumulo2.properties
@@ -0,0 +1,23 @@
+# 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.
+
+general.rpc.timeout=123s
+instance.crypto.service=DummyService
+instance.secret=mysecret
+instance.volumes=hdfs://localhost:8020/accumulo123
+instance.zookeeper.host=myhost123:2181
+table.durability=flush
+tserver.memory.maps.native.enabled=false
+tserver.walog.max.size=256M