You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/08/24 15:06:24 UTC

[GitHub] mikewalch closed pull request #613: Removed all use of SiteConfiguration.getInstance()

mikewalch closed pull request #613: Removed all use of SiteConfiguration.getInstance()
URL: https://github.com/apache/accumulo/pull/613
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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
new file mode 100644
index 0000000000..37d68245b4
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ConfigOpts.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.cli;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.converters.IParameterSplitter;
+
+public class ConfigOpts extends Help {
+
+  private static final Logger log = LoggerFactory.getLogger(ConfigOpts.class);
+
+  @Parameter(names = {"--site", "-site"}, description = "Sets path to accumulo-site.xml. The"
+      + " classpath will be searched if this property is not set")
+  private String sitePath;
+
+  public String getSitePath() {
+    return sitePath;
+  }
+
+  public static class NullSplitter implements IParameterSplitter {
+    @Override
+    public List<String> split(String value) {
+      return Collections.singletonList(value);
+    }
+  }
+
+  @Parameter(names = "-o", splitter = NullSplitter.class,
+      description = "Overrides configuration set in accumulo-site.xml (but NOT system-wide config"
+          + " set in Zookeeper). Expected format: -o <key>=<value>")
+  private List<String> overrides = new ArrayList<>();
+
+  private SiteConfiguration siteConfig = null;
+
+  public synchronized SiteConfiguration getSiteConfiguration() {
+    if (siteConfig == null) {
+      if (sitePath != null) {
+        siteConfig = new SiteConfiguration(new File(sitePath), getOverrides());
+      } else {
+        siteConfig = new SiteConfiguration(SiteConfiguration.getAccumuloSiteLocation(),
+            getOverrides());
+      }
+    }
+    return siteConfig;
+  }
+
+  public Map<String,String> getOverrides() {
+    Map<String,String> config = new HashMap<>();
+    for (String prop : overrides) {
+      String[] propArgs = prop.split("=", 2);
+      if (propArgs.length == 2) {
+        String key = propArgs[0].trim();
+        String value = propArgs[1].trim();
+        if (key.isEmpty() || value.isEmpty()) {
+          throw new IllegalArgumentException("Invalid command line -o option: " + prop);
+        } else {
+          config.put(key, value);
+        }
+      } else {
+        throw new IllegalArgumentException("Invalid command line -o option: " + prop);
+      }
+    }
+    return config;
+  }
+
+  @Override
+  public void parseArgs(String programName, String[] args, Object... others) {
+    super.parseArgs(programName, args, others);
+    if (getOverrides().size() > 0) {
+      log.info("The following configuration was set on the command line:");
+      for (Map.Entry<String,String> entry : getOverrides().entrySet()) {
+        String key = entry.getKey();
+        log.info(key + " = " + (Property.isSensitive(key) ? "<hidden>" : entry.getValue()));
+      }
+    }
+  }
+}
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 c66ac174f4..1373444f92 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
@@ -53,17 +53,28 @@
   private static final Logger log = LoggerFactory.getLogger(SiteConfiguration.class);
 
   private static final AccumuloConfiguration parent = DefaultConfiguration.getInstance();
-  private static SiteConfiguration instance = null;
 
   private final Map<String,String> overrides;
-  private static Configuration xmlConfig;
+  private final Configuration xmlConfig;
   private final Map<String,String> staticConfigs;
 
