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/29 11:38:00 UTC

[accumulo] branch master updated: Simplify SiteConfiguration initialization (#1240)

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 f81516e  Simplify SiteConfiguration initialization (#1240)
f81516e is described below

commit f81516e18ddd62916fa80784e45821917534f5b9
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Sat Jun 29 07:37:55 2019 -0400

    Simplify SiteConfiguration initialization (#1240)
    
    * Use CompositeConfiguration to load configuration from the site
    properties file, the command-line overrides, and the resolved credential
    provider values in direct way to make it more readable and maintainable
    during its initial generation of its internal static map of property
    key-value pairs.
    * Use builder/fluent syntax to get rid of overloaded SiteConfiguration
    constructors.
    * Use var to shorten lines affected by SiteConfiguration, wherever
    possible, as long as the type was already sufficiently clear.
---
 .../org/apache/accumulo/core/cli/ConfigOpts.java   |   7 +-
 .../accumulo/core/conf/SiteConfiguration.java      | 230 +++++++++++++--------
 .../apache/accumulo/core/file/rfile/PrintInfo.java |  22 +-
 .../accumulo/core/file/rfile/bcfile/PrintInfo.java |   7 +-
 .../accumulo/core/conf/SiteConfigurationTest.java  |  17 +-
 .../standalone/StandaloneAccumuloCluster.java      |   3 +-
 .../accumulo/minicluster/MiniAccumuloInstance.java |   5 +-
 .../miniclusterImpl/MiniAccumuloClusterImpl.java   |  52 +++--
 .../miniclusterImpl/MiniAccumuloConfigImpl.java    |   2 +-
 .../org/apache/accumulo/server/AbstractServer.java |   3 +-
 .../apache/accumulo/server/cli/ServerUtilOpts.java |   4 +-
 .../accumulo/server/conf/ConfigSanityCheck.java    |   2 +-
 .../apache/accumulo/server/init/Initialize.java    |   4 +-
 .../accumulo/server/problems/ProblemReports.java   |  68 +++---
 .../apache/accumulo/server/util/ChangeSecret.java  |  49 ++---
 .../accumulo/server/util/CleanZookeeper.java       |   2 +-
 .../accumulo/server/util/DeleteZooInstance.java    |   5 +-
 .../java/org/apache/accumulo/server/util/Info.java |   2 +-
 .../apache/accumulo/server/util/ListInstances.java |   8 +-
 .../accumulo/server/util/ListVolumesUsed.java      |  20 +-
 .../accumulo/server/util/LoginProperties.java      |   2 +-
 .../accumulo/server/util/RestoreZookeeper.java     |   2 +-
 .../accumulo/server/util/TabletServerLocks.java    |   2 +-
 .../apache/accumulo/server/util/ZooKeeperMain.java |   5 +-
 .../org/apache/accumulo/server/util/ZooZap.java    |   9 +-
 .../conf/ServerConfigurationFactoryTest.java       |   4 +-
 .../BaseHostRegexTableLoadBalancerTest.java        |   5 +-
 .../master/balancer/TableLoadBalancerTest.java     |   2 +-
 .../server/security/SystemCredentialsTest.java     |   2 +-
 .../accumulo/server/util/TServerUtilsTest.java     |   2 +-
 .../accumulo/gc/SimpleGarbageCollectorTest.java    |   2 +-
 .../apache/accumulo/master/state/SetGoalState.java |   2 +-
 .../org/apache/accumulo/master/util/FateAdmin.java |   2 +-
 .../monitor/util/AccumuloMonitorAppender.java      |   2 +-
 .../apache/accumulo/tserver/logger/LogReader.java  |   8 +-
 .../accumulo/shell/commands/FateCommand.java       |   8 +-
 .../org/apache/accumulo/test/GetMasterStats.java   |   8 +-
 .../accumulo/test/functional/CacheTestClean.java   |   2 +-
 .../accumulo/test/functional/CacheTestWriter.java  |   6 +-
 .../accumulo/test/functional/SplitRecoveryIT.java  |   2 +-
 .../accumulo/test/functional/ZombieTServer.java    |   2 +-
 .../accumulo/test/performance/NullTserver.java     |   2 +-
 .../test/server/security/SystemCredentialsIT.java  |   5 +-
 .../org/apache/accumulo/test/util/CertUtils.java   |   4 +-
 44 files changed, 344 insertions(+), 258 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ConfigOpts.java b/core/src/main/java/org/apache/accumulo/core/cli/ConfigOpts.java
index b0d627e..e7dccf9 100644
--- a/core/src/main/java/org/apache/accumulo/core/cli/ConfigOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ConfigOpts.java
@@ -42,9 +42,6 @@ public class ConfigOpts extends Help {
   private String propsPath;
 
   public synchronized String getPropertiesPath() {
-    if (propsPath == null) {
-      propsPath = SiteConfiguration.getAccumuloPropsLocation().getFile();
-    }
     return propsPath;
   }
 
@@ -66,7 +63,9 @@ public class ConfigOpts extends Help {
       justification = "process runs in same security context as admin who provided path")
   public synchronized SiteConfiguration getSiteConfiguration() {
     if (siteConfig == null) {
-      siteConfig = new SiteConfiguration(new File(getPropertiesPath()), getOverrides());
+      String propsPath = getPropertiesPath();
+      siteConfig = (propsPath == null ? SiteConfiguration.fromEnv()
+          : SiteConfiguration.fromFile(new File(propsPath))).withOverrides(getOverrides()).build();
     }
     return siteConfig;
   }
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 27fd9df..b33f76c 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,8 @@
  */
 package org.apache.accumulo.core.conf;
 
+import static java.util.Objects.requireNonNull;
+
 import java.io.File;
 import java.net.MalformedURLException;
 import java.net.URI;
@@ -25,8 +27,11 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.function.Predicate;
+import java.util.stream.Stream;
 
+import org.apache.commons.configuration2.AbstractConfiguration;
 import org.apache.commons.configuration2.CompositeConfiguration;
+import org.apache.commons.configuration2.MapConfiguration;
 import org.apache.commons.configuration2.PropertiesConfiguration;
 import org.apache.commons.configuration2.builder.FileBasedConfigurationBuilder;
 import org.apache.commons.configuration2.builder.fluent.Parameters;
@@ -54,116 +59,171 @@ public class SiteConfiguration extends AccumuloConfiguration {
 
   private static final AccumuloConfiguration parent = DefaultConfiguration.getInstance();
 
-  private final Map<String,String> config;
-
-  public SiteConfiguration() {
-    this(getAccumuloPropsLocation());
+  public interface Buildable {
+    SiteConfiguration build();
   }
 
-  public SiteConfiguration(Map<String,String> overrides) {
-    this(getAccumuloPropsLocation(), overrides);
+  public interface OverridesOption extends Buildable {
+    Buildable withOverrides(Map<String,String> overrides);
   }
 
-  public SiteConfiguration(File accumuloPropsFile) {
-    this(accumuloPropsFile, Collections.emptyMap());
-  }
+  static class Builder implements OverridesOption, Buildable {
+    private URL url = null;
+    private Map<String,String> overrides = Collections.emptyMap();
 
-  public SiteConfiguration(File accumuloPropsFile, Map<String,String> overrides) {
-    this(toURL(accumuloPropsFile), overrides);
-  }
+    // visible to package-private for testing only
+    Builder() {}
 
-  public SiteConfiguration(URL accumuloPropsLocation) {
-    this(accumuloPropsLocation, Collections.emptyMap());
-  }
+    // exists for testing only
+    OverridesOption noFile() {
+      return this;
+    }
 
-  public SiteConfiguration(URL accumuloPropsLocation, Map<String,String> overrides) {
-    config = createMap(accumuloPropsLocation, overrides);
-    ConfigSanityCheck.validate(config.entrySet());
-  }
+    // exists for testing only
+    OverridesOption fromUrl(URL propertiesFileUrl) {
+      url = requireNonNull(propertiesFileUrl);
+      return this;
+    }
 
-  @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD",
-      justification = "location of props is specified by an admin")
-  private static Map<String,String> createMap(URL accumuloPropsLocation,
-      Map<String,String> overrides) {
-    CompositeConfiguration config = new CompositeConfiguration();
-    if (accumuloPropsLocation != null) {
-      FileBasedConfigurationBuilder<PropertiesConfiguration> propsBuilder =
-          new FileBasedConfigurationBuilder<>(PropertiesConfiguration.class)
-              .configure(new Parameters().properties().setURL(accumuloPropsLocation));
+    public OverridesOption fromEnv() {
+      URL siteUrl = SiteConfiguration.class.getClassLoader().getResource("accumulo-site.xml");
+      if (siteUrl != null) {
+        throw new IllegalArgumentException("Found deprecated config file 'accumulo-site.xml' on "
+            + "classpath. Since 2.0.0, this file was replaced by 'accumulo.properties'. Run the "
+            + "following command to convert an old 'accumulo-site.xml' file to the new format: "
+            + "accumulo convert-config -x /old/accumulo-site.xml -p /new/accumulo.properties");
+      }
+
+      String configFile = System.getProperty("accumulo.properties", "accumulo.properties");
+      if (configFile.startsWith("file://")) {
+        File f;
+        try {
+          f = new File(new URI(configFile));
+        } catch (URISyntaxException e) {
+          throw new IllegalArgumentException(
+              "Failed to load Accumulo configuration from " + configFile, e);
+        }
+        if (f.exists() && !f.isDirectory()) {
+          log.info("Found Accumulo configuration at {}", configFile);
+          return fromFile(f);
+        } else {
+          throw new IllegalArgumentException(
+              "Failed to load Accumulo configuration at " + configFile);
+        }
+      } else {
+        URL accumuloConfigUrl = SiteConfiguration.class.getClassLoader().getResource(configFile);
+        if (accumuloConfigUrl == null) {
+          throw new IllegalArgumentException(
+              "Failed to load Accumulo configuration '" + configFile + "' from classpath");
+        } else {
+          log.info("Found Accumulo configuration on classpath at {}", accumuloConfigUrl.getFile());
+          url = accumuloConfigUrl;
+          return this;
+        }
+      }
+    }
+
+    public OverridesOption fromFile(File propertiesFileLocation) {
       try {
-        config.addConfiguration(propsBuilder.getConfiguration());
-      } catch (ConfigurationException e) {
+        url = requireNonNull(propertiesFileLocation).toURI().toURL();
+      } catch (MalformedURLException e) {
         throw new IllegalArgumentException(e);
       }
+      return this;
+    }
+
+    @Override
+    public Buildable withOverrides(Map<String,String> overrides) {
+      this.overrides = requireNonNull(overrides);
+      return this;
     }
 
-    // Add all properties in config file
-    Map<String,String> result = new HashMap<>();
-    config.getKeys().forEachRemaining(key -> result.put(key, config.getString(key)));
-
-    // Add all overrides
-    overrides.forEach(result::put);
-
-    // 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();
-      HadoopCredentialProvider.setPath(hadoopConf, credProvider);
-      for (Property property : Property.values()) {
-        if (property.isSensitive()) {
-          char[] value = HadoopCredentialProvider.getValue(hadoopConf, property.getKey());
-          if (value != null) {
-            result.put(property.getKey(), new String(value));
-          }
+    @SuppressFBWarnings(value = "URLCONNECTION_SSRF_FD",
+        justification = "location of props is specified by an admin")
+    @Override
+    public SiteConfiguration build() {
+      // load properties from configuration file
+      var propsFileConfig = getPropsFileConfig(url);
+
+      // load properties from command-line overrides
+      var overrideConfig = new MapConfiguration(overrides);
+
+      // load credential provider property
+      var credProviderProps = new HashMap<String,String>();
+      for (var c : new AbstractConfiguration[] {propsFileConfig, overrideConfig}) {
+        var credProvider =
+            c.getString(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
+        if (credProvider != null && !credProvider.isEmpty()) {
+          loadCredProviderProps(credProvider, credProviderProps);
+          break;
         }
       }
+      var credProviderConfig = new MapConfiguration(credProviderProps);
+
+      var config = new CompositeConfiguration();
+      // add in specific order; use credential provider first, then overrides, then properties file
+      config.addConfiguration(credProviderConfig);
+      config.addConfiguration(overrideConfig);
+      config.addConfiguration(propsFileConfig);
+
+      var result = new HashMap<String,String>();
+      config.getKeys().forEachRemaining(k -> result.put(k, config.getString(k)));
+      return new SiteConfiguration(Collections.unmodifiableMap(result));
     }
-    return Map.copyOf(result);
   }
 
-  private static URL toURL(File f) {
-    try {
-      return f.toURI().toURL();
-    } catch (MalformedURLException e) {
-      throw new IllegalArgumentException(e);
-    }
+  /**
+   * Build a SiteConfiguration from the environmental configuration with the option to override.
+   */
+  public static SiteConfiguration.OverridesOption fromEnv() {
+    return new SiteConfiguration.Builder().fromEnv();
   }
 
-  public static URL getAccumuloPropsLocation() {
+  /**
+   * Build a SiteConfiguration from the provided properties file with the option to override.
+   */
+  public static SiteConfiguration.OverridesOption fromFile(File propertiesFileLocation) {
+    return new SiteConfiguration.Builder().fromFile(propertiesFileLocation);
+  }
 
-    URL siteUrl = SiteConfiguration.class.getClassLoader().getResource("accumulo-site.xml");
-    if (siteUrl != null) {
-      throw new IllegalArgumentException("Found deprecated config file 'accumulo-site.xml' on "
-          + "classpath. Since 2.0.0, this file was replaced by 'accumulo.properties'. Run the "
-          + "following command to convert an old 'accumulo-site.xml' file to the new format: "
-          + "accumulo convert-config -x /old/accumulo-site.xml -p /new/accumulo.properties");
-    }
+  /**
+   * Build a SiteConfiguration from the environmental configuration and no overrides.
+   */
+  public static SiteConfiguration auto() {
+    return new SiteConfiguration.Builder().fromEnv().build();
+  }
+
+  private final Map<String,String> config;
 
-    String configFile = System.getProperty("accumulo.properties", "accumulo.properties");
-    if (configFile.startsWith("file://")) {
+  private SiteConfiguration(Map<String,String> config) {
+    ConfigSanityCheck.validate(config.entrySet());
+    this.config = config;
+  }
+
+  // load properties from config file
+  private static AbstractConfiguration getPropsFileConfig(URL accumuloPropsLocation) {
+    if (accumuloPropsLocation != null) {
+      var propsBuilder = new FileBasedConfigurationBuilder<>(PropertiesConfiguration.class)
+          .configure(new Parameters().properties().setURL(accumuloPropsLocation));
       try {
-        File f = new File(new URI(configFile));
-        if (f.exists() && !f.isDirectory()) {
-          log.info("Found Accumulo configuration at {}", configFile);
-          return f.toURI().toURL();
-        } else {
-          throw new IllegalArgumentException(
-              "Failed to load Accumulo configuration at " + configFile);
-        }
-      } catch (MalformedURLException | URISyntaxException e) {
-        throw new IllegalArgumentException(
-            "Failed to load Accumulo configuration from " + configFile, e);
-      }
-    } else {
-      URL accumuloConfigUrl = SiteConfiguration.class.getClassLoader().getResource(configFile);
-      if (accumuloConfigUrl == null) {
-        throw new IllegalArgumentException(
-            "Failed to load Accumulo configuration '" + configFile + "' from classpath");
-      } else {
-        log.info("Found Accumulo configuration on classpath at {}", accumuloConfigUrl.getFile());
-        return accumuloConfigUrl;
+        return propsBuilder.getConfiguration();
+      } catch (ConfigurationException e) {
+        throw new IllegalArgumentException(e);
       }
     }
+    return new PropertiesConfiguration();
+  }
+
+  // load sensitive properties from Hadoop credential provider
+  private static void loadCredProviderProps(String provider, Map<String,String> props) {
+    var hadoopConf = new org.apache.hadoop.conf.Configuration();
+    HadoopCredentialProvider.setPath(hadoopConf, provider);
+    Stream.of(Property.values()).filter(Property::isSensitive).forEach(p -> {
+      char[] value = HadoopCredentialProvider.getValue(hadoopConf, p.getKey());
+      if (value != null) {
+        props.put(p.getKey(), new String(value));
+      }
+    });
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
index 2a50a16..f195701 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.accumulo.core.cli.ConfigOpts;
-import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory;
 import org.apache.accumulo.core.crypto.CryptoServiceFactory.ClassloaderType;
 import org.apache.accumulo.core.crypto.CryptoUtils;
@@ -147,7 +146,7 @@ public class PrintInfo implements KeywordExecutable {
       System.err.println("No files were given");
       System.exit(1);
     }
-    SiteConfiguration siteConfig = opts.getSiteConfiguration();
+    var siteConfig = opts.getSiteConfiguration();
 
     Configuration conf = new Configuration();
     for (String confFile : opts.configFiles) {
@@ -166,9 +165,9 @@ public class PrintInfo implements KeywordExecutable {
     for (String arg : opts.files) {
       Path path = new Path(arg);
       FileSystem fs;
-      if (arg.contains(":"))
+      if (arg.contains(":")) {
         fs = path.getFileSystem(conf);
-      else {
+      } else {
         log.warn(
             "Attempting to find file across filesystems. Consider providing URI instead of path");
         fs = hadoopFs.exists(path) ? hadoopFs : localFs; // fall back to local
@@ -183,13 +182,16 @@ public class PrintInfo implements KeywordExecutable {
       Reader iter = new RFile.Reader(cb);
       MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> vmg = new VisMetricsGatherer();
 
-      if (opts.vis || opts.hash)
+      if (opts.vis || opts.hash) {
         iter.registerMetrics(vmg);
+      }
 
       iter.printInfo(opts.printIndex);
       System.out.println();
-      org.apache.accumulo.core.file.rfile.bcfile.PrintInfo
-          .main(new String[] {"-props", opts.getPropertiesPath(), arg});
+      String propsPath = opts.getPropertiesPath();
+      String[] mainArgs =
+          propsPath == null ? new String[] {arg} : new String[] {"-props", propsPath, arg};
+      org.apache.accumulo.core.file.rfile.bcfile.PrintInfo.main(mainArgs);
 
       Map<String,ArrayList<ByteSequence>> localityGroupCF = null;
 
@@ -223,8 +225,9 @@ public class PrintInfo implements KeywordExecutable {
             Value value = dataIter.getTopValue();
             if (opts.dump) {
               System.out.println(key + " -> " + value);
-              if (System.out.checkError())
+              if (System.out.checkError()) {
                 return;
+              }
             }
             if (opts.histogram) {
               kvHistogram.add(key.getSize() + value.getSize());
@@ -262,8 +265,9 @@ public class PrintInfo implements KeywordExecutable {
         indexKeyStats.print("\t");
       }
       // If the output stream has closed, there is no reason to keep going.
-      if (System.out.checkError())
+      if (System.out.checkError()) {
         return;
+      }
     }
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
index 820ae2a..9263e27 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
@@ -76,16 +76,17 @@ public class PrintInfo {
       System.err.println("No files were given");
       System.exit(-1);
     }
-    SiteConfiguration siteConfig = opts.getSiteConfiguration();
+    var siteConfig = opts.getSiteConfiguration();
     Configuration conf = new Configuration();
     FileSystem hadoopFs = FileSystem.get(conf);
     FileSystem localFs = FileSystem.getLocal(conf);
     Path path = new Path(opts.file);
     FileSystem fs;
-    if (opts.file.contains(":"))
+    if (opts.file.contains(":")) {
       fs = path.getFileSystem(conf);
-    else
+    } else {
       fs = hadoopFs.exists(path) ? hadoopFs : localFs; // fall back to local
+    }
     printMetaBlockInfo(siteConfig, conf, fs, path);
   }
 }
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 3a05b9d..8efac67 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
@@ -42,8 +42,9 @@ public class SiteConfigurationTest {
     assertNotNull(keystore);
     String credProvPath = "jceks://file" + new File(keystore.getFile()).getAbsolutePath();
 
-    SiteConfiguration config = new SiteConfiguration(
-        Map.of(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), credProvPath));
+    var overrides =
+        Map.of(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), credProvPath);
+    var config = new SiteConfiguration.Builder().noFile().withOverrides(overrides).build();
 
     assertEquals("mysecret", config.get(Property.INSTANCE_SECRET));
     assertNull(config.get("ignored.property"));
@@ -53,7 +54,7 @@ public class SiteConfigurationTest {
 
   @Test
   public void testDefault() {
-    SiteConfiguration conf = new SiteConfiguration();
+    var conf = SiteConfiguration.auto();
     assertEquals("localhost:2181", conf.get(Property.INSTANCE_ZK_HOST));
     assertEquals("DEFAULT", conf.get(Property.INSTANCE_SECRET));
     assertEquals("", conf.get(Property.INSTANCE_VOLUMES));
@@ -66,7 +67,7 @@ public class SiteConfigurationTest {
   @Test
   public void testFile() {
     URL propsUrl = getClass().getClassLoader().getResource("accumulo2.properties");
-    SiteConfiguration conf = new SiteConfiguration(propsUrl);
+    var conf = new SiteConfiguration.Builder().fromUrl(propsUrl).build();
     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));
@@ -78,14 +79,14 @@ public class SiteConfigurationTest {
 
   @Test
   public void testConfigOverrides() {
-    SiteConfiguration conf = new SiteConfiguration();
+    var conf = SiteConfiguration.auto();
     assertEquals("localhost:2181", conf.get(Property.INSTANCE_ZK_HOST));
 
-    conf = new SiteConfiguration((URL) null,
-        Map.of(Property.INSTANCE_ZK_HOST.getKey(), "myhost:2181"));
+    conf = new SiteConfiguration.Builder().noFile()
+        .withOverrides(Map.of(Property.INSTANCE_ZK_HOST.getKey(), "myhost:2181")).build();
     assertEquals("myhost:2181", conf.get(Property.INSTANCE_ZK_HOST));
 
-    Map<String,String> results = new HashMap<>();
+    var results = new HashMap<String,String>();
     conf.getProperties(results, p -> p.startsWith("instance"));
     assertEquals("myhost:2181", results.get(Property.INSTANCE_ZK_HOST.getKey()));
   }
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
index 90b5853..e4bfca4 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
@@ -74,7 +74,8 @@ public class StandaloneAccumuloCluster implements AccumuloCluster {
     this.tmp = tmp;
     this.users = users;
     this.serverAccumuloConfDir = serverAccumuloConfDir;
-    siteConfig = new SiteConfiguration(new File(serverAccumuloConfDir, "accumulo.properties"));
+    siteConfig =
+        SiteConfiguration.fromFile(new File(serverAccumuloConfDir, "accumulo.properties")).build();
   }
 
   public String getAccumuloHome() {
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
index eeacdb1..43f5eec 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
@@ -41,10 +41,11 @@ public class MiniAccumuloInstance extends org.apache.accumulo.core.client.ZooKee
 
   // Keep this private to avoid bringing it into the public API
   private static String getZooKeepersFromDir(File directory) {
-    if (!directory.isDirectory())
+    if (!directory.isDirectory()) {
       throw new IllegalArgumentException("Not a directory " + directory.getPath());
+    }
     File configFile = new File(new File(directory, "conf"), "accumulo.properties");
-    SiteConfiguration conf = new SiteConfiguration(configFile);
+    var conf = SiteConfiguration.fromFile(configFile).build();
     return conf.get(Property.INSTANCE_ZK_HOST);
   }
 }
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
index e968564..a996588 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -146,8 +146,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
   public ProcessInfo exec(Class<?> clazz, List<String> jvmArgs, String... args) throws IOException {
     ArrayList<String> jvmArgs2 = new ArrayList<>(1 + (jvmArgs == null ? 0 : jvmArgs.size()));
     jvmArgs2.add("-Xmx" + config.getDefaultMemory());
-    if (jvmArgs != null)
+    if (jvmArgs != null) {
       jvmArgs2.addAll(jvmArgs);
+    }
     return _exec(clazz, jvmArgs2, args);
   }
 
@@ -155,9 +156,10 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     StringBuilder classpathBuilder = new StringBuilder();
     classpathBuilder.append(config.getConfDir().getAbsolutePath());
 
-    if (config.getHadoopConfDir() != null)
+    if (config.getHadoopConfDir() != null) {
       classpathBuilder.append(File.pathSeparator)
           .append(config.getHadoopConfDir().getAbsolutePath());
+    }
 
     if (config.getClasspathItems() == null) {
       String javaClassPath = System.getProperty("java.class.path");
@@ -166,8 +168,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       }
       classpathBuilder.append(File.pathSeparator).append(javaClassPath);
     } else {
-      for (String s : config.getClasspathItems())
+      for (String s : config.getClasspathItems()) {
         classpathBuilder.append(File.pathSeparator).append(s);
+      }
     }
 
     return classpathBuilder.toString();
@@ -236,14 +239,17 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
 
     // if we're running under accumulo.start, we forward these env vars
     String env = System.getenv("HADOOP_HOME");
-    if (env != null)
+    if (env != null) {
       builder.environment().put("HADOOP_HOME", env);
+    }
     env = System.getenv("ZOOKEEPER_HOME");
-    if (env != null)
+    if (env != null) {
       builder.environment().put("ZOOKEEPER_HOME", env);
+    }
     builder.environment().put("ACCUMULO_CONF_DIR", config.getConfDir().getAbsolutePath());
-    if (config.getHadoopConfDir() != null)
+    if (config.getHadoopConfDir() != null) {
       builder.environment().put("HADOOP_CONF_DIR", config.getHadoopConfDir().getAbsolutePath());
+    }
 
     log.debug("Starting MiniAccumuloCluster process with class: " + clazz.getSimpleName()
         + "\n, jvmOpts: " + extraJvmOpts + "\n, classpath: " + classpath + "\n, args: " + argList
@@ -311,8 +317,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     mkdirs(config.getLibExtDir());
 
     if (!config.useExistingInstance()) {
-      if (!config.useExistingZooKeepers())
+      if (!config.useExistingZooKeepers()) {
         mkdirs(config.getZooKeeperDir());
+      }
       mkdirs(config.getAccumuloDir());
     }
 
@@ -333,10 +340,11 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       conf.set("dfs.datanode.data.dir.perm", MiniDFSUtil.computeDatanodeDirectoryPermission());
       String oldTestBuildData = System.setProperty("test.build.data", dfs.getAbsolutePath());
       miniDFS = new MiniDFSCluster.Builder(conf).build();
-      if (oldTestBuildData == null)
+      if (oldTestBuildData == null) {
         System.clearProperty("test.build.data");
-      else
+      } else {
         System.setProperty("test.build.data", oldTestBuildData);
+      }
       miniDFS.waitClusterUp();
       InetSocketAddress dfsAddress = miniDFS.getNameNode().getNameNodeAddress();
       dfsUri = "hdfs://" + dfsAddress.getHostName() + ":" + dfsAddress.getPort();
@@ -376,7 +384,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
 
     File siteFile = new File(config.getConfDir(), "accumulo.properties");
     writeConfigProperties(siteFile, config.getSiteConfig());
-    siteConfig = new SiteConfiguration(siteFile);
+    siteConfig = SiteConfiguration.fromFile(siteFile).build();
 
     if (!config.useExistingInstance() && !config.useExistingZooKeepers()) {
       zooCfgFile = new File(config.getConfDir(), "zoo.cfg");
@@ -423,8 +431,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
   private void writeConfigProperties(File file, Map<String,String> settings) throws IOException {
     FileWriter fileWriter = new FileWriter(file);
 
-    for (Entry<String,String> entry : settings.entrySet())
+    for (Entry<String,String> entry : settings.entrySet()) {
       fileWriter.append(entry.getKey() + "=" + entry.getValue() + "\n");
+    }
     fileWriter.close();
   }
 
@@ -474,13 +483,15 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       } catch (KeeperException e) {
         throw new RuntimeException("Unable to read instance name from zookeeper.", e);
       }
-      if (instanceName == null)
+      if (instanceName == null) {
         throw new RuntimeException("Unable to read instance name from zookeeper.");
+      }
 
       config.setInstanceName(instanceName);
-      if (!AccumuloStatus.isAccumuloOffline(zrw, rootPath))
+      if (!AccumuloStatus.isAccumuloOffline(zrw, rootPath)) {
         throw new RuntimeException(
             "The Accumulo instance being used is already running. Aborting.");
+      }
     } else {
       if (!initialized) {
         Runtime.getRuntime().addShutdownHook(new Thread(() -> {
@@ -494,8 +505,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
         }));
       }
 
-      if (!config.useExistingZooKeepers())
+      if (!config.useExistingZooKeepers()) {
         control.start(ServerType.ZOOKEEPER);
+      }
 
       if (!initialized) {
         if (!config.useExistingZooKeepers()) {
@@ -510,8 +522,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
               s.getOutputStream().flush();
               byte[] buffer = new byte[100];
               int n = s.getInputStream().read(buffer);
-              if (n >= 4 && new String(buffer, 0, 4).equals("imok"))
+              if (n >= 4 && new String(buffer, 0, 4).equals("imok")) {
                 break;
+              }
             } catch (Exception e) {
               if (System.currentTimeMillis() - startTime >= config.getZooKeeperStartupTime()) {
                 throw new ZooKeeperBindException("Zookeeper did not start within "
@@ -521,8 +534,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
               // Don't spin absurdly fast
               sleepUninterruptibly(250, TimeUnit.MILLISECONDS);
             } finally {
-              if (s != null)
+              if (s != null) {
                 s.close();
+              }
             }
           }
         }
@@ -561,8 +575,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
     for (int i = 0; i < 5; i++) {
       ret = exec(Main.class, SetGoalState.class.getName(), MasterGoalState.NORMAL.toString())
           .getProcess().waitFor();
-      if (ret == 0)
+      if (ret == 0) {
         break;
+      }
       sleepUninterruptibly(1, TimeUnit.SECONDS);
     }
     if (ret != 0) {
@@ -670,8 +685,9 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       executor = null;
     }
 
-    if (config.useMiniDFS() && miniDFS != null)
+    if (config.useMiniDFS() && miniDFS != null) {
       miniDFS.shutdown();
+    }
     for (Process p : cleanup) {
       p.destroy();
       p.waitFor();
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 7b8c9e1..db9a203 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
@@ -719,7 +719,7 @@ public class MiniAccumuloConfigImpl {
     System.setProperty("accumulo.properties", "accumulo.properties");
     this.hadoopConfDir = hadoopConfDir;
     hadoopConf = new Configuration(false);
-    accumuloConf = new SiteConfiguration(accumuloProps);
+    accumuloConf = SiteConfiguration.fromFile(accumuloProps).build();
     File coreSite = new File(hadoopConfDir, "core-site.xml");
     File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml");
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java
index 0eb7a61..9029459 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java
@@ -21,7 +21,6 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.server.metrics.Metrics;
 import org.apache.accumulo.server.security.SecurityUtil;
@@ -42,7 +41,7 @@ public abstract class AbstractServer implements AutoCloseable, Runnable {
     this.applicationName = appName;
     this.hostname = Objects.requireNonNull(opts.getAddress());
     opts.parseArgs(appName, args);
-    SiteConfiguration siteConfig = opts.getSiteConfiguration();
+    var siteConfig = opts.getSiteConfiguration();
     context = new ServerContext(siteConfig);
     SecurityUtil.serverLogin(siteConfig);
     log.info("Version " + Constants.VERSION);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java
index 1578b9a..4635239 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ServerUtilOpts.java
@@ -27,9 +27,9 @@ public class ServerUtilOpts extends ClientOpts {
   public synchronized ServerContext getServerContext() {
     if (context == null) {
       if (getClientConfigFile() == null) {
-        context = new ServerContext(new SiteConfiguration());
+        context = new ServerContext(SiteConfiguration.auto());
       } else {
-        context = new ServerContext(new SiteConfiguration(), getClientProps());
+        context = new ServerContext(SiteConfiguration.auto(), getClientProps());
       }
     }
     return context;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java
index fcffcfb..f7d89b7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java
@@ -26,7 +26,7 @@ import com.google.auto.service.AutoService;
 public class ConfigSanityCheck implements KeywordExecutable {
 
   public static void main(String[] args) {
-    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+    try (var context = new ServerContext(SiteConfiguration.auto())) {
       context.getServerConfFactory().getSystemConfiguration();
     }
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index b6b1a8d..a1a4e60 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -403,7 +403,7 @@ public class Initialize implements KeywordExecutable {
       // If they did not, fall back to the credentials present in accumulo.properties that the
       // servers will use themselves.
       try {
-        final SiteConfiguration siteConf = context.getServerConfFactory().getSiteConfiguration();
+        final var siteConf = context.getServerConfFactory().getSiteConfiguration();
         if (siteConf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) {
           final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
           // We don't have any valid creds to talk to HDFS
@@ -946,7 +946,7 @@ public class Initialize implements KeywordExecutable {
   public void execute(final String[] args) {
     Opts opts = new Opts();
     opts.parseArgs("accumulo init", args);
-    SiteConfiguration siteConfig = new SiteConfiguration();
+    var siteConfig = SiteConfiguration.auto();
 
     try {
       setZooReaderWriter(new ZooReaderWriter(siteConfig));
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
index b903cf2..3d8d483 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
@@ -85,28 +85,23 @@ public class ProblemReports implements Iterable<ProblemReport> {
       problemReports.put(pr, System.currentTimeMillis());
     }
 
-    Runnable r = new Runnable() {
-
-      @Override
-      public void run() {
-
-        log.debug("Filing problem report {} {} {}", pr.getTableId(), pr.getProblemType(),
-            pr.getResource());
-
-        try {
-          if (isMeta(pr.getTableId())) {
-            // file report in zookeeper
-            pr.saveToZooKeeper(context);
-          } else {
-            // file report in metadata table
-            pr.saveToMetadataTable(context);
-          }
-        } catch (Exception e) {
-          log.error("Failed to file problem report " + pr.getTableId() + " " + pr.getProblemType()
-              + " " + pr.getResource(), e);
+    Runnable r = () -> {
+
+      log.debug("Filing problem report {} {} {}", pr.getTableId(), pr.getProblemType(),
+          pr.getResource());
+
+      try {
+        if (isMeta(pr.getTableId())) {
+          // file report in zookeeper
+          pr.saveToZooKeeper(context);
+        } else {
+          // file report in metadata table
+          pr.saveToMetadataTable(context);
         }
+      } catch (Exception e) {
+        log.error("Failed to file problem report " + pr.getTableId() + " " + pr.getProblemType()
+            + " " + pr.getResource(), e);
       }
-
     };
 
     try {
@@ -128,22 +123,18 @@ public class ProblemReports implements Iterable<ProblemReport> {
   public void deleteProblemReport(TableId table, ProblemType pType, String resource) {
     final ProblemReport pr = new ProblemReport(table, pType, resource, null);
 
-    Runnable r = new Runnable() {
-
-      @Override
-      public void run() {
-        try {
-          if (isMeta(pr.getTableId())) {
-            // file report in zookeeper
-            pr.removeFromZooKeeper(context);
-          } else {
-            // file report in metadata table
-            pr.removeFromMetadataTable(context);
-          }
-        } catch (Exception e) {
-          log.error("Failed to delete problem report {} {} {}", pr.getTableId(),
-              pr.getProblemType(), pr.getResource(), e);
+    Runnable r = () -> {
+      try {
+        if (isMeta(pr.getTableId())) {
+          // file report in zookeeper
+          pr.removeFromZooKeeper(context);
+        } else {
+          // file report in metadata table
+          pr.removeFromMetadataTable(context);
         }
+      } catch (Exception e) {
+        log.error("Failed to delete problem report {} {} {}", pr.getTableId(), pr.getProblemType(),
+            pr.getResource(), e);
       }
     };
 
@@ -180,8 +171,9 @@ public class ProblemReports implements Iterable<ProblemReport> {
       delMut.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier());
     }
 
-    if (hasProblems)
+    if (hasProblems) {
       MetadataTableUtil.getMetadataTable(context).update(delMut);
+    }
   }
 
   private static boolean isMeta(TableId tableId) {
@@ -191,7 +183,7 @@ public class ProblemReports implements Iterable<ProblemReport> {
   public Iterator<ProblemReport> iterator(final TableId table) {
     try {
 
-      return new Iterator<ProblemReport>() {
+      return new Iterator<>() {
 
         IZooReaderWriter zoo = context.getZooReaderWriter();
         private int iter1Count = 0;
@@ -296,7 +288,7 @@ public class ProblemReports implements Iterable<ProblemReport> {
   }
 
   public static void main(String[] args) {
-    ServerContext context = new ServerContext(new SiteConfiguration());
+    var context = new ServerContext(SiteConfiguration.auto());
     getInstance(context).printProblems();
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java b/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
index 7980462..a8a7d3b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
@@ -61,7 +61,7 @@ public class ChangeSecret {
   }
 
   public static void main(String[] args) throws Exception {
-    SiteConfiguration siteConfig = new SiteConfiguration();
+    var siteConfig = SiteConfiguration.auto();
     VolumeManager fs = VolumeManagerImpl.get(siteConfig, new Configuration());
     verifyHdfsWritePermission(fs);
 
@@ -108,12 +108,10 @@ public class ChangeSecret {
         context.getZooKeepersSessionTimeOut(), oldPassword);
     String root = context.getZooKeeperRoot();
     final List<String> ephemerals = new ArrayList<>();
-    recurse(zooReader, root, new Visitor() {
-      @Override
-      public void visit(ZooReader zoo, String path) throws Exception {
-        Stat stat = zoo.getStatus(path);
-        if (stat.getEphemeralOwner() != 0)
-          ephemerals.add(path);
+    recurse(zooReader, root, (zoo, path) -> {
+      Stat stat = zoo.getStatus(path);
+      if (stat.getEphemeralOwner() != 0) {
+        ephemerals.add(path);
       }
     });
     if (ephemerals.size() > 0) {
@@ -133,28 +131,25 @@ public class ChangeSecret {
         context.getZooKeepersSessionTimeOut(), newPass);
 
     String root = context.getZooKeeperRoot();
-    recurse(orig, root, new Visitor() {
-      @Override
-      public void visit(ZooReader zoo, String path) throws Exception {
-        String newPath = path.replace(context.getInstanceID(), newInstanceId);
-        byte[] data = zoo.getData(path, null);
-        List<ACL> acls = orig.getZooKeeper().getACL(path, new Stat());
-        if (acls.containsAll(Ids.READ_ACL_UNSAFE)) {
-          new_.putPersistentData(newPath, data, NodeExistsPolicy.FAIL);
-        } else {
-          // upgrade
-          if (acls.containsAll(Ids.OPEN_ACL_UNSAFE)) {
-            // make user nodes private, they contain the user's password
-            String[] parts = path.split("/");
-            if (parts[parts.length - 2].equals("users")) {
-              new_.putPrivatePersistentData(newPath, data, NodeExistsPolicy.FAIL);
-            } else {
-              // everything else can have the readable acl
-              new_.putPersistentData(newPath, data, NodeExistsPolicy.FAIL);
-            }
-          } else {
+    recurse(orig, root, (zoo, path) -> {
+      String newPath = path.replace(context.getInstanceID(), newInstanceId);
+      byte[] data = zoo.getData(path, null);
+      List<ACL> acls = orig.getZooKeeper().getACL(path, new Stat());
+      if (acls.containsAll(Ids.READ_ACL_UNSAFE)) {
+        new_.putPersistentData(newPath, data, NodeExistsPolicy.FAIL);
+      } else {
+        // upgrade
+        if (acls.containsAll(Ids.OPEN_ACL_UNSAFE)) {
+          // make user nodes private, they contain the user's password
+          String[] parts = path.split("/");
+          if (parts[parts.length - 2].equals("users")) {
             new_.putPrivatePersistentData(newPath, data, NodeExistsPolicy.FAIL);
+          } else {
+            // everything else can have the readable acl
+            new_.putPersistentData(newPath, data, NodeExistsPolicy.FAIL);
           }
+        } else {
+          new_.putPrivatePersistentData(newPath, data, NodeExistsPolicy.FAIL);
         }
       }
     });
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java b/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
index b2ce11b..aee6ae0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
@@ -50,7 +50,7 @@ public class CleanZookeeper {
     Opts opts = new Opts();
     opts.parseArgs(CleanZookeeper.class.getName(), args);
 
-    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+    try (var context = new ServerContext(SiteConfiguration.auto())) {
 
       String root = Constants.ZROOT;
       IZooReaderWriter zk = context.getZooReaderWriter();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/DeleteZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/util/DeleteZooInstance.java
index de06b32..7e37415 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/DeleteZooInstance.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/DeleteZooInstance.java
@@ -59,7 +59,7 @@ public class DeleteZooInstance {
     Opts opts = new Opts();
     opts.parseArgs(DeleteZooInstance.class.getName(), args);
 
-    ZooReaderWriter zk = new ZooReaderWriter(new SiteConfiguration());
+    var zk = new ZooReaderWriter(SiteConfiguration.auto());
     // try instance name:
     Set<String> instances = new HashSet<>(zk.getChildren(Constants.ZROOT + Constants.ZINSTANCES));
     Set<String> uuids = new HashSet<>(zk.getChildren(Constants.ZROOT));
@@ -74,8 +74,9 @@ public class DeleteZooInstance {
       for (String instance : instances) {
         String path = Constants.ZROOT + Constants.ZINSTANCES + "/" + instance;
         byte[] data = zk.getData(path, null);
-        if (opts.instance.equals(new String(data, UTF_8)))
+        if (opts.instance.equals(new String(data, UTF_8))) {
           deleteRetry(zk, path);
+        }
       }
       deleteRetry(zk, Constants.ZROOT + "/" + opts.instance);
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Info.java b/server/base/src/main/java/org/apache/accumulo/server/util/Info.java
index 5a768c7..7b16159 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Info.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Info.java
@@ -44,7 +44,7 @@ public class Info implements KeywordExecutable {
 
   @Override
   public void execute(final String[] args) throws KeeperException, InterruptedException {
-    ServerContext context = new ServerContext(new SiteConfiguration());
+    var context = new ServerContext(SiteConfiguration.auto());
     System.out.println("monitor: " + MonitorUtil.getLocation(context));
     System.out.println("masters: " + context.getMasterLocations());
     System.out.println("zookeepers: " + context.getZooKeepers());
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
index 17c8798..9e9333d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
@@ -65,7 +65,7 @@ public class ListInstances {
     opts.parseArgs(ListInstances.class.getName(), args);
 
     if (opts.keepers == null) {
-      SiteConfiguration siteConfig = new SiteConfiguration();
+      var siteConfig = SiteConfiguration.auto();
       opts.keepers = siteConfig.get(Property.INSTANCE_ZK_HOST);
     }
 
@@ -126,8 +126,9 @@ public class ListInstances {
     public void formatTo(Formatter formatter, int flags, int width, int precision) {
 
       StringBuilder sb = new StringBuilder();
-      for (int i = 0; i < width; i++)
+      for (int i = 0; i < width; i++) {
         sb.append(c);
+      }
       formatter.format(sb.toString());
     }
 
@@ -212,8 +213,9 @@ public class ListInstances {
       List<String> children = zk.getChildren(Constants.ZROOT);
 
       for (String iid : children) {
-        if (iid.equals("instances"))
+        if (iid.equals("instances")) {
           continue;
+        }
         try {
           ts.add(UUID.fromString(iid));
         } catch (Exception e) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
index e65b001..5e758a2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
@@ -37,20 +37,22 @@ import org.apache.hadoop.fs.Path;
 public class ListVolumesUsed {
 
   public static void main(String[] args) throws Exception {
-    listVolumes(new ServerContext(new SiteConfiguration()));
+    listVolumes(new ServerContext(SiteConfiguration.auto()));
   }
 
   private static String getTableURI(String rootTabletDir) {
     Path ret = FileType.TABLE.getVolume(new Path(rootTabletDir));
-    if (ret == null)
+    if (ret == null) {
       return "RELATIVE";
+    }
     return ret.toString();
   }
 
   private static String getLogURI(String logEntry) {
     Path ret = FileType.WAL.getVolume(new Path(logEntry));
-    if (ret == null)
+    if (ret == null) {
       return "RELATIVE";
+    }
     return ret.toString();
   }
 
@@ -70,8 +72,9 @@ public class ListVolumesUsed {
       getLogURIs(volumes, logEntry);
     }
 
-    for (String volume : volumes)
+    for (String volume : volumes) {
       System.out.println("\tVolume : " + volume);
+    }
 
   }
 
@@ -102,8 +105,9 @@ public class ListVolumesUsed {
       }
     }
 
-    for (String volume : volumes)
+    for (String volume : volumes) {
       System.out.println("\tVolume : " + volume);
+    }
 
     volumes.clear();
 
@@ -119,8 +123,9 @@ public class ListVolumesUsed {
     System.out.println("Listing volumes referenced in " + name
         + " deletes section (volume replacement occurrs at deletion time)");
 
-    for (String volume : volumes)
+    for (String volume : volumes) {
       System.out.println("\tVolume : " + volume);
+    }
 
     volumes.clear();
 
@@ -131,8 +136,9 @@ public class ListVolumesUsed {
 
     System.out.println("Listing volumes referenced in " + name + " current logs");
 
-    for (String volume : volumes)
+    for (String volume : volumes) {
       System.out.println("\tVolume : " + volume);
+    }
   }
 
   public static void listVolumes(ServerContext context) throws Exception {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java b/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java
index df2e5dd..f83ba80 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java
@@ -43,7 +43,7 @@ public class LoginProperties implements KeywordExecutable {
 
   @Override
   public void execute(String[] args) throws Exception {
-    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+    try (var context = new ServerContext(SiteConfiguration.auto())) {
       AccumuloConfiguration config = context.getServerConfFactory().getSystemConfiguration();
       Authenticator authenticator = AccumuloVFSClassLoader.getClassLoader()
           .loadClass(config.get(Property.INSTANCE_SECURITY_AUTHENTICATOR))
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java b/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
index 4ef08bf..4084df9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RestoreZookeeper.java
@@ -119,7 +119,7 @@ public class RestoreZookeeper {
     Opts opts = new Opts();
     opts.parseArgs(RestoreZookeeper.class.getName(), args);
 
-    ZooReaderWriter zoo = new ZooReaderWriter(new SiteConfiguration());
+    var zoo = new ZooReaderWriter(SiteConfiguration.auto());
 
     InputStream in = System.in;
     if (opts.file != null) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
index 52e8d6c..08591ab 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
@@ -41,7 +41,7 @@ public class TabletServerLocks {
 
   public static void main(String[] args) throws Exception {
 
-    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+    try (var context = new ServerContext(SiteConfiguration.auto())) {
       String tserverPath = context.getZooKeeperRoot() + Constants.ZTSERVERS;
       Opts opts = new Opts();
       opts.parseArgs(TabletServerLocks.class.getName(), args);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java b/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
index 7760836..b3823bb 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
@@ -64,7 +64,7 @@ public class ZooKeeperMain implements KeywordExecutable {
   public void execute(final String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(ZooKeeperMain.class.getName(), args);
-    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+    try (var context = new ServerContext(SiteConfiguration.auto())) {
       FileSystem fs = context.getVolumeManager().getDefaultVolume().getFileSystem();
       String baseDir = ServerConstants.getBaseUris(context)[0];
       System.out.println("Using " + fs.makeQualified(new Path(baseDir + "/instance_id"))
@@ -73,8 +73,9 @@ public class ZooKeeperMain implements KeywordExecutable {
         opts.servers = context.getZooKeepers();
       }
       System.out.println("The accumulo instance id is " + context.getInstanceID());
-      if (!opts.servers.contains("/"))
+      if (!opts.servers.contains("/")) {
         opts.servers += "/accumulo/" + context.getInstanceID();
+      }
       org.apache.zookeeper.ZooKeeperMain
           .main(new String[] {"-server", opts.servers, "-timeout", "" + (opts.timeout * 1000)});
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
index 05ae191..eb98192 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
@@ -42,8 +42,9 @@ public class ZooZap {
   private static final Logger log = LoggerFactory.getLogger(ZooZap.class);
 
   private static void message(String msg, Opts opts) {
-    if (opts.verbose)
+    if (opts.verbose) {
       System.out.println(msg);
+    }
   }
 
   static class Opts extends Help {
@@ -66,7 +67,7 @@ public class ZooZap {
       return;
     }
 
-    SiteConfiguration siteConf = new SiteConfiguration();
+    var siteConf = SiteConfiguration.auto();
     Configuration hadoopConf = new Configuration();
     // Login as the server on secure HDFS
     if (siteConf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) {
@@ -95,9 +96,9 @@ public class ZooZap {
         for (String child : children) {
           message("Deleting " + tserversPath + "/" + child + " from zookeeper", opts);
 
-          if (opts.zapMaster)
+          if (opts.zapMaster) {
             zoo.recursiveDelete(tserversPath + "/" + child, NodeMissingPolicy.SKIP);
-          else {
+          } else {
             String path = tserversPath + "/" + child;
             if (zoo.getChildren(path).size() > 0) {
               if (!ZooLock.deleteLock(zoo, path, "tserver")) {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java
index bf9dbb9..7a4f90a 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/ServerConfigurationFactoryTest.java
@@ -51,7 +51,7 @@ public class ServerConfigurationFactoryTest {
   // use the same mock ZooCacheFactory and ZooCache for all tests
   private static ZooCacheFactory zcf;
   private static ZooCache zc;
-  private static SiteConfiguration siteConfig = new SiteConfiguration();
+  private static SiteConfiguration siteConfig = SiteConfiguration.auto();
 
   @BeforeClass
   public static void setUpClass() {
@@ -107,7 +107,7 @@ public class ServerConfigurationFactoryTest {
   @Test
   public void testGetSiteConfiguration() {
     ready();
-    SiteConfiguration c = scf.getSiteConfiguration();
+    var c = scf.getSiteConfiguration();
     assertNotNull(c);
   }
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
index e7944a0..69fab92 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
@@ -88,7 +88,7 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
         TestDefaultBalancer.class.getName());
   }
 
-  private static SiteConfiguration siteConfg = new SiteConfiguration();
+  private static SiteConfiguration siteConfg = SiteConfiguration.auto();
 
   protected static class TestServerConfigurationFactory extends ServerConfigurationFactory {
 
@@ -253,10 +253,11 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
         && (host.equals("192.168.0.6") || host.equals("192.168.0.7") || host.equals("192.168.0.8")
             || host.equals("192.168.0.9") || host.equals("192.168.0.10"))) {
       return true;
-    } else
+    } else {
       return tid.equals("3") && (host.equals("192.168.0.11") || host.equals("192.168.0.12")
           || host.equals("192.168.0.13") || host.equals("192.168.0.14")
           || host.equals("192.168.0.15"));
+    }
   }
 
   protected String idToTableName(TableId id) {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
index f596684..d5616ca 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
@@ -148,7 +148,7 @@ public class TableLoadBalancerTest {
     final ServerContext context = createMockContext();
     replay(context);
     ServerConfigurationFactory confFactory =
-        new ServerConfigurationFactory(context, new SiteConfiguration()) {
+        new ServerConfigurationFactory(context, SiteConfiguration.auto()) {
           @Override
           public TableConfiguration getTableConfiguration(TableId tableId) {
             // create a dummy namespaceConfiguration to satisfy requireNonNull in TableConfiguration
diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
index a419840..5a2d7a4 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
@@ -42,7 +42,7 @@ public class SystemCredentialsTest {
   @Rule
   public TestName test = new TestName();
 
-  private static SiteConfiguration siteConfig = new SiteConfiguration();
+  private static SiteConfiguration siteConfig = SiteConfiguration.auto();
   private String instanceId =
       UUID.nameUUIDFromBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 0}).toString();
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
index 4f855db..4e40bb9 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
@@ -62,7 +62,7 @@ public class TServerUtilsTest {
     private ConfigurationCopy conf = null;
 
     public TestServerConfigurationFactory(ServerContext context) {
-      super(context, new SiteConfiguration());
+      super(context, SiteConfiguration.auto());
       conf = new ConfigurationCopy(DefaultConfiguration.getInstance());
     }
 
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
index 8bf950e..5c2435f 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
@@ -49,7 +49,7 @@ public class SimpleGarbageCollectorTest {
   private Credentials credentials;
   private SimpleGarbageCollector gc;
   private ConfigurationCopy systemConfig;
-  private static SiteConfiguration siteConfig = new SiteConfiguration();
+  private static SiteConfiguration siteConfig = SiteConfiguration.auto();
 
   @Before
   public void setUp() {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/state/SetGoalState.java b/server/master/src/main/java/org/apache/accumulo/master/state/SetGoalState.java
index ca764fa..ecbf85f 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/state/SetGoalState.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/state/SetGoalState.java
@@ -38,7 +38,7 @@ public class SetGoalState {
       System.exit(-1);
     }
 
-    ServerContext context = new ServerContext(new SiteConfiguration());
+    var context = new ServerContext(SiteConfiguration.auto());
     SecurityUtil.serverLogin(context.getConfiguration());
     ServerUtil.waitForZookeeperAndHdfs(context);
     context.getZooReaderWriter().putPersistentData(
diff --git a/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
index 21e0850..a6a5d74 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
@@ -77,7 +77,7 @@ public class FateAdmin {
 
     AdminUtil<Master> admin = new AdminUtil<>();
 
-    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+    try (var context = new ServerContext(SiteConfiguration.auto())) {
       final String zkRoot = context.getZooKeeperRoot();
       String path = zkRoot + Constants.ZFATE;
       String masterPath = zkRoot + Constants.ZMASTER_LOCK;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
index da37c12..d8c582d 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/AccumuloMonitorAppender.java
@@ -145,7 +145,7 @@ public class AccumuloMonitorAppender extends AsyncAppender implements AutoClosea
     public MonitorLocation get() {
       // lazily set up path and zooCache (see comment in constructor)
       if (this.context == null) {
-        this.context = new ServerContext(new SiteConfiguration());
+        this.context = new ServerContext(SiteConfiguration.auto());
         this.path = context.getZooKeeperRoot() + Constants.ZMONITOR_LOG4J_ADDR;
         this.zooCache = context.getZooCache();
       }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
index 2f38fd9..2149d85 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
@@ -76,7 +76,7 @@ public class LogReader {
   public static void main(String[] args) throws IOException {
     Opts opts = new Opts();
     opts.parseArgs(LogReader.class.getName(), args);
-    SiteConfiguration siteConfig = new SiteConfiguration();
+    var siteConfig = SiteConfiguration.auto();
     VolumeManager fs = VolumeManagerImpl.get(siteConfig, new Configuration());
 
     Matcher rowMatcher = null;
@@ -86,8 +86,9 @@ public class LogReader {
       new JCommander(opts).usage();
       return;
     }
-    if (opts.row != null)
+    if (opts.row != null) {
       row = new Text(opts.row);
+    }
     if (opts.extent != null) {
       String[] sa = opts.extent.split(";");
       ke = new KeyExtent(TableId.of(sa[0]), new Text(sa[1]), new Text(sa[2]));
@@ -174,8 +175,9 @@ public class LogReader {
           }
         }
 
-        if (!found)
+        if (!found) {
           return;
+        }
       } else {
         return;
       }
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
index 00ac9df..ad491e4 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
@@ -113,7 +113,7 @@ public class FateCommand extends Command {
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
       throws ParseException, KeeperException, InterruptedException, IOException {
     ClientContext context = shellState.getContext();
-    SiteConfiguration siteConfig = new SiteConfiguration();
+    var siteConfig = SiteConfiguration.auto();
     String[] args = cl.getArgs();
     if (args.length <= 0) {
       throw new ParseException("Must provide a command to execute");
@@ -174,11 +174,11 @@ public class FateCommand extends Command {
       if (cl.hasOption(statusOption.getOpt())) {
         filterStatus = EnumSet.noneOf(TStatus.class);
         String[] tstat = cl.getOptionValues(statusOption.getOpt());
-        for (int i = 0; i < tstat.length; i++) {
+        for (String element : tstat) {
           try {
-            filterStatus.add(TStatus.valueOf(tstat[i]));
+            filterStatus.add(TStatus.valueOf(element));
           } catch (IllegalArgumentException iae) {
-            System.out.printf("Invalid transaction status name: %s%n", tstat[i]);
+            System.out.printf("Invalid transaction status name: %s%n", element);
             return 1;
           }
         }
diff --git a/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java b/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java
index fcdec02..56ca506 100644
--- a/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/GetMasterStats.java
@@ -41,7 +41,7 @@ public class GetMasterStats {
   public static void main(String[] args) throws Exception {
     MasterClientService.Iface client = null;
     MasterMonitorInfo stats = null;
-    ServerContext context = new ServerContext(new SiteConfiguration());
+    var context = new ServerContext(SiteConfiguration.auto());
     while (true) {
       try {
         client = MasterClient.getConnectionWithRetry(context);
@@ -51,8 +51,9 @@ public class GetMasterStats {
         // Let it loop, fetching a new location
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } finally {
-        if (client != null)
+        if (client != null) {
           MasterClient.close(client);
+        }
       }
     }
     out(0, "State: " + stats.state.name());
@@ -109,8 +110,9 @@ public class GetMasterStats {
         out(2, "Time Difference: %.1f", ((now - server.lastContact) / 1000.));
         out(2, "Total Records: %d", summary.recs);
         out(2, "Lookups: %d", server.lookups);
-        if (server.holdTime > 0)
+        if (server.holdTime > 0) {
           out(2, "Hold Time: %d", server.holdTime);
+        }
         if (server.tableMap != null && server.tableMap.size() > 0) {
           out(2, "Tables");
           for (Entry<String,TableInfo> status : server.tableMap.entrySet()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java
index 3ac49c1..99dd7c3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestClean.java
@@ -34,7 +34,7 @@ public class CacheTestClean {
     String rootDir = args[0];
     File reportDir = new File(args[1]);
 
-    SiteConfiguration siteConfig = new SiteConfiguration();
+    var siteConfig = SiteConfiguration.auto();
     IZooReaderWriter zoo = new ZooReaderWriter(siteConfig);
 
     if (zoo.exists(rootDir)) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
index f8e473d..25bcab3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CacheTestWriter.java
@@ -32,7 +32,6 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -46,7 +45,7 @@ public class CacheTestWriter {
   @SuppressFBWarnings(value = {"PATH_TRAVERSAL_IN", "OBJECT_DESERIALIZATION"},
       justification = "path provided by test; object deserialization is okay for test")
   public static void main(String[] args) throws Exception {
-    IZooReaderWriter zk = new ZooReaderWriter(new SiteConfiguration());
+    var zk = new ZooReaderWriter(SiteConfiguration.auto());
 
     String rootDir = args[0];
     File reportDir = new File(args[1]);
@@ -159,8 +158,9 @@ public class CacheTestWriter {
             }
           }
 
-          if (ok)
+          if (ok) {
             break;
+          }
         }
 
         sleepUninterruptibly(5, TimeUnit.MILLISECONDS);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
index 704299b..4ecb5ef 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
@@ -292,7 +292,7 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
   }
 
   public static void main(String[] args) throws Exception {
-    new SplitRecoveryIT().run(new ServerContext(new SiteConfiguration()));
+    new SplitRecoveryIT().run(new ServerContext(SiteConfiguration.auto()));
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
index 79a9291..3f6f03a 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
@@ -101,7 +101,7 @@ public class ZombieTServer {
     Random random = new SecureRandom();
     random.setSeed(System.currentTimeMillis() % 1000);
     int port = random.nextInt(30000) + 2000;
-    ServerContext context = new ServerContext(new SiteConfiguration());
+    var context = new ServerContext(SiteConfiguration.auto());
     TransactionWatcher watcher = new TransactionWatcher(context);
     final ThriftClientHandler tch = new ThriftClientHandler(context, watcher);
     Processor<Iface> processor = new Processor<>(tch);
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
index be015dd..13d3b28 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java
@@ -292,7 +292,7 @@ public class NullTserver {
     // modify metadata
     int zkTimeOut =
         (int) DefaultConfiguration.getInstance().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
-    SiteConfiguration siteConfig = new SiteConfiguration();
+    var siteConfig = SiteConfiguration.auto();
     ServerContext context = new ServerContext(siteConfig, opts.iname, opts.keepers, zkTimeOut);
     TransactionWatcher watcher = new TransactionWatcher(context);
     ThriftClientHandler tch = new ThriftClientHandler(context, watcher);
diff --git a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
index 222d757..7f35548 100644
--- a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
@@ -58,12 +58,13 @@ public class SystemCredentialsIT extends ConfigurableMacBase {
   }
 
   public static void main(final String[] args) throws AccumuloException, TableNotFoundException {
-    SiteConfiguration siteConfig = new SiteConfiguration();
+    var siteConfig = SiteConfiguration.auto();
     try (ServerContext context = new ServerContext(siteConfig)) {
       Credentials creds;
       String badInstanceID = SystemCredentials.class.getName();
-      if (args.length < 2)
+      if (args.length < 2) {
         throw new RuntimeException("Incorrect usage; expected to be run by test only");
+      }
       switch (args[0]) {
         case "bad":
           creds = SystemCredentials.get(badInstanceID, siteConfig);
diff --git a/test/src/main/java/org/apache/accumulo/test/util/CertUtils.java b/test/src/main/java/org/apache/accumulo/test/util/CertUtils.java
index 349fad5..2ed1134 100644
--- a/test/src/main/java/org/apache/accumulo/test/util/CertUtils.java
+++ b/test/src/main/java/org/apache/accumulo/test/util/CertUtils.java
@@ -127,9 +127,9 @@ public class CertUtils {
     @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "path provided by test")
     public SiteConfiguration getSiteConfiguration() {
       if (accumuloPropsFile == null) {
-        return new SiteConfiguration();
+        return SiteConfiguration.auto();
       } else {
-        return new SiteConfiguration(new File(accumuloPropsFile));
+        return SiteConfiguration.fromFile(new File(accumuloPropsFile)).build();
       }
     }
   }