-  private SiteConfiguration() {
-    this(getAccumuloSiteLocation(), Collections.emptyMap());
+  public SiteConfiguration() {
+    this(getAccumuloSiteLocation());
   }
 
-  private SiteConfiguration(URL accumuloSiteLocation, Map<String,String> overrides) {
+  public SiteConfiguration(File accumuloSiteFile) {
+    this(accumuloSiteFile, Collections.emptyMap());
+  }
+
+  public SiteConfiguration(File accumuloSiteFile, Map<String,String> overrides) {
+    this(toURL(accumuloSiteFile), overrides);
+  }
+
+  public SiteConfiguration(URL accumuloSiteLocation) {
+    this(accumuloSiteLocation, Collections.emptyMap());
+  }
+
+  public SiteConfiguration(URL accumuloSiteLocation, Map<String,String> overrides) {
     this.overrides = overrides;
     /*
      * Make a read-only copy of static configs so we can avoid lock contention on the Hadoop
@@ -99,53 +110,6 @@ private static URL toURL(File f) {
     }
   }
 
-  synchronized public static SiteConfiguration create() {
-    if (instance == null) {
-      instance = new SiteConfiguration();
-      ConfigSanityCheck.validate(instance);
-    } else {
-      log.warn("SiteConfiguration was previously created! Returning previous instance.");
-    }
-    return instance;
-  }
-
-  synchronized public static SiteConfiguration create(URL accumuloSiteUrl,
-      Map<String,String> overrides) {
-    if (instance == null) {
-      instance = new SiteConfiguration(accumuloSiteUrl, overrides);
-      ConfigSanityCheck.validate(instance);
-    } else {
-      log.warn("SiteConfiguration was previously created! Returning previous instance.");
-    }
-    return instance;
-  }
-
-  public static SiteConfiguration create(URL accumuloSiteUrl) {
-    return create(accumuloSiteUrl, Collections.emptyMap());
-  }
-
-  public static SiteConfiguration create(File accumuloSiteFile, Map<String,String> overrides) {
-    return create(toURL(accumuloSiteFile), overrides);
-  }
-
-  public static SiteConfiguration create(File accumuloSiteFile) {
-    return create(toURL(accumuloSiteFile), Collections.emptyMap());
-  }
-
-  /**
-   * Gets an instance of this class. A new instance is only created on the first call.
-   *
-   * @throws IllegalArgumentException
-   *           if the configuration is invalid or accumulo-site.xml is not on classpath.
-   */
-  synchronized public static SiteConfiguration getInstance() {
-    if (instance == null) {
-      instance = new SiteConfiguration();
-      ConfigSanityCheck.validate(instance);
-    }
-    return instance;
-  }
-
   public static URL getAccumuloSiteLocation() {
     String configFile = System.getProperty("accumulo.configuration", "accumulo-site.xml");
     if (configFile.startsWith("file://")) {
@@ -292,14 +256,6 @@ protected Configuration getHadoopConfiguration() {
     return null;
   }
 
-  /**
-   * Clears the configuration properties in this configuration (but not the parent). This method
-   * supports testing and should not be called.
-   */
-  synchronized public static void clearInstance() {
-    instance = null;
-  }
-
   /**
    * Sets a property. This method supports testing and should not be called.
    *
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 3f75c2f994..2beb686eed 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
@@ -22,7 +22,7 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.accumulo.core.cli.Help;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -54,7 +54,7 @@
 
   private static final Logger log = LoggerFactory.getLogger(PrintInfo.class);
 
-  static class Opts extends Help {
+  static class Opts extends ConfigOpts {
     @Parameter(names = {"-d", "--dump"}, description = "dump the key/value pairs")
     boolean dump = false;
     @Parameter(names = {"-v", "--vis"}, description = "show visibility metrics")
@@ -142,7 +142,7 @@ public void execute(final String[] args) throws Exception {
       System.err.println("No files were given");
       System.exit(-1);
     }
-    SiteConfiguration siteConfig = SiteConfiguration.create();
+    SiteConfiguration siteConfig = opts.getSiteConfiguration();
 
     Configuration conf = new Configuration();
     for (String confFile : opts.configFiles) {
@@ -183,7 +183,8 @@ public void execute(final String[] args) throws Exception {
 
       iter.printInfo(opts.printIndex);
       System.out.println();
-      org.apache.accumulo.core.file.rfile.bcfile.PrintInfo.main(new String[] {arg});
+      org.apache.accumulo.core.file.rfile.bcfile.PrintInfo
+          .main(new String[] {"-site", opts.getSitePath(), arg});
 
       Map<String,ArrayList<ByteSequence>> localityGroupCF = null;
 
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 43b97db674..9b91c3a971 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
@@ -21,6 +21,7 @@
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile.MetaIndexEntry;
 import org.apache.accumulo.core.security.crypto.CryptoServiceFactory;
@@ -29,6 +30,8 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
+import com.beust.jcommander.Parameter;
+
 public class PrintInfo {
   public static void printMetaBlockInfo(SiteConfiguration siteConfig, Configuration conf,
       FileSystem fs, Path path) throws IOException {
@@ -58,14 +61,27 @@ public static void printMetaBlockInfo(SiteConfiguration siteConfig, Configuratio
     }
   }
 
+  static class Opts extends ConfigOpts {
+
+    @Parameter(description = " <file>")
+    String file;
+
+  }
+
   public static void main(String[] args) throws Exception {
-    SiteConfiguration siteConfig = SiteConfiguration.create();
+    Opts opts = new Opts();
+    opts.parseArgs("PrintInfo", args);
+    if (opts.file.isEmpty()) {
+      System.err.println("No files were given");
+      System.exit(-1);
+    }
+    SiteConfiguration siteConfig = opts.getSiteConfiguration();
     Configuration conf = new Configuration();
     FileSystem hadoopFs = FileSystem.get(conf);
     FileSystem localFs = FileSystem.getLocal(conf);
-    Path path = new Path(args[0]);
+    Path path = new Path(opts.file);
     FileSystem fs;
-    if (args[0].contains(":"))
+    if (opts.file.contains(":"))
       fs = path.getFileSystem(conf);
     else
       fs = hadoopFs.exists(path) ? hadoopFs : localFs; // fall back to local
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 d2449ba3ac..3632646613 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
@@ -45,7 +45,7 @@ public static void checkCredentialProviderAvailable() {
 
   @Test
   public void testOnlySensitivePropertiesExtractedFromCredentialProvider()
-      throws SecurityException, NoSuchMethodException {
+      throws SecurityException {
     if (!isCredentialProviderAvailable) {
       return;
     }
@@ -81,11 +81,10 @@ public void testOnlySensitivePropertiesExtractedFromCredentialProvider()
 
   @Test
   public void testConfigOverrides() {
-    SiteConfiguration conf = SiteConfiguration.create();
+    SiteConfiguration conf = new SiteConfiguration();
     Assert.assertEquals("localhost:2181", conf.get(Property.INSTANCE_ZK_HOST));
 
-    SiteConfiguration.clearInstance();
-    conf = SiteConfiguration.create((URL) null,
+    conf = new SiteConfiguration((URL) null,
         ImmutableMap.of(Property.INSTANCE_ZK_HOST.getKey(), "myhost:2181"));
     Assert.assertEquals("myhost:2181", conf.get(Property.INSTANCE_ZK_HOST));
 
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
index 8fdda8345d..759028df5f 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
@@ -76,6 +76,11 @@ Connector getConnector(String user, AuthenticationToken token)
    */
   AccumuloConfiguration getSiteConfiguration();
 
+  /**
+   * Get path to accumulo-site.xml
+   */
+  String getSitePath();
+
   /**
    * Get an object that can manage a cluster
    *
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 281aa48915..b99feeda48 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
@@ -71,7 +71,7 @@ public StandaloneAccumuloCluster(ClientInfo info, Path tmp, List<ClusterUser> us
     this.tmp = tmp;
     this.users = users;
     this.serverAccumuloConfDir = serverAccumuloConfDir;
-    siteConfig = SiteConfiguration.create(new File(serverAccumuloConfDir, "accumulo-site.xml"));
+    siteConfig = new SiteConfiguration(new File(serverAccumuloConfDir, "accumulo-site.xml"));
   }
 
   public String getAccumuloHome() {
@@ -215,4 +215,9 @@ public ClusterUser getUser(int offset) {
   public AccumuloConfiguration getSiteConfiguration() {
     return new ConfigurationCopy(siteConfig);
   }
+
+  @Override
+  public String getSitePath() {
+    return new File(serverAccumuloConfDir, "accumulo-site.xml").toString();
+  }
 }
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index c0a1069803..472603ff64 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -463,8 +463,7 @@ public MiniAccumuloClusterImpl(MiniAccumuloConfigImpl config) throws IOException
 
     File siteFile = new File(config.getConfDir(), "accumulo-site.xml");
     writeConfig(siteFile, config.getSiteConfig().entrySet());
-    SiteConfiguration.clearInstance();
-    siteConfig = SiteConfiguration.create(siteFile);
+    siteConfig = new SiteConfiguration(siteFile);
 
     if (!config.useExistingInstance() && !config.useExistingZooKeepers()) {
       zooCfgFile = new File(config.getConfDir(), "zoo.cfg");
@@ -888,4 +887,9 @@ public AccumuloConfiguration getSiteConfiguration() {
     return new ConfigurationCopy(
         Iterables.concat(DefaultConfiguration.getInstance(), config.getSiteConfig().entrySet()));
   }
+
+  @Override
+  public String getSitePath() {
+    return new File(config.getConfDir(), "accumulo-site.xml").toString();
+  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
index 4fe7f62049..9db6d06478 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
@@ -38,10 +38,8 @@
 import org.apache.accumulo.fate.ReadOnlyStore;
 import org.apache.accumulo.fate.ReadOnlyTStore;
 import org.apache.accumulo.fate.ZooStore;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.util.time.SimpleTimer;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -106,15 +104,14 @@ public static synchronized Path getAccumuloInstanceIdPath(VolumeManager fs) {
     return ServerConstants.getInstanceIdLocation(v);
   }
 
-  public static void init(VolumeManager fs, String instanceID,
-      ServerConfigurationFactory serverConfig, String application) throws IOException {
-    final AccumuloConfiguration conf = serverConfig.getSystemConfiguration();
+  public static void init(ServerContext context, String application) throws IOException {
+    final AccumuloConfiguration conf = context.getConfiguration();
 
     log.info("{} starting", application);
-    log.info("Instance {}", instanceID);
-    int dataVersion = Accumulo.getAccumuloPersistentVersion(fs);
+    log.info("Instance {}", context.getInstanceID());
+    int dataVersion = Accumulo.getAccumuloPersistentVersion(context.getVolumeManager());
     log.info("Data Version {}", dataVersion);
-    Accumulo.waitForZookeeperAndHdfs(fs);
+    Accumulo.waitForZookeeperAndHdfs(context);
 
     if (!(canUpgradeFromDataVersion(dataVersion))) {
       throw new RuntimeException("This version of accumulo (" + Constants.VERSION
@@ -193,11 +190,11 @@ public void run() {
     }, 1000, 10 * 60 * 1000);
   }
 
-  public static void waitForZookeeperAndHdfs(VolumeManager fs) {
+  public static void waitForZookeeperAndHdfs(ServerContext context) {
     log.info("Attempting to talk to zookeeper");
     while (true) {
       try {
-        ZooReaderWriter.getInstance().getChildren(Constants.ZROOT);
+        context.getZooReaderWriter().getChildren(Constants.ZROOT);
         break;
       } catch (InterruptedException e) {
         // ignored
@@ -211,7 +208,7 @@ public static void waitForZookeeperAndHdfs(VolumeManager fs) {
     int unknownHostTries = 3;
     while (true) {
       try {
-        if (fs.isReady())
+        if (context.getVolumeManager().isReady())
           break;
         log.warn("Waiting for the NameNode to leave safemode");
       } catch (IOException ex) {
@@ -264,7 +261,7 @@ public static void waitForZookeeperAndHdfs(VolumeManager fs) {
   public static void abortIfFateTransactions(ServerContext context) {
     try {
       final ReadOnlyTStore<Accumulo> fate = new ReadOnlyStore<>(new ZooStore<>(
-          context.getZooKeeperRoot() + Constants.ZFATE, ZooReaderWriter.getInstance()));
+          context.getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter()));
       if (!(fate.list().isEmpty())) {
         throw new AccumuloException("Aborting upgrade because there are"
             + " outstanding FATE transactions from a previous Accumulo version."
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java b/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
index 51a54f9d77..d45d210c5a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
@@ -24,8 +24,8 @@
 import java.util.HashSet;
 import java.util.List;
 
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.volume.Volume;
@@ -88,16 +88,16 @@
   private static List<Pair<Path,Path>> replacementsList = null;
 
   // these are functions to delay loading the Accumulo configuration unless we must
-  public static synchronized String[] getBaseUris() {
+  public static synchronized String[] getBaseUris(AccumuloConfiguration conf) {
     if (baseUris == null) {
-      baseUris = checkBaseUris(VolumeConfiguration.getVolumeUris(SiteConfiguration.getInstance()),
-          false);
+      baseUris = checkBaseUris(conf, VolumeConfiguration.getVolumeUris(conf), false);
     }
 
     return baseUris;
   }
 
-  public static String[] checkBaseUris(String[] configuredBaseDirs, boolean ignore) {
+  public static String[] checkBaseUris(AccumuloConfiguration conf, String[] configuredBaseDirs,
+      boolean ignore) {
     // all base dirs must have same instance id and data version, any dirs that have neither should
     // be ignored
     String firstDir = null;
@@ -109,7 +109,7 @@
       String currentIid;
       Integer currentVersion;
       try {
-        currentIid = ZooUtil.getInstanceIDFromHdfs(path, SiteConfiguration.getInstance());
+        currentIid = ZooUtil.getInstanceIDFromHdfs(path, conf);
         Path vpath = new Path(baseDir, VERSION_DIR);
         currentVersion = Accumulo.getAccumuloPersistentVersion(
             vpath.getFileSystem(CachedConfiguration.getInstance()), vpath);
@@ -148,16 +148,16 @@
   public static final String RECOVERY_DIR = "recovery";
   public static final String WAL_DIR = "wal";
 
-  public static String[] getTablesDirs() {
-    return VolumeConfiguration.prefix(getBaseUris(), TABLE_DIR);
+  public static String[] getTablesDirs(AccumuloConfiguration conf) {
+    return VolumeConfiguration.prefix(getBaseUris(conf), TABLE_DIR);
   }
 
-  public static String[] getRecoveryDirs() {
-    return VolumeConfiguration.prefix(getBaseUris(), RECOVERY_DIR);
+  public static String[] getRecoveryDirs(AccumuloConfiguration conf) {
+    return VolumeConfiguration.prefix(getBaseUris(conf), RECOVERY_DIR);
   }
 
-  public static String[] getWalDirs() {
-    return VolumeConfiguration.prefix(getBaseUris(), WAL_DIR);
+  public static String[] getWalDirs(AccumuloConfiguration conf) {
+    return VolumeConfiguration.prefix(getBaseUris(conf), WAL_DIR);
   }
 
   public static Path getInstanceIdLocation(Volume v) {
@@ -170,11 +170,11 @@ public static Path getDataVersionLocation(Volume v) {
     return v.prefixChild(VERSION_DIR);
   }
 
-  public static synchronized List<Pair<Path,Path>> getVolumeReplacements() {
+  public static synchronized List<Pair<Path,Path>> getVolumeReplacements(
+      AccumuloConfiguration conf) {
 
     if (replacementsList == null) {
-      String replacements = SiteConfiguration.getInstance()
-          .get(Property.INSTANCE_VOLUMES_REPLACEMENTS);
+      String replacements = conf.get(Property.INSTANCE_VOLUMES_REPLACEMENTS);
 
       replacements = replacements.trim();
 
@@ -217,7 +217,7 @@ public static Path getDataVersionLocation(Volume v) {
       }
 
       HashSet<Path> baseDirs = new HashSet<>();
-      for (String baseDir : getBaseUris()) {
+      for (String baseDir : getBaseUris(conf)) {
         // normalize using path
         baseDirs.add(new Path(baseDir));
       }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index dc71a4d032..7246e7e32e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -45,6 +45,7 @@
 import org.apache.accumulo.server.security.delegation.AuthenticationTokenSecretManager;
 import org.apache.accumulo.server.tables.TableManager;
 import org.apache.accumulo.server.tablets.UniqueNameAllocator;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -60,6 +61,7 @@
   private final ServerInfo info;
   private TableManager tableManager;
   private UniqueNameAllocator nameAllocator;
+  private ZooReaderWriter zooReaderWriter;
   private ServerConfigurationFactory serverConfFactory = null;
   private String applicationName = null;
   private String applicationClassName = null;
@@ -84,6 +86,7 @@ public ServerContext(SiteConfiguration siteConfig, ClientInfo info) {
   private ServerContext(ServerInfo info) {
     super(info, info.getSiteConfiguration());
     this.info = info;
+    zooReaderWriter = new ZooReaderWriter(info.getSiteConfiguration());
   }
 
   public void setupServer(String appName, String appClassName, String hostname) {
@@ -94,7 +97,7 @@ public void setupServer(String appName, String appClassName, String hostname) {
     log.info("Version " + Constants.VERSION);
     log.info("Instance " + info.getInstanceID());
     try {
-      Accumulo.init(getVolumeManager(), getInstanceID(), getServerConfFactory(), applicationName);
+      Accumulo.init(this, applicationName);
     } catch (IOException e) {
       throw new IllegalStateException(e);
     }
@@ -178,6 +181,10 @@ public VolumeManager getVolumeManager() {
     return info.getVolumeManager();
   }
 
+  public ZooReaderWriter getZooReaderWriter() {
+    return zooReaderWriter;
+  }
+
   /**
    * Retrieve the SSL/TLS configuration for starting up a listening service
    */
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java b/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java
index 77e73b0dd6..d7c69b9e06 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java
@@ -56,6 +56,11 @@
     this.instanceName = instanceName;
     this.zooKeepers = zooKeepers;
     this.zooKeepersSessionTimeOut = zooKeepersSessionTimeOut;
+    try {
+      volumeManager = VolumeManagerImpl.get(siteConfig);
+    } catch (IOException e) {
+      throw new IllegalStateException(e);
+    }
     zooCache = new ZooCacheFactory().getZooCache(zooKeepers, zooKeepersSessionTimeOut);
     String instanceNamePath = Constants.ZROOT + Constants.ZINSTANCES + "/" + instanceName;
     byte[] iidb = zooCache.get(instanceNamePath);
@@ -77,7 +82,7 @@
   ServerInfo(SiteConfiguration config) {
     siteConfig = config;
     try {
-      volumeManager = VolumeManagerImpl.get();
+      volumeManager = VolumeManagerImpl.get(siteConfig);
     } catch (IOException e) {
       throw new IllegalStateException(e);
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java b/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java
index 355e412649..67601554f7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java
@@ -16,88 +16,18 @@
  */
 package org.apache.accumulo.server;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.accumulo.core.cli.Help;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.accumulo.core.cli.ConfigOpts;
 
 import com.beust.jcommander.Parameter;
-import com.beust.jcommander.converters.IParameterSplitter;
-
-public class ServerOpts extends Help {
 
-  private static final Logger log = LoggerFactory.getLogger(ServerOpts.class);
+public class ServerOpts extends ConfigOpts {
 
   @Parameter(names = {"-a", "--address"}, description = "address to bind to")
   private String address = null;
 
-  public static class NullSplitter implements IParameterSplitter {
-    @Override
-    public List<String> split(String value) {
-      return Collections.singletonList(value);
-    }
-  }
-
-  @Parameter(names = "-o", splitter = NullSplitter.class,
-      description = "Overrides configuration set in accumulo-site.xml (but NOT system-wide config"
-          + " set in Zookeeper). Expected format: -o <key>=<value>")
-  private List<String> properties = new ArrayList<>();
-
   public String getAddress() {
     if (address != null)
       return address;
     return "0.0.0.0";
   }
-
-  public List<String> getProperties() {
-    return properties;
-  }
-
-  private SiteConfiguration siteConfig = null;
-
-  public synchronized SiteConfiguration getSiteConfiguration() {
-    if (siteConfig == null) {
-      siteConfig = SiteConfiguration.create(SiteConfiguration.getAccumuloSiteLocation(),
-          getConfig());
-    }
-    return siteConfig;
-  }
-
-  public Map<String,String> getConfig() {
-    Map<String,String> config = new HashMap<>();
-    for (String prop : getProperties()) {
-      String[] propArgs = prop.split("=", 2);
-      if (propArgs.length == 2) {
-        String key = propArgs[0].trim();
-        String value = propArgs[1].trim();
-        if (key.isEmpty() || value.isEmpty()) {
-          throw new IllegalArgumentException("Invalid command line -o option: " + prop);
-        } else {
-          config.put(key, value);
-        }
-      } else {
-        throw new IllegalArgumentException("Invalid command line -o option: " + prop);
-      }
-    }
-    return config;
-  }
-
-  @Override
-  public void parseArgs(String programName, String[] args, Object... others) {
-    super.parseArgs(programName, args, others);
-    if (getConfig().size() > 0) {
-      log.info("The following configuration was set on the command line:");
-      for (Map.Entry<String,String> entry : getConfig().entrySet()) {
-        String key = entry.getKey();
-        log.info(key + " = " + (Property.isSensitive(key) ? "<hidden>" : entry.getValue()));
-      }
-    }
-  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
index b0f4f96602..02a9e04c6b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
@@ -29,9 +29,9 @@
   public synchronized ServerContext getServerContext() {
     if (context == null) {
       if (instance == null) {
-        context = new ServerContext(SiteConfiguration.create());
+        context = new ServerContext(new SiteConfiguration());
       } else {
-        context = new ServerContext(SiteConfiguration.create(), getClientInfo());
+        context = new ServerContext(new SiteConfiguration(), getClientInfo());
       }
     }
     return context;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
index bacd7698c0..98ee62f7c6 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
@@ -29,9 +29,9 @@
   public synchronized ServerContext getServerContext() {
     if (context == null) {
       if (instance == null) {
-        context = new ServerContext(SiteConfiguration.create());
+        context = new ServerContext(new SiteConfiguration());
       } else {
-        context = new ServerContext(SiteConfiguration.create(), getClientInfo());
+        context = new ServerContext(new SiteConfiguration(), getClientInfo());
       }
     }
     return context;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
index ba4caa74a1..8f5ed5cf59 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
@@ -34,9 +34,9 @@ public ClientContext getClientContext() {
   public synchronized ServerContext getServerContext() {
     if (context == null) {
       if (instance == null) {
-        context = new ServerContext(SiteConfiguration.create());
+        context = new ServerContext(new SiteConfiguration());
       } else {
-        context = new ServerContext(SiteConfiguration.create(), getClientInfo());
+        context = new ServerContext(new SiteConfiguration(), getClientInfo());
       }
     }
     return context;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
deleted file mode 100644
index 854f88303b..0000000000
--- a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.client;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientInfo;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.impl.ClientConfConverter;
-import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.client.impl.ConnectorImpl;
-import org.apache.accumulo.core.client.impl.InstanceOperationsImpl;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.util.OpTimer;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
-import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
-import org.apache.accumulo.server.Accumulo;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Joiner;
-
-/**
- * An implementation of Instance that looks in HDFS and ZooKeeper to find the master and root tablet
- * location.
- *
- * @deprecated since 2.0.0, Use {@link ServerContext} instead
- */
-@Deprecated
-public class HdfsZooInstance implements org.apache.accumulo.core.client.Instance {
-
-  private final AccumuloConfiguration site = SiteConfiguration.getInstance();
-
-  private HdfsZooInstance() {
-    zooCache = new ZooCacheFactory().getZooCache(site.get(Property.INSTANCE_ZK_HOST),
-        (int) site.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
-  }
-
-  private static final HdfsZooInstance cachedHdfsZooInstance = new HdfsZooInstance();
-
-  public static org.apache.accumulo.core.client.Instance getInstance() {
-    return cachedHdfsZooInstance;
-  }
-
-  private final ZooCache zooCache;
-  private static String instanceId = null;
-  private static final Logger log = LoggerFactory.getLogger(HdfsZooInstance.class);
-
-  @Override
-  public String getRootTabletLocation() {
-    String zRootLocPath = ZooUtil.getRoot(getInstanceID()) + RootTable.ZROOT_TABLET_LOCATION;
-
-    OpTimer timer = null;
-
-    if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up root tablet location in zoocache.",
-          Thread.currentThread().getId());
-      timer = new OpTimer().start();
-    }
-
-    byte[] loc = zooCache.get(zRootLocPath);
-
-    if (timer != null) {
-      timer.stop();
-      log.trace("tid={} Found root tablet at {} in {}", Thread.currentThread().getId(),
-          (loc == null ? "null" : new String(loc, UTF_8)),
-          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
-    }
-
-    if (loc == null) {
-      return null;
-    }
-
-    return new String(loc, UTF_8).split("\\|")[0];
-  }
-
-  @Override
-  public List<String> getMasterLocations() {
-
-    String masterLocPath = ZooUtil.getRoot(getInstanceID()) + Constants.ZMASTER_LOCK;
-
-    OpTimer timer = null;
-
-    if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up master location in zoocache.", Thread.currentThread().getId());
-      timer = new OpTimer().start();
-    }
-
-    byte[] loc = ZooLock.getLockData(zooCache, masterLocPath, null);
-
-    if (timer != null) {
-      timer.stop();
-      log.trace("tid={} Found master at {} in {}", Thread.currentThread().getId(),
-          (loc == null ? "null" : new String(loc, UTF_8)),
-          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
-    }
-
-    if (loc == null) {
-      return Collections.emptyList();
-    }
-
-    return Collections.singletonList(new String(loc, UTF_8));
-  }
-
-  @Override
-  public String getInstanceID() {
-    if (instanceId == null)
-      _getInstanceID();
-    return instanceId;
-  }
-
-  private static synchronized void _getInstanceID() {
-    if (instanceId == null) {
-      AccumuloConfiguration acuConf = SiteConfiguration.getInstance();
-      // InstanceID should be the same across all volumes, so just choose one
-      VolumeManager fs;
-      try {
-        fs = VolumeManagerImpl.get();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      Path instanceIdPath = Accumulo.getAccumuloInstanceIdPath(fs);
-      log.trace("Looking for instanceId from {}", instanceIdPath);
-      instanceId = ZooUtil.getInstanceIDFromHdfs(instanceIdPath, acuConf);
-    }
-  }
-
-  @Override
-  public String getInstanceName() {
-    return InstanceOperationsImpl.lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
-  }
-
-  @Override
-  public String getZooKeepers() {
-    return site.get(Property.INSTANCE_ZK_HOST);
-  }
-
-  @Override
-  public int getZooKeepersSessionTimeOut() {
-    return (int) site.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
-  }
-
-  @Override
-  public Connector getConnector(String principal, AuthenticationToken token)
-      throws AccumuloException, AccumuloSecurityException {
-    Properties properties = ClientConfConverter.toProperties(site);
-    properties.setProperty(ClientProperty.INSTANCE_NAME.getKey(), getInstanceName());
-    properties.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS.getKey(), getZooKeepers());
-    properties.setProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getKey(),
-        Integer.toString(getZooKeepersSessionTimeOut()));
-    properties.setProperty(ClientProperty.AUTH_PRINCIPAL.getKey(), principal);
-    ClientProperty.setAuthenticationToken(properties, token);
-    return new ConnectorImpl(new ClientContext(ClientInfo.from(properties, token)));
-  }
-
-  public static void main(String[] args) {
-    org.apache.accumulo.core.client.Instance instance = HdfsZooInstance.getInstance();
-    System.out.println("Instance Name: " + instance.getInstanceName());
-    System.out.println("Instance ID: " + instance.getInstanceID());
-    System.out.println("ZooKeepers: " + instance.getZooKeepers());
-    System.out.println("Masters: " + Joiner.on(", ").join(instance.getMasterLocations()));
-  }
-}
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 9c50a10aa9..00932d3766 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 @@
 public class ConfigSanityCheck implements KeywordExecutable {
 
   public static void main(String[] args) {
-    ServerContext context = new ServerContext(SiteConfiguration.create());
+    ServerContext context = new ServerContext(new SiteConfiguration());
     context.getServerConfFactory().getSystemConfiguration();
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfigurationFactory.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfigurationFactory.java
index efb4902ac4..9449fe8446 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfigurationFactory.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfigurationFactory.java
@@ -16,20 +16,13 @@
  */
 package org.apache.accumulo.server.conf;
 
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
-import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.hadoop.fs.Path;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 
@@ -53,24 +46,9 @@
    */
   ZooConfiguration getInstance(ServerContext context, ZooCacheFactory zcf,
       AccumuloConfiguration parent) {
-    String instanceId;
-    if (context == null) {
-      // InstanceID should be the same across all volumes, so just choose one
-      VolumeManager fs;
-      try {
-        fs = VolumeManagerImpl.get();
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      Path instanceIdPath = Accumulo.getAccumuloInstanceIdPath(fs);
-      instanceId = ZooUtil.getInstanceIDFromHdfs(instanceIdPath, parent);
-    } else {
-      instanceId = context.getInstanceID();
-    }
-
     ZooConfiguration config;
     synchronized (instances) {
-      config = instances.get(instanceId);
+      config = instances.get(context.getInstanceID());
       if (config == null) {
         ZooCache propCache;
 
@@ -81,15 +59,10 @@ ZooConfiguration getInstance(ServerContext context, ZooCacheFactory zcf,
           @Override
           public void process(WatchedEvent arg0) {}
         };
-        if (context == null) {
-          propCache = zcf.getZooCache(parent.get(Property.INSTANCE_ZK_HOST),
-              (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), watcher);
-        } else {
-          propCache = zcf.getZooCache(context.getZooKeepers(),
-              context.getZooKeepersSessionTimeOut(), watcher);
-        }
-        config = new ZooConfiguration(instanceId, propCache, parent);
-        instances.put(instanceId, config);
+        propCache = zcf.getZooCache(context.getZooKeepers(), context.getZooKeepersSessionTimeOut(),
+            watcher);
+        config = new ZooConfiguration(context.getInstanceID(), propCache, parent);
+        instances.put(context.getInstanceID(), config);
       }
     }
     return config;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
index 4ac08bc69c..b19b56fc6b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java
@@ -262,7 +262,7 @@ private static boolean isValidColumn(ColumnUpdate cu) {
         } else if (new ColumnFQ(columnUpdate)
             .equals(TabletsSection.ServerColumnFamily.LOCK_COLUMN)) {
           if (zooCache == null) {
-            zooCache = new ZooCache();
+            zooCache = new ZooCache(context);
           }
 
           if (zooRoot == null) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index 4a73dac4e0..02bb9c42aa 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -32,7 +32,6 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.file.rfile.RFile;
@@ -334,11 +333,6 @@ public short getDefaultReplication(Path path) {
     return v.getFileSystem().getDefaultReplication(path);
   }
 
-  public static VolumeManager get() throws IOException {
-    AccumuloConfiguration conf = SiteConfiguration.getInstance();
-    return get(conf);
-  }
-
   static private final String DEFAULT = "";
 
   public static VolumeManager get(AccumuloConfiguration conf) throws IOException {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
index a548064930..936590ba0b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
@@ -57,13 +57,13 @@
   private static final Logger log = LoggerFactory.getLogger(VolumeUtil.class);
   private static final SecureRandom rand = new SecureRandom();
 
-  private static boolean isActiveVolume(Path dir) {
+  private static boolean isActiveVolume(ServerContext context, Path dir) {
 
     // consider relative path as active and take no action
     if (!dir.toString().contains(":"))
       return true;
 
-    for (String tableDir : ServerConstants.getTablesDirs()) {
+    for (String tableDir : ServerConstants.getTablesDirs(context.getConfiguration())) {
       // use Path to normalize tableDir
       if (dir.toString().startsWith(new Path(tableDir).toString()))
         return true;
@@ -166,7 +166,7 @@ public TabletFiles(String dir, List<LogEntry> logEntries,
   public static String switchRootTableVolume(ServerContext context, String location)
       throws IOException {
     String newLocation = switchVolume(location, FileType.TABLE,
-        ServerConstants.getVolumeReplacements());
+        ServerConstants.getVolumeReplacements(context.getConfiguration()));
     if (newLocation != null) {
       MetadataTableUtil.setRootTabletDir(context, newLocation);
       log.info("Volume replaced: {} -> {}", location, newLocation);
@@ -183,7 +183,8 @@ public static String switchRootTableVolume(ServerContext context, String locatio
   public static TabletFiles updateTabletVolumes(ServerContext context, ZooLock zooLock,
       VolumeManager vm, KeyExtent extent, TabletFiles tabletFiles, boolean replicate)
       throws IOException {
-    List<Pair<Path,Path>> replacements = ServerConstants.getVolumeReplacements();
+    List<Pair<Path,Path>> replacements = ServerConstants
+        .getVolumeReplacements(context.getConfiguration());
     log.trace("Using volume replacements: {}", replacements);
 
     List<LogEntry> logsToRemove = new ArrayList<>();
@@ -265,7 +266,7 @@ public static TabletFiles updateTabletVolumes(ServerContext context, ZooLock zoo
   private static String decommisionedTabletDir(ServerContext context, ZooLock zooLock,
       VolumeManager vm, KeyExtent extent, String metaDir) throws IOException {
     Path dir = new Path(metaDir);
-    if (isActiveVolume(dir))
+    if (isActiveVolume(context, dir))
       return metaDir;
 
     if (!dir.getParent().getParent().getName().equals(ServerConstants.TABLE_DIR)) {
@@ -274,9 +275,10 @@ private static String decommisionedTabletDir(ServerContext context, ZooLock zooL
 
     VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(extent.getTableId(),
         context);
-    Path newDir = new Path(vm.choose(chooserEnv, ServerConstants.getBaseUris()) + Path.SEPARATOR
-        + ServerConstants.TABLE_DIR + Path.SEPARATOR + dir.getParent().getName() + Path.SEPARATOR
-        + dir.getName());
+    Path newDir = new Path(
+        vm.choose(chooserEnv, ServerConstants.getBaseUris(context.getConfiguration()))
+            + Path.SEPARATOR + ServerConstants.TABLE_DIR + Path.SEPARATOR
+            + dir.getParent().getName() + Path.SEPARATOR + dir.getName());
 
     log.info("Updating directory for {} from {} to {}", extent, dir, newDir);
     if (extent.isRootTablet()) {
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 ec7aad55dc..592dd559c0 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
@@ -78,7 +78,6 @@
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.Accumulo;
@@ -132,7 +131,7 @@
   private static final String TABLE_TABLETS_TABLET_DIR = "/table_info";
 
   private static ConsoleReader reader = null;
-  private static IZooReaderWriter zoo = null;
+  private static ZooReaderWriter zoo = null;
 
   private static ConsoleReader getConsoleReader() throws IOException {
     if (reader == null)
@@ -143,11 +142,11 @@ private static ConsoleReader getConsoleReader() throws IOException {
   /**
    * Sets this class's ZooKeeper reader/writer.
    *
-   * @param izoo
+   * @param zooReaderWriter
    *          reader/writer
    */
-  static void setZooReaderWriter(IZooReaderWriter izoo) {
-    zoo = izoo;
+  static void setZooReaderWriter(ZooReaderWriter zooReaderWriter) {
+    zoo = zooReaderWriter;
   }
 
   /**
@@ -155,7 +154,7 @@ static void setZooReaderWriter(IZooReaderWriter izoo) {
    *
    * @return reader/writer
    */
-  static IZooReaderWriter getZooReaderWriter() {
+  static ZooReaderWriter getZooReaderWriter() {
     return zoo;
   }
 
@@ -483,15 +482,15 @@ private void initFileSystem(SiteConfiguration siteConfig, VolumeManager fs, UUID
     initDirs(fs, uuid, VolumeConfiguration.getVolumeUris(siteConfig), false);
 
     // initialize initial system tables config in zookeeper
-    initSystemTablesConfig(Constants.ZROOT + "/" + uuid);
+    initSystemTablesConfig(zoo, Constants.ZROOT + "/" + uuid);
 
     VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ChooserScope.INIT);
-    String tableMetadataTabletDir = fs.choose(chooserEnv, ServerConstants.getBaseUris())
+    String tableMetadataTabletDir = fs.choose(chooserEnv, ServerConstants.getBaseUris(siteConfig))
         + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + MetadataTable.ID + TABLE_TABLETS_TABLET_DIR;
-    String replicationTableDefaultTabletDir = fs.choose(chooserEnv, ServerConstants.getBaseUris())
-        + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + ReplicationTable.ID
-        + Constants.DEFAULT_TABLET_LOCATION;
-    String defaultMetadataTabletDir = fs.choose(chooserEnv, ServerConstants.getBaseUris())
+    String replicationTableDefaultTabletDir = fs.choose(chooserEnv,
+        ServerConstants.getBaseUris(siteConfig)) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
+        + ReplicationTable.ID + Constants.DEFAULT_TABLET_LOCATION;
+    String defaultMetadataTabletDir = fs.choose(chooserEnv, ServerConstants.getBaseUris(siteConfig))
         + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + MetadataTable.ID
         + Constants.DEFAULT_TABLET_LOCATION;
 
@@ -609,15 +608,15 @@ private static void initZooKeeper(Opts opts, String uuid, String instanceNamePat
         NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZNAMESPACES, new byte[0],
         NodeExistsPolicy.FAIL);
-    TableManager.prepareNewNamespaceState(uuid, Namespace.ID.DEFAULT, Namespace.DEFAULT,
+    TableManager.prepareNewNamespaceState(zoo, uuid, Namespace.ID.DEFAULT, Namespace.DEFAULT,
         NodeExistsPolicy.FAIL);
-    TableManager.prepareNewNamespaceState(uuid, Namespace.ID.ACCUMULO, Namespace.ACCUMULO,
+    TableManager.prepareNewNamespaceState(zoo, uuid, Namespace.ID.ACCUMULO, Namespace.ACCUMULO,
         NodeExistsPolicy.FAIL);
-    TableManager.prepareNewTableState(uuid, RootTable.ID, Namespace.ID.ACCUMULO, RootTable.NAME,
-        TableState.ONLINE, NodeExistsPolicy.FAIL);
-    TableManager.prepareNewTableState(uuid, MetadataTable.ID, Namespace.ID.ACCUMULO,
+    TableManager.prepareNewTableState(zoo, uuid, RootTable.ID, Namespace.ID.ACCUMULO,
+        RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
+    TableManager.prepareNewTableState(zoo, uuid, MetadataTable.ID, Namespace.ID.ACCUMULO,
         MetadataTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
-    TableManager.prepareNewTableState(uuid, ReplicationTable.ID, Namespace.ID.ACCUMULO,
+    TableManager.prepareNewTableState(zoo, uuid, ReplicationTable.ID, Namespace.ID.ACCUMULO,
         ReplicationTable.NAME, TableState.OFFLINE, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTSERVERS, EMPTY_BYTE_ARRAY,
         NodeExistsPolicy.FAIL);
@@ -774,7 +773,8 @@ private static void initSecurity(ServerContext context, Opts opts, String iid, S
         rootUser, opts.rootpass);
   }
 
-  public static void initSystemTablesConfig(String zooKeeperRoot) throws IOException {
+  public static void initSystemTablesConfig(ZooReaderWriter zoo, String zooKeeperRoot)
+      throws IOException {
     try {
       Configuration conf = CachedConfiguration.getInstance();
       int max = conf.getInt("dfs.replication.max", 512);
@@ -786,23 +786,23 @@ public static void initSystemTablesConfig(String zooKeeperRoot) throws IOExcepti
       if (min > 5)
         setMetadataReplication(min, "min");
       for (Entry<String,String> entry : initialMetadataConf.entrySet()) {
-        if (!TablePropUtil.setTableProperty(zooKeeperRoot, RootTable.ID, entry.getKey(),
+        if (!TablePropUtil.setTableProperty(zoo, zooKeeperRoot, RootTable.ID, entry.getKey(),
             entry.getValue()))
           throw new IOException("Cannot create per-table property " + entry.getKey());
-        if (!TablePropUtil.setTableProperty(zooKeeperRoot, MetadataTable.ID, entry.getKey(),
+        if (!TablePropUtil.setTableProperty(zoo, zooKeeperRoot, MetadataTable.ID, entry.getKey(),
             entry.getValue()))
           throw new IOException("Cannot create per-table property " + entry.getKey());
       }
       // Only add combiner config to accumulo.metadata table (ACCUMULO-3077)
       for (Entry<String,String> entry : initialMetadataCombinerConf.entrySet()) {
-        if (!TablePropUtil.setTableProperty(zooKeeperRoot, MetadataTable.ID, entry.getKey(),
+        if (!TablePropUtil.setTableProperty(zoo, zooKeeperRoot, MetadataTable.ID, entry.getKey(),
             entry.getValue()))
           throw new IOException("Cannot create per-table property " + entry.getKey());
       }
 
       // add configuration to the replication table
       for (Entry<String,String> entry : initialReplicationTableConf.entrySet()) {
-        if (!TablePropUtil.setTableProperty(zooKeeperRoot, ReplicationTable.ID, entry.getKey(),
+        if (!TablePropUtil.setTableProperty(zoo, zooKeeperRoot, ReplicationTable.ID, entry.getKey(),
             entry.getValue()))
           throw new IOException("Cannot create per-table property " + entry.getKey());
       }
@@ -842,7 +842,8 @@ private static void addVolumes(VolumeManager fs, SiteConfiguration siteConfig)
     String[] volumeURIs = VolumeConfiguration.getVolumeUris(siteConfig);
 
     HashSet<String> initializedDirs = new HashSet<>();
-    initializedDirs.addAll(Arrays.asList(ServerConstants.checkBaseUris(volumeURIs, true)));
+    initializedDirs
+        .addAll(Arrays.asList(ServerConstants.checkBaseUris(siteConfig, volumeURIs, true)));
 
     HashSet<String> uinitializedDirs = new HashSet<>();
     uinitializedDirs.addAll(Arrays.asList(volumeURIs));
@@ -853,7 +854,7 @@ private static void addVolumes(VolumeManager fs, SiteConfiguration siteConfig)
     Path versionPath = new Path(aBasePath, ServerConstants.VERSION_DIR);
 
     UUID uuid = UUID.fromString(ZooUtil.getInstanceIDFromHdfs(iidPath, siteConfig));
-    for (Pair<Path,Path> replacementVolume : ServerConstants.getVolumeReplacements()) {
+    for (Pair<Path,Path> replacementVolume : ServerConstants.getVolumeReplacements(siteConfig)) {
       if (aBasePath.equals(replacementVolume.getFirst()))
         log.error(
             "{} is set to be replaced in {} and should not appear in {}."
@@ -918,10 +919,10 @@ public String description() {
   public void execute(final String[] args) {
     Opts opts = new Opts();
     opts.parseArgs("accumulo init", args);
-    SiteConfiguration siteConfig = SiteConfiguration.create();
+    SiteConfiguration siteConfig = new SiteConfiguration();
 
     try {
-      zoo = ZooReaderWriter.getInstance();
+      setZooReaderWriter(new ZooReaderWriter(siteConfig));
       SecurityUtil.serverLogin(siteConfig);
       Configuration hadoopConfig = CachedConfiguration.getInstance();
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java
index 864f48ff4b..2503e30f9a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java
@@ -29,6 +29,7 @@
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.Path;
@@ -92,9 +93,9 @@ public WalMarkerException(Exception ex) {
 
   private volatile boolean checkedExistance = false;
 
-  public WalStateManager(ClientContext context, ZooReaderWriter zoo) {
+  public WalStateManager(ServerContext context) {
     this.context = context;
-    this.zoo = zoo;
+    this.zoo = context.getZooReaderWriter();
   }
 
   private String root() throws WalMarkerException {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
index eb41d70b71..97e57e24d7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java
@@ -27,7 +27,6 @@
 import java.util.Set;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.impl.KeyExtent;
@@ -41,12 +40,12 @@
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.core.util.ServerServices;
 import org.apache.accumulo.fate.zookeeper.ZooCache.ZcStat;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.util.Halt;
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
@@ -67,7 +66,7 @@
   private static final Logger log = LoggerFactory.getLogger(LiveTServerSet.class);
 
   private final Listener cback;
-  private final ClientContext context;
+  private final ServerContext context;
   private ZooCache zooCache;
 
   public class TServerConnection {
@@ -241,14 +240,14 @@ public boolean isActive(long tid) throws TException {
   // The set of entries in zookeeper without locks, and the first time each was noticed
   private Map<String,Long> locklessServers = new HashMap<>();
 
-  public LiveTServerSet(ClientContext context, Listener cback) {
+  public LiveTServerSet(ServerContext context, Listener cback) {
     this.cback = cback;
     this.context = context;
   }
 
   public synchronized ZooCache getZooCache() {
     if (zooCache == null)
-      zooCache = new ZooCache(this);
+      zooCache = new ZooCache(context.getZooReaderWriter(), this);
     return zooCache;
   }
 
@@ -288,7 +287,7 @@ public synchronized void scanServers() {
 
   private void deleteServerNode(String serverNode) throws InterruptedException, KeeperException {
     try {
-      ZooReaderWriter.getInstance().delete(serverNode, -1);
+      context.getZooReaderWriter().delete(serverNode, -1);
     } catch (NotEmptyException | NoNodeException ex) {
       // race condition: tserver created the lock after our last check; we'll see it at the next
       // check
@@ -437,7 +436,7 @@ public synchronized void remove(TServerInstance server) {
     log.info("Removing zookeeper lock for {}", server);
     String fullpath = context.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + zPath;
     try {
-      ZooReaderWriter.getInstance().recursiveDelete(fullpath, SKIP);
+      context.getZooReaderWriter().recursiveDelete(fullpath, SKIP);
     } catch (Exception e) {
       String msg = "error removing tablet server lock";
       // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j compatibility
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java
index 0be5f45844..0d27c018c1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java
@@ -22,9 +22,9 @@
 import java.util.List;
 
 import org.apache.accumulo.core.master.thrift.DeadServer;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.data.Stat;
@@ -32,14 +32,17 @@
 import org.slf4j.LoggerFactory;
 
 public class DeadServerList {
+
   private static final Logger log = LoggerFactory.getLogger(DeadServerList.class);
+
   private final String path;
+  private final ZooReaderWriter zoo;
 
-  public DeadServerList(String path) {
+  public DeadServerList(ServerContext context, String path) {
     this.path = path;
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    zoo = context.getZooReaderWriter();
     try {
-      zoo.mkdirs(path);
+      context.getZooReaderWriter().mkdirs(path);
     } catch (Exception ex) {
       log.error("Unable to make parent directories of " + path, ex);
     }
@@ -47,7 +50,6 @@ public DeadServerList(String path) {
 
   public List<DeadServer> getList() {
     List<DeadServer> result = new ArrayList<>();
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     try {
       List<String> children = zoo.getChildren(path);
       if (children != null) {
@@ -73,7 +75,6 @@ public DeadServerList(String path) {
   }
 
   public void delete(String server) {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     try {
       zoo.recursiveDelete(path + "/" + server, NodeMissingPolicy.SKIP);
     } catch (Exception ex) {
@@ -82,7 +83,6 @@ public void delete(String server) {
   }
 
   public void post(String server, String cause) {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     try {
       zoo.putPersistentData(path + "/" + server, cause.getBytes(UTF_8), NodeExistsPolicy.SKIP);
     } catch (Exception ex) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooStore.java
index dd671df702..44c5ac84fe 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooStore.java
@@ -21,12 +21,11 @@
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.zookeeper.ZooCache;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,18 +33,17 @@
 
   private static final Logger log = LoggerFactory.getLogger(ZooStore.class);
 
-  String basePath;
+  private ServerContext context;
+  private String basePath;
+  private ZooCache cache;
 
-  ZooCache cache = new ZooCache();
-
-  public ZooStore(String basePath) throws IOException {
-    if (basePath.endsWith("/"))
-      basePath = basePath.substring(0, basePath.length() - 1);
-    this.basePath = basePath;
-  }
-
-  public ZooStore(ClientContext context) throws IOException {
-    this(context.getZooKeeperRoot());
+  public ZooStore(ServerContext context) throws IOException {
+    this.context = context;
+    cache = new ZooCache(context);
+    String zkRoot = context.getZooKeeperRoot();
+    if (zkRoot.endsWith("/"))
+      zkRoot = zkRoot.substring(0, zkRoot.length() - 1);
+    this.basePath = zkRoot;
   }
 
   @Override
@@ -74,7 +72,7 @@ private String relative(String path) {
   public void put(String path, byte[] bs) throws DistributedStoreException {
     try {
       path = relative(path);
-      ZooReaderWriter.getInstance().putPersistentData(path, bs, NodeExistsPolicy.OVERWRITE);
+      context.getZooReaderWriter().putPersistentData(path, bs, NodeExistsPolicy.OVERWRITE);
       cache.clear();
       log.debug("Wrote {} to {}", new String(bs, UTF_8), path);
     } catch (Exception ex) {
@@ -87,7 +85,7 @@ public void remove(String path) throws DistributedStoreException {
     try {
       log.debug("Removing {}", path);
       path = relative(path);
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      IZooReaderWriter zoo = context.getZooReaderWriter();
       if (zoo.exists(path))
         zoo.recursiveDelete(path, NodeMissingPolicy.SKIP);
       cache.clear();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
index bb86cdcb0f..32f8cf03b6 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
@@ -26,10 +26,10 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
@@ -44,7 +44,7 @@ public ZooTabletStateStore(DistributedStore store) {
     this.store = store;
   }
 
-  public ZooTabletStateStore(ClientContext context) throws DistributedStoreException {
+  public ZooTabletStateStore(ServerContext context) throws DistributedStoreException {
     try {
       store = new ZooStore(context);
     } catch (IOException ex) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/monitor/LogService.java b/server/base/src/main/java/org/apache/accumulo/server/monitor/LogService.java
index 2fa5417de9..0d85673dfc 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/monitor/LogService.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/monitor/LogService.java
@@ -30,7 +30,7 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.Daemon;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.net.SocketNode;
@@ -83,16 +83,14 @@ public void run() {
   /**
    * Place the host:port advertisement for the Monitor's Log4j listener in ZooKeeper
    *
-   * @param conf
-   *          configuration for the instance
-   * @param instanceId
-   *          instanceId for the instance
+   * @param context
+   *          Server context
    * @param hostAddress
    *          Address that monitor process is bound to
    */
-  public static void startLogListener(AccumuloConfiguration conf, String instanceId,
-      String hostAddress) {
+  public static void startLogListener(ServerContext context, String hostAddress) {
     try {
+      AccumuloConfiguration conf = context.getConfiguration();
       SocketServer server = new SocketServer(conf.getPort(Property.MONITOR_LOG4J_PORT)[0]);
 
       // getLocalPort will return the actual ephemeral port used when '0' was provided.
@@ -100,8 +98,8 @@ public static void startLogListener(AccumuloConfiguration conf, String instanceI
 
       log.debug("Setting monitor log4j log-forwarding address to: {}", logForwardingAddr);
 
-      final String path = ZooUtil.getRoot(instanceId) + Constants.ZMONITOR_LOG4J_ADDR;
-      final ZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      final String path = context.getZooKeeperRoot() + Constants.ZMONITOR_LOG4J_ADDR;
+      final ZooReaderWriter zoo = context.getZooReaderWriter();
 
       // Delete before we try to re-create in case the previous session hasn't yet expired
       try {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
index 19d886793a..baf0da8f4a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
@@ -150,7 +150,7 @@ void saveToMetadataTable(ServerContext context) throws Exception {
   }
 
   void removeFromZooKeeper(ServerContext context) throws Exception {
-    removeFromZooKeeper(ZooReaderWriter.getInstance(), context);
+    removeFromZooKeeper(context.getZooReaderWriter(), context);
   }
 
   void removeFromZooKeeper(ZooReaderWriter zoorw, ServerContext context)
@@ -159,13 +159,9 @@ void removeFromZooKeeper(ZooReaderWriter zoorw, ServerContext context)
     zoorw.recursiveDelete(zpath, NodeMissingPolicy.SKIP);
   }
 
-  void saveToZooKeeper(ServerContext context) throws Exception {
-    saveToZooKeeper(ZooReaderWriter.getInstance(), context);
-  }
-
-  void saveToZooKeeper(ZooReaderWriter zoorw, ServerContext context)
+  void saveToZooKeeper(ServerContext context)
       throws IOException, KeeperException, InterruptedException {
-    zoorw.putPersistentData(getZPath(context.getZooKeeperRoot()), encode(),
+    context.getZooReaderWriter().putPersistentData(getZPath(context.getZooKeeperRoot()), encode(),
         NodeExistsPolicy.OVERWRITE);
   }
 
@@ -182,12 +178,8 @@ private String getZPath(String zkRoot) throws IOException {
         + Encoding.encodeAsBase64FileName(new Text(baos.toByteArray()));
   }
 
-  static ProblemReport decodeZooKeeperEntry(ServerContext context, String node) throws Exception {
-    return decodeZooKeeperEntry(node, ZooReaderWriter.getInstance(), context);
-  }
-
-  static ProblemReport decodeZooKeeperEntry(String node, ZooReaderWriter zoorw,
-      ServerContext context) throws IOException, KeeperException, InterruptedException {
+  static ProblemReport decodeZooKeeperEntry(ServerContext context, String node)
+      throws IOException, KeeperException, InterruptedException {
     byte bytes[] = Encoding.decodeBase64FileName(node);
 
     ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
@@ -198,7 +190,7 @@ static ProblemReport decodeZooKeeperEntry(String node, ZooReaderWriter zoorw,
     String resource = dis.readUTF();
 
     String zpath = context.getZooKeeperRoot() + Constants.ZPROBLEMS + "/" + node;
-    byte[] enc = zoorw.getData(zpath, null);
+    byte[] enc = context.getZooReaderWriter().getData(zpath, null);
 
     return new ProblemReport(tableId, ProblemType.valueOf(problemType), resource, enc);
 
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 490199493b..4e2fd75e6e 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
@@ -49,7 +49,6 @@
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
@@ -196,7 +195,7 @@ private static boolean isMeta(Table.ID tableId) {
 
       return new Iterator<ProblemReport>() {
 
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+        IZooReaderWriter zoo = context.getZooReaderWriter();
         private int iter1Count = 0;
         private Iterator<String> iter1;
 
@@ -301,7 +300,7 @@ public static synchronized ProblemReports getInstance(ServerContext context) {
   }
 
   public static void main(String args[]) throws Exception {
-    ServerContext context = new ServerContext(SiteConfiguration.create());
+    ServerContext context = new ServerContext(new SiteConfiguration());
     getInstance(context).printProblems();
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
index 41bf351ad7..dd32543f7a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
@@ -68,7 +68,7 @@
   private final ReplicaSystemFactory factory;
 
   public ReplicationUtil(ServerContext context) {
-    this(context, new ZooCache(), new ReplicaSystemFactory());
+    this(context, new ZooCache(context), new ReplicaSystemFactory());
   }
 
   public ReplicationUtil(ServerContext context, ZooCache cache, ReplicaSystemFactory factory) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 5fa3248087..2ad5d3abaf 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -90,7 +90,7 @@ public static synchronized SecurityOperation getInstance(ServerContext context,
   protected static Authorizor getAuthorizor(ServerContext context, boolean initialize) {
     Authorizor toRet = Property.createInstanceFromPropertyName(context.getConfiguration(),
         Property.INSTANCE_SECURITY_AUTHORIZOR, Authorizor.class, ZKAuthorizor.getInstance());
-    toRet.initialize(context.getInstanceID(), initialize);
+    toRet.initialize(context, initialize);
     return toRet;
   }
 
@@ -106,14 +106,14 @@ protected static PermissionHandler getPermHandler(ServerContext context, boolean
     PermissionHandler toRet = Property.createInstanceFromPropertyName(context.getConfiguration(),
         Property.INSTANCE_SECURITY_PERMISSION_HANDLER, PermissionHandler.class,
         ZKPermHandler.getInstance());
-    toRet.initialize(context.getInstanceID(), initialize);
+    toRet.initialize(context, initialize);
     return toRet;
   }
 
   protected SecurityOperation(ServerContext context) {
     this.context = context;
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
-    zooCache = new ZooCache();
+    zooCache = new ZooCache(context);
   }
 
   public SecurityOperation(ServerContext context, Authorizor author, Authenticator authent,
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java
index cd8d5f421c..327529c44d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/Authorizor.java
@@ -23,6 +23,7 @@
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.server.ServerContext;
 
 /**
  * This interface is used for the system which will be used for getting a users Authorizations. If
@@ -34,7 +35,7 @@
   /**
    * Sets up the authorizor for a new instance of Accumulo
    */
-  void initialize(String instanceId, boolean initialize);
+  void initialize(ServerContext context, boolean initialize);
 
   /**
    * Used to validate that the Authorizor, Authenticator, and permission handler can coexist
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
index c1baa284e0..c06bf43483 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
@@ -24,6 +24,7 @@
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.server.ServerContext;
 
 /**
  * This is a Permission Handler implementation that doesn't actually do any security. Use at your
@@ -32,7 +33,7 @@
 public class InsecurePermHandler implements PermissionHandler {
 
   @Override
-  public void initialize(String instanceId, boolean initialize) {}
+  public void initialize(ServerContext context, boolean initialize) {}
 
   @Override
   public boolean validSecurityHandlers(Authenticator authent, Authorizor author) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java
index 1b6e512155..01561af74a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthenticator.java
@@ -40,7 +40,6 @@
 import org.apache.accumulo.server.security.UserImpersonation;
 import org.apache.accumulo.server.security.UserImpersonation.UsersWithHosts;
 import org.apache.accumulo.server.zookeeper.ZooCache;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -56,18 +55,15 @@
       .newHashSet(KerberosToken.class.getName(), SystemToken.class.getName());
 
   private final ZKAuthenticator zkAuthenticator = new ZKAuthenticator();
-  private final ZooCache zooCache;
+  private ZooCache zooCache;
   private ServerContext context;
   private String zkUserPath;
   private UserImpersonation impersonation;
 
-  public KerberosAuthenticator() {
-    zooCache = new ZooCache();
-  }
-
   @Override
   public void initialize(ServerContext context, boolean initialize) {
     this.context = context;
+    zooCache = new ZooCache(context);
     impersonation = new UserImpersonation(context.getConfiguration());
     zkAuthenticator.initialize(context, initialize);
     zkUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
@@ -81,7 +77,7 @@ public boolean validSecurityHandlers(Authorizor auth, PermissionHandler pm) {
   private void createUserNodeInZk(String principal) throws KeeperException, InterruptedException {
     synchronized (zooCache) {
       zooCache.clear();
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      IZooReaderWriter zoo = context.getZooReaderWriter();
       zoo.putPrivatePersistentData(zkUserPath + "/" + principal, new byte[0],
           NodeExistsPolicy.FAIL);
     }
@@ -92,7 +88,7 @@ public void initializeSecurity(TCredentials credentials, String principal, byte[
       throws AccumuloSecurityException, ThriftSecurityException {
     try {
       // remove old settings from zookeeper first, if any
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      IZooReaderWriter zoo = context.getZooReaderWriter();
       synchronized (zooCache) {
         zooCache.clear();
         if (zoo.exists(zkUserPath)) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthorizor.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthorizor.java
index 25c391185c..b78ffa2a7e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthorizor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosAuthorizor.java
@@ -26,6 +26,7 @@
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.server.ServerContext;
 
 /**
  * Kerberos principals might contains identifiers that are not valid ZNodes ('/'). Base64-encodes
@@ -40,8 +41,8 @@ public KerberosAuthorizor() {
   }
 
   @Override
-  public void initialize(String instanceId, boolean initialize) {
-    zkAuthorizor.initialize(instanceId, initialize);
+  public void initialize(ServerContext context, boolean initialize) {
+    zkAuthorizor.initialize(context, initialize);
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosPermissionHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosPermissionHandler.java
index 54e2189df9..0f0e427c9b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosPermissionHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/KerberosPermissionHandler.java
@@ -29,6 +29,7 @@
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.server.ServerContext;
 
 /**
  * Kerberos principals might contains identifiers that are not valid ZNodes ('/'). Base64-encodes
@@ -43,8 +44,8 @@ public KerberosPermissionHandler() {
   }
 
   @Override
-  public void initialize(String instanceId, boolean initialize) {
-    zkPermissionHandler.initialize(instanceId, initialize);
+  public void initialize(ServerContext context, boolean initialize) {
+    zkPermissionHandler.initialize(context, initialize);
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
index 5e8626ab5c..a775584f67 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
@@ -25,6 +25,7 @@
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.server.ServerContext;
 
 /**
  * This interface is used for the system which will be used for getting a users permissions. If the
@@ -36,7 +37,7 @@
   /**
    * Sets up the permission handler for a new instance of Accumulo
    */
-  void initialize(String instanceId, boolean initialize);
+  void initialize(ServerContext context, boolean initialize);
 
   /**
    * Used to validate that the Authorizor, Authenticator, and permission handler can coexist
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
index 491c2cfd4a..862ca13c7f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
@@ -34,7 +34,6 @@
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.zookeeper.ZooCache;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,7 +45,7 @@
 
   private ServerContext context;
   private String ZKUserPath;
-  private final ZooCache zooCache;
+  private ZooCache zooCache;
 
   public static synchronized Authenticator getInstance() {
     if (zkAuthenticatorInstance == null)
@@ -54,13 +53,10 @@ public static synchronized Authenticator getInstance() {
     return zkAuthenticatorInstance;
   }
 
-  public ZKAuthenticator() {
-    zooCache = new ZooCache();
-  }
-
   @Override
   public void initialize(ServerContext context, boolean initialize) {
     this.context = context;
+    zooCache = new ZooCache(context);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
   }
 
@@ -69,7 +65,7 @@ public void initializeSecurity(TCredentials credentials, String principal, byte[
       throws AccumuloSecurityException {
     try {
       // remove old settings from zookeeper first, if any
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      IZooReaderWriter zoo = context.getZooReaderWriter();
       synchronized (zooCache) {
         zooCache.clear();
         if (zoo.exists(ZKUserPath)) {
@@ -96,7 +92,7 @@ private void constructUser(String user, byte[] pass)
       throws KeeperException, InterruptedException {
     synchronized (zooCache) {
       zooCache.clear();
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      IZooReaderWriter zoo = context.getZooReaderWriter();
       zoo.putPrivatePersistentData(ZKUserPath + "/" + user, pass, NodeExistsPolicy.FAIL);
     }
   }
@@ -132,7 +128,7 @@ public void dropUser(String user) throws AccumuloSecurityException {
     try {
       synchronized (zooCache) {
         zooCache.clear();
-        ZooReaderWriter.getInstance().recursiveDelete(ZKUserPath + "/" + user,
+        context.getZooReaderWriter().recursiveDelete(ZKUserPath + "/" + user,
             NodeMissingPolicy.FAIL);
       }
     } catch (InterruptedException e) {
@@ -157,7 +153,7 @@ public void changePassword(String principal, AuthenticationToken token)
       try {
         synchronized (zooCache) {
           zooCache.clear(ZKUserPath + "/" + principal);
-          ZooReaderWriter.getInstance().putPrivatePersistentData(ZKUserPath + "/" + principal,
+          context.getZooReaderWriter().putPrivatePersistentData(ZKUserPath + "/" + principal,
               ZKSecurityTool.createPass(pt.getPassword()), NodeExistsPolicy.OVERWRITE);
         }
       } catch (KeeperException e) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
index 809b6a21b6..12bbcd9423 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
@@ -39,6 +39,7 @@
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.zookeeper.KeeperException;
@@ -51,8 +52,9 @@
 
   private final String ZKUserAuths = "/Authorizations";
 
+  private ServerContext context;
   private String ZKUserPath;
-  private final ZooCache zooCache;
+  private ZooCache zooCache;
 
   public static synchronized Authorizor getInstance() {
     if (zkAuthorizorInstance == null)
@@ -60,13 +62,11 @@ public static synchronized Authorizor getInstance() {
     return zkAuthorizorInstance;
   }
 
-  public ZKAuthorizor() {
-    zooCache = new ZooCache();
-  }
-
   @Override
-  public void initialize(String instanceId, boolean initialize) {
-    ZKUserPath = ZKSecurityTool.getInstancePath(instanceId) + "/users";
+  public void initialize(ServerContext context, boolean initialize) {
+    this.context = context;
+    zooCache = new ZooCache(context);
+    ZKUserPath = ZKSecurityTool.getInstancePath(context.getInstanceID()) + "/users";
   }
 
   @Override
@@ -85,7 +85,7 @@ public boolean validSecurityHandlers(Authenticator auth, PermissionHandler pm) {
   @Override
   public void initializeSecurity(TCredentials itw, String rootuser)
       throws AccumuloSecurityException {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    ZooReaderWriter zoo = context.getZooReaderWriter();
 
     // create the root user with all system privileges, no table privileges, and no record-level
     // authorizations
@@ -113,7 +113,7 @@ public void initializeSecurity(TCredentials itw, String rootuser)
 
   @Override
   public void initUser(String user) throws AccumuloSecurityException {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = context.getZooReaderWriter();
     try {
       zoo.putPersistentData(ZKUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP);
     } catch (KeeperException e) {
@@ -129,7 +129,7 @@ public void initUser(String user) throws AccumuloSecurityException {
   public void dropUser(String user) throws AccumuloSecurityException {
     try {
       synchronized (zooCache) {
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+        IZooReaderWriter zoo = context.getZooReaderWriter();
         zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserAuths, NodeMissingPolicy.SKIP);
         zooCache.clear(ZKUserPath + "/" + user);
       }
@@ -151,7 +151,7 @@ public void changeAuthorizations(String user, Authorizations authorizations)
     try {
       synchronized (zooCache) {
         zooCache.clear();
-        ZooReaderWriter.getInstance().putPersistentData(ZKUserPath + "/" + user + ZKUserAuths,
+        context.getZooReaderWriter().putPersistentData(ZKUserPath + "/" + user + ZKUserAuths,
             ZKSecurityTool.convertAuthorizations(authorizations), NodeExistsPolicy.OVERWRITE);
       }
     } catch (KeeperException e) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
index 925f6fa3df..6cf77d342d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
@@ -37,9 +37,9 @@
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.TCredentials;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.zookeeper.KeeperException;
@@ -51,10 +51,11 @@
   private static final Logger log = LoggerFactory.getLogger(ZKPermHandler.class);
   private static PermissionHandler zkPermHandlerInstance = null;
 
+  private ZooReaderWriter zoo;
   private String ZKUserPath;
   private String ZKTablePath;
   private String ZKNamespacePath;
-  private final ZooCache zooCache;
+  private ZooCache zooCache;
   private final String ZKUserSysPerms = "/System";
   private final String ZKUserTablePerms = "/Tables";
   private final String ZKUserNamespacePerms = "/Namespaces";
@@ -66,31 +67,29 @@ public static synchronized PermissionHandler getInstance() {
   }
 
   @Override
-  public void initialize(String instanceId, boolean initialize) {
+  public void initialize(ServerContext context, boolean initialize) {
+    zooCache = new ZooCache(context);
+    zoo = context.getZooReaderWriter();
+    String instanceId = context.getInstanceID();
     ZKUserPath = ZKSecurityTool.getInstancePath(instanceId) + "/users";
     ZKTablePath = ZKSecurityTool.getInstancePath(instanceId) + "/tables";
     ZKNamespacePath = ZKSecurityTool.getInstancePath(instanceId) + "/namespaces";
   }
 
-  public ZKPermHandler() {
-    zooCache = new ZooCache();
-  }
-
   @Override
   public boolean hasTablePermission(String user, String table, TablePermission permission)
       throws TableNotFoundException {
     byte[] serializedPerms;
-    final ZooReaderWriter zrw = ZooReaderWriter.getInstance();
     try {
       String path = ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table;
-      zrw.sync(path);
-      serializedPerms = zrw.getData(path, null);
+      zoo.sync(path);
+      serializedPerms = zoo.getData(path, null);
     } catch (KeeperException e) {
       if (e.code() == Code.NONODE) {
         // maybe the table was just deleted?
         try {
           // check for existence:
-          zrw.getData(ZKTablePath + "/" + table, null);
+          zoo.getData(ZKTablePath + "/" + table, null);
           // it's there, you don't have permission
           return false;
         } catch (InterruptedException ex) {
@@ -131,17 +130,16 @@ public boolean hasCachedTablePermission(String user, String table, TablePermissi
   public boolean hasNamespacePermission(String user, Namespace.ID namespace,
       NamespacePermission permission) throws NamespaceNotFoundException {
     byte[] serializedPerms;
-    final ZooReaderWriter zrw = ZooReaderWriter.getInstance();
     try {
       String path = ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace;
-      zrw.sync(path);
-      serializedPerms = zrw.getData(path, null);
+      zoo.sync(path);
+      serializedPerms = zoo.getData(path, null);
     } catch (KeeperException e) {
       if (e.code() == Code.NONODE) {
         // maybe the namespace was just deleted?
         try {
           // check for existence:
-          zrw.getData(ZKNamespacePath + "/" + namespace, null);
+          zoo.getData(ZKNamespacePath + "/" + namespace, null);
           // it's there, you don't have permission
           return false;
         } catch (InterruptedException ex) {
@@ -196,7 +194,7 @@ public void grantSystemPermission(String user, SystemPermission permission)
       if (perms.add(permission)) {
         synchronized (zooCache) {
           zooCache.clear();
-          ZooReaderWriter.getInstance().putPersistentData(ZKUserPath + "/" + user + ZKUserSysPerms,
+          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserSysPerms,
               ZKSecurityTool.convertSystemPermissions(perms), NodeExistsPolicy.OVERWRITE);
         }
       }
@@ -223,8 +221,7 @@ public void grantTablePermission(String user, String table, TablePermission perm
       if (tablePerms.add(permission)) {
         synchronized (zooCache) {
           zooCache.clear(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
-          ZooReaderWriter.getInstance().putPersistentData(
-              ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
+          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
               ZKSecurityTool.convertTablePermissions(tablePerms), NodeExistsPolicy.OVERWRITE);
         }
       }
@@ -252,8 +249,7 @@ public void grantNamespacePermission(String user, Namespace.ID namespace,
       if (namespacePerms.add(permission)) {
         synchronized (zooCache) {
           zooCache.clear(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
-          ZooReaderWriter.getInstance().putPersistentData(
-              ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
+          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
               ZKSecurityTool.convertNamespacePermissions(namespacePerms),
               NodeExistsPolicy.OVERWRITE);
         }
@@ -282,7 +278,7 @@ public void revokeSystemPermission(String user, SystemPermission permission)
       if (sysPerms.remove(permission)) {
         synchronized (zooCache) {
           zooCache.clear();
-          ZooReaderWriter.getInstance().putPersistentData(ZKUserPath + "/" + user + ZKUserSysPerms,
+          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserSysPerms,
               ZKSecurityTool.convertSystemPermissions(sysPerms), NodeExistsPolicy.OVERWRITE);
         }
       }
@@ -308,7 +304,6 @@ public void revokeTablePermission(String user, String table, TablePermission per
     try {
       if (tablePerms.remove(permission)) {
         zooCache.clear();
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
         if (tablePerms.size() == 0)
           zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
               NodeMissingPolicy.SKIP);
@@ -340,7 +335,6 @@ public void revokeNamespacePermission(String user, Namespace.ID namespace,
     try {
       if (namespacePerms.remove(permission)) {
         zooCache.clear();
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
         if (namespacePerms.size() == 0)
           zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
               NodeMissingPolicy.SKIP);
@@ -363,7 +357,6 @@ public void cleanTablePermissions(String table) throws AccumuloSecurityException
     try {
       synchronized (zooCache) {
         zooCache.clear();
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
         for (String user : zooCache.getChildren(ZKUserPath))
           zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
               NodeMissingPolicy.SKIP);
@@ -382,7 +375,6 @@ public void cleanNamespacePermissions(Namespace.ID namespace) throws AccumuloSec
     try {
       synchronized (zooCache) {
         zooCache.clear();
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
         for (String user : zooCache.getChildren(ZKUserPath))
           zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
               NodeMissingPolicy.SKIP);
@@ -399,7 +391,6 @@ public void cleanNamespacePermissions(Namespace.ID namespace) throws AccumuloSec
   @Override
   public void initializeSecurity(TCredentials itw, String rootuser)
       throws AccumuloSecurityException {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 
     // create the root user with all system privileges, no table privileges, and no record-level
     // authorizations
@@ -437,7 +428,6 @@ public void initializeSecurity(TCredentials itw, String rootuser)
 
   @Override
   public void initUser(String user) throws AccumuloSecurityException {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     try {
       zoo.putPersistentData(ZKUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP);
       zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms, new byte[0],
@@ -461,8 +451,7 @@ private void createTablePerm(String user, Table.ID table, Set<TablePermission> p
       throws KeeperException, InterruptedException {
     synchronized (zooCache) {
       zooCache.clear();
-      ZooReaderWriter.getInstance().putPersistentData(
-          ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
+      zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table,
           ZKSecurityTool.convertTablePermissions(perms), NodeExistsPolicy.FAIL);
     }
   }
@@ -475,8 +464,7 @@ private void createNamespacePerm(String user, Namespace.ID namespace,
       Set<NamespacePermission> perms) throws KeeperException, InterruptedException {
     synchronized (zooCache) {
       zooCache.clear();
-      ZooReaderWriter.getInstance().putPersistentData(
-          ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
+      zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
           ZKSecurityTool.convertNamespacePermissions(perms), NodeExistsPolicy.FAIL);
     }
   }
@@ -485,7 +473,6 @@ private void createNamespacePerm(String user, Namespace.ID namespace,
   public void cleanUser(String user) throws AccumuloSecurityException {
     try {
       synchronized (zooCache) {
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
         zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserSysPerms, NodeMissingPolicy.SKIP);
         zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms, NodeMissingPolicy.SKIP);
         zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms, NodeMissingPolicy.SKIP);
@@ -509,8 +496,8 @@ public boolean hasSystemPermission(String user, SystemPermission permission)
     byte[] perms;
     try {
       String path = ZKUserPath + "/" + user + ZKUserSysPerms;
-      ZooReaderWriter.getInstance().sync(path);
-      perms = ZooReaderWriter.getInstance().getData(path, null);
+      zoo.sync(path);
+      perms = zoo.getData(path, null);
     } catch (KeeperException e) {
       if (e.code() == Code.NONODE) {
         return false;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
index adfc96cec0..b563f03b29 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
@@ -60,22 +60,22 @@
   private final ServerContext context;
   private final String zkRoot;
   private final String instanceID;
+  private final ZooReaderWriter zoo;
   private ZooCache zooStateCache;
 
-  public static void prepareNewNamespaceState(String instanceId, Namespace.ID namespaceId,
-      String namespace, NodeExistsPolicy existsPolicy)
+  public static void prepareNewNamespaceState(IZooReaderWriter zoo, String instanceId,
+      Namespace.ID namespaceId, String namespace, NodeExistsPolicy existsPolicy)
       throws KeeperException, InterruptedException {
     log.debug("Creating ZooKeeper entries for new namespace {} (ID: {})", namespace, namespaceId);
     String zPath = Constants.ZROOT + "/" + instanceId + Constants.ZNAMESPACES + "/" + namespaceId;
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     zoo.putPersistentData(zPath, new byte[0], existsPolicy);
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_NAME, namespace.getBytes(UTF_8),
         existsPolicy);
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_CONF, new byte[0], existsPolicy);
   }
 
-  public static void prepareNewTableState(String instanceId, Table.ID tableId,
+  public static void prepareNewTableState(IZooReaderWriter zoo, String instanceId, Table.ID tableId,
       Namespace.ID namespaceId, String tableName, TableState state, NodeExistsPolicy existsPolicy)
       throws KeeperException, InterruptedException {
     // state gets created last
@@ -84,7 +84,6 @@ public static void prepareNewTableState(String instanceId, Table.ID tableId,
     Pair<String,String> qualifiedTableName = Tables.qualify(tableName);
     tableName = qualifiedTableName.getSecond();
     String zTablePath = Constants.ZROOT + "/" + instanceId + Constants.ZTABLES + "/" + tableId;
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     zoo.putPersistentData(zTablePath, new byte[0], existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_CONF, new byte[0], existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAMESPACE, namespaceId.getUtf8(),
@@ -102,7 +101,8 @@ public TableManager(ServerContext context) {
     this.context = context;
     zkRoot = context.getZooKeeperRoot();
     instanceID = context.getInstanceID();
-    zooStateCache = new ZooCache(new TableStateWatcher());
+    zoo = context.getZooReaderWriter();
+    zooStateCache = new ZooCache(zoo, new TableStateWatcher());
     updateTableStateCache();
   }
 
@@ -152,38 +152,36 @@ public synchronized void transitionTableState(final Table.ID tableId, final Tabl
     String statePath = zkRoot + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE;
 
     try {
-      ZooReaderWriter.getInstance().mutate(statePath, newState.name().getBytes(UTF_8),
-          ZooUtil.PUBLIC, new Mutator() {
-            @Override
-            public byte[] mutate(byte[] oldData) throws Exception {
-              TableState oldState = TableState.UNKNOWN;
-              if (oldData != null)
-                oldState = TableState.valueOf(new String(oldData, UTF_8));
-              boolean transition = true;
-              // +--------+
-              // v |
-              // NEW -> (ONLINE|OFFLINE)+--- DELETING
-              switch (oldState) {
-                case NEW:
-                  transition = (newState == TableState.OFFLINE || newState == TableState.ONLINE);
-                  break;
-                case ONLINE: // fall-through intended
-                case UNKNOWN:// fall through intended
-                case OFFLINE:
-                  transition = (newState != TableState.NEW);
-                  break;
-                case DELETING:
-                  // Can't transition to any state from DELETING
-                  transition = false;
-                  break;
-              }
-              if (!transition)
-                throw new IllegalTableTransitionException(oldState, newState);
-              log.debug("Transitioning state for table {} from {} to {}", tableId, oldState,
-                  newState);
-              return newState.name().getBytes(UTF_8);
-            }
-          });
+      zoo.mutate(statePath, newState.name().getBytes(UTF_8), ZooUtil.PUBLIC, new Mutator() {
+        @Override
+        public byte[] mutate(byte[] oldData) throws Exception {
+          TableState oldState = TableState.UNKNOWN;
+          if (oldData != null)
+            oldState = TableState.valueOf(new String(oldData, UTF_8));
+          boolean transition = true;
+          // +--------+
+          // v |
+          // NEW -> (ONLINE|OFFLINE)+--- DELETING
+          switch (oldState) {
+            case NEW:
+              transition = (newState == TableState.OFFLINE || newState == TableState.ONLINE);
+              break;
+            case ONLINE: // fall-through intended
+            case UNKNOWN:// fall through intended
+            case OFFLINE:
+              transition = (newState != TableState.NEW);
+              break;
+            case DELETING:
+              // Can't transition to any state from DELETING
+              transition = false;
+              break;
+          }
+          if (!transition)
+            throw new IllegalTableTransitionException(oldState, newState);
+          log.debug("Transitioning state for table {} from {} to {}", tableId, oldState, newState);
+          return newState.name().getBytes(UTF_8);
+        }
+      });
     } catch (Exception e) {
       // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j compatibility
       log.error("FATAL Failed to transition table to state {}", newState);
@@ -221,29 +219,29 @@ public TableState updateTableStateCache(Table.ID tableId) {
   public void addTable(Table.ID tableId, Namespace.ID namespaceId, String tableName,
       NodeExistsPolicy existsPolicy)
       throws KeeperException, InterruptedException, NamespaceNotFoundException {
-    prepareNewTableState(instanceID, tableId, namespaceId, tableName, TableState.NEW, existsPolicy);
+    prepareNewTableState(zoo, instanceID, tableId, namespaceId, tableName, TableState.NEW,
+        existsPolicy);
     updateTableStateCache(tableId);
   }
 
   public void cloneTable(Table.ID srcTableId, Table.ID tableId, String tableName,
       Namespace.ID namespaceId, Map<String,String> propertiesToSet, Set<String> propertiesToExclude,
       NodeExistsPolicy existsPolicy) throws KeeperException, InterruptedException {
-    prepareNewTableState(instanceID, tableId, namespaceId, tableName, TableState.NEW, existsPolicy);
+    prepareNewTableState(zoo, instanceID, tableId, namespaceId, tableName, TableState.NEW,
+        existsPolicy);
 
     String srcTablePath = Constants.ZROOT + "/" + instanceID + Constants.ZTABLES + "/" + srcTableId
         + Constants.ZTABLE_CONF;
     String newTablePath = Constants.ZROOT + "/" + instanceID + Constants.ZTABLES + "/" + tableId
         + Constants.ZTABLE_CONF;
-    ZooReaderWriter.getInstance().recursiveCopyPersistent(srcTablePath, newTablePath,
-        NodeExistsPolicy.OVERWRITE);
+    zoo.recursiveCopyPersistent(srcTablePath, newTablePath, NodeExistsPolicy.OVERWRITE);
 
     for (Entry<String,String> entry : propertiesToSet.entrySet())
       TablePropUtil.setTableProperty(context, tableId, entry.getKey(), entry.getValue());
 
     for (String prop : propertiesToExclude)
-      ZooReaderWriter.getInstance().recursiveDelete(Constants.ZROOT + "/" + instanceID
-          + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_CONF + "/" + prop,
-          NodeMissingPolicy.SKIP);
+      zoo.recursiveDelete(Constants.ZROOT + "/" + instanceID + Constants.ZTABLES + "/" + tableId
+          + Constants.ZTABLE_CONF + "/" + prop, NodeMissingPolicy.SKIP);
 
     updateTableStateCache(tableId);
   }
@@ -251,11 +249,9 @@ public void cloneTable(Table.ID srcTableId, Table.ID tableId, String tableName,
   public void removeTable(Table.ID tableId) throws KeeperException, InterruptedException {
     synchronized (tableStateCache) {
       tableStateCache.remove(tableId);
-      ZooReaderWriter.getInstance().recursiveDelete(
-          zkRoot + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE,
-          NodeMissingPolicy.SKIP);
-      ZooReaderWriter.getInstance().recursiveDelete(zkRoot + Constants.ZTABLES + "/" + tableId,
+      zoo.recursiveDelete(zkRoot + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE,
           NodeMissingPolicy.SKIP);
+      zoo.recursiveDelete(zkRoot + Constants.ZTABLES + "/" + tableId, NodeMissingPolicy.SKIP);
     }
   }
 
@@ -342,8 +338,7 @@ public void process(WatchedEvent event) {
 
   public void removeNamespace(Namespace.ID namespaceId)
       throws KeeperException, InterruptedException {
-    ZooReaderWriter.getInstance().recursiveDelete(
-        zkRoot + Constants.ZNAMESPACES + "/" + namespaceId, NodeMissingPolicy.SKIP);
+    zoo.recursiveDelete(zkRoot + Constants.ZNAMESPACES + "/" + namespaceId, NodeMissingPolicy.SKIP);
   }
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java b/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java
index 93829573cf..1cc8177c15 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java
@@ -34,12 +34,14 @@
  */
 public class UniqueNameAllocator {
 
+  private ServerContext context;
   private long next = 0;
   private long maxAllocated = 0;
   private String nextNamePath;
   private Random rand;
 
   public UniqueNameAllocator(ServerContext context) {
+    this.context = context;
     nextNamePath = Constants.ZROOT + "/" + context.getInstanceID() + Constants.ZNEXT_FILE;
     rand = new Random();
   }
@@ -50,7 +52,7 @@ public synchronized String getNextName() {
       final int allocate = 100 + rand.nextInt(100);
 
       try {
-        byte[] max = ZooReaderWriter.getInstance().mutate(nextNamePath, null, ZooUtil.PRIVATE,
+        byte[] max = context.getZooReaderWriter().mutate(nextNamePath, null, ZooUtil.PRIVATE,
             new ZooReaderWriter.Mutator() {
               @Override
               public byte[] mutate(byte[] currentValue) throws Exception {
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 cb8c5b4cce..4817fa40ee 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
@@ -24,6 +24,7 @@
 import java.util.List;
 import java.util.UUID;
 
+import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
@@ -58,7 +59,8 @@
   }
 
   public static void main(String[] args) throws Exception {
-    VolumeManager fs = VolumeManagerImpl.get();
+    SiteConfiguration siteConfig = new SiteConfiguration();
+    VolumeManager fs = VolumeManagerImpl.get(siteConfig);
     verifyHdfsWritePermission(fs);
 
     Opts opts = new Opts();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
index 3fcbcb1b37..1ea4c2cc18 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/CheckForMetadataProblems.java
@@ -35,7 +35,6 @@
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.io.Text;
 
 public class CheckForMetadataProblems {
@@ -169,7 +168,7 @@ public static void main(String[] args) throws Exception {
     ClientOpts opts = new ClientOpts();
     opts.parseArgs(CheckForMetadataProblems.class.getName(), args);
 
-    VolumeManager fs = VolumeManagerImpl.get();
+    VolumeManager fs = opts.getServerContext().getVolumeManager();
 
     checkMetadataAndRootTableEntries(RootTable.NAME, opts, fs);
     checkMetadataAndRootTableEntries(MetadataTable.NAME, opts, fs);
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 6ecaf15c31..6af70e1ff1 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
@@ -24,7 +24,6 @@
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,14 +50,14 @@ public static void main(String[] args) {
     Opts opts = new Opts();
     opts.parseArgs(CleanZookeeper.class.getName(), args);
 
+    ServerContext context = new ServerContext(new SiteConfiguration());
+
     String root = Constants.ZROOT;
-    IZooReaderWriter zk = ZooReaderWriter.getInstance();
+    IZooReaderWriter zk = context.getZooReaderWriter();
     if (opts.auth != null) {
       zk.getZooKeeper().addAuthInfo("digest", ("accumulo:" + opts.auth).getBytes(UTF_8));
     }
 
-    ServerContext context = new ServerContext(SiteConfiguration.create());
-
     try {
       for (String child : zk.getChildren(root)) {
         if (Constants.ZINSTANCES.equals("/" + child)) {
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 0c4578c8b4..c40708dd19 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
@@ -23,6 +23,7 @@
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.cli.Help;
+import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
@@ -58,7 +59,7 @@ public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(DeleteZooInstance.class.getName(), args);
 
-    IZooReaderWriter zk = ZooReaderWriter.getInstance();
+    ZooReaderWriter zk = new ZooReaderWriter(new SiteConfiguration());
     // try instance name:
     Set<String> instances = new HashSet<>(zk.getChildren(Constants.ZROOT + Constants.ZINSTANCES));
     Set<String> uuids = new HashSet<>(zk.getChildren(Constants.ZROOT));
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/DumpZookeeper.java b/server/base/src/main/java/org/apache/accumulo/server/util/DumpZookeeper.java
index d89425ac68..22fcbd7ff5 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/DumpZookeeper.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/DumpZookeeper.java
@@ -23,6 +23,7 @@
 import java.util.Base64;
 
 import org.apache.accumulo.core.cli.Help;
+import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.log4j.Level;
@@ -60,7 +61,7 @@ public static void main(String[] args) {
     Logger.getRootLogger().setLevel(Level.WARN);
     PrintStream out = System.out;
     try {
-      zk = ZooReaderWriter.getInstance();
+      zk = new ZooReaderWriter(new SiteConfiguration());
 
       write(out, 0, "<dump root='%s'>", opts.root);
       for (String child : zk.getChildren(opts.root, null))
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 acbcd17025..5a768c72dd 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 String description() {
 
   @Override
   public void execute(final String[] args) throws KeeperException, InterruptedException {
-    ServerContext context = new ServerContext(SiteConfiguration.create());
+    ServerContext context = new ServerContext(new SiteConfiguration());
     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 bd0036ed6e..081ca5c9b1 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 static void main(String[] args) {
     opts.parseArgs(ListInstances.class.getName(), args);
 
     if (opts.keepers == null) {
-      SiteConfiguration siteConfig = SiteConfiguration.create();
+      SiteConfiguration siteConfig = new SiteConfiguration();
       opts.keepers = siteConfig.get(Property.INSTANCE_ZK_HOST);
     }
 
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 d96273da54..56abd4805d 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
@@ -32,13 +32,12 @@
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.log.WalStateManager;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.Path;
 
 public class ListVolumesUsed {
 
   public static void main(String[] args) throws Exception {
-    listVolumes(new ServerContext(SiteConfiguration.create()));
+    listVolumes(new ServerContext(new SiteConfiguration()));
   }
 
   private static String getTableURI(String rootTabletDir) {
@@ -124,7 +123,7 @@ private static void listTable(String name, ServerContext context) throws Excepti
 
     volumes.clear();
 
-    WalStateManager wals = new WalStateManager(context, ZooReaderWriter.getInstance());
+    WalStateManager wals = new WalStateManager(context);
     for (Path path : wals.getAllState().keySet()) {
       volumes.add(getLogURI(path.toString()));
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java b/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
index 8849359b00..5281485a30 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
@@ -33,7 +33,6 @@
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,7 +44,7 @@ public int run(String[] args) throws Exception {
     ClientOpts opts = new ClientOpts();
     opts.parseArgs(LocalityCheck.class.getName(), args);
 
-    VolumeManager fs = VolumeManagerImpl.get();
+    VolumeManager fs = opts.getServerContext().getVolumeManager();
     Connector connector = opts.getConnector();
     Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
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 608e8c1312..c6bf4fe792 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
@@ -47,7 +47,7 @@ public String description() {
 
   @Override
   public void execute(String[] args) throws Exception {
-    ServerContext context = new ServerContext(SiteConfiguration.create());
+    ServerContext context = new ServerContext(new SiteConfiguration());
     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/MasterMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
index 2d4472bef5..2e86ba517e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
@@ -53,11 +53,8 @@
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.io.Text;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -163,7 +160,6 @@ private static KeyExtent fixSplit(ServerContext context, Table.ID tableId, Text
     try (ScannerImpl scanner2 = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY)) {
       scanner2.setRange(new Range(prevRowKey, prevRowKey.followingKey(PartialKey.ROW)));
 
-      VolumeManager fs = VolumeManagerImpl.get();
       if (!scanner2.iterator().hasNext()) {
         log.info("Rolling back incomplete split {} {}", metadataEntry, metadataPrevEndRow);
         MetadataTableUtil.rollBackSplit(metadataEntry, KeyExtent.decodePrevEndRow(oper), context,
@@ -186,7 +182,7 @@ private static KeyExtent fixSplit(ServerContext context, Table.ID tableId, Text
 
           for (Entry<Key,Value> entry : scanner3) {
             if (entry.getKey().compareColumnFamily(DataFileColumnFamily.NAME) == 0) {
-              origDatafileSizes.put(new FileRef(fs, entry.getKey()),
+              origDatafileSizes.put(new FileRef(context.getVolumeManager(), entry.getKey()),
                   new DataFileValue(entry.getValue().get()));
             }
           }
@@ -288,7 +284,7 @@ private static void updateRootTabletDataFile(ServerContext context, KeyExtent ex
       FileRef path, FileRef mergeFile, DataFileValue dfv, String time,
       Set<FileRef> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs,
       TServerInstance lastLocation, long flushId) {
-    IZooReaderWriter zk = ZooReaderWriter.getInstance();
+    IZooReaderWriter zk = context.getZooReaderWriter();
     String root = MetadataTableUtil.getZookeeperLogLocation(context);
     for (String entry : unusedWalLogs) {
       String[] parts = entry.split("/");
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index 71a4f4b1e8..4082017ca2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@ -87,10 +87,8 @@
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.zookeeper.ZooLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -265,7 +263,7 @@ private static void retryZooKeeperUpdate(ServerContext context, ZooLock zooLock,
       ZooOperation op) {
     while (true) {
       try {
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+        IZooReaderWriter zoo = context.getZooReaderWriter();
         if (zoo.isLockHeld(zooLock.getLockID())) {
           op.run(zoo);
         }
@@ -297,7 +295,6 @@ public void run(IZooReaderWriter rw)
     try (Scanner mdScanner = new ScannerImpl(context, MetadataTable.ID, Authorizations.EMPTY)) {
       mdScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
       Text row = extent.getMetadataEntry();
-      VolumeManager fs = VolumeManagerImpl.get();
 
       Key endKey = new Key(row, DataFileColumnFamily.NAME, new Text(""));
       endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
@@ -308,7 +305,7 @@ public void run(IZooReaderWriter rw)
         if (!entry.getKey().getRow().equals(row))
           break;
         DataFileValue dfv = new DataFileValue(entry.getValue().get());
-        sizes.put(new FileRef(fs, entry.getKey()), dfv);
+        sizes.put(new FileRef(context.getVolumeManager(), entry.getKey()), dfv);
       }
 
       return sizes;
@@ -368,18 +365,20 @@ public static void addDeleteEntries(KeyExtent extent, Set<FileRef> datafilesToDe
     // TODO could use batch writer,would need to handle failure and retry like update does -
     // ACCUMULO-1294
     for (FileRef pathToRemove : datafilesToDelete) {
-      update(context, createDeleteMutation(tableId, pathToRemove.path().toString()), extent);
+      update(context, createDeleteMutation(context, tableId, pathToRemove.path().toString()),
+          extent);
     }
   }
 
   public static void addDeleteEntry(ServerContext context, Table.ID tableId, String path)
       throws IOException {
-    update(context, createDeleteMutation(tableId, path), new KeyExtent(tableId, null, null));
+    update(context, createDeleteMutation(context, tableId, path),
+        new KeyExtent(tableId, null, null));
   }
 
-  public static Mutation createDeleteMutation(Table.ID tableId, String pathToRemove)
-      throws IOException {
-    Path path = VolumeManagerImpl.get().getFullPath(tableId, pathToRemove);
+  public static Mutation createDeleteMutation(ServerContext context, Table.ID tableId,
+      String pathToRemove) throws IOException {
+    Path path = context.getVolumeManager().getFullPath(tableId, pathToRemove);
     Mutation delFlag = new Mutation(new Text(MetadataSchema.DeletesSection.getRowPrefix() + path));
     delFlag.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
     return delFlag;
@@ -465,12 +464,12 @@ public static void deleteTable(Table.ID tableId, boolean insertDeletes, ServerCo
           Key key = cell.getKey();
 
           if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-            FileRef ref = new FileRef(VolumeManagerImpl.get(), key);
-            bw.addMutation(createDeleteMutation(tableId, ref.meta().toString()));
+            FileRef ref = new FileRef(context.getVolumeManager(), key);
+            bw.addMutation(createDeleteMutation(context, tableId, ref.meta().toString()));
           }
 
           if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
-            bw.addMutation(createDeleteMutation(tableId, cell.getValue().toString()));
+            bw.addMutation(createDeleteMutation(context, tableId, cell.getValue().toString()));
           }
         }
 
@@ -507,7 +506,7 @@ static String getZookeeperLogLocation(ServerContext context) {
   }
 
   public static void setRootTabletDir(ServerContext context, String dir) throws IOException {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = context.getZooReaderWriter();
     String zpath = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET_PATH;
     try {
       zoo.putPersistentData(zpath, dir.getBytes(UTF_8), -1, NodeExistsPolicy.OVERWRITE);
@@ -520,7 +519,7 @@ public static void setRootTabletDir(ServerContext context, String dir) throws IO
   }
 
   public static String getRootTabletDir(ServerContext context) throws IOException {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = context.getZooReaderWriter();
     String zpath = context.getZooKeeperRoot() + RootTable.ZROOT_TABLET_PATH;
     try {
       return new String(zoo.getData(zpath, null), UTF_8);
@@ -538,7 +537,7 @@ public static String getRootTabletDir(ServerContext context) throws IOException
     ArrayList<LogEntry> result = new ArrayList<>();
     TreeMap<FileRef,DataFileValue> sizes = new TreeMap<>();
 
-    VolumeManager fs = VolumeManagerImpl.get();
+    VolumeManager fs = context.getVolumeManager();
     if (extent.isRootTablet()) {
       getRootLogEntries(context, result);
       Path rootDir = new Path(getRootTabletDir(context));
@@ -606,7 +605,7 @@ public static String getRootTabletDir(ServerContext context) throws IOException
 
   static void getRootLogEntries(ServerContext context, final ArrayList<LogEntry> result)
       throws KeeperException, InterruptedException, IOException {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = context.getZooReaderWriter();
     String root = getZookeeperLogLocation(context);
     // there's a little race between getting the children and fetching
     // the data. The log can be removed in between.
@@ -928,8 +927,9 @@ public static void cloneTable(ServerContext context, Table.ID srcTableId, Table.
         Mutation m = new Mutation(k.getRow());
         m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
         VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableId, context);
-        String dir = volumeManager.choose(chooserEnv, ServerConstants.getBaseUris())
-            + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + tableId + Path.SEPARATOR + new String(
+        String dir = volumeManager.choose(chooserEnv,
+            ServerConstants.getBaseUris(context.getConfiguration())) + Constants.HDFS_TABLES_DIR
+            + Path.SEPARATOR + tableId + Path.SEPARATOR + new String(
                 FastFormat.toZeroPaddedString(dirCount++, 8, 16, Constants.CLONE_PREFIX_BYTES));
         TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(dir.getBytes(UTF_8)));
 
@@ -966,12 +966,12 @@ public static void removeBulkLoadEntries(Connector conn, Table.ID tableId, long
     }
   }
 
-  public static List<FileRef> getBulkFilesLoaded(Connector conn, KeyExtent extent, long tid)
-      throws IOException {
+  public static List<FileRef> getBulkFilesLoaded(ServerContext context, Connector conn,
+      KeyExtent extent, long tid) throws IOException {
     List<FileRef> result = new ArrayList<>();
     try (Scanner mscanner = new IsolatedScanner(conn.createScanner(
         extent.isMeta() ? RootTable.NAME : MetadataTable.NAME, Authorizations.EMPTY))) {
-      VolumeManager fs = VolumeManagerImpl.get();
+      VolumeManager fs = context.getVolumeManager();
       mscanner.setRange(extent.toMetadataRange());
       mscanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
       for (Entry<Key,Value> entry : mscanner) {
@@ -992,7 +992,7 @@ public static void removeBulkLoadEntries(Connector conn, Table.ID tableId, long
     Text metadataRow = extent.getMetadataEntry();
     Map<Long,List<FileRef>> result = new HashMap<>();
 
-    VolumeManager fs = VolumeManagerImpl.get();
+    VolumeManager fs = context.getVolumeManager();
     try (Scanner scanner = new ScannerImpl(context,
         extent.isMeta() ? RootTable.ID : MetadataTable.ID, Authorizations.EMPTY)) {
       scanner.setRange(new Range(metadataRow));
@@ -1038,9 +1038,9 @@ public static void createReplicationTable(ServerContext context) throws IOExcept
 
     VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ReplicationTable.ID,
         context);
-    String dir = VolumeManagerImpl.get().choose(chooserEnv, ServerConstants.getBaseUris())
-        + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + ReplicationTable.ID
-        + Constants.DEFAULT_TABLET_LOCATION;
+    String dir = context.getVolumeManager().choose(chooserEnv,
+        ServerConstants.getBaseUris(context.getConfiguration())) + Constants.HDFS_TABLES_DIR
+        + Path.SEPARATOR + ReplicationTable.ID + Constants.DEFAULT_TABLET_LOCATION;
 
     Mutation m = new Mutation(new Text(KeyExtent.getMetadataEntry(ReplicationTable.ID, null)));
     m.put(DIRECTORY_COLUMN.getColumnFamily(), DIRECTORY_COLUMN.getColumnQualifier(), 0,
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
index 5dec31578a..9a133de230 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
@@ -21,10 +21,10 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.impl.Namespace;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.zookeeper.KeeperException;
 
 public class NamespacePropUtil {
@@ -33,16 +33,16 @@ public static boolean setNamespaceProperty(ServerContext context, Namespace.ID n
     if (!isPropertyValid(property, value))
       return false;
 
+    IZooReaderWriter zoo = context.getZooReaderWriter();
+
     // create the zk node for per-namespace properties for this namespace if it doesn't already
     // exist
     String zkNamespacePath = getPath(context, namespaceId);
-    ZooReaderWriter.getInstance().putPersistentData(zkNamespacePath, new byte[0],
-        NodeExistsPolicy.SKIP);
+    zoo.putPersistentData(zkNamespacePath, new byte[0], NodeExistsPolicy.SKIP);
 
     // create the zk node for this property and set it's data to the specified value
     String zPath = zkNamespacePath + "/" + property;
-    ZooReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(UTF_8),
-        NodeExistsPolicy.OVERWRITE);
+    zoo.putPersistentData(zPath, value.getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
 
     return true;
   }
@@ -56,7 +56,7 @@ public static boolean isPropertyValid(String property, String value) {
   public static void removeNamespaceProperty(ServerContext context, Namespace.ID namespaceId,
       String property) throws InterruptedException, KeeperException {
     String zPath = getPath(context, namespaceId) + "/" + property;
-    ZooReaderWriter.getInstance().recursiveDelete(zPath, NodeMissingPolicy.SKIP);
+    context.getZooReaderWriter().recursiveDelete(zPath, NodeMissingPolicy.SKIP);
   }
 
   private static String getPath(ServerContext context, Namespace.ID namespaceId) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java b/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java
index a0af94a744..8d81fe0a89 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RandomizeVolumes.java
@@ -43,7 +43,6 @@
 import org.apache.accumulo.server.cli.ClientOnRequiredTable;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -72,7 +71,7 @@ public static void main(String[] args) throws AccumuloException, AccumuloSecurit
 
   public static int randomize(ServerContext context, Connector c, String tableName)
       throws IOException, AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    final VolumeManager vm = VolumeManagerImpl.get();
+    final VolumeManager vm = context.getVolumeManager();
     if (vm.getVolumes().size() < 2) {
       log.error("There are not enough volumes configured");
       return 1;
@@ -114,9 +113,9 @@ public static int randomize(ServerContext context, Connector c, String tableName
       Mutation m = new Mutation(key.getRow());
 
       VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableId, context);
-      final String newLocation = vm.choose(chooserEnv, ServerConstants.getBaseUris())
-          + Path.SEPARATOR + ServerConstants.TABLE_DIR + Path.SEPARATOR + tableId + Path.SEPARATOR
-          + directory;
+      final String newLocation = vm.choose(chooserEnv,
+          ServerConstants.getBaseUris(context.getConfiguration())) + Path.SEPARATOR
+          + ServerConstants.TABLE_DIR + Path.SEPARATOR + tableId + Path.SEPARATOR + directory;
       m.put(key.getColumnFamily(), key.getColumnQualifier(),
           new Value(newLocation.getBytes(UTF_8)));
       if (log.isTraceEnabled()) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
index 585c72b92c..5c9d6f33a6 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
@@ -32,7 +32,6 @@
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.data.Key;
@@ -45,9 +44,9 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.hadoop.fs.Path;
 
@@ -118,7 +117,7 @@ public void run() {
     }
   }
 
-  private static int checkTable(ClientContext context, String tableName, Range range, boolean fix)
+  private static int checkTable(ServerContext context, String tableName, Range range, boolean fix)
       throws Exception {
 
     @SuppressWarnings({"rawtypes"})
@@ -128,7 +127,7 @@ private static int checkTable(ClientContext context, String tableName, Range ran
 
     System.out.printf("Scanning : %s %s\n", tableName, range);
 
-    VolumeManager fs = VolumeManagerImpl.get();
+    VolumeManager fs = context.getVolumeManager();
     Connector connector = context.getConnector();
     Scanner metadata = connector.createScanner(tableName, Authorizations.EMPTY);
     metadata.setRange(range);
@@ -182,7 +181,7 @@ private static int checkTable(ClientContext context, String tableName, Range ran
     return missing.get();
   }
 
-  static int checkAllTables(ClientContext context, boolean fix) throws Exception {
+  static int checkAllTables(ServerContext context, boolean fix) throws Exception {
     int missing = checkTable(context, RootTable.NAME, MetadataSchema.TabletsSection.getRange(),
         fix);
 
@@ -192,7 +191,7 @@ static int checkAllTables(ClientContext context, boolean fix) throws Exception {
       return missing;
   }
 
-  static int checkTable(ClientContext context, String tableName, boolean fix) throws Exception {
+  static int checkTable(ServerContext context, String tableName, boolean fix) throws Exception {
     if (tableName.equals(RootTable.NAME)) {
       throw new IllegalArgumentException("Can not check root table");
     } else if (tableName.equals(MetadataTable.NAME)) {
@@ -210,7 +209,6 @@ public static void main(String[] args) throws Exception {
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(RemoveEntriesForMissingFiles.class.getName(), args, scanOpts, bwOpts);
 
-    Connector conn = opts.getConnector();
-    checkAllTables(new ClientContext(conn.info()), opts.fix);
+    checkAllTables(opts.getServerContext(), opts.fix);
   }
 }
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 12bba35242..d25b98e40c 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
@@ -27,6 +27,7 @@
 import javax.xml.parsers.SAXParserFactory;
 
 import org.apache.accumulo.core.cli.Help;
+import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
@@ -114,6 +115,8 @@ public static void main(String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(RestoreZookeeper.class.getName(), args);
 
+    ZooReaderWriter zoo = new ZooReaderWriter(new SiteConfiguration());
+
     InputStream in = System.in;
     if (opts.file != null) {
       in = new FileInputStream(opts.file);
@@ -124,7 +127,7 @@ public static void main(String[] args) throws Exception {
     // is a simple switch to remove any chance of external entities causing problems.
     factory.setFeature("http://apache.org/xml/features/disallow-doctype-decl", true);
     SAXParser parser = factory.newSAXParser();
-    parser.parse(in, new Restore(ZooReaderWriter.getInstance(), opts.overwrite));
+    parser.parse(in, new Restore(zoo, opts.overwrite));
     in.close();
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java
index 32c8a225a9..82b2502b5d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java
@@ -24,7 +24,6 @@
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -63,13 +62,13 @@ public static boolean setSystemProperty(ServerContext context, String property,
     // create the zk node for this property and set it's data to the specified value
     String zPath = context.getZooKeeperRoot() + Constants.ZCONFIG + "/" + property;
 
-    return ZooReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(UTF_8),
+    return context.getZooReaderWriter().putPersistentData(zPath, value.getBytes(UTF_8),
         NodeExistsPolicy.OVERWRITE);
   }
 
   public static void removeSystemProperty(ServerContext context, String property)
       throws InterruptedException, KeeperException {
     String zPath = context.getZooKeeperRoot() + Constants.ZCONFIG + "/" + property;
-    ZooReaderWriter.getInstance().recursiveDelete(zPath, NodeMissingPolicy.FAIL);
+    context.getZooReaderWriter().recursiveDelete(zPath, NodeMissingPolicy.FAIL);
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
index 7ed7055634..8c62040d38 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TableDiskUsage.java
@@ -45,7 +45,6 @@
 import org.apache.accumulo.core.util.NumUtil;
 import org.apache.accumulo.server.cli.ClientOpts;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
@@ -297,10 +296,10 @@ public static void printDiskUsage(Collection<String> tableNames, VolumeManager f
   }
 
   public static void main(String[] args) throws Exception {
-    VolumeManager fs = VolumeManagerImpl.get();
     Opts opts = new Opts();
     opts.parseArgs(TableDiskUsage.class.getName(), args);
     Connector conn = opts.getConnector();
+    VolumeManager fs = opts.getServerContext().getVolumeManager();
     org.apache.accumulo.server.util.TableDiskUsage.printDiskUsage(opts.tables, fs, conn, false);
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
index ab5afb4ccd..f96edeed89 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TablePropUtil.java
@@ -31,23 +31,22 @@
 
   public static boolean setTableProperty(ServerContext context, Table.ID tableId, String property,
       String value) throws KeeperException, InterruptedException {
-    return setTableProperty(context.getZooKeeperRoot(), tableId, property, value);
+    return setTableProperty(context.getZooReaderWriter(), context.getZooKeeperRoot(), tableId,
+        property, value);
   }
 
-  public static boolean setTableProperty(String zkRoot, Table.ID tableId, String property,
-      String value) throws KeeperException, InterruptedException {
+  public static boolean setTableProperty(ZooReaderWriter zoo, String zkRoot, Table.ID tableId,
+      String property, String value) throws KeeperException, InterruptedException {
     if (!isPropertyValid(property, value))
       return false;
 
     // create the zk node for per-table properties for this table if it doesn't already exist
     String zkTablePath = getTablePath(zkRoot, tableId);
-    ZooReaderWriter.getInstance().putPersistentData(zkTablePath, new byte[0],
-        NodeExistsPolicy.SKIP);
+    zoo.putPersistentData(zkTablePath, new byte[0], NodeExistsPolicy.SKIP);
 
     // create the zk node for this property and set it's data to the specified value
     String zPath = zkTablePath + "/" + property;
-    ZooReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(UTF_8),
-        NodeExistsPolicy.OVERWRITE);
+    zoo.putPersistentData(zPath, value.getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
 
     return true;
   }
@@ -61,7 +60,7 @@ public static boolean isPropertyValid(String property, String value) {
   public static void removeTableProperty(ServerContext context, Table.ID tableId, String property)
       throws InterruptedException, KeeperException {
     String zPath = getTablePath(context.getZooKeeperRoot(), tableId) + "/" + property;
-    ZooReaderWriter.getInstance().recursiveDelete(zPath, NodeMissingPolicy.SKIP);
+    context.getZooReaderWriter().recursiveDelete(zPath, NodeMissingPolicy.SKIP);
   }
 
   private static String getTablePath(String zkRoot, Table.ID tableId) {
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 d471ce76a7..a813f13e38 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
@@ -23,7 +23,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.cli.Help;
 import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.zookeeper.ZooLock;
@@ -42,15 +41,15 @@
 
   public static void main(String[] args) throws Exception {
 
-    ServerContext context = new ServerContext(SiteConfiguration.create());
+    ServerContext context = new ServerContext(new SiteConfiguration());
     String tserverPath = context.getZooKeeperRoot() + Constants.ZTSERVERS;
     Opts opts = new Opts();
     opts.parseArgs(TabletServerLocks.class.getName(), args);
 
     ZooCache cache = context.getZooCache();
+    ZooReaderWriter zoo = context.getZooReaderWriter();
 
     if (opts.list) {
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 
       List<String> tabletServers = zoo.getChildren(tserverPath);
 
@@ -64,7 +63,7 @@ public static void main(String[] args) throws Exception {
         System.out.printf("%32s %16s%n", tabletServer, holder);
       }
     } else if (opts.delete != null) {
-      ZooLock.deleteLock(tserverPath + "/" + args[1]);
+      ZooLock.deleteLock(zoo, tserverPath + "/" + args[1]);
     } else {
       System.out.println(
           "Usage : " + TabletServerLocks.class.getName() + " -list|-delete <tserver lock>");
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 e0ed7b4e88..edcc619425 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
@@ -20,7 +20,6 @@
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.start.spi.KeywordExecutable;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -65,11 +64,11 @@ public String description() {
   public void execute(final String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(ZooKeeperMain.class.getName(), args);
-    FileSystem fs = VolumeManagerImpl.get().getDefaultVolume().getFileSystem();
-    String baseDir = ServerConstants.getBaseUris()[0];
+    ServerContext context = new ServerContext(new SiteConfiguration());
+    FileSystem fs = context.getVolumeManager().getDefaultVolume().getFileSystem();
+    String baseDir = ServerConstants.getBaseUris(context.getConfiguration())[0];
     System.out.println("Using " + fs.makeQualified(new Path(baseDir + "/instance_id"))
         + " to lookup accumulo instance");
-    ServerContext context = new ServerContext(SiteConfiguration.create());
     if (opts.servers == null) {
       opts.servers = context.getZooKeepers();
     }
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 7ba2a5dd85..d7a44e653a 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
@@ -65,7 +65,7 @@ public static void main(String[] args) {
       return;
     }
 
-    SiteConfiguration siteConf = SiteConfiguration.create();
+    SiteConfiguration siteConf = new SiteConfiguration();
     // Login as the server on secure HDFS
     if (siteConf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) {
       SecurityUtil.serverLogin(siteConf);
@@ -74,7 +74,7 @@ public static void main(String[] args) {
     String volDir = VolumeConfiguration.getVolumeUris(siteConf)[0];
     Path instanceDir = new Path(volDir, "instance_id");
     String iid = ZooUtil.getInstanceIDFromHdfs(instanceDir, siteConf);
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    ZooReaderWriter zoo = new ZooReaderWriter(siteConf);
 
     if (opts.zapMaster) {
       String masterLockPath = Constants.ZROOT + "/" + iid + Constants.ZMASTER_LOCK;
@@ -94,12 +94,11 @@ public static void main(String[] args) {
           message("Deleting " + tserversPath + "/" + child + " from zookeeper", opts);
 
           if (opts.zapMaster)
-            ZooReaderWriter.getInstance().recursiveDelete(tserversPath + "/" + child,
-                NodeMissingPolicy.SKIP);
+            zoo.recursiveDelete(tserversPath + "/" + child, NodeMissingPolicy.SKIP);
           else {
             String path = tserversPath + "/" + child;
             if (zoo.getChildren(path).size() > 0) {
-              if (!ZooLock.deleteLock(path, "tserver")) {
+              if (!ZooLock.deleteLock(zoo, path, "tserver")) {
                 message("Did not delete " + tserversPath + "/" + child, opts);
               }
             }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
index ce2d551e75..5a74023b8a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
@@ -51,7 +51,7 @@
   private static final Logger log = LoggerFactory.getLogger(DistributedWorkQueue.class);
 
   private ThreadPoolExecutor threadPool;
-  private ZooReaderWriter zoo = ZooReaderWriter.getInstance();
+  private ZooReaderWriter zoo;
   private String path;
   private AccumuloConfiguration config;
   private long timerInitialDelay, timerPeriod;
@@ -169,6 +169,11 @@ public DistributedWorkQueue(String path, AccumuloConfiguration config, long time
     this.config = config;
     this.timerInitialDelay = timerInitialDelay;
     this.timerPeriod = timerPeriod;
+    zoo = new ZooReaderWriter(config);
+  }
+
+  public ZooReaderWriter getZooReaderWriter() {
+    return zoo;
   }
 
   public void startProcessing(final Processor processor, ThreadPoolExecutor executorService)
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java
index 4cbe48a24b..68d4ece083 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java
@@ -48,7 +48,7 @@ public boolean transactionAlive(String type, long tid) throws Exception {
 
     public static void start(ServerContext context, String type, long tid)
         throws KeeperException, InterruptedException {
-      IZooReaderWriter writer = ZooReaderWriter.getInstance();
+      IZooReaderWriter writer = context.getZooReaderWriter();
       writer.putPersistentData(context.getZooKeeperRoot() + "/" + type, new byte[] {},
           NodeExistsPolicy.OVERWRITE);
       writer.putPersistentData(context.getZooKeeperRoot() + "/" + type + "/" + tid, new byte[] {},
@@ -59,14 +59,14 @@ public static void start(ServerContext context, String type, long tid)
 
     public static void stop(ServerContext context, String type, long tid)
         throws KeeperException, InterruptedException {
-      IZooReaderWriter writer = ZooReaderWriter.getInstance();
+      IZooReaderWriter writer = context.getZooReaderWriter();
       writer.recursiveDelete(context.getZooKeeperRoot() + "/" + type + "/" + tid,
           NodeMissingPolicy.SKIP);
     }
 
     public static void cleanup(ServerContext context, String type, long tid)
         throws KeeperException, InterruptedException {
-      IZooReaderWriter writer = ZooReaderWriter.getInstance();
+      IZooReaderWriter writer = context.getZooReaderWriter();
       writer.recursiveDelete(context.getZooKeeperRoot() + "/" + type + "/" + tid,
           NodeMissingPolicy.SKIP);
       writer.recursiveDelete(context.getZooKeeperRoot() + "/" + type + "/" + tid + "-running",
@@ -75,7 +75,7 @@ public static void cleanup(ServerContext context, String type, long tid)
 
     public static Set<Long> allTransactionsAlive(ServerContext context, String type)
         throws KeeperException, InterruptedException {
-      final IZooReader reader = ZooReaderWriter.getInstance();
+      final IZooReader reader = context.getZooReaderWriter();
       final Set<Long> result = new HashSet<>();
       final String parent = context.getZooKeeperRoot() + "/" + type;
       reader.sync(parent);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java
index 9550197dcc..78bc9f3fd5 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java
@@ -16,15 +16,20 @@
  */
 package org.apache.accumulo.server.zookeeper;
 
+import org.apache.accumulo.server.ServerContext;
 import org.apache.zookeeper.Watcher;
 
 public class ZooCache extends org.apache.accumulo.fate.zookeeper.ZooCache {
-  public ZooCache() {
-    this(null);
+
+  public ZooCache(ServerContext context) {
+    this(context.getZooReaderWriter(), null);
   }
 
-  public ZooCache(Watcher watcher) {
-    super(ZooReaderWriter.getInstance(), watcher);
+  public ZooCache(ZooReaderWriter zoo) {
+    this(zoo, null);
   }
 
+  public ZooCache(ZooReaderWriter zoo, Watcher watcher) {
+    super(zoo, watcher);
+  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java
index 3d51d816fe..606e9c6a29 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java
@@ -16,20 +16,9 @@
  */
 package org.apache.accumulo.server.zookeeper;
 
-import org.apache.zookeeper.KeeperException;
-
 public class ZooLock extends org.apache.accumulo.fate.zookeeper.ZooLock {
 
-  public ZooLock(String path) {
-    super(new ZooCache(), ZooReaderWriter.getInstance(), path);
-  }
-
-  public static void deleteLock(String path) throws InterruptedException, KeeperException {
-    deleteLock(ZooReaderWriter.getInstance(), path);
-  }
-
-  public static boolean deleteLock(String path, String lockData)
-      throws InterruptedException, KeeperException {
-    return deleteLock(ZooReaderWriter.getInstance(), path, lockData);
+  public ZooLock(ZooReaderWriter zoo, String path) {
+    super(new ZooCache(zoo), zoo, path);
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java
index 31e64b010e..633e710165 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooQueueLock.java
@@ -21,17 +21,20 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 
+import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock;
 import org.apache.zookeeper.KeeperException;
 
 public class ZooQueueLock extends org.apache.accumulo.fate.zookeeper.ZooQueueLock {
 
-  public ZooQueueLock(String path, boolean ephemeral) throws KeeperException, InterruptedException {
-    super(ZooReaderWriter.getInstance(), path, ephemeral);
+  public ZooQueueLock(ZooReaderWriter zoo, String path, boolean ephemeral)
+      throws KeeperException, InterruptedException {
+    super(zoo, path, ephemeral);
   }
 
   public static void main(String args[]) throws InterruptedException, KeeperException {
-    ZooQueueLock lock = new ZooQueueLock("/lock", true);
+    ZooReaderWriter zoo = new ZooReaderWriter(new SiteConfiguration());
+    ZooQueueLock lock = new ZooQueueLock(zoo, "/lock", true);
     DistributedReadWriteLock rlocker = new DistributedReadWriteLock(lock, "reader".getBytes(UTF_8));
     DistributedReadWriteLock wlocker = new DistributedReadWriteLock(lock,
         "wlocker".getBytes(UTF_8));
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java
index 2a9b908ca5..4f78e77ee1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java
@@ -20,25 +20,18 @@
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
 
 public class ZooReaderWriter extends org.apache.accumulo.fate.zookeeper.ZooReaderWriter {
   private static final String SCHEME = "digest";
   private static final String USER = "accumulo";
-  private static ZooReaderWriter instance = null;
 
-  public ZooReaderWriter(String string, int timeInMillis, String secret) {
-    super(string, timeInMillis, SCHEME, (USER + ":" + secret).getBytes(UTF_8));
+  public ZooReaderWriter(AccumuloConfiguration conf) {
+    this(conf.get(Property.INSTANCE_ZK_HOST),
+        (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT),
+        conf.get(Property.INSTANCE_SECRET));
   }
 
-  public static synchronized ZooReaderWriter getInstance() {
-    if (instance == null) {
-      AccumuloConfiguration conf = SiteConfiguration.getInstance();
-      instance = new ZooReaderWriter(conf.get(Property.INSTANCE_ZK_HOST),
-          (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT),
-          conf.get(Property.INSTANCE_SECRET));
-    }
-    return instance;
+  public ZooReaderWriter(String string, int timeInMillis, String secret) {
+    super(string, timeInMillis, SCHEME, (USER + ":" + secret).getBytes(UTF_8));
   }
-
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriterFactory.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriterFactory.java
index 2640204a62..b7683e4e9d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriterFactory.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriterFactory.java
@@ -18,9 +18,6 @@
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 
@@ -30,7 +27,6 @@
 public class ZooReaderWriterFactory {
   private static final String SCHEME = "digest";
   private static final String USER = "accumulo";
-  private static IZooReaderWriter instance = null;
 
   /**
    * Gets a new reader/writer.
@@ -46,22 +42,4 @@
   public IZooReaderWriter getZooReaderWriter(String string, int timeInMillis, String secret) {
     return new ZooReaderWriter(string, timeInMillis, SCHEME, (USER + ":" + secret).getBytes(UTF_8));
   }
-
-  /**
-   * Gets a reader/writer, retrieving ZooKeeper information from the site configuration. The same
-   * instance may be returned for multiple calls.
-   *
-   * @return reader/writer
-   */
-  public IZooReaderWriter getInstance() {
-    synchronized (ZooReaderWriterFactory.class) {
-      if (instance == null) {
-        AccumuloConfiguration conf = SiteConfiguration.getInstance();
-        instance = getZooReaderWriter(conf.get(Property.INSTANCE_ZK_HOST),
-            (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT),
-            conf.get(Property.INSTANCE_SECRET));
-      }
-      return instance;
-    }
-  }
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java b/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java
index d6da1ba65a..ecbde92788 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java
@@ -23,6 +23,8 @@
 import java.util.List;
 import java.util.UUID;
 
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -33,6 +35,9 @@
 import org.junit.rules.TemporaryFolder;
 
 public class ServerConstantsTest {
+
+  AccumuloConfiguration conf = DefaultConfiguration.getInstance();
+
   @Rule
   public TemporaryFolder folder = new TemporaryFolder(
       new File(System.getProperty("user.dir") + "/target"));
@@ -63,17 +68,17 @@ public void testCheckBaseDirs() throws IOException {
   }
 
   private void verifyAllPass(ArrayList<String> paths) {
-    Assert.assertEquals(paths, Arrays
-        .asList(ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), true)));
-    Assert.assertEquals(paths, Arrays
-        .asList(ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), false)));
+    Assert.assertEquals(paths, Arrays.asList(
+        ServerConstants.checkBaseUris(conf, paths.toArray(new String[paths.size()]), true)));
+    Assert.assertEquals(paths, Arrays.asList(
+        ServerConstants.checkBaseUris(conf, paths.toArray(new String[paths.size()]), false)));
   }
 
   private void verifySomePass(ArrayList<String> paths, int numExpected) {
-    Assert.assertEquals(paths.subList(0, 2), Arrays
-        .asList(ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), true)));
+    Assert.assertEquals(paths.subList(0, 2), Arrays.asList(
+        ServerConstants.checkBaseUris(conf, paths.toArray(new String[paths.size()]), true)));
     try {
-      ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), false);
+      ServerConstants.checkBaseUris(conf, paths.toArray(new String[paths.size()]), false);
       Assert.fail();
     } catch (Exception e) {
       // ignored
@@ -82,14 +87,14 @@ private void verifySomePass(ArrayList<String> paths, int numExpected) {
 
   private void verifyError(ArrayList<String> paths) {
     try {
-      ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), true);
+      ServerConstants.checkBaseUris(conf, paths.toArray(new String[paths.size()]), true);
       Assert.fail();
     } catch (Exception e) {
       // ignored
     }
 
     try {
-      ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), false);
+      ServerConstants.checkBaseUris(conf, paths.toArray(new String[paths.size()]), false);
       Assert.fail();
     } catch (Exception e) {
       // ignored
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 f73d95bdbb..61ae9c2f9b 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
@@ -50,7 +50,7 @@
   // use the same mock ZooCacheFactory and ZooCache for all tests
   private static ZooCacheFactory zcf;
   private static ZooCache zc;
-  private static SiteConfiguration siteConfig = SiteConfiguration.create();
+  private static SiteConfiguration siteConfig = new SiteConfiguration();
 
   @BeforeClass
   public static void setUpClass() throws Exception {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java b/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java
index d8ee0b1eb1..b87ae3198a 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java
@@ -28,8 +28,8 @@
 
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.junit.After;
@@ -44,8 +44,8 @@
   private Configuration conf;
   private VolumeManager fs;
   private SiteConfiguration sconf;
-  private IZooReaderWriter zooOrig;
-  private IZooReaderWriter zoo;
+  private ZooReaderWriter zooOrig;
+  private ZooReaderWriter zoo;
 
   @SuppressWarnings("deprecation")
   @Before
@@ -56,7 +56,7 @@ public void setUp() throws Exception {
     expect(sconf.get(Property.INSTANCE_VOLUMES)).andReturn("").anyTimes();
     expect(sconf.get(Property.INSTANCE_DFS_DIR)).andReturn("/bar").anyTimes();
     expect(sconf.get(Property.INSTANCE_ZK_HOST)).andReturn("zk1").anyTimes();
-    zoo = createMock(IZooReaderWriter.class);
+    zoo = createMock(ZooReaderWriter.class);
     zooOrig = Initialize.getZooReaderWriter();
     Initialize.setZooReaderWriter(zoo);
   }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/LiveTServerSetTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/LiveTServerSetTest.java
index 69bf2b6d6a..c94e92faa2 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/LiveTServerSetTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/LiveTServerSetTest.java
@@ -22,8 +22,8 @@
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.master.LiveTServerSet.Listener;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerInfo;
@@ -42,7 +42,7 @@ public void testSessionIds() {
         new TServerInstance(HostAndPort.fromParts("localhost", 1234), "5555"), mockConn);
     servers.put("server1", server1);
 
-    LiveTServerSet tservers = new LiveTServerSet(EasyMock.createMock(ClientContext.class),
+    LiveTServerSet tservers = new LiveTServerSet(EasyMock.createMock(ServerContext.class),
         EasyMock.createMock(Listener.class));
 
     assertEquals(server1.instance, tservers.find(servers, "localhost:1234"));
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 7c4bdfdd7e..0ea7d970b1 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
@@ -91,7 +91,7 @@ public String getTableName() {
         TestDefaultBalancer.class.getName());
   }
 
-  private static SiteConfiguration siteConfg = SiteConfiguration.create();
+  private static SiteConfiguration siteConfg = new SiteConfiguration();
 
   protected static class TestServerConfigurationFactory extends ServerConfigurationFactory {
 
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 fb46537257..347b5fde22 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
@@ -155,7 +155,7 @@ public void test() throws Exception {
     final ServerContext context = createMockContext();
     replay(context);
     ServerConfigurationFactory confFactory = new ServerConfigurationFactory(context,
-        SiteConfiguration.create()) {
+        new SiteConfiguration()) {
       @Override
       public TableConfiguration getTableConfiguration(Table.ID tableId) {
         // create a dummy namespaceConfiguration to satisfy requireNonNull in TableConfiguration
diff --git a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
index af63ccfdf5..ad800d4142 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
@@ -55,9 +55,10 @@
   @Before
   public void setUp() throws Exception {
     context = createMock(ServerContext.class);
+    zoorw = createMock(ZooReaderWriter.class);
     expect(context.getZooKeeperRoot()).andReturn("/accumulo/instance");
+    expect(context.getZooReaderWriter()).andReturn(zoorw).anyTimes();
     replay(context);
-    zoorw = createMock(ZooReaderWriter.class);
   }
 
   @Test
@@ -174,7 +175,7 @@ public void testSaveToZooKeeper() throws Exception {
         .andReturn(true);
     replay(zoorw);
 
-    r.saveToZooKeeper(zoorw, context);
+    r.saveToZooKeeper(context);
     verify(zoorw);
   }
 
@@ -189,7 +190,7 @@ public void testDecodeZooKeeperEntry() throws Exception {
         .andReturn(encoded);
     replay(zoorw);
 
-    r = ProblemReport.decodeZooKeeperEntry(node, zoorw, context);
+    r = ProblemReport.decodeZooKeeperEntry(context, node);
     assertEquals(TABLE_ID, r.getTableId());
     assertSame(ProblemType.FILE_READ, r.getProblemType());
     assertEquals(RESOURCE, r.getResource());
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 503af6596a..958061e105 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
@@ -38,7 +38,7 @@
   @Rule
   public TestName test = new TestName();
 
-  private static SiteConfiguration siteConfig = SiteConfiguration.create();
+  private static SiteConfiguration siteConfig = new SiteConfiguration();
   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 2516b37ed5..374c07d753 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
@@ -59,7 +59,7 @@
     private ConfigurationCopy conf = null;
 
     public TestServerConfigurationFactory(ServerContext context) {
-      super(context, SiteConfiguration.create());
+      super(context, new SiteConfiguration());
       conf = new ConfigurationCopy(DefaultConfiguration.getInstance());
     }
 
@@ -126,6 +126,7 @@ public void testStopTServer_Null() {
 
   private static ServerContext createMockContext() {
     ServerContext context = EasyMock.createMock(ServerContext.class);
+    expect(context.getZooReaderWriter()).andReturn(null);
     expect(context.getProperties()).andReturn(new Properties()).anyTimes();
     expect(context.getZooKeepers()).andReturn("").anyTimes();
     expect(context.getInstanceName()).andReturn("instance").anyTimes();
@@ -295,6 +296,7 @@ private int getFreePort(int startingAddress) throws UnknownHostException {
 
   private ServerAddress startServer() throws Exception {
     ServerContext ctx = createMock(ServerContext.class);
+    expect(ctx.getZooReaderWriter()).andReturn(null).anyTimes();
     expect(ctx.getInstanceID()).andReturn("instance").anyTimes();
     expect(ctx.getConfiguration()).andReturn(factory.getSystemConfiguration()).anyTimes();
     expect(ctx.getThriftServerType()).andReturn(ThriftServerType.THREADPOOL);
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index c81d267329..c086d7fa31 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@ -59,7 +59,6 @@
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletLocationState;
 import org.apache.accumulo.server.master.state.TabletState;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -104,7 +103,7 @@ public void update(LiveTServerSet current, Set<TServerInstance> deleted,
       }
     });
     liveServers.startListeningForTabletServerChanges();
-    this.walMarker = new WalStateManager(context, ZooReaderWriter.getInstance());
+    this.walMarker = new WalStateManager(context);
     this.store = new Iterable<TabletLocationState>() {
       @Override
       public Iterator<TabletLocationState> iterator() {
@@ -417,7 +416,7 @@ private long getCurrent(Map<TServerInstance,Set<UUID>> logsByServer,
   protected Map<UUID,Path> getSortedWALogs() throws IOException {
     Map<UUID,Path> result = new HashMap<>();
 
-    for (String dir : ServerConstants.getRecoveryDirs()) {
+    for (String dir : ServerConstants.getRecoveryDirs(context.getConfiguration())) {
       Path recoveryDir = new Path(dir);
       if (fs.exists(recoveryDir)) {
         for (FileStatus status : fs.listStatus(recoveryDir)) {
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 9908fe9798..c455a82771 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -350,7 +350,8 @@ public void delete(SortedMap<String,String> confirmedDeletes)
       ExecutorService deleteThreadPool = Executors.newFixedThreadPool(getNumDeleteThreads(),
           new NamingThreadFactory("deleting"));
 
-      final List<Pair<Path,Path>> replacements = ServerConstants.getVolumeReplacements();
+      final List<Pair<Path,Path>> replacements = ServerConstants
+          .getVolumeReplacements(getConfiguration());
 
       for (final String delete : confirmedDeletes.values()) {
 
@@ -451,7 +452,7 @@ public void run() {
     public void deleteTableDirIfEmpty(Table.ID tableID) throws IOException {
       // if dir exist and is empty, then empty list is returned...
       // hadoop 2.0 will throw an exception if the file does not exist
-      for (String dir : ServerConstants.getTablesDirs()) {
+      for (String dir : ServerConstants.getTablesDirs(context.getConfiguration())) {
         FileStatus[] tabletDirs = null;
         try {
           tabletDirs = fs.listStatus(new Path(dir + "/" + tableID));
@@ -649,7 +650,7 @@ public void run() {
     };
 
     while (true) {
-      lock = new ZooLock(path);
+      lock = new ZooLock(context.getZooReaderWriter(), path);
       if (lock.tryLock(lockWatcher,
           new ServerServices(addr.toString(), Service.GC_CLIENT).toString().getBytes())) {
         log.debug("Got GC ZooKeeper lock");
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
index 22ca046964..cc89e6b1ae 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
@@ -51,7 +51,6 @@
 import org.apache.accumulo.server.log.WalStateManager.WalState;
 import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
@@ -132,7 +131,7 @@ public void run() {
    * @return The Set of WALs that are referenced in the metadata table
    */
   protected HashSet<String> getClosedLogs() {
-    WalStateManager wals = new WalStateManager(context, ZooReaderWriter.getInstance());
+    WalStateManager wals = new WalStateManager(context);
 
     HashSet<String> result = new HashSet<>();
     try {
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 663333c649..69fa1d1ecb 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
@@ -53,7 +53,7 @@
   private Opts opts;
   private SimpleGarbageCollector gc;
   private ConfigurationCopy systemConfig;
-  private static SiteConfiguration siteConfig = SiteConfiguration.create();
+  private static SiteConfiguration siteConfig = new SiteConfiguration();
 
   @Before
   public void setUp() {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index e382e3fe14..043e49866d 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -283,7 +283,7 @@ private void moveRootTabletToRootTable(IZooReaderWriter zoo) throws Exception {
       Path oldPath = fs.getFullPath(FileType.TABLE, "/" + MetadataTable.ID + "/root_tablet");
       if (fs.exists(oldPath)) {
         VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(RootTable.ID, context);
-        String newPath = fs.choose(chooserEnv, ServerConstants.getBaseUris())
+        String newPath = fs.choose(chooserEnv, ServerConstants.getBaseUris(getConfiguration()))
             + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + RootTable.ID;
         fs.mkdirs(new Path(newPath));
         if (!fs.rename(oldPath, new Path(newPath))) {
@@ -295,7 +295,7 @@ private void moveRootTabletToRootTable(IZooReaderWriter zoo) throws Exception {
 
       Path location = null;
 
-      for (String basePath : ServerConstants.getTablesDirs()) {
+      for (String basePath : ServerConstants.getTablesDirs(getConfiguration())) {
         Path path = new Path(basePath + "/" + RootTable.ID + RootTable.ROOT_TABLET_LOCATION);
         if (fs.exists(path)) {
           if (location != null) {
@@ -337,7 +337,7 @@ private void upgradeZookeeper() {
       try {
         log.info("Upgrading zookeeper");
 
-        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+        IZooReaderWriter zoo = context.getZooReaderWriter();
         final String zooRoot = getZooKeeperRoot();
 
         log.debug("Handling updates for version {}", accumuloPersistentVersion);
@@ -404,20 +404,21 @@ private void upgradeZookeeper() {
           Namespace.ID id = namespace.getSecond();
           log.debug("Upgrade creating namespace \"{}\" (ID: {})", ns, id);
           if (!Namespaces.exists(context, id))
-            TableManager.prepareNewNamespaceState(getInstanceID(), id, ns, NodeExistsPolicy.SKIP);
+            TableManager.prepareNewNamespaceState(zoo, getInstanceID(), id, ns,
+                NodeExistsPolicy.SKIP);
         }
 
         // create replication table in zk
         log.debug("Upgrade creating table {} (ID: {})", ReplicationTable.NAME, ReplicationTable.ID);
-        TableManager.prepareNewTableState(getInstanceID(), ReplicationTable.ID,
+        TableManager.prepareNewTableState(zoo, getInstanceID(), ReplicationTable.ID,
             Namespace.ID.ACCUMULO, ReplicationTable.NAME, TableState.OFFLINE,
             NodeExistsPolicy.SKIP);
 
         // create root table
         log.debug("Upgrade creating table {} (ID: {})", RootTable.NAME, RootTable.ID);
-        TableManager.prepareNewTableState(getInstanceID(), RootTable.ID, Namespace.ID.ACCUMULO,
+        TableManager.prepareNewTableState(zoo, getInstanceID(), RootTable.ID, Namespace.ID.ACCUMULO,
             RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.SKIP);
-        Initialize.initSystemTablesConfig(context.getZooKeeperRoot());
+        Initialize.initSystemTablesConfig(context.getZooReaderWriter(), context.getZooKeeperRoot());
         // ensure root user can flush root table
         security.grantTablePermission(context.rpcCreds(), security.getRootUsername(), RootTable.ID,
             TablePermission.ALTER_TABLE, Namespace.ID.ACCUMULO);
@@ -446,7 +447,7 @@ private void upgradeZookeeper() {
 
         // add system namespace permissions to existing users
         ZKPermHandler perm = new ZKPermHandler();
-        perm.initialize(getInstanceID(), true);
+        perm.initialize(getContext(), true);
         String users = getZooKeeperRoot() + "/users";
         for (String user : zoo.getChildren(users)) {
           zoo.putPersistentData(users + "/" + user + "/Namespaces", new byte[0],
@@ -683,7 +684,7 @@ public Master(ServerContext context) throws IOException {
       log.info("SASL is enabled, creating delegation token key manager and distributor");
       final long tokenUpdateInterval = aconf
           .getTimeInMillis(Property.GENERAL_DELEGATION_TOKEN_UPDATE_INTERVAL);
-      keyDistributor = new ZooAuthenticationKeyDistributor(ZooReaderWriter.getInstance(),
+      keyDistributor = new ZooAuthenticationKeyDistributor(context.getZooReaderWriter(),
           getZooKeeperRoot() + Constants.ZDELEGATION_TOKEN_KEYS);
       authenticationTokenKeyManager = new AuthenticationTokenKeyManager(context.getSecretManager(),
           keyDistributor, tokenUpdateInterval, tokenLifetime);
@@ -715,9 +716,9 @@ public MergeInfo getMergeInfo(Table.ID tableId) {
     synchronized (mergeLock) {
       try {
         String path = getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + "/merge";
-        if (!ZooReaderWriter.getInstance().exists(path))
+        if (!context.getZooReaderWriter().exists(path))
           return new MergeInfo();
-        byte[] data = ZooReaderWriter.getInstance().getData(path, new Stat());
+        byte[] data = context.getZooReaderWriter().getData(path, new Stat());
         DataInputBuffer in = new DataInputBuffer();
         in.reset(data, data.length);
         MergeInfo info = new MergeInfo();
@@ -740,7 +741,7 @@ public void setMergeState(MergeInfo info, MergeState state)
           + "/merge";
       info.setState(state);
       if (state.equals(MergeState.NONE)) {
-        ZooReaderWriter.getInstance().recursiveDelete(path, NodeMissingPolicy.SKIP);
+        context.getZooReaderWriter().recursiveDelete(path, NodeMissingPolicy.SKIP);
       } else {
         DataOutputBuffer out = new DataOutputBuffer();
         try {
@@ -748,7 +749,7 @@ public void setMergeState(MergeInfo info, MergeState state)
         } catch (IOException ex) {
           throw new RuntimeException("Unlikely", ex);
         }
-        ZooReaderWriter.getInstance().putPersistentData(path, out.getData(),
+        context.getZooReaderWriter().putPersistentData(path, out.getData(),
             state.equals(MergeState.STARTED) ? ZooUtil.NodeExistsPolicy.FAIL
                 : ZooUtil.NodeExistsPolicy.OVERWRITE);
       }
@@ -761,7 +762,7 @@ public void clearMergeState(Table.ID tableId)
       throws IOException, KeeperException, InterruptedException {
     synchronized (mergeLock) {
       String path = getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + "/merge";
-      ZooReaderWriter.getInstance().recursiveDelete(path, NodeMissingPolicy.SKIP);
+      context.getZooReaderWriter().recursiveDelete(path, NodeMissingPolicy.SKIP);
       mergeLock.notifyAll();
     }
     nextEvent.event("Merge state of %s cleared", tableId);
@@ -769,7 +770,7 @@ public void clearMergeState(Table.ID tableId)
 
   void setMasterGoalState(MasterGoalState state) {
     try {
-      ZooReaderWriter.getInstance().putPersistentData(
+      context.getZooReaderWriter().putPersistentData(
           getZooKeeperRoot() + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
           NodeExistsPolicy.OVERWRITE);
     } catch (Exception ex) {
@@ -780,7 +781,7 @@ void setMasterGoalState(MasterGoalState state) {
   MasterGoalState getMasterGoalState() {
     while (true)
       try {
-        byte[] data = ZooReaderWriter.getInstance()
+        byte[] data = context.getZooReaderWriter()
             .getData(getZooKeeperRoot() + Constants.ZMASTER_GOAL_STATE, null);
         return MasterGoalState.valueOf(new String(data));
       } catch (Exception e) {
@@ -1291,7 +1292,7 @@ public void run() throws IOException, InterruptedException, KeeperException {
 
     tserverSet.startListeningForTabletServerChanges();
 
-    ZooReaderWriter zReaderWriter = ZooReaderWriter.getInstance();
+    ZooReaderWriter zReaderWriter = context.getZooReaderWriter();
 
     zReaderWriter.getChildren(zroot + Constants.ZRECOVERY, new Watcher() {
       @Override
@@ -1299,7 +1300,7 @@ public void process(WatchedEvent event) {
         nextEvent.event("Noticed recovery changes", event.getType());
         try {
           // watcher only fires once, add it back
-          ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, this);
+          zReaderWriter.getChildren(zroot + Constants.ZRECOVERY, this);
         } catch (Exception e) {
           log.error("Failed to add log recovery watcher back", e);
         }
@@ -1326,7 +1327,7 @@ boolean canSuspendTablets() {
           }
         });
 
-    watchers.add(new TabletGroupWatcher(this, new ZooTabletStateStore(new ZooStore(zroot)),
+    watchers.add(new TabletGroupWatcher(this, new ZooTabletStateStore(new ZooStore(context)),
         watchers.get(1)) {
       @Override
       boolean canSuspendTablets() {
@@ -1343,7 +1344,7 @@ boolean canSuspendTablets() {
 
     try {
       final AgeOffStore<Master> store = new AgeOffStore<>(new org.apache.accumulo.fate.ZooStore<>(
-          getZooKeeperRoot() + Constants.ZFATE, ZooReaderWriter.getInstance()), 1000 * 60 * 60 * 8);
+          getZooKeeperRoot() + Constants.ZFATE, context.getZooReaderWriter()), 1000 * 60 * 60 * 8);
 
       int threads = getConfiguration().getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
 
@@ -1404,7 +1405,7 @@ public void run() {
     replicationWorkAssigner.start();
 
     // Advertise that port we used so peers don't have to be told what it is
-    ZooReaderWriter.getInstance().putPersistentData(
+    context.getZooReaderWriter().putPersistentData(
         getZooKeeperRoot() + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR,
         replAddress.address.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
 
@@ -1527,7 +1528,7 @@ private void getMasterLock(final String zMasterLoc) throws KeeperException, Inte
     while (true) {
 
       MasterLockWatcher masterLockWatcher = new MasterLockWatcher();
-      masterLock = new ZooLock(zMasterLoc);
+      masterLock = new ZooLock(getContext().getZooReaderWriter(), zMasterLoc);
       masterLock.lockAsync(masterLockWatcher, masterClientAddress.getBytes());
 
       masterLockWatcher.waitForChange();
@@ -1565,7 +1566,7 @@ public static void main(String[] args) throws Exception {
   @Override
   public void update(LiveTServerSet current, Set<TServerInstance> deleted,
       Set<TServerInstance> added) {
-    DeadServerList obit = new DeadServerList(getZooKeeperRoot() + Constants.ZDEADTSERVERS);
+    DeadServerList obit = new DeadServerList(context, getZooKeeperRoot() + Constants.ZDEADTSERVERS);
     if (added.size() > 0) {
       log.info("New servers: {}", added);
       for (TServerInstance up : added)
@@ -1752,7 +1753,7 @@ public MasterMonitorInfo getMasterMonitorInfo() {
       for (TServerInstance server : serversToShutdown)
         result.serversShuttingDown.add(server.hostPort());
     }
-    DeadServerList obit = new DeadServerList(getZooKeeperRoot() + Constants.ZDEADTSERVERS);
+    DeadServerList obit = new DeadServerList(context, getZooKeeperRoot() + Constants.ZDEADTSERVERS);
     result.deadTabletServers = obit.getList();
     result.bulkImports = bulkImportStatus.getBulkLoadStatus();
     return result;
@@ -1783,7 +1784,7 @@ public boolean delegationTokensAvailable() {
 
   public void markDeadServerLogsAsClosed(Map<TServerInstance,List<Path>> logsForDeadServers)
       throws WalMarkerException {
-    WalStateManager mgr = new WalStateManager(context, ZooReaderWriter.getInstance());
+    WalStateManager mgr = new WalStateManager(context);
     for (Entry<TServerInstance,List<Path>> server : logsForDeadServers.entrySet()) {
       for (Path path : server.getValue()) {
         mgr.closeWal(server.getKey(), path);
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index 3d070ee337..83582866f9 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -93,7 +93,6 @@
 import org.apache.accumulo.server.util.NamespacePropUtil;
 import org.apache.accumulo.server.util.SystemPropUtil;
 import org.apache.accumulo.server.util.TablePropUtil;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.Token;
 import org.apache.thrift.TException;
@@ -125,7 +124,7 @@ public long initiateFlush(TInfo tinfo, TCredentials c, String tableIdStr)
     String zTablePath = Constants.ZROOT + "/" + master.getInstanceID() + Constants.ZTABLES + "/"
         + tableId + Constants.ZTABLE_FLUSH_ID;
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = master.getContext().getZooReaderWriter();
     byte fid[];
     try {
       fid = zoo.mutate(zTablePath, null, null, new Mutator() {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterTime.java b/server/master/src/main/java/org/apache/accumulo/master/MasterTime.java
index cbfa2b3a34..240670cdf2 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterTime.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterTime.java
@@ -50,7 +50,7 @@
 
   public MasterTime(Master master) throws IOException {
     this.zPath = master.getZooKeeperRoot() + Constants.ZMASTER_TICK;
-    this.zk = ZooReaderWriter.getInstance();
+    this.zk = master.getContext().getZooReaderWriter();
     this.master = master;
 
     try {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index c4d1cf0cb7..3ca5cc5cdf 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -89,7 +89,6 @@
 import org.apache.accumulo.server.master.state.TabletStateStore;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
@@ -147,7 +146,7 @@ public void run() {
     int[] oldCounts = new int[TabletState.values().length];
     EventCoordinator.Listener eventListener = this.master.nextEvent.getListener();
 
-    WalStateManager wals = new WalStateManager(master.getContext(), ZooReaderWriter.getInstance());
+    WalStateManager wals = new WalStateManager(master.getContext());
 
     while (this.master.stillMaster()) {
       // slow things down a little, otherwise we spam the logs when there are many wake-up events
@@ -671,9 +670,9 @@ private void deleteTablets(MergeInfo info) throws AccumuloException {
         Master.log.debug("Recreating the last tablet to point to {}", extent.getPrevEndRow());
         VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(extent.getTableId(),
             master.getContext());
-        String tdir = master.getFileSystem().choose(chooserEnv, ServerConstants.getBaseUris())
-            + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + extent.getTableId()
-            + Constants.DEFAULT_TABLET_LOCATION;
+        String tdir = master.getFileSystem().choose(chooserEnv,
+            ServerConstants.getBaseUris(master.getConfiguration())) + Constants.HDFS_TABLES_DIR
+            + Path.SEPARATOR + extent.getTableId() + Constants.DEFAULT_TABLET_LOCATION;
         MetadataTableUtil.addTablet(
             new KeyExtent(extent.getTableId(), null, extent.getPrevEndRow()), tdir,
             master.getContext(), timeType, this.master.masterLock);
@@ -729,8 +728,8 @@ private void mergeMetadataRecords(MergeInfo info) throws AccumuloException {
         } else if (TabletsSection.ServerColumnFamily.TIME_COLUMN.hasColumns(key)) {
           maxLogicalTime = TabletTime.maxMetadataTime(maxLogicalTime, value.toString());
         } else if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
-          bw.addMutation(MetadataTableUtil.createDeleteMutation(range.getTableId(),
-              entry.getValue().toString()));
+          bw.addMutation(MetadataTableUtil.createDeleteMutation(master.getContext(),
+              range.getTableId(), entry.getValue().toString()));
         }
       }
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
index 59cf756795..8417e76817 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
@@ -64,11 +64,11 @@
   public RecoveryManager(Master master) {
     this.master = master;
     executor = Executors.newScheduledThreadPool(4, new NamingThreadFactory("Walog sort starter "));
-    zooCache = new ZooCache();
+    zooCache = new ZooCache(master.getContext());
     try {
-      AccumuloConfiguration aconf = master.getConfiguration();
       List<String> workIDs = new DistributedWorkQueue(
-          master.getZooKeeperRoot() + Constants.ZRECOVERY, aconf).getWorkQueued();
+          master.getZooKeeperRoot() + Constants.ZRECOVERY, master.getConfiguration())
+              .getWorkQueued();
       sortsQueued.addAll(workIDs);
     } catch (Exception e) {
       log.warn("{}", e.getMessage(), e);
@@ -92,14 +92,14 @@ public LogSortTask(LogCloser closer, String source, String destination, String s
     public void run() {
       boolean rescheduled = false;
       try {
-        AccumuloConfiguration aconf = master.getConfiguration();
-        long time = closer.close(aconf, master.getFileSystem(), new Path(source));
+        long time = closer.close(master.getConfiguration(), master.getFileSystem(),
+            new Path(source));
 
         if (time > 0) {
           executor.schedule(this, time, TimeUnit.MILLISECONDS);
           rescheduled = true;
         } else {
-          initiateSort(sortId, source, destination, aconf);
+          initiateSort(sortId, source, destination);
         }
       } catch (FileNotFoundException e) {
         log.debug("Unable to initate log sort for " + source + ": " + e);
@@ -116,11 +116,11 @@ public void run() {
 
   }
 
-  private void initiateSort(String sortId, String source, final String destination,
-      AccumuloConfiguration aconf) throws KeeperException, InterruptedException, IOException {
+  private void initiateSort(String sortId, String source, final String destination)
+      throws KeeperException, InterruptedException, IOException {
     String work = source + "|" + destination;
-    new DistributedWorkQueue(master.getZooKeeperRoot() + Constants.ZRECOVERY, aconf).addWork(sortId,
-        work.getBytes(UTF_8));
+    new DistributedWorkQueue(master.getZooKeeperRoot() + Constants.ZRECOVERY,
+        master.getConfiguration()).addWork(sortId, work.getBytes(UTF_8));
 
     synchronized (this) {
       sortsQueued.add(sortId);
@@ -138,7 +138,7 @@ public boolean recoverLogs(KeyExtent extent, Collection<Collection<String>> walo
       for (String walog : logs) {
 
         String switchedWalog = VolumeUtil.switchVolume(walog, FileType.WAL,
-            ServerConstants.getVolumeReplacements());
+            ServerConstants.getVolumeReplacements(master.getConfiguration()));
         if (switchedWalog != null) {
           // replaces the volume used for sorting, but do not change entry in metadata table. When
           // the tablet loads it will change the metadata table entry. If
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
index 53cb3fd3bb..c3dade4316 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
@@ -133,7 +133,7 @@ public void assignWork() {
     initializeQueuedWork();
 
     if (null == zooCache) {
-      zooCache = new ZooCache();
+      zooCache = new ZooCache(workQueue.getZooReaderWriter());
     }
 
     // Get the maximum number of entries we want to queue work for (or the default)
diff --git a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
index c8a39db08e..4ec37ed9f6 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
@@ -262,13 +262,14 @@ public static void main(String[] args) throws Exception {
 
     Connector conn = opts.getConnector();
     Map<String,String> tableIdMap = conn.tableOperations().tableIdMap();
+    ZooReaderWriter zooReaderWriter = opts.getServerContext().getZooReaderWriter();
     for (Entry<String,String> entry : tableIdMap.entrySet()) {
       final String table = entry.getKey(), tableId = entry.getValue();
       String path = ZooUtil.getRoot(conn.getInstanceID()) + Constants.ZTABLES + "/" + tableId
           + "/merge";
       MergeInfo info = new MergeInfo();
-      if (ZooReaderWriter.getInstance().exists(path)) {
-        byte[] data = ZooReaderWriter.getInstance().getData(path, new Stat());
+      if (zooReaderWriter.exists(path)) {
+        byte[] data = zooReaderWriter.getData(path, new Stat());
         DataInputBuffer in = new DataInputBuffer();
         in.reset(data, data.length);
         info.readFields(in);
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 6d3227bf51..8ea12847af 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
@@ -25,7 +25,6 @@
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.security.SecurityUtil;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 
 public class SetGoalState {
 
@@ -39,10 +38,10 @@ public static void main(String[] args) throws Exception {
       System.exit(-1);
     }
 
-    ServerContext context = new ServerContext(SiteConfiguration.create());
+    ServerContext context = new ServerContext(new SiteConfiguration());
     SecurityUtil.serverLogin(context.getConfiguration());
-    Accumulo.waitForZookeeperAndHdfs(context.getVolumeManager());
-    ZooReaderWriter.getInstance().putPersistentData(
+    Accumulo.waitForZookeeperAndHdfs(context);
+    context.getZooReaderWriter().putPersistentData(
         context.getZooKeeperRoot() + Constants.ZMASTER_GOAL_STATE, args[0].getBytes(UTF_8),
         NodeExistsPolicy.OVERWRITE);
   }
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
index e3f68edc65..2f245d8539 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
@@ -26,7 +26,6 @@
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
 import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 
 public class CancelCompactions extends MasterRepo {
 
@@ -52,7 +51,7 @@ public long isReady(long tid, Master env) throws Exception {
     String zCancelID = Constants.ZROOT + "/" + environment.getInstanceID() + Constants.ZTABLES + "/"
         + tableId + Constants.ZTABLE_COMPACT_CANCEL_ID;
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = environment.getContext().getZooReaderWriter();
 
     byte[] currentValue = zoo.getData(zCompactID, null);
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChooseDir.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChooseDir.java
index d24e943c91..bd5be4b5f6 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChooseDir.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChooseDir.java
@@ -43,9 +43,9 @@ public long isReady(long tid, Master environment) throws Exception {
     // one here
     VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableInfo.tableId,
         master.getContext());
-    tableInfo.dir = master.getFileSystem().choose(chooserEnv, ServerConstants.getBaseUris())
-        + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + tableInfo.tableId
-        + Constants.DEFAULT_TABLET_LOCATION;
+    tableInfo.dir = master.getFileSystem().choose(chooserEnv,
+        ServerConstants.getBaseUris(master.getConfiguration())) + Constants.HDFS_TABLES_DIR
+        + Path.SEPARATOR + tableInfo.tableId + Constants.DEFAULT_TABLET_LOCATION;
     return new CreateDir(tableInfo);
   }
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CleanUp.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CleanUp.java
index debda60aba..41fab91bc7 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CleanUp.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CleanUp.java
@@ -170,7 +170,7 @@ public long isReady(long tid, Master master) throws Exception {
       // delete the map files
       try {
         VolumeManager fs = master.getFileSystem();
-        for (String dir : ServerConstants.getTablesDirs()) {
+        for (String dir : ServerConstants.getTablesDirs(master.getConfiguration())) {
           fs.deleteRecursively(new Path(dir, tableId.canonicalID()));
         }
       } catch (IOException e) {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
index 88df1b9768..1ee224c477 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
@@ -35,7 +35,6 @@
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.master.tableOps.UserCompactionConfig;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
@@ -93,7 +92,7 @@ public long isReady(long tid, Master env) throws Exception {
     String zTablePath = Constants.ZROOT + "/" + env.getInstanceID() + Constants.ZTABLES + "/"
         + tableId + Constants.ZTABLE_COMPACT_ID;
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = env.getContext().getZooReaderWriter();
     byte[] cid;
     try {
       cid = zoo.mutate(zTablePath, null, null, new Mutator() {
@@ -146,7 +145,7 @@ static void removeIterators(Master environment, final long txid, Table.ID tableI
     String zTablePath = Constants.ZROOT + "/" + environment.getInstanceID() + Constants.ZTABLES
         + "/" + tableId + Constants.ZTABLE_COMPACT_ID;
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = environment.getContext().getZooReaderWriter();
 
     zoo.mutate(zTablePath, null, null, new Mutator() {
       @Override
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
index 98984adc8d..61ba17b3ae 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
@@ -47,7 +47,6 @@
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
 import org.apache.accumulo.server.master.state.TServerInstance;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
 import org.slf4j.LoggerFactory;
@@ -77,7 +76,7 @@ public long isReady(long tid, Master master) throws Exception {
     String zCancelID = Constants.ZROOT + "/" + master.getInstanceID() + Constants.ZTABLES + "/"
         + tableId + Constants.ZTABLE_COMPACT_CANCEL_ID;
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = master.getContext().getZooReaderWriter();
 
     if (Long.parseLong(new String(zoo.getData(zCancelID, null))) >= compactId) {
       // compaction was canceled
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateImportDir.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateImportDir.java
index 4fdec0bdd9..106afb56a2 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateImportDir.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateImportDir.java
@@ -44,7 +44,7 @@
     UniqueNameAllocator namer = master.getContext().getUniqueNameAllocator();
 
     Path exportDir = new Path(tableInfo.exportDir);
-    String[] tableDirs = ServerConstants.getTablesDirs();
+    String[] tableDirs = ServerConstants.getTablesDirs(master.getConfiguration());
 
     log.info("Looking for matching filesystem for " + exportDir + " from options "
         + Arrays.toString(tableDirs));
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
index c98ad722ea..4db30726f9 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
@@ -105,7 +105,7 @@
       // hdfs://localhost:8020/path/to/accumulo/tables/...
       final String bulkDir = tableInfo.importDir;
 
-      final String[] tableDirs = ServerConstants.getTablesDirs();
+      final String[] tableDirs = ServerConstants.getTablesDirs(master.getConfiguration());
 
       ZipEntry zipEntry;
       while ((zipEntry = zis.getNextEntry()) != null) {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateZookeeperWithNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateZookeeperWithNamespace.java
index 96e5c4ecc1..50293499fd 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateZookeeperWithNamespace.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateZookeeperWithNamespace.java
@@ -50,8 +50,9 @@ public long isReady(long id, Master environment) throws Exception {
       Utils.checkNamespaceDoesNotExist(master.getContext(), namespaceInfo.namespaceName,
           namespaceInfo.namespaceId, TableOperation.CREATE);
 
-      TableManager.prepareNewNamespaceState(master.getInstanceID(), namespaceInfo.namespaceId,
-          namespaceInfo.namespaceName, NodeExistsPolicy.OVERWRITE);
+      TableManager.prepareNewNamespaceState(master.getContext().getZooReaderWriter(),
+          master.getInstanceID(), namespaceInfo.namespaceId, namespaceInfo.namespaceName,
+          NodeExistsPolicy.OVERWRITE);
 
       for (Entry<String,String> entry : namespaceInfo.props.entrySet())
         NamespacePropUtil.setNamespaceProperty(master.getContext(), namespaceInfo.namespaceId,
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
index 5621fdf81c..e50d7e3b54 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
@@ -26,7 +26,6 @@
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
 import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.slf4j.LoggerFactory;
 
 public class RenameNamespace extends MasterRepo {
@@ -50,7 +49,7 @@ public RenameNamespace(Namespace.ID namespaceId, String oldName, String newName)
   @Override
   public Repo<Master> call(long id, Master master) throws Exception {
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = master.getContext().getZooReaderWriter();
 
     Utils.tableNameLock.lock();
     try {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index 3001911554..60c3619d2f 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -32,7 +32,6 @@
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
 import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.slf4j.LoggerFactory;
 
 public class RenameTable extends MasterRepo {
@@ -69,7 +68,7 @@ public RenameTable(Namespace.ID namespaceId, Table.ID tableId, String oldTableNa
           TableOperation.RENAME, TableOperationExceptionType.INVALID_NAME,
           "Namespace in new table name does not match the old table name");
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = master.getContext().getZooReaderWriter();
 
     Utils.tableNameLock.lock();
     try {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
index 75db68dc68..b8ebaeddb2 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
@@ -27,7 +27,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.impl.AbstractId;
 import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
-import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Namespace;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Table;
@@ -39,8 +38,8 @@
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooReservation;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.zookeeper.ZooQueueLock;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -49,7 +48,7 @@
   private static final byte[] ZERO_BYTE = {'0'};
   private static final Logger log = LoggerFactory.getLogger(Utils.class);
 
-  static void checkTableDoesNotExist(ClientContext context, String tableName, Table.ID tableId,
+  static void checkTableDoesNotExist(ServerContext context, String tableName, Table.ID tableId,
       TableOperation operation) throws AcceptableThriftTableOperationException {
 
     Table.ID id = Tables.getNameToIdMap(context).get(tableName);
@@ -59,10 +58,10 @@ static void checkTableDoesNotExist(ClientContext context, String tableName, Tabl
           TableOperationExceptionType.EXISTS, null);
   }
 
-  static <T extends AbstractId> T getNextId(String name, ClientContext context,
+  static <T extends AbstractId> T getNextId(String name, ServerContext context,
       Function<String,T> newIdFunction) throws AcceptableThriftTableOperationException {
     try {
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      IZooReaderWriter zoo = context.getZooReaderWriter();
       final String ntp = context.getZooKeeperRoot() + Constants.ZTABLES;
       byte[] nid = zoo.mutate(ntp, ZERO_BYTE, ZooUtil.PUBLIC, currentValue -> {
         BigInteger nextId = new BigInteger(new String(currentValue, UTF_8), Character.MAX_RADIX);
@@ -84,7 +83,7 @@ public static long reserveTable(Master env, Table.ID tableId, long tid, boolean
       boolean tableMustExist, TableOperation op) throws Exception {
     if (getLock(env.getContext(), tableId, tid, writeLock).tryLock()) {
       if (tableMustExist) {
-        IZooReaderWriter zk = ZooReaderWriter.getInstance();
+        IZooReaderWriter zk = env.getContext().getZooReaderWriter();
         if (!zk.exists(env.getContext().getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId))
           throw new AcceptableThriftTableOperationException(tableId.canonicalID(), "", op,
               TableOperationExceptionType.NOTFOUND, "Table does not exist");
@@ -114,7 +113,7 @@ public static long reserveNamespace(Master env, Namespace.ID namespaceId, long i
       boolean writeLock, boolean mustExist, TableOperation op) throws Exception {
     if (getLock(env.getContext(), namespaceId, id, writeLock).tryLock()) {
       if (mustExist) {
-        IZooReaderWriter zk = ZooReaderWriter.getInstance();
+        IZooReaderWriter zk = env.getContext().getZooReaderWriter();
         if (!zk.exists(
             env.getContext().getZooKeeperRoot() + Constants.ZNAMESPACES + "/" + namespaceId))
           throw new AcceptableThriftTableOperationException(namespaceId.canonicalID(), "", op,
@@ -132,7 +131,7 @@ public static long reserveHdfsDirectory(Master env, String directory, long tid)
     String resvPath = env.getContext().getZooKeeperRoot() + Constants.ZHDFS_RESERVATIONS + "/"
         + Base64.getEncoder().encodeToString(directory.getBytes(UTF_8));
 
-    IZooReaderWriter zk = ZooReaderWriter.getInstance();
+    IZooReaderWriter zk = env.getContext().getZooReaderWriter();
 
     if (ZooReservation.attempt(zk, resvPath, String.format("%016x", tid), "")) {
       return 0;
@@ -144,13 +143,14 @@ public static void unreserveHdfsDirectory(Master env, String directory, long tid
       throws KeeperException, InterruptedException {
     String resvPath = env.getContext().getZooKeeperRoot() + Constants.ZHDFS_RESERVATIONS + "/"
         + Base64.getEncoder().encodeToString(directory.getBytes(UTF_8));
-    ZooReservation.release(ZooReaderWriter.getInstance(), resvPath, String.format("%016x", tid));
+    ZooReservation.release(env.getContext().getZooReaderWriter(), resvPath,
+        String.format("%016x", tid));
   }
 
-  private static Lock getLock(ClientContext context, AbstractId id, long tid, boolean writeLock)
+  private static Lock getLock(ServerContext context, AbstractId id, long tid, boolean writeLock)
       throws Exception {
     byte[] lockData = String.format("%016x", tid).getBytes(UTF_8);
-    ZooQueueLock qlock = new ZooQueueLock(
+    ZooQueueLock qlock = new ZooQueueLock(context.getZooReaderWriter(),
         context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS + "/" + id, false);
     Lock lock = DistributedReadWriteLock.recoverLock(qlock, lockData);
     if (lock == null) {
@@ -167,7 +167,7 @@ public static Lock getReadLock(Master env, AbstractId tableId, long tid) throws
     return Utils.getLock(env.getContext(), tableId, tid, false);
   }
 
-  static void checkNamespaceDoesNotExist(ClientContext context, String namespace,
+  static void checkNamespaceDoesNotExist(ServerContext context, String namespace,
       Namespace.ID namespaceId, TableOperation operation)
       throws AcceptableThriftTableOperationException {
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
index aa59d07cf8..d3ebfbf6e4 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
@@ -150,7 +150,8 @@ public long isReady(long tid, Master master) throws Exception {
 
   private Path createNewBulkDir(ServerContext context, VolumeManager fs, Table.ID tableId)
       throws IOException {
-    Path tempPath = fs.matchingFileSystem(new Path(sourceDir), ServerConstants.getTablesDirs());
+    Path tempPath = fs.matchingFileSystem(new Path(sourceDir),
+        ServerConstants.getTablesDirs(context.getConfiguration()));
     if (tempPath == null)
       throw new IOException(sourceDir + " is not in a volume configured for Accumulo");
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImport.java
index 21fa939630..b69658bf4c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImport.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/PrepBulkImport.java
@@ -224,7 +224,7 @@ private void checkForMerge(final Master master) throws Exception {
   private Path createNewBulkDir(ServerContext context, VolumeManager fs, Table.ID tableId)
       throws IOException {
     Path tempPath = fs.matchingFileSystem(new Path(bulkInfo.sourceDir),
-        ServerConstants.getTablesDirs());
+        ServerConstants.getTablesDirs(context.getConfiguration()));
     if (tempPath == null)
       throw new IOException(bulkInfo.sourceDir + " is not in a volume configured for Accumulo");
 
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tserverOps/ShutdownTServer.java b/server/master/src/main/java/org/apache/accumulo/master/tserverOps/ShutdownTServer.java
index 8ac83effc1..79c626040c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tserverOps/ShutdownTServer.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tserverOps/ShutdownTServer.java
@@ -21,7 +21,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.fate.Repo;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.master.tableOps.MasterRepo;
@@ -91,10 +90,10 @@ public long isReady(long tid, Master master) throws Exception {
   public Repo<Master> call(long tid, Master master) throws Exception {
     // suppress assignment of tablets to the server
     if (force) {
+      ZooReaderWriter zoo = master.getContext().getZooReaderWriter();
       String path = master.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + server.getLocation();
-      ZooLock.deleteLock(path);
+      ZooLock.deleteLock(zoo, path);
       path = master.getZooKeeperRoot() + Constants.ZDEADTSERVERS + "/" + server.getLocation();
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
       zoo.putPersistentData(path, "forced down".getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
     }
 
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 ebb46ed35d..f2cdd3c228 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
@@ -30,7 +30,6 @@
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
@@ -78,11 +77,11 @@ public static void main(String[] args) throws Exception {
 
     AdminUtil<Master> admin = new AdminUtil<>();
 
-    ServerContext context = new ServerContext(SiteConfiguration.create());
+    ServerContext context = new ServerContext(new SiteConfiguration());
     final String zkRoot = context.getZooKeeperRoot();
     String path = zkRoot + Constants.ZFATE;
     String masterPath = zkRoot + Constants.ZMASTER_LOCK;
-    IZooReaderWriter zk = ZooReaderWriter.getInstance();
+    IZooReaderWriter zk = context.getZooReaderWriter();
     ZooStore<Master> zs = new ZooStore<>(path, zk);
 
     if (jc.getParsedCommand().equals("fail")) {
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/MasterReplicationCoordinatorTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/MasterReplicationCoordinatorTest.java
index 6516ec1578..2dab2693ab 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/MasterReplicationCoordinatorTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/MasterReplicationCoordinatorTest.java
@@ -73,6 +73,7 @@ public void randomServerFromMany() {
     ServerContext context = EasyMock.createMock(ServerContext.class);
     EasyMock.expect(context.getConfiguration()).andReturn(config).anyTimes();
     EasyMock.expect(context.getInstanceID()).andReturn("1234").anyTimes();
+    EasyMock.expect(context.getZooReaderWriter()).andReturn(null).anyTimes();
     EasyMock.expect(master.getInstanceID()).andReturn("1234").anyTimes();
     EasyMock.expect(master.getContext()).andReturn(context).anyTimes();
     EasyMock.replay(master, context, reader);
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index fcd819aecb..dfc158fb53 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -408,7 +408,7 @@ private static GCStatus fetchGcStatus() {
     HostAndPort address = null;
     try {
       // Read the gc location from its lock
-      ZooReaderWriter zk = ZooReaderWriter.getInstance();
+      ZooReaderWriter zk = context.getZooReaderWriter();
       String path = context.getZooKeeperRoot() + Constants.ZGC_LOCK;
       List<String> locks = zk.getChildren(path, null);
       if (locks != null && locks.size() > 0) {
@@ -490,7 +490,7 @@ public void run() {
 
     try {
       String monitorAddress = HostAndPort.fromParts(advertiseHost, server.getPort()).toString();
-      ZooReaderWriter.getInstance().putPersistentData(
+      context.getZooReaderWriter().putPersistentData(
           context.getZooKeeperRoot() + Constants.ZMONITOR_HTTP_ADDR, monitorAddress.getBytes(UTF_8),
           NodeExistsPolicy.OVERWRITE);
       log.info("Set monitor address in zookeeper to {}", monitorAddress);
@@ -499,8 +499,7 @@ public void run() {
     }
 
     if (null != advertiseHost) {
-      LogService.startLogListener(Monitor.getContext().getConfiguration(), context.getInstanceID(),
-          advertiseHost);
+      LogService.startLogListener(context, advertiseHost);
     } else {
       log.warn("Not starting log4j listener as we could not determine address to use");
     }
@@ -633,7 +632,7 @@ private void getMonitorLock() throws KeeperException, InterruptedException {
     final String monitorLockPath = zRoot + Constants.ZMONITOR_LOCK;
 
     // Ensure that everything is kosher with ZK as this has changed.
-    ZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    ZooReaderWriter zoo = context.getZooReaderWriter();
     if (zoo.exists(monitorPath)) {
       byte[] data = zoo.getData(monitorPath, null);
       // If the node isn't empty, it's from a previous install (has hostname:port for HTTP server)
@@ -661,7 +660,7 @@ private void getMonitorLock() throws KeeperException, InterruptedException {
     // Get a ZooLock for the monitor
     while (true) {
       MoniterLockWatcher monitorLockWatcher = new MoniterLockWatcher();
-      monitorLock = new ZooLock(monitorLockPath);
+      monitorLock = new ZooLock(zoo, monitorLockPath);
       monitorLock.lockAsync(monitorLockWatcher, new byte[0]);
 
       monitorLockWatcher.waitForChange();
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
index 679bf338c0..e246e71bef 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServerResource.java
@@ -102,7 +102,7 @@ public TabletServers getTserverSummary() {
   @Consumes(MediaType.TEXT_PLAIN)
   public void clearDeadServer(
       @QueryParam("server") @NotNull @Pattern(regexp = SERVER_REGEX) String server) {
-    DeadServerList obit = new DeadServerList(
+    DeadServerList obit = new DeadServerList(Monitor.getContext(),
         Monitor.getContext().getZooKeeperRoot() + Constants.ZDEADTSERVERS);
     obit.delete(server);
   }
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 d97d081d03..d24ca2ab3f 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 int hashCode() {
     public MonitorLocation get() {
       // lazily set up path and zooCache (see comment in constructor)
       if (this.context == null) {
-        this.context = new ServerContext(SiteConfiguration.create());
+        this.context = new ServerContext(new SiteConfiguration());
         this.path = context.getZooKeeperRoot() + Constants.ZMONITOR_LOG4J_ADDR;
         this.zooCache = context.getZooCache();
       }
diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
index 8e714089a1..b277135e63 100644
--- a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
+++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
@@ -57,7 +57,6 @@
 import org.apache.accumulo.server.metrics.MetricsSystemHelper;
 import org.apache.accumulo.server.security.SecurityUtil;
 import org.apache.accumulo.server.util.time.SimpleTimer;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.apache.accumulo.tracer.thrift.RemoteSpan;
 import org.apache.accumulo.tracer.thrift.SpanReceiver.Iface;
@@ -348,7 +347,7 @@ private void resetWriter() {
   }
 
   private void registerInZooKeeper(String name, String root) throws Exception {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = context.getZooReaderWriter();
     zoo.putPersistentData(root, new byte[0], NodeExistsPolicy.SKIP);
     log.info("Registering tracer {} at {}", name, root);
     String path = zoo.putEphemeralSequential(root + "/trace-", name.getBytes(UTF_8));
@@ -404,14 +403,13 @@ public static void main(String[] args) throws Exception {
     ServerContext context = new ServerContext(opts.getSiteConfiguration());
     loginTracer(context.getConfiguration());
     MetricsSystemHelper.configure(TraceServer.class.getSimpleName());
-    Accumulo.init(context.getVolumeManager(), context.getInstanceID(),
-        context.getServerConfFactory(), app);
+    Accumulo.init(context, app);
     TraceServer server = new TraceServer(context, opts.getAddress());
     try {
       server.run();
     } finally {
       log.info("tracer stopping");
-      ZooReaderWriter.getInstance().getZooKeeper().close();
+      context.getZooReaderWriter().getZooKeeper().close();
     }
   }
 
@@ -427,7 +425,7 @@ public void process(WatchedEvent event) {
     }
     if (event.getPath() != null) {
       try {
-        if (ZooReaderWriter.getInstance().exists(event.getPath(), this))
+        if (context.getZooReaderWriter().exists(event.getPath(), this))
           return;
       } catch (Exception ex) {
         log.error("{}", ex.getMessage(), ex);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/BulkFailedCopyProcessor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/BulkFailedCopyProcessor.java
index 90e14a4701..2d6f0ac606 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/BulkFailedCopyProcessor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/BulkFailedCopyProcessor.java
@@ -20,10 +20,9 @@
 
 import java.io.IOException;
 
-import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue.Processor;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -38,9 +37,15 @@
 
   private static final Logger log = LoggerFactory.getLogger(BulkFailedCopyProcessor.class);
 
+  private ServerContext context;
+
+  BulkFailedCopyProcessor(ServerContext context) {
+    this.context = context;
+  }
+
   @Override
   public Processor newProcessor() {
-    return new BulkFailedCopyProcessor();
+    return new BulkFailedCopyProcessor(context);
   }
 
   @Override
@@ -52,8 +57,8 @@ public void process(String workID, byte[] data) {
     Path dest = new Path(paths[1]);
     Path tmp = new Path(dest.getParent(), dest.getName() + ".tmp");
 
+    VolumeManager vm = context.getVolumeManager();
     try {
-      VolumeManager vm = VolumeManagerImpl.get(SiteConfiguration.getInstance());
       FileSystem origFs = vm.getVolumeByPath(orig).getFileSystem();
       FileSystem destFs = vm.getVolumeByPath(dest).getFileSystem();
 
@@ -62,7 +67,6 @@ public void process(String workID, byte[] data) {
       log.debug("copied {} to {}", orig, dest);
     } catch (IOException ex) {
       try {
-        VolumeManager vm = VolumeManagerImpl.get(SiteConfiguration.getInstance());
         FileSystem destFs = vm.getVolumeByPath(dest).getFileSystem();
         destFs.create(dest).close();
         log.warn(" marked " + dest + " failed", ex);
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
index 2a09a15a43..23bd6c91dc 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
@@ -42,7 +42,6 @@
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.Key;
@@ -643,7 +642,7 @@ private synchronized FileSKVIterator getReader() throws IOException {
         FileSystem fs = FileSystem.getLocal(conf);
 
         reader = new RFileOperations().newReaderBuilder().forFile(memDumpFile, fs, conf)
-            .withTableConfiguration(SiteConfiguration.getInstance())
+            .withTableConfiguration(context.getConfiguration())
             .withCryptoService(context.getCryptoService()).seekToBeginning().build();
         if (iflag != null)
           reader.setInterruptFlag(iflag);
@@ -808,14 +807,14 @@ public void delete(long waitTime) {
         Configuration newConf = new Configuration(conf);
         newConf.setInt("io.seqfile.compress.blocksize", 100000);
 
-        AccumuloConfiguration siteConf = SiteConfiguration.getInstance();
+        AccumuloConfiguration aconf = context.getConfiguration();
 
         if (getOrCreateSampler() != null) {
-          siteConf = createSampleConfig(siteConf);
+          aconf = createSampleConfig(aconf);
         }
 
         FileSKVWriter out = new RFileOperations().newWriterBuilder().forFile(tmpFile, fs, newConf)
-            .withTableConfiguration(siteConf).withCryptoService(context.getCryptoService()).build();
+            .withTableConfiguration(aconf).withCryptoService(context.getCryptoService()).build();
 
         InterruptibleIterator iter = map.skvIterator(null);
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index e5c9eb0489..e0a36056a2 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -167,7 +167,6 @@
 import org.apache.accumulo.fate.util.LoggingRunnable;
 import org.apache.accumulo.fate.util.Retry;
 import org.apache.accumulo.fate.util.Retry.RetryFactory;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockWatcher;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
@@ -286,7 +285,7 @@
 
   private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
   private final TransactionWatcher watcher;
-  private final ZooCache masterLockCache = new ZooCache();
+  private ZooCache masterLockCache;
 
   private final TabletServerLogger logger;
 
@@ -356,6 +355,7 @@ public Metrics getMinCMetrics() {
 
   public TabletServer(ServerContext context) {
     this.context = context;
+    this.masterLockCache = new ZooCache(context);
     this.watcher = new TransactionWatcher(context);
     this.confFactory = context.getServerConfFactory();
     this.fs = context.getVolumeManager();
@@ -426,7 +426,7 @@ public void run() {
         TabletLocator.clearLocators();
       }
     }, jitter(TIME_BETWEEN_LOCATOR_CACHE_CLEARS), jitter(TIME_BETWEEN_LOCATOR_CACHE_CLEARS));
-    walMarker = new WalStateManager(context, ZooReaderWriter.getInstance());
+    walMarker = new WalStateManager(context);
 
     // Create the secret manager
     context.setSecretManager(new AuthenticationTokenSecretManager(context.getInstanceID(),
@@ -435,7 +435,7 @@ public void run() {
       log.info("SASL is enabled, creating ZooKeeper watcher for AuthenticationKeys");
       // Watcher to notice new AuthenticationKeys which enable delegation tokens
       authKeyWatcher = new ZooAuthenticationKeyWatcher(context.getSecretManager(),
-          ZooReaderWriter.getInstance(),
+          context.getZooReaderWriter(),
           context.getZooKeeperRoot() + Constants.ZDELEGATION_TOKEN_KEYS);
     } else {
       authKeyWatcher = null;
@@ -2513,7 +2513,7 @@ public void run() {
             getTableConfiguration(extent));
         TabletData data;
         if (extent.isRootTablet()) {
-          data = new TabletData(context, fs, ZooReaderWriter.getInstance(),
+          data = new TabletData(context, fs, context.getZooReaderWriter(),
               getTableConfiguration(extent));
         } else {
           data = new TabletData(context, extent, fs, tabletsKeyValues.entrySet().iterator());
@@ -2697,7 +2697,7 @@ private HostAndPort startReplicationService() throws UnknownHostException {
       // The replication service is unique to the thrift service for a tserver, not just a host.
       // Advertise the host and port for replication service given the host and port for the
       // tserver.
-      ZooReaderWriter.getInstance().putPersistentData(
+      context.getZooReaderWriter().putPersistentData(
           context.getZooKeeperRoot() + ReplicationConstants.ZOO_TSERVERS + "/" + clientAddress,
           sp.address.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
     } catch (Exception e) {
@@ -2713,7 +2713,7 @@ public ZooLock getLock() {
   }
 
   private void announceExistence() {
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    ZooReaderWriter zoo = context.getZooReaderWriter();
     try {
       String zPath = context.getZooKeeperRoot() + Constants.ZTSERVERS + "/"
           + getClientAddressString();
@@ -2728,7 +2728,7 @@ private void announceExistence() {
         throw e;
       }
 
-      tabletServerLock = new ZooLock(zPath);
+      tabletServerLock = new ZooLock(zoo, zPath);
 
       LockWatcher lw = new LockWatcher() {
 
@@ -2787,7 +2787,7 @@ public void run() {
     // To make things easier on users/devs, and to avoid creating an upgrade path to 1.7
     // We can just make the zookeeper paths before we try to use.
     try {
-      ZooKeeperInitialization.ensureZooKeeperInitialized(ZooReaderWriter.getInstance(),
+      ZooKeeperInitialization.ensureZooKeeperInitialized(context.getZooReaderWriter(),
           context.getZooKeeperRoot());
     } catch (KeeperException | InterruptedException e) {
       log.error("Could not ensure that ZooKeeper is properly initialized", e);
@@ -2839,7 +2839,7 @@ public void run() {
     bulkFailedCopyQ = new DistributedWorkQueue(
         context.getZooKeeperRoot() + Constants.ZBULK_FAILED_COPYQ, getConfiguration());
     try {
-      bulkFailedCopyQ.startProcessing(new BulkFailedCopyProcessor(), distWorkQThreadPool);
+      bulkFailedCopyQ.startProcessing(new BulkFailedCopyProcessor(context), distWorkQThreadPool);
     } catch (Exception e1) {
       throw new RuntimeException("Failed to start distributed work queue for copying ", e1);
     }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 28b60dd2ad..b0f3229cc3 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -411,8 +411,9 @@ public synchronized void open(String address) throws IOException {
 
     VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ChooserScope.LOGGER,
         context);
-    logPath = fs.choose(chooserEnv, ServerConstants.getBaseUris()) + Path.SEPARATOR
-        + ServerConstants.WAL_DIR + Path.SEPARATOR + logger + Path.SEPARATOR + filename;
+    logPath = fs.choose(chooserEnv, ServerConstants.getBaseUris(context.getConfiguration()))
+        + Path.SEPARATOR + ServerConstants.WAL_DIR + Path.SEPARATOR + logger + Path.SEPARATOR
+        + filename;
 
     metaReference = toString();
     LoggerOperation op = null;
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 6bb2ea9d1c..f879c670f0 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
@@ -75,8 +75,8 @@
   public static void main(String[] args) throws IOException {
     Opts opts = new Opts();
     opts.parseArgs(LogReader.class.getName(), args);
-    SiteConfiguration siteConfig = SiteConfiguration.create();
-    VolumeManager fs = VolumeManagerImpl.get();
+    SiteConfiguration siteConfig = new SiteConfiguration();
+    VolumeManager fs = VolumeManagerImpl.get(siteConfig);
 
     Matcher rowMatcher = null;
     KeyExtent ke = null;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
index 87c1230222..e2026a2487 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
@@ -49,7 +49,6 @@
 import org.apache.accumulo.server.util.MasterMetadataUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -225,7 +224,8 @@ public void importMapFiles(long tid, Map<FileRef,DataFileValue> pathsString, boo
 
       boolean inTheRightDirectory = false;
       Path parent = tpath.path().getParent().getParent();
-      for (String tablesDir : ServerConstants.getTablesDirs()) {
+      for (String tablesDir : ServerConstants
+          .getTablesDirs(tablet.getContext().getConfiguration())) {
         if (parent.equals(new Path(tablesDir, tablet.getExtent().getTableId().canonicalID()))) {
           inTheRightDirectory = true;
           break;
@@ -346,7 +346,7 @@ void unreserveMergingMinorCompactionFile(FileRef file) {
   void bringMinorCompactionOnline(FileRef tmpDatafile, FileRef newDatafile, FileRef absMergeFile,
       DataFileValue dfv, CommitSession commitSession, long flushId) throws IOException {
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    IZooReaderWriter zoo = tablet.getContext().getZooReaderWriter();
     if (tablet.getExtent().isRootTablet()) {
       try {
         if (!zoo.isLockHeld(tablet.getTabletServer().getLock().getLockID())) {
@@ -547,7 +547,7 @@ void bringMajorCompactionOnline(Set<FileRef> oldDatafiles, FileRef tmpDatafile,
 
       t1 = System.currentTimeMillis();
 
-      IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+      IZooReaderWriter zoo = tablet.getContext().getZooReaderWriter();
 
       tablet.incrementDataSourceDeletions();
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index a850a4452b..e783f3f10e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -116,7 +116,6 @@
 import org.apache.accumulo.server.util.MasterMetadataUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.apache.accumulo.tserver.ConditionCheckerContext.ConditionChecker;
 import org.apache.accumulo.tserver.InMemoryMap;
@@ -1114,7 +1113,7 @@ public long getFlushID() throws NoNodeException {
       String zTablePath = Constants.ZROOT + "/" + tabletServer.getInstanceID() + Constants.ZTABLES
           + "/" + extent.getTableId() + Constants.ZTABLE_FLUSH_ID;
       return Long
-          .parseLong(new String(ZooReaderWriter.getInstance().getData(zTablePath, null), UTF_8));
+          .parseLong(new String(context.getZooReaderWriter().getData(zTablePath, null), UTF_8));
     } catch (InterruptedException | NumberFormatException e) {
       throw new RuntimeException(e);
     } catch (KeeperException ke) {
@@ -1132,7 +1131,7 @@ long getCompactionCancelID() {
 
     try {
       return Long
-          .parseLong(new String(ZooReaderWriter.getInstance().getData(zTablePath, null), UTF_8));
+          .parseLong(new String(context.getZooReaderWriter().getData(zTablePath, null), UTF_8));
     } catch (KeeperException | InterruptedException e) {
       throw new RuntimeException(e);
     }
@@ -1143,7 +1142,7 @@ long getCompactionCancelID() {
       String zTablePath = Constants.ZROOT + "/" + tabletServer.getInstanceID() + Constants.ZTABLES
           + "/" + extent.getTableId() + Constants.ZTABLE_COMPACT_ID;
 
-      String[] tokens = new String(ZooReaderWriter.getInstance().getData(zTablePath, null), UTF_8)
+      String[] tokens = new String(context.getZooReaderWriter().getData(zTablePath, null), UTF_8)
           .split(",");
       long compactID = Long.parseLong(tokens[0]);
 
@@ -2848,8 +2847,8 @@ private static String createTabletDirectory(ServerContext context, VolumeManager
 
     UniqueNameAllocator namer = context.getUniqueNameAllocator();
     VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableId, context);
-    String volume = fs.choose(chooserEnv, ServerConstants.getBaseUris()) + Constants.HDFS_TABLES_DIR
-        + Path.SEPARATOR;
+    String volume = fs.choose(chooserEnv, ServerConstants.getBaseUris(context.getConfiguration()))
+        + Constants.HDFS_TABLES_DIR + Path.SEPARATOR;
 
     while (true) {
       try {
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java
index d9a8119d73..2ab007a097 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/InMemoryMapTest.java
@@ -112,6 +112,8 @@ public static void setUp() throws Exception {
   public static ServerContext getServerContext() {
     ServerContext context = EasyMock.createMock(ServerContext.class);
     EasyMock.expect(context.getCryptoService()).andReturn(new NoCryptoService()).anyTimes();
+    EasyMock.expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance())
+        .anyTimes();
     EasyMock.replay(context);
     return context;
   }
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 e401026509..9facfd983c 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 JsonElement serialize(byte[] link, Type type, JsonSerializationContext co
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState)
       throws ParseException, KeeperException, InterruptedException, IOException {
     ClientContext context = shellState.getContext();
-    SiteConfiguration siteConfig = SiteConfiguration.getInstance();
+    SiteConfiguration siteConfig = new SiteConfiguration();
     String[] args = cl.getArgs();
     if (args.length <= 0) {
       throw new ParseException("Must provide a command to execute");
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 b0c7b72f77..346b2dce82 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 static void main(String[] args) throws Exception {
     MasterClientService.Iface client = null;
     MasterMonitorInfo stats = null;
-    ServerContext context = new ServerContext(SiteConfiguration.create());
+    ServerContext context = new ServerContext(new SiteConfiguration());
     while (true) {
       try {
         client = MasterClient.getConnectionWithRetry(context);
diff --git a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
index d07e2a8b7c..7d4fe6d7a7 100644
--- a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
@@ -26,7 +26,6 @@
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -36,6 +35,7 @@
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.log.WalStateManager;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
@@ -78,7 +78,7 @@ public void test() throws Exception {
     c.tableOperations().create(bigTable);
     c.tableOperations().create(lilTable);
 
-    ClientContext context = getClientContext();
+    ServerContext context = getServerContext();
     zk = new ZooReaderWriter(c.info().getZooKeepers(), c.info().getZooKeepersSessionTimeOut(), "");
 
     // put some data in a log that should be replayed for both tables
@@ -86,11 +86,11 @@ public void test() throws Exception {
     scanSomeData(c, bigTable, 0, 10, 0, 10);
     writeSomeData(c, lilTable, 0, 1, 0, 1);
     scanSomeData(c, lilTable, 0, 1, 0, 1);
-    assertEquals(2, getWALCount(context, zk));
+    assertEquals(2, getWALCount(context));
 
     // roll the logs by pushing data into bigTable
     writeSomeData(c, bigTable, 0, 3000, 0, 1000);
-    assertEquals(3, getWALCount(context, zk));
+    assertEquals(3, getWALCount(context));
 
     // put some data in the latest log
     writeSomeData(c, lilTable, 1, 10, 0, 10);
@@ -130,8 +130,8 @@ private void scanSomeData(Connector c, String table, int startRow, int rowCount,
     }
   }
 
-  private int getWALCount(ClientContext context, ZooReaderWriter zk) throws Exception {
-    WalStateManager wals = new WalStateManager(context, zk);
+  private int getWALCount(ServerContext context) throws Exception {
+    WalStateManager wals = new WalStateManager(context);
     int result = 0;
     for (Entry<TServerInstance,List<UUID>> entry : wals.getAllMarkers().entrySet()) {
       result += entry.getValue().size();
diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
index 56fd3045e8..0a85dc3869 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java
@@ -70,7 +70,6 @@
 import org.apache.accumulo.server.log.WalStateManager.WalMarkerException;
 import org.apache.accumulo.server.log.WalStateManager.WalState;
 import org.apache.accumulo.server.util.Admin;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -449,9 +448,7 @@ private void verifyVolumesUsed(String tableName, boolean shouldExist, Path... pa
 
       // keep retrying until WAL state information in ZooKeeper stabilizes or until test times out
       retry: while (true) {
-        ZooReaderWriter zk = new ZooReaderWriter(conn.info().getZooKeepers(),
-            conn.info().getZooKeepersSessionTimeOut(), "");
-        WalStateManager wals = new WalStateManager(getClientContext(), zk);
+        WalStateManager wals = new WalStateManager(getServerContext());
         try {
           outer: for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
             for (Path path : paths) {
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 64123f056b..e6b094f228 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
@@ -19,6 +19,7 @@
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
@@ -30,7 +31,8 @@ public static void main(String[] args) throws Exception {
     String rootDir = args[0];
     File reportDir = new File(args[1]);
 
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    SiteConfiguration siteConfig = new SiteConfiguration();
+    IZooReaderWriter zoo = new ZooReaderWriter(siteConfig);
 
     if (zoo.exists(rootDir)) {
       zoo.recursiveDelete(rootDir, NodeMissingPolicy.FAIL);
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 32c2f9a317..6438275bb1 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
@@ -30,6 +30,7 @@
 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.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -40,7 +41,7 @@
   static final int NUM_DATA = 3;
 
   public static void main(String[] args) throws Exception {
-    IZooReaderWriter zk = ZooReaderWriter.getInstance();
+    IZooReaderWriter zk = new ZooReaderWriter(new SiteConfiguration());
 
     String rootDir = args[0];
     File reportDir = new File(args[1]);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index c64eb53921..cac98437ec 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -448,6 +448,8 @@ private void verifyLocalityGroupsInRFile(final Connector connector, final String
           System.setOut(newOut);
           List<String> args = new ArrayList<>();
           args.add(entry.getKey().getColumnQualifier().toString());
+          args.add("--site");
+          args.add(getCluster().getSitePath());
           if (ClusterType.STANDALONE == getClusterType() && saslEnabled()) {
             args.add("--config");
             StandaloneAccumuloCluster sac = (StandaloneAccumuloCluster) cluster;
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 79309ae251..b931887505 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
@@ -47,7 +47,6 @@
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.ColumnFQ;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockWatcher;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
@@ -80,9 +79,9 @@ private KeyExtent nke(String table, String endRow, String prevEndRow) {
 
   private void run(ServerContext c) throws Exception {
     String zPath = c.getZooKeeperRoot() + "/testLock";
-    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    ZooReaderWriter zoo = c.getZooReaderWriter();
     zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.OVERWRITE);
-    ZooLock zl = new ZooLock(zPath);
+    ZooLock zl = new ZooLock(zoo, zPath);
     boolean gotLock = zl.tryLock(new LockWatcher() {
 
       @Override
@@ -140,7 +139,8 @@ private void runSplitRecoveryTest(ServerContext context, int failPoint, String m
     for (int i = 0; i < extents.length; i++) {
       KeyExtent extent = extents[i];
 
-      String tdir = ServerConstants.getTablesDirs()[0] + "/" + extent.getTableId() + "/dir_" + i;
+      String tdir = ServerConstants.getTablesDirs(context.getConfiguration())[0] + "/"
+          + extent.getTableId() + "/dir_" + i;
       MetadataTableUtil.addTablet(extent, tdir, context, TabletTime.LOGICAL_TIME_ID, zl);
       SortedMap<FileRef,DataFileValue> mapFiles = new TreeMap<>();
       mapFiles.put(new FileRef(tdir + "/" + RFile.EXTENSION + "_000_000"),
@@ -286,7 +286,7 @@ private void verifySame(SortedMap<FileRef,DataFileValue> datafileSizes,
   }
 
   public static void main(String[] args) throws Exception {
-    new SplitRecoveryIT().run(new ServerContext(SiteConfiguration.create()));
+    new SplitRecoveryIT().run(new ServerContext(new SiteConfiguration()));
   }
 
   @Test
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index c6bd8415bd..89ce65ede0 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -42,7 +42,6 @@
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -55,9 +54,9 @@
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterControl;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.log.WalStateManager;
 import org.apache.accumulo.server.log.WalStateManager.WalState;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
@@ -96,7 +95,7 @@ public void test() throws Exception {
     MiniAccumuloClusterImpl mac = getCluster();
     MiniAccumuloClusterControl control = mac.getClusterControl();
     control.stop(GARBAGE_COLLECTOR);
-    ClientContext context = getClientContext();
+    ServerContext context = getServerContext();
     Connector c = context.getConnector();
     String tableName = getUniqueNames(1)[0];
     c.tableOperations().create(tableName);
@@ -223,7 +222,7 @@ private void writeSomeData(Connector conn, String tableName, int row, int col) t
   private final int TIMES_TO_COUNT = 20;
   private final int PAUSE_BETWEEN_COUNTS = 100;
 
-  private Map<String,Boolean> getWALsAndAssertCount(ClientContext c, int expectedCount)
+  private Map<String,Boolean> getWALsAndAssertCount(ServerContext c, int expectedCount)
       throws Exception {
     // see https://issues.apache.org/jira/browse/ACCUMULO-4110. Sometimes this test counts the logs
     // before
@@ -261,11 +260,9 @@ private int getWaitFactor() {
     return waitLonger;
   }
 
-  private Map<String,Boolean> _getWals(ClientContext c) throws Exception {
+  private Map<String,Boolean> _getWals(ServerContext c) throws Exception {
     Map<String,Boolean> result = new HashMap<>();
-    ZooReaderWriter zk = new ZooReaderWriter(c.getClientInfo().getZooKeepers(),
-        c.getClientInfo().getZooKeepersSessionTimeOut(), "");
-    WalStateManager wals = new WalStateManager(c, zk);
+    WalStateManager wals = new WalStateManager(c);
     for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
       // WALs are in use if they are not unreferenced
       result.put(entry.getKey().toString(), entry.getValue() != WalState.UNREFERENCED);
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 bd15e46a0f..ebb8f3190c 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
@@ -100,7 +100,7 @@ synchronized public void halt(TInfo tinfo, TCredentials credentials, String lock
   public static void main(String[] args) throws Exception {
     Random random = new Random(System.currentTimeMillis() % 1000);
     int port = random.nextInt(30000) + 2000;
-    ServerContext context = new ServerContext(SiteConfiguration.create());
+    ServerContext context = new ServerContext(new SiteConfiguration());
     TransactionWatcher watcher = new TransactionWatcher(context);
     final ThriftClientHandler tch = new ThriftClientHandler(context, watcher);
     Processor<Iface> processor = new Processor<>(tch);
@@ -110,10 +110,10 @@ public static void main(String[] args) throws Exception {
 
     String addressString = serverPort.address.toString();
     String zPath = context.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + addressString;
-    ZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    ZooReaderWriter zoo = context.getZooReaderWriter();
     zoo.putPersistentData(zPath, new byte[] {}, NodeExistsPolicy.SKIP);
 
-    ZooLock zlock = new ZooLock(zPath);
+    ZooLock zlock = new ZooLock(zoo, zPath);
 
     LockWatcher lw = new LockWatcher() {
       @Override
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 3cb65e9c12..11bb5cbc6b 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
@@ -310,7 +310,7 @@ public static void main(String[] args) throws Exception {
     // modify metadata
     int zkTimeOut = (int) DefaultConfiguration.getInstance()
         .getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
-    SiteConfiguration siteConfig = SiteConfiguration.create();
+    SiteConfiguration siteConfig = new SiteConfiguration();
     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/performance/scan/CollectTabletStats.java b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
index 2e0e96556b..d0474309b1 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
@@ -70,7 +70,6 @@
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
@@ -147,7 +146,7 @@ public static void main(String[] args) throws Exception {
       System.out.println("\t *** Information about tablet " + ke.getUUID() + " *** ");
       System.out.println("\t\t# files in tablet : " + tabletFiles.get(ke).size());
       System.out.println("\t\ttablet location   : " + tabletLocations.get(ke));
-      reportHdfsBlockLocations(tabletFiles.get(ke));
+      reportHdfsBlockLocations(context, tabletFiles.get(ke));
     }
 
     System.out.println("%n*** RUNNING TEST ***%n");
@@ -397,8 +396,9 @@ private static void runTest(String desc, List<Test> tests, int numThreads,
     return new ArrayList<>(MetadataTableUtil.getDataFileSizes(ke, context).keySet());
   }
 
-  private static void reportHdfsBlockLocations(List<FileRef> files) throws Exception {
-    VolumeManager fs = VolumeManagerImpl.get();
+  private static void reportHdfsBlockLocations(ServerContext context, List<FileRef> files)
+      throws Exception {
+    VolumeManager fs = context.getVolumeManager();
 
     System.out.println("\t\tFile block report : ");
     for (FileRef file : files) {
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index 443531249d..9ddcdab52b 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -46,10 +46,10 @@
 import org.apache.accumulo.core.trace.Tracer;
 import org.apache.accumulo.core.util.HostAndPort;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.log.WalStateManager;
 import org.apache.accumulo.server.log.WalStateManager.WalState;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -100,15 +100,13 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
    * Fetch all of the WALs referenced by tablets in the metadata table for this table
    */
   private Set<String> getWalsForTable(String tableName) throws Exception {
-    final ClientContext context = getClientContext();
+    final ServerContext context = getServerContext();
     final Connector conn = context.getConnector();
     final String tableId = conn.tableOperations().tableIdMap().get(tableName);
 
     Assert.assertNotNull("Could not determine table ID for " + tableName, tableId);
 
-    ZooReaderWriter zk = new ZooReaderWriter(conn.info().getZooKeepers(),
-        conn.info().getZooKeepersSessionTimeOut(), "");
-    WalStateManager wals = new WalStateManager(context, zk);
+    WalStateManager wals = new WalStateManager(context);
 
     Set<String> result = new HashSet<>();
     for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
index 7a909a987d..79051092f7 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@ -47,7 +47,6 @@
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.conf.ClientProperty;
 import org.apache.accumulo.core.conf.Property;
@@ -80,6 +79,7 @@
 import org.apache.accumulo.gc.SimpleGarbageCollector;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.log.WalStateManager;
 import org.apache.accumulo.server.log.WalStateManager.WalState;
 import org.apache.accumulo.server.master.state.TServerInstance;
@@ -89,7 +89,6 @@
 import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -141,7 +140,7 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite)
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 
-  private Multimap<String,Table.ID> getLogs(ClientContext context) throws Exception {
+  private Multimap<String,Table.ID> getLogs(ServerContext context) throws Exception {
     // Map of server to tableId
     Connector conn = context.getConnector();
     Multimap<TServerInstance,String> serverToTableID = HashMultimap.create();
@@ -156,9 +155,7 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite)
       }
       // Map of logs to tableId
       Multimap<String,Table.ID> logs = HashMultimap.create();
-      ZooReaderWriter zk = new ZooReaderWriter(conn.info().getZooKeepers(),
-          conn.info().getZooKeepersSessionTimeOut(), "");
-      WalStateManager wals = new WalStateManager(context, zk);
+      WalStateManager wals = new WalStateManager(context);
       for (Entry<TServerInstance,List<UUID>> entry : wals.getAllMarkers().entrySet()) {
         for (UUID id : entry.getValue()) {
           Pair<WalState,Path> state = wals.state(entry.getKey(), id);
@@ -171,7 +168,7 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite)
     }
   }
 
-  private Multimap<String,Table.ID> getAllLogs(ClientContext context) throws Exception {
+  private Multimap<String,Table.ID> getAllLogs(ServerContext context) throws Exception {
     Multimap<String,Table.ID> logs = getLogs(context);
     try (Scanner scanner = context.getConnector().createScanner(ReplicationTable.NAME,
         Authorizations.EMPTY)) {
@@ -342,10 +339,8 @@ public void correctRecordsCompleteFile() throws Exception {
 
     Set<String> wals = new HashSet<>();
     attempts = 5;
-    ZooReaderWriter zk = new ZooReaderWriter(conn.info().getZooKeepers(),
-        conn.info().getZooKeepersSessionTimeOut(), "");
     while (wals.isEmpty() && attempts > 0) {
-      WalStateManager markers = new WalStateManager(getClientContext(), zk);
+      WalStateManager markers = new WalStateManager(getServerContext());
       for (Entry<Path,WalState> entry : markers.getAllState().entrySet()) {
         wals.add(entry.getKey().toString());
       }
@@ -530,7 +525,7 @@ private void writeSomeData(Connector conn, String table, int rows, int cols) thr
 
   @Test
   public void replicationEntriesPrecludeWalDeletion() throws Exception {
-    final ClientContext context = getClientContext();
+    final ServerContext context = getServerContext();
     final Connector conn = getConnector();
     String table1 = "table1", table2 = "table2", table3 = "table3";
     final Multimap<String,Table.ID> logs = HashMultimap.create();
@@ -1102,7 +1097,7 @@ public void correctClusterNameInWorkEntry() throws Exception {
   public void replicationRecordsAreClosedAfterGarbageCollection() throws Exception {
     getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
 
-    final ClientContext context = getClientContext();
+    final ServerContext context = getServerContext();
     final Connector conn = getConnector();
 
     ReplicationTable.setOnline(conn);
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 05d7e30af4..cfe599623c 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
@@ -59,7 +59,7 @@ public void testSystemCredentials() throws Exception {
 
   public static void main(final String[] args)
       throws AccumuloException, TableNotFoundException, AccumuloSecurityException {
-    SiteConfiguration siteConfig = SiteConfiguration.create();
+    SiteConfiguration siteConfig = new SiteConfiguration();
     ServerContext context = new ServerContext(siteConfig);
     Credentials creds = null;
     String badInstanceID = SystemCredentials.class.getName();
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 483a5e23ca..bc86cc5634 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
@@ -124,9 +124,9 @@
 
     public SiteConfiguration getSiteConfiguration() {
       if (siteFile == null) {
-        return SiteConfiguration.create();
+        return new SiteConfiguration();
       } else {
-        return SiteConfiguration.create(new File(siteFile));
+        return new SiteConfiguration(new File(siteFile));
       }
     }
   }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services