You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2019/02/11 23:12:21 UTC

[accumulo] branch master updated: Avoid exposing internal types in new VolumeChooser SPIs (#941)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e78e4d9  Avoid exposing internal types in new VolumeChooser SPIs (#941)
e78e4d9 is described below

commit e78e4d9266b5d9fd1881ff3662ca8da08975a639
Author: Keith Turner <kt...@apache.org>
AuthorDate: Mon Feb 11 18:12:16 2019 -0500

    Avoid exposing internal types in new VolumeChooser SPIs (#941)
    
    For 2.0.0 new method were added to VolumeChooser that expose internal
    types like ServerContext.  These types are very unstable and will
    likely change alot in the future as we seek to remove static state
    in Accumulo.  This PR introduces a simple SPI interface that exposes
    some of the functionality of ServerContext in a more stable way.
---
 .../core/spi/common/ServiceEnvironment.java        | 151 +++++++++++++++++++++
 .../accumulo/core/spi/scan/ScanDispatcher.java     |   9 ++
 .../apache/accumulo/core/spi/scan/ScanInfo.java    |   3 +-
 .../accumulo/core/spi/scan/ScanPrioritizer.java    |   4 +
 .../spi/scan/IdleRatioScanPrioritizerTest.java     |  15 +-
 .../core/spi/scan/SimpleScanDispatcherTest.java    |  25 +++-
 .../accumulo/core/spi/scan/TestScanInfo.java       |   3 +-
 .../accumulo/server/ServiceEnvironmentImpl.java    | 141 +++++++++++++++++++
 .../accumulo/server/conf/TableConfiguration.java   |  21 ++-
 .../accumulo/server/fs/PerTableVolumeChooser.java  |  92 +++++--------
 .../accumulo/server/fs/PreferredVolumeChooser.java |  65 +++------
 .../server/fs/SpaceAwareVolumeChooser.java         |  15 +-
 .../server/fs/VolumeChooserEnvironment.java        |  76 +++--------
 ...ment.java => VolumeChooserEnvironmentImpl.java} |  56 +++++---
 .../org/apache/accumulo/server/fs/VolumeUtil.java  |   2 +-
 .../apache/accumulo/server/init/Initialize.java    |   5 +-
 .../accumulo/server/util/MetadataTableUtil.java    |   7 +-
 .../accumulo/server/util/RandomizeVolumes.java     |   3 +-
 .../server/fs/PerTableVolumeChooserTest.java       | 138 ++++++++++---------
 .../server/fs/PreferredVolumeChooserTest.java      | 128 +++++++++--------
 .../server/fs/SpaceAwareVolumeChooserTest.java     |  68 ++++------
 .../accumulo/server/fs/VolumeManagerImplTest.java  |   2 +-
 .../java/org/apache/accumulo/master/Master.java    |   3 +-
 .../apache/accumulo/master/TabletGroupWatcher.java |   3 +-
 .../accumulo/master/tableOps/create/ChooseDir.java |   5 +-
 .../tableImport/PopulateMetadataTable.java         |   5 +-
 .../tableOps/tableImport/ImportTableTest.java      |   3 +-
 .../accumulo/tserver/ConditionCheckerContext.java  |   2 +-
 .../tserver/TabletServerResourceManager.java       |  21 ++-
 .../tserver/compaction/MajorCompactionRequest.java |  14 +-
 .../org/apache/accumulo/tserver/log/DfsLogger.java |   3 +-
 .../accumulo/tserver/session/MultiScanSession.java |   5 +-
 .../tserver/session/SingleScanSession.java         |   5 +-
 .../accumulo/tserver/tablet/ScanDataSource.java    |   2 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |   4 +-
 .../org/apache/accumulo/test/ShellConfigIT.java    |  12 +-
 .../accumulo/test/VolumeChooserFailureIT.java      |  16 ++-
 .../org/apache/accumulo/test/VolumeChooserIT.java  |  35 +++--
 38 files changed, 746 insertions(+), 421 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/spi/common/ServiceEnvironment.java b/core/src/main/java/org/apache/accumulo/core/spi/common/ServiceEnvironment.java
new file mode 100644
index 0000000..7e9e71a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/common/ServiceEnvironment.java
@@ -0,0 +1,151 @@
+/*
+ * 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.spi.common;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.TableId;
+
+/**
+ * This interface exposes Accumulo system level information to plugins in a stable manner. The
+ * purpose of this interface is to insulate plugins from internal refactorings and changes to
+ * Accumulo.
+ *
+ * @since 2.0.0
+ */
+public interface ServiceEnvironment {
+
+  /**
+   * @since 2.0.0
+   */
+  public interface Configuration extends Iterable<Entry<String,String>> {
+
+    /**
+     * @return The value for a single property or null if not present. Sensitive properties are
+     *         intentionally not returned in order to prevent inadvertent logging of them. If your
+     *         plugin needs sensitive properties a getSensitive method could be added.
+     */
+    String get(String key);
+
+    /**
+     * Users can set arbitrary custom properties in Accumulo using the prefix
+     * {@code general.custom.}. This method will return all properties with that prefix, stripping
+     * the prefix. For example, assume the following properties were set :
+     *
+     * <pre>
+     *  {@code
+     *   general.custom.prop1=123
+     *   general.custom.prop2=abc
+     * }
+     * </pre>
+     *
+     * Then this function would return a map containing {@code [prop1=123,prop2=abc]}.
+     *
+     */
+    Map<String,String> getCustom();
+
+    /**
+     * This method appends the prefix {@code general.custom} and gets the property.
+     *
+     * @return The same as calling {@code getCustom().get(keySuffix)} OR
+     *         {@code get("general.custom."+keySuffix)}
+     */
+    String getCustom(String keySuffix);
+
+    /**
+     * Users can set arbitrary custom table properties in Accumulo using the prefix
+     * {@code table.custom.}. This method will return all properties with that prefix, stripping the
+     * prefix. For example, assume the following properties were set :
+     *
+     * <pre>
+     *  {@code
+     *   table.custom.tp1=ch1
+     *   table.custom.tp2=bh2
+     * }
+     * </pre>
+     *
+     * Then this function would return a map containing {@code [tp1=ch1,tp2=bh2]}.
+     *
+     */
+    Map<String,String> getTableCustom();
+
+    /**
+     * This method appends the prefix {@code table.custom} and gets the property.
+     *
+     * @return The same as calling {@code getTableCustom().get(keySuffix)} OR
+     *         {@code get("table.custom."+keySuffix)}
+     */
+    String getTableCustom(String keySuffix);
+
+    /**
+     * Returns an iterator over all properties. This may be inefficient, consider opening an issue
+     * if you have a use case that is only satisfied by this. Sensitive properties are intentionally
+     * suppressed in order to prevent inadvertent logging of them.
+     */
+    @Override
+    Iterator<Entry<String,String>> iterator();
+  }
+
+  /**
+   * @return A view of Accumulo's system level configuration. This is backed by system level config
+   *         in zookeeper, which falls back to site configuration, which falls back to the default
+   *         configuration.
+   */
+  Configuration getConfiguration();
+
+  /**
+   * @return a view of a table's configuration. When requesting properties that start with
+   *         {@code table.} the returned configuration may give different values for different
+   *         tables. For other properties the returned configuration will return the same value as
+   *         {@link #getConfiguration()}.
+   *
+   */
+  Configuration getConfiguration(TableId tableId);
+
+  /**
+   * Many Accumulo plugins are given table IDs as this is what Accumulo uses internally to identify
+   * tables. If a plugin needs to log debugging information it can call this method to get the table
+   * name.
+   */
+  String getTableName(TableId tableId) throws TableNotFoundException;
+
+  /**
+   * Instantiate a class using Accumulo's system classloader. The class must have a no argument
+   * constructor.
+   *
+   * @param className
+   *          Fully qualified name of the class.
+   * @param base
+   *          The expected super type of the class.
+   */
+  <T> T instantiate(String className, Class<T> base) throws Exception;
+
+  /**
+   * Instantiate a class using Accumulo's per table classloader. The class must have a no argument
+   * constructor.
+   *
+   * @param className
+   *          Fully qualified name of the class.
+   * @param base
+   *          The expected super type of the class.
+   */
+  <T> T instantiate(TableId tableId, String className, Class<T> base) throws Exception;
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java
index 9fc9d91..c9538ab 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java
@@ -18,6 +18,9 @@ package org.apache.accumulo.core.spi.scan;
 
 import java.util.Map;
 
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+
 import com.google.common.base.Preconditions;
 
 /**
@@ -44,6 +47,10 @@ public interface ScanDispatcher {
      *         {@code p1=abc} and {@code p9=123}.
      */
     Map<String,String> getOptions();
+
+    TableId getTableId();
+
+    ServiceEnvironment getServiceEnv();
   }
 
   /**
@@ -70,6 +77,8 @@ public interface ScanDispatcher {
      * @return the currently configured scan executors
      */
     Map<String,ScanExecutor> getScanExecutors();
+
+    ServiceEnvironment getServiceEnv();
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanInfo.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanInfo.java
index 3b2aa63..bc9e612 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanInfo.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase;
 import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.spi.common.IteratorConfiguration;
 import org.apache.accumulo.core.spi.common.Stats;
 
@@ -52,7 +53,7 @@ public interface ScanInfo {
 
   Type getScanType();
 
-  String getTableId();
+  TableId getTableId();
 
   /**
    * Returns the first time a tablet knew about a scan over its portion of data. This is the time a
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanPrioritizer.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanPrioritizer.java
index 07fdf91..4d40d38 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanPrioritizer.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanPrioritizer.java
@@ -19,6 +19,8 @@ package org.apache.accumulo.core.spi.scan;
 import java.util.Comparator;
 import java.util.Map;
 
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+
 /**
  * A factory for creating comparators used for prioritizing scans. For information about
  * configuring, find the documentation for the {@code tserver.scan.executors.} property.
@@ -41,6 +43,8 @@ public interface ScanPrioritizer {
      *         {@code <key>=<value>} portions of those properties ends up in the returned map.
      */
     Map<String,String> getOptions();
+
+    ServiceEnvironment getServiceEnv();
   }
 
   Comparator<ScanInfo> createComparator(CreateParameters params);
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/scan/IdleRatioScanPrioritizerTest.java b/core/src/test/java/org/apache/accumulo/core/spi/scan/IdleRatioScanPrioritizerTest.java
index b1873ed..fa53f6e 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/scan/IdleRatioScanPrioritizerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/scan/IdleRatioScanPrioritizerTest.java
@@ -23,7 +23,9 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
+import java.util.Map;
 
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.scan.ScanInfo.Type;
 import org.junit.Test;
 
@@ -48,7 +50,18 @@ public class IdleRatioScanPrioritizerTest {
     Collections.shuffle(scans);
 
     Comparator<ScanInfo> comparator = new IdleRatioScanPrioritizer()
-        .createComparator(Collections::emptyMap);
+        .createComparator(new ScanPrioritizer.CreateParameters() {
+
+          @Override
+          public Map<String,String> getOptions() {
+            return Collections.emptyMap();
+          }
+
+          @Override
+          public ServiceEnvironment getServiceEnv() {
+            throw new UnsupportedOperationException();
+          }
+        });
 
     Collections.sort(scans, comparator);
 
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcherTest.java b/core/src/test/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcherTest.java
index 5f53cea..a35e527 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcherTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/scan/SimpleScanDispatcherTest.java
@@ -24,6 +24,8 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.scan.ScanDispatcher.DispatchParmaters;
 import org.apache.accumulo.core.spi.scan.ScanInfo.Type;
 import org.junit.Test;
@@ -59,6 +61,11 @@ public class SimpleScanDispatcherTest {
       return se;
     }
 
+    @Override
+    public ServiceEnvironment getServiceEnv() {
+      throw new UnsupportedOperationException();
+    }
+
   }
 
   private void runTest(Map<String,String> opts, Map<String,String> hints, String expectedSingle,
@@ -70,7 +77,23 @@ public class SimpleScanDispatcherTest {
 
     SimpleScanDispatcher ssd1 = new SimpleScanDispatcher();
 
-    ssd1.init(() -> opts);
+    ssd1.init(new ScanDispatcher.InitParameters() {
+
+      @Override
+      public TableId getTableId() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public Map<String,String> getOptions() {
+        return opts;
+      }
+
+      @Override
+      public ServiceEnvironment getServiceEnv() {
+        throw new UnsupportedOperationException();
+      }
+    });
 
     Map<String,ScanExecutor> executors = new HashMap<>();
     executors.put("E1", null);
diff --git a/core/src/test/java/org/apache/accumulo/core/spi/scan/TestScanInfo.java b/core/src/test/java/org/apache/accumulo/core/spi/scan/TestScanInfo.java
index ee3db08..a9c72f8 100644
--- a/core/src/test/java/org/apache/accumulo/core/spi/scan/TestScanInfo.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/scan/TestScanInfo.java
@@ -24,6 +24,7 @@ import java.util.OptionalLong;
 import java.util.Set;
 
 import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.spi.common.IteratorConfiguration;
 import org.apache.accumulo.core.spi.common.Stats;
 import org.apache.accumulo.core.util.Stat;
@@ -61,7 +62,7 @@ public class TestScanInfo implements ScanInfo {
   }
 
   @Override
-  public String getTableId() {
+  public TableId getTableId() {
     throw new UnsupportedOperationException();
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
new file mode 100644
index 0000000..10e2255
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServiceEnvironmentImpl.java
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.StreamSupport;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.clientImpl.Tables;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMap.Builder;
+
+public class ServiceEnvironmentImpl implements ServiceEnvironment {
+
+  private final ServerContext srvCtx;
+  private final Configuration conf;
+
+  public ServiceEnvironmentImpl(ServerContext ctx) {
+    this.srvCtx = ctx;
+    this.conf = new ConfigurationImpl(srvCtx.getConfiguration());
+  }
+
+  private static class ConfigurationImpl implements Configuration {
+
+    private final AccumuloConfiguration acfg;
+    private Map<String,String> customProps;
+    private Map<String,String> tableCustomProps;
+
+    ConfigurationImpl(AccumuloConfiguration acfg) {
+      this.acfg = acfg;
+    }
+
+    @Override
+    public String get(String key) {
+      // Get prop to check if sensitive, also looking up by prop may be more efficient.
+      Property prop = Property.getPropertyByKey(key);
+      if (prop != null) {
+        if (prop.isSensitive())
+          return null;
+        return acfg.get(prop);
+      } else {
+        return acfg.get(key);
+      }
+    }
+
+    @Override
+    public Map<String,String> getCustom() {
+      if (customProps == null)
+        customProps = buildCustom(Property.GENERAL_ARBITRARY_PROP_PREFIX);
+
+      return customProps;
+    }
+
+    @Override
+    public String getCustom(String keySuffix) {
+      return getCustom().get(keySuffix);
+    }
+
+    @Override
+    public Map<String,String> getTableCustom() {
+      if (tableCustomProps == null)
+        tableCustomProps = buildCustom(Property.TABLE_ARBITRARY_PROP_PREFIX);
+
+      return tableCustomProps;
+    }
+
+    @Override
+    public String getTableCustom(String keySuffix) {
+      return getTableCustom().get(keySuffix);
+    }
+
+    private Map<String,String> buildCustom(Property customPrefix) {
+      // This could be optimized as described in #947
+      Map<String,String> props = acfg.getAllPropertiesWithPrefix(customPrefix);
+      Builder<String,String> builder = ImmutableMap.builder();
+      props.forEach((k, v) -> {
+        builder.put(k.substring(customPrefix.getKey().length()), v);
+      });
+
+      return builder.build();
+    }
+
+    @Override
+    public Iterator<Entry<String,String>> iterator() {
+      return StreamSupport.stream(acfg.spliterator(), false)
+          .filter(e -> !Property.isSensitive(e.getKey())).iterator();
+    }
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  @Override
+  public Configuration getConfiguration(TableId tableId) {
+    return new ConfigurationImpl(srvCtx.getServerConfFactory().getTableConfiguration(tableId));
+  }
+
+  @Override
+  public String getTableName(TableId tableId) throws TableNotFoundException {
+    return Tables.getTableName(srvCtx, tableId);
+  }
+
+  @Override
+  public <T> T instantiate(String className, Class<T> base)
+      throws ClassNotFoundException, InstantiationException, IllegalAccessException, IOException {
+    return ConfigurationTypeHelper.getClassInstance(null, className, base);
+  }
+
+  @Override
+  public <T> T instantiate(TableId tableId, String className, Class<T> base)
+      throws ClassNotFoundException, InstantiationException, IllegalAccessException, IOException {
+    String ctx = srvCtx.getServerConfFactory().getTableConfiguration(tableId)
+        .get(Property.TABLE_CLASSPATH);
+    return ConfigurationTypeHelper.getClassInstance(ctx, className, base);
+  }
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
index 749b04b..5165c5f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
@@ -35,10 +35,12 @@ import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.scan.ScanDispatcher;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServiceEnvironmentImpl;
 import org.apache.accumulo.server.conf.ZooCachePropertyAccessor.PropCacheKey;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -193,7 +195,7 @@ public class TableConfiguration extends ObservableConfiguration {
       return tableOpts;
     }
 
-    public String getContext() {
+    public String getServiceEnv() {
       return context;
     }
   }
@@ -242,7 +244,22 @@ public class TableConfiguration extends ObservableConfiguration {
 
       Map<String,String> opts = builder.build();
 
-      newDispatcher.init(() -> opts);
+      newDispatcher.init(new ScanDispatcher.InitParameters() {
+        @Override
+        public TableId getTableId() {
+          return tableId;
+        }
+
+        @Override
+        public Map<String,String> getOptions() {
+          return opts;
+        }
+
+        @Override
+        public ServiceEnvironment getServiceEnv() {
+          return new ServiceEnvironmentImpl(context);
+        }
+      });
 
       TablesScanDispatcher newRef = new TablesScanDispatcher(newDispatcher, count);
       scanDispatcherRef.compareAndSet(currRef, newRef);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
index 82268f0..13d4d4d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
@@ -16,24 +16,18 @@
  */
 package org.apache.accumulo.server.fs;
 
-import java.io.IOException;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * A {@link VolumeChooser} that delegates to another volume chooser based on other properties:
- * table.custom.volume.chooser for tables, and general.custom.scoped.volume.chooser for scopes.
- * general.custor.{scope}.volume.chooser can override the system wide setting for
- * general.custom.scoped.volume.chooser. At the this this was written, the only known scope was
+ * table.custom.volume.chooser for tables, and general.custom.volume.chooser.scoped for scopes.
+ * general.custom.volume.chooser.{scope} can override the system wide setting for
+ * general.custom.volume.chooser.scoped. At the this this was written, the only known scope was
  * "logger".
  */
 public class PerTableVolumeChooser implements VolumeChooser {
@@ -45,19 +39,13 @@ public class PerTableVolumeChooser implements VolumeChooser {
   private final ConcurrentHashMap<ChooserScope,VolumeChooser> scopeSpecificChooserCache = new ConcurrentHashMap<>();
   private final RandomVolumeChooser randomChooser = new RandomVolumeChooser();
 
-  // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see
-  // ACCUMULO-3411
-  private volatile ServerConfigurationFactory lazyConfFactory = null;
+  private static final String TABLE_CUSTOM_SUFFIX = "volume.chooser";
 
-  public static final String TABLE_VOLUME_CHOOSER = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey()
-      + "volume.chooser";
-
-  public static final String getPropertyNameForScope(ChooserScope scope) {
-    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope.name().toLowerCase()
-        + ".volume.chooser";
+  private static final String getCustomPropertySuffix(ChooserScope scope) {
+    return "volume.chooser." + scope.name().toLowerCase();
   }
 
-  private static final String DEFAULT_SCOPED_VOLUME_CHOOSER = getPropertyNameForScope(
+  private static final String DEFAULT_SCOPED_VOLUME_CHOOSER = getCustomPropertySuffix(
       ChooserScope.DEFAULT);
 
   @Override
@@ -76,55 +64,47 @@ public class PerTableVolumeChooser implements VolumeChooser {
             ChooserScope.INIT, RandomVolumeChooser.class.getName());
         return randomChooser;
       case TABLE:
-        return getVolumeChooserForTable(env, loadConfFactory(env));
+        return getVolumeChooserForTable(env);
       default:
-        return getVolumeChooserForScope(env, loadConfFactory(env));
+        return getVolumeChooserForScope(env);
     }
   }
 
-  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env,
-      ServerConfigurationFactory confFactory) {
-    log.trace("Looking up property {} for table id: {}", TABLE_VOLUME_CHOOSER, env.getTableId());
-    final TableConfiguration tableConf = confFactory.getTableConfiguration(env.getTableId());
-    String clazz = tableConf.get(TABLE_VOLUME_CHOOSER);
+  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env) {
+    log.trace("Looking up property {} for table id: {}", TABLE_CUSTOM_SUFFIX, env.getTableId());
+
+    String clazz = env.getServiceEnv().getConfiguration(env.getTableId())
+        .getTableCustom(TABLE_CUSTOM_SUFFIX);
 
     // fall back to global default scope, so setting only one default is necessary, rather than a
     // separate default for TABLE scope than other scopes
     if (clazz == null || clazz.isEmpty()) {
-      clazz = confFactory.getSystemConfiguration().get(DEFAULT_SCOPED_VOLUME_CHOOSER);
+      clazz = env.getServiceEnv().getConfiguration().getCustom(DEFAULT_SCOPED_VOLUME_CHOOSER);
     }
 
     if (clazz == null || clazz.isEmpty()) {
-      String msg = "Property " + TABLE_VOLUME_CHOOSER + " or " + DEFAULT_SCOPED_VOLUME_CHOOSER
+      String msg = "Property " + TABLE_CUSTOM_SUFFIX + " or " + DEFAULT_SCOPED_VOLUME_CHOOSER
           + " must be a valid " + VolumeChooser.class.getSimpleName() + " to use the "
           + getClass().getSimpleName();
       throw new VolumeChooserException(msg);
     }
 
-    String context = getTableContext(tableConf); // can be null
-    return createVolumeChooser(context, clazz, TABLE_VOLUME_CHOOSER, env.getTableId(),
+    return createVolumeChooser(env, clazz, TABLE_CUSTOM_SUFFIX, env.getTableId(),
         tableSpecificChooserCache);
   }
 
-  // visible (not private) for testing
-  String getTableContext(TableConfiguration tableConf) {
-    return tableConf.get(Property.TABLE_CLASSPATH);
-  }
-
-  private VolumeChooser getVolumeChooserForScope(VolumeChooserEnvironment env,
-      ServerConfigurationFactory confFactory) {
+  private VolumeChooser getVolumeChooserForScope(VolumeChooserEnvironment env) {
     ChooserScope scope = env.getScope();
-    String property = getPropertyNameForScope(scope);
+    String property = getCustomPropertySuffix(scope);
     log.trace("Looking up property {} for scope: {}", property, scope);
 
-    AccumuloConfiguration systemConfiguration = confFactory.getSystemConfiguration();
-    String clazz = systemConfiguration.get(property);
+    String clazz = env.getServiceEnv().getConfiguration().getCustom(property);
 
     // fall back to global default scope if this scope isn't configured (and not already default
     // scope)
     if ((clazz == null || clazz.isEmpty()) && scope != ChooserScope.DEFAULT) {
       log.debug("{} not found; using {}", property, DEFAULT_SCOPED_VOLUME_CHOOSER);
-      clazz = systemConfiguration.get(DEFAULT_SCOPED_VOLUME_CHOOSER);
+      clazz = env.getServiceEnv().getConfiguration().getCustom(DEFAULT_SCOPED_VOLUME_CHOOSER);
 
       if (clazz == null || clazz.isEmpty()) {
         String msg = "Property " + property + " or " + DEFAULT_SCOPED_VOLUME_CHOOSER
@@ -136,8 +116,7 @@ public class PerTableVolumeChooser implements VolumeChooser {
       property = DEFAULT_SCOPED_VOLUME_CHOOSER;
     }
 
-    String context = null;
-    return createVolumeChooser(context, clazz, property, scope, scopeSpecificChooserCache);
+    return createVolumeChooser(env, clazz, property, scope, scopeSpecificChooserCache);
   }
 
   /**
@@ -154,8 +133,8 @@ public class PerTableVolumeChooser implements VolumeChooser {
    *          The cache
    * @return The volume chooser instance
    */
-  private <T> VolumeChooser createVolumeChooser(String context, String clazz, String property,
-      T key, ConcurrentHashMap<T,VolumeChooser> cache) {
+  private <T> VolumeChooser createVolumeChooser(VolumeChooserEnvironment env, String clazz,
+      String property, T key, ConcurrentHashMap<T,VolumeChooser> cache) {
     final String className = clazz.trim();
     // create a new instance, unless another thread beat us with one of the same class name, then
     // use theirs
@@ -169,26 +148,17 @@ public class PerTableVolumeChooser implements VolumeChooser {
         log.trace("Change detected for {} for {}", property, key);
       }
       try {
-        return ConfigurationTypeHelper.getClassInstance(context, className, VolumeChooser.class);
-      } catch (ClassNotFoundException | InstantiationException | IllegalAccessException
-          | IOException e) {
+        if (key instanceof TableId) {
+          TableId tableId = (TableId) key;
+          return env.getServiceEnv().instantiate(tableId, className, VolumeChooser.class);
+        } else {
+          return env.getServiceEnv().instantiate(className, VolumeChooser.class);
+        }
+      } catch (Exception e) {
         String msg = "Failed to create instance for " + key + " configured to use " + className
             + " via " + property;
         throw new VolumeChooserException(msg, e);
       }
     });
   }
-
-  // visible (not private) for testing
-  ServerConfigurationFactory loadConfFactory(VolumeChooserEnvironment env) {
-    // This local variable is an intentional component of the single-check idiom.
-    ServerConfigurationFactory localConf = lazyConfFactory;
-    if (localConf == null) {
-      // If we're under contention when first getting here we'll throw away some initializations.
-      localConf = env.getServerContext().getServerConfFactory();
-      lazyConfFactory = localConf;
-    }
-    return localConf;
-  }
-
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java
index 452acf6..afd07ba 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java
@@ -20,11 +20,7 @@ import java.util.Arrays;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.volume.Volume;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
@@ -33,28 +29,22 @@ import org.slf4j.LoggerFactory;
 /**
  * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset
  * of those options preferred for a particular table. Defaults to selecting from all of the options
- * presented. Can be customized via the table property table.custom.preferredVolumes, which should
+ * presented. Can be customized via the table property table.custom.volume.preferred, which should
  * contain a comma separated list of {@link Volume} URIs. Note that both the property name and the
  * format of its value are specific to this particular implementation.
  */
 public class PreferredVolumeChooser extends RandomVolumeChooser {
   private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
 
-  public static final String TABLE_PREFERRED_VOLUMES = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey()
-      + "preferred.volumes";
+  private static final String TABLE_CUSTOM_SUFFIX = "volume.preferred";
 
-  public static final String getPropertyNameForScope(ChooserScope scope) {
-    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope.name().toLowerCase()
-        + ".preferred.volumes";
+  private static final String getCustomPropertySuffix(ChooserScope scope) {
+    return "volume.preferred." + scope.name().toLowerCase();
   }
 
-  private static final String DEFAULT_SCOPED_PREFERRED_VOLUMES = getPropertyNameForScope(
+  private static final String DEFAULT_SCOPED_PREFERRED_VOLUMES = getCustomPropertySuffix(
       ChooserScope.DEFAULT);
 
-  // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see
-  // ACCUMULO-3411
-  private volatile ServerConfigurationFactory lazyConfFactory = null;
-
   @Override
   public String choose(VolumeChooserEnvironment env, String[] options)
       throws VolumeChooserException {
@@ -74,51 +64,49 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
             ChooserScope.INIT);
         return options;
       case TABLE:
-        return getPreferredVolumesForTable(env, loadConfFactory(env), options);
+        return getPreferredVolumesForTable(env, options);
       default:
-        return getPreferredVolumesForScope(env, loadConfFactory(env), options);
+        return getPreferredVolumesForScope(env, options);
     }
   }
 
-  private String[] getPreferredVolumesForTable(VolumeChooserEnvironment env,
-      ServerConfigurationFactory confFactory, String[] options) {
-    log.trace("Looking up property {} + for Table id: {}", TABLE_PREFERRED_VOLUMES,
-        env.getTableId());
+  private String[] getPreferredVolumesForTable(VolumeChooserEnvironment env, String[] options) {
+    log.trace("Looking up property {} + for Table id: {}", TABLE_CUSTOM_SUFFIX, env.getTableId());
 
-    final TableConfiguration tableConf = confFactory.getTableConfiguration(env.getTableId());
-    String preferredVolumes = tableConf.get(TABLE_PREFERRED_VOLUMES);
+    String preferredVolumes = env.getServiceEnv().getConfiguration(env.getTableId())
+        .getTableCustom(TABLE_CUSTOM_SUFFIX);
 
     // fall back to global default scope, so setting only one default is necessary, rather than a
     // separate default for TABLE scope than other scopes
     if (preferredVolumes == null || preferredVolumes.isEmpty()) {
-      preferredVolumes = confFactory.getSystemConfiguration().get(DEFAULT_SCOPED_PREFERRED_VOLUMES);
+      preferredVolumes = env.getServiceEnv().getConfiguration()
+          .getCustom(DEFAULT_SCOPED_PREFERRED_VOLUMES);
     }
 
     // throw an error if volumes not specified or empty
     if (preferredVolumes == null || preferredVolumes.isEmpty()) {
-      String msg = "Property " + TABLE_PREFERRED_VOLUMES + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
+      String msg = "Property " + TABLE_CUSTOM_SUFFIX + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
           + " must be a subset of " + Arrays.toString(options) + " to use the "
           + getClass().getSimpleName();
       throw new VolumeChooserException(msg);
     }
 
-    return parsePreferred(TABLE_PREFERRED_VOLUMES, preferredVolumes, options);
+    return parsePreferred(TABLE_CUSTOM_SUFFIX, preferredVolumes, options);
   }
 
-  private String[] getPreferredVolumesForScope(VolumeChooserEnvironment env,
-      ServerConfigurationFactory confFactory, String[] options) {
+  private String[] getPreferredVolumesForScope(VolumeChooserEnvironment env, String[] options) {
     ChooserScope scope = env.getScope();
-    String property = getPropertyNameForScope(scope);
+    String property = getCustomPropertySuffix(scope);
     log.trace("Looking up property {} for scope: {}", property, scope);
 
-    AccumuloConfiguration systemConfiguration = confFactory.getSystemConfiguration();
-    String preferredVolumes = systemConfiguration.get(property);
+    String preferredVolumes = env.getServiceEnv().getConfiguration().getCustom(property);
 
     // fall back to global default scope if this scope isn't configured (and not already default
     // scope)
     if ((preferredVolumes == null || preferredVolumes.isEmpty()) && scope != ChooserScope.DEFAULT) {
       log.debug("{} not found; using {}", property, DEFAULT_SCOPED_PREFERRED_VOLUMES);
-      preferredVolumes = systemConfiguration.get(DEFAULT_SCOPED_PREFERRED_VOLUMES);
+      preferredVolumes = env.getServiceEnv().getConfiguration()
+          .getCustom(DEFAULT_SCOPED_PREFERRED_VOLUMES);
 
       // only if the custom property is not set to we fall back to the default scoped preferred
       // volumes
@@ -155,17 +143,4 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
 
     return preferred.toArray(new String[preferred.size()]);
   }
-
-  // visible (not private) for testing
-  ServerConfigurationFactory loadConfFactory(VolumeChooserEnvironment env) {
-    // Get the current table's properties, and find the preferred volumes property
-    // This local variable is an intentional component of the single-check idiom.
-    ServerConfigurationFactory localConf = lazyConfFactory;
-    if (localConf == null) {
-      // If we're under contention when first getting here we'll throw away some initializations.
-      localConf = env.getServerContext().getServerConfFactory();
-      lazyConfFactory = localConf;
-    }
-    return localConf;
-  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
index d98c8fb..b86cd4e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
@@ -25,13 +25,9 @@ import java.util.TreeMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsStatus;
-import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,8 +42,7 @@ import com.google.common.cache.LoadingCache;
  */
 public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
 
-  public static final String HDFS_SPACE_RECOMPUTE_INTERVAL = Property.GENERAL_ARBITRARY_PROP_PREFIX
-      .getKey() + "spaceaware.volume.chooser.recompute.interval";
+  public static final String RECOMPUTE_INTERVAL = "spaceaware.volume.chooser.recompute.interval";
 
   // Default time to wait in ms. Defaults to 5 min
   private long defaultComputationCacheDuration = 300000;
@@ -72,9 +67,7 @@ public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
       VolumeChooserEnvironment env) {
 
     if (choiceCache == null) {
-      ServerConfigurationFactory scf = loadConfFactory(env);
-      AccumuloConfiguration systemConfiguration = scf.getSystemConfiguration();
-      String propertyValue = systemConfiguration.get(HDFS_SPACE_RECOMPUTE_INTERVAL);
+      String propertyValue = env.getServiceEnv().getConfiguration().getCustom(RECOMPUTE_INTERVAL);
 
       long computationCacheDuration = StringUtils.isNotBlank(propertyValue)
           ? Long.parseLong(propertyValue)
@@ -106,11 +99,9 @@ public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
         throw new IllegalStateException("Options was empty! No valid volumes to choose from.");
       }
 
-      VolumeManager manager = env.getServerContext().getVolumeManager();
-
       // Compute percentage space available on each volume
       for (String option : options) {
-        FileSystem pathFs = manager.getVolumeByPath(new Path(option)).getFileSystem();
+        FileSystem pathFs = env.getFileSystem(option);
         try {
           FsStatus optionStatus = pathFs.getStatus();
           double percentFree = ((double) optionStatus.getRemaining() / optionStatus.getCapacity());
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
index 37f5571..8e478fd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
@@ -14,15 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.accumulo.server.fs;
 
-import java.util.Objects;
+package org.apache.accumulo.server.fs;
 
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
 
-public class VolumeChooserEnvironment {
+public interface VolumeChooserEnvironment {
 
   /**
    * A scope the volume chooser environment; a TABLE scope should be accompanied by a tableId.
@@ -33,68 +33,30 @@ public class VolumeChooserEnvironment {
     DEFAULT, TABLE, INIT, LOGGER
   }
 
-  private final ServerContext context;
-  private final ChooserScope scope;
-  private final TableId tableId;
-  private final Text endRow;
-
-  public VolumeChooserEnvironment(ChooserScope scope, ServerContext context) {
-    this.context = context;
-    this.scope = Objects.requireNonNull(scope);
-    this.tableId = null;
-    this.endRow = null;
-  }
-
-  public VolumeChooserEnvironment(TableId tableId, Text endRow, ServerContext context) {
-    this.context = context;
-    this.scope = ChooserScope.TABLE;
-    this.tableId = Objects.requireNonNull(tableId);
-    this.endRow = endRow;
-
-  }
-
   /**
    * The end row of the tablet for which a volume is being chosen. Only call this when the scope is
    * TABLE
    *
    * @since 2.0.0
    */
-  public Text getEndRow() {
-    if (scope != ChooserScope.TABLE)
-      throw new IllegalStateException("Can only request end row for tables, not for " + scope);
-    return endRow;
-  }
+  public Text getEndRow();
 
-  public TableId getTableId() {
-    return tableId;
-  }
+  public boolean hasTableId();
 
-  public ChooserScope getScope() {
-    return this.scope;
-  }
+  public TableId getTableId();
 
-  public ServerContext getServerContext() {
-    if (context == null) {
-      throw new IllegalStateException("Requested ServerContext from " + getClass().getSimpleName()
-          + " that was created without it");
-    }
-    return context;
-  }
+  /**
+   * @since 2.0.0
+   */
+  public ChooserScope getScope();
 
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == this) {
-      return true;
-    }
-    if (obj == null || !(obj instanceof VolumeChooserEnvironment)) {
-      return false;
-    }
-    VolumeChooserEnvironment other = (VolumeChooserEnvironment) obj;
-    return getScope() == other.getScope() && Objects.equals(getTableId(), other.getTableId());
-  }
+  /**
+   * @since 2.0.0
+   */
+  public ServiceEnvironment getServiceEnv();
 
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(scope) * 31 + Objects.hashCode(tableId);
-  }
+  /**
+   * @since 2.0.0
+   */
+  public FileSystem getFileSystem(String option);
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java
similarity index 62%
copy from server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
copy to server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java
index 37f5571..88c3d83 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java
@@ -19,33 +19,34 @@ package org.apache.accumulo.server.fs;
 import java.util.Objects;
 
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServiceEnvironmentImpl;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
-public class VolumeChooserEnvironment {
-
-  /**
-   * A scope the volume chooser environment; a TABLE scope should be accompanied by a tableId.
-   *
-   * @since 2.0.0
-   */
-  public static enum ChooserScope {
-    DEFAULT, TABLE, INIT, LOGGER
-  }
+/**
+ * Volume chooser authors should avoid using this class when testing their volume chooser. The
+ * constructors for this class may change at any time. For testing purposes mocking the interface
+ * {@link VolumeChooserEnvironment} should result in more stable code over time than using this
+ * class.
+ */
+public class VolumeChooserEnvironmentImpl implements VolumeChooserEnvironment {
 
   private final ServerContext context;
   private final ChooserScope scope;
   private final TableId tableId;
   private final Text endRow;
 
-  public VolumeChooserEnvironment(ChooserScope scope, ServerContext context) {
+  public VolumeChooserEnvironmentImpl(ChooserScope scope, ServerContext context) {
     this.context = context;
     this.scope = Objects.requireNonNull(scope);
     this.tableId = null;
     this.endRow = null;
   }
 
-  public VolumeChooserEnvironment(TableId tableId, Text endRow, ServerContext context) {
+  public VolumeChooserEnvironmentImpl(TableId tableId, Text endRow, ServerContext context) {
     this.context = context;
     this.scope = ChooserScope.TABLE;
     this.tableId = Objects.requireNonNull(tableId);
@@ -59,26 +60,41 @@ public class VolumeChooserEnvironment {
    *
    * @since 2.0.0
    */
+  @Override
   public Text getEndRow() {
     if (scope != ChooserScope.TABLE)
       throw new IllegalStateException("Can only request end row for tables, not for " + scope);
     return endRow;
   }
 
+  @Override
+  public boolean hasTableId() {
+    return scope == ChooserScope.TABLE;
+  }
+
+  @Override
   public TableId getTableId() {
+    if (scope != ChooserScope.TABLE)
+      throw new IllegalStateException("Can only request table id for tables, not for " + scope);
     return tableId;
   }
 
+  /**
+   * @since 2.0.0
+   */
+  @Override
   public ChooserScope getScope() {
     return this.scope;
   }
 
-  public ServerContext getServerContext() {
-    if (context == null) {
-      throw new IllegalStateException("Requested ServerContext from " + getClass().getSimpleName()
-          + " that was created without it");
-    }
-    return context;
+  @Override
+  public ServiceEnvironment getServiceEnv() {
+    return new ServiceEnvironmentImpl(context);
+  }
+
+  @Override
+  public FileSystem getFileSystem(String option) {
+    return context.getVolumeManager().getVolumeByPath(new Path(option)).getFileSystem();
   }
 
   @Override
@@ -86,10 +102,10 @@ public class VolumeChooserEnvironment {
     if (obj == this) {
       return true;
     }
-    if (obj == null || !(obj instanceof VolumeChooserEnvironment)) {
+    if (obj == null || !(obj instanceof VolumeChooserEnvironmentImpl)) {
       return false;
     }
-    VolumeChooserEnvironment other = (VolumeChooserEnvironment) obj;
+    VolumeChooserEnvironmentImpl other = (VolumeChooserEnvironmentImpl) obj;
     return getScope() == other.getScope() && Objects.equals(getTableId(), other.getTableId());
   }
 
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 0653134..e5d5f6e 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
@@ -272,7 +272,7 @@ public class VolumeUtil {
       throw new IllegalArgumentException("Unexpected table dir " + dir);
     }
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(extent.getTableId(),
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(extent.getTableId(),
         extent.getEndRow(), context);
 
     Path newDir = new Path(vm.choose(chooserEnv, ServerConstants.getBaseUris(context))
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 c895f20..da91eab 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
@@ -88,6 +88,7 @@ import org.apache.accumulo.server.ServerUtil;
 import org.apache.accumulo.server.constraints.MetadataConstraints;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.iterators.MetadataBulkLoadFilter;
@@ -355,7 +356,7 @@ public class Initialize implements KeywordExecutable {
     UUID uuid = UUID.randomUUID();
     // the actual disk locations of the root table and tablets
     String[] configuredVolumes = VolumeConfiguration.getVolumeUris(siteConfig, hadoopConf);
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ChooserScope.INIT, null);
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(ChooserScope.INIT, null);
     final String rootTabletDir = new Path(
         fs.choose(chooserEnv, configuredVolumes) + Path.SEPARATOR + ServerConstants.TABLE_DIR
             + Path.SEPARATOR + RootTable.ID + RootTable.ROOT_TABLET_LOCATION).toString();
@@ -489,7 +490,7 @@ public class Initialize implements KeywordExecutable {
     // initialize initial system tables config in zookeeper
     initSystemTablesConfig(zoo, Constants.ZROOT + "/" + uuid, hadoopConf);
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ChooserScope.INIT, null);
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(ChooserScope.INIT, null);
     String tableMetadataTabletDir = fs.choose(chooserEnv,
         ServerConstants.getBaseUris(siteConfig, hadoopConf)) + Constants.HDFS_TABLES_DIR
         + Path.SEPARATOR + MetadataTable.ID + TABLE_TABLETS_TABLET_DIR;
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 83c6831..121eb40 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,6 +87,7 @@ import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.hadoop.fs.FileStatus;
@@ -914,7 +915,7 @@ public class MetadataTableUtil {
         Key k = entry.getKey();
         Mutation m = new Mutation(k.getRow());
         m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
-        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableId,
+        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(tableId,
             new KeyExtent(k.getRow(), (Text) null).getEndRow(), context);
         String dir = volumeManager.choose(chooserEnv, ServerConstants.getBaseUris(context))
             + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + tableId + Path.SEPARATOR + new String(
@@ -1024,8 +1025,8 @@ public class MetadataTableUtil {
    */
   public static void createReplicationTable(ServerContext context) {
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ReplicationTable.ID, null,
-        context);
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(ReplicationTable.ID,
+        null, context);
     String dir = context.getVolumeManager().choose(chooserEnv, ServerConstants.getBaseUris(context))
         + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + ReplicationTable.ID
         + Constants.DEFAULT_TABLET_LOCATION;
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 854cf39..c23474d 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
@@ -42,6 +42,7 @@ import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.cli.ServerUtilOnRequiredTable;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -107,7 +108,7 @@ public class RandomizeVolumes {
       Key key = entry.getKey();
       Mutation m = new Mutation(key.getRow());
 
-      VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableId,
+      VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(tableId,
           new KeyExtent(key.getRow(), (Text) null).getEndRow(), context);
       final String newLocation = vm.choose(chooserEnv, ServerConstants.getBaseUris(context))
           + Path.SEPARATOR + ServerConstants.TABLE_DIR + Path.SEPARATOR + tableId + Path.SEPARATOR
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
index cdf8926..cde5116 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
@@ -24,10 +24,9 @@ import static org.easymock.EasyMock.verify;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.fail;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
 import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.junit.After;
@@ -38,10 +37,16 @@ import org.junit.rules.ExpectedException;
 
 public class PerTableVolumeChooserTest {
 
-  private ServerConfigurationFactory confFactory;
-  private TableConfiguration tableConf;
+  private static final String TABLE_CUSTOM_SUFFIX = "volume.chooser";
+
+  private static final String getCustomPropertySuffix(ChooserScope scope) {
+    return "volume.chooser." + scope.name().toLowerCase();
+  }
+
+  private ServiceEnvironment serviceEnv;
+  private Configuration tableConf;
   private PerTableVolumeChooser chooser;
-  private AccumuloConfiguration systemConf;
+  private Configuration systemConf;
 
   public static class MockChooser1 extends RandomVolumeChooser {}
 
@@ -52,65 +57,68 @@ public class PerTableVolumeChooserTest {
 
   @Before
   public void before() {
-    confFactory = createStrictMock(ServerConfigurationFactory.class);
+    serviceEnv = createStrictMock(ServiceEnvironment.class);
 
-    chooser = new PerTableVolumeChooser() {
-      @Override
-      ServerConfigurationFactory loadConfFactory(VolumeChooserEnvironment env) {
-        return confFactory;
-      }
+    chooser = new PerTableVolumeChooser();
 
-      @Override
-      String getTableContext(TableConfiguration tableConf) {
-        return null;
-      }
-    };
-
-    tableConf = createStrictMock(TableConfiguration.class);
-    systemConf = createStrictMock(AccumuloConfiguration.class);
-    expect(confFactory.getTableConfiguration(anyObject())).andReturn(tableConf).anyTimes();
-    expect(confFactory.getSystemConfiguration()).andReturn(systemConf).anyTimes();
+    tableConf = createStrictMock(Configuration.class);
+    systemConf = createStrictMock(Configuration.class);
+    expect(serviceEnv.getConfiguration(anyObject())).andReturn(tableConf).anyTimes();
+    expect(serviceEnv.getConfiguration()).andReturn(systemConf).anyTimes();
   }
 
   @After
   public void after() {
-    verify(confFactory, tableConf, systemConf);
+    verify(serviceEnv, tableConf, systemConf);
   }
 
   private VolumeChooser getTableDelegate() {
-    VolumeChooserEnvironment env = new VolumeChooserEnvironment(TableId.of("testTable"), null,
-        null);
+    VolumeChooserEnvironment env = new VolumeChooserEnvironmentImpl(TableId.of("testTable"), null,
+        null) {
+      @Override
+      public ServiceEnvironment getServiceEnv() {
+        return serviceEnv;
+      }
+    };
     return chooser.getDelegateChooser(env);
   }
 
   private VolumeChooser getDelegate(ChooserScope scope) {
-    VolumeChooserEnvironment env = new VolumeChooserEnvironment(scope, null);
+    VolumeChooserEnvironment env = new VolumeChooserEnvironmentImpl(scope, null) {
+      @Override
+      public ServiceEnvironment getServiceEnv() {
+        return serviceEnv;
+      }
+    };
     return chooser.getDelegateChooser(env);
   }
 
   @Test
   public void testInitScopeSelectsRandomChooser() {
-    replay(confFactory, tableConf, systemConf);
+    replay(serviceEnv, tableConf, systemConf);
     VolumeChooser delegate = getDelegate(ChooserScope.INIT);
     assertSame(RandomVolumeChooser.class, delegate.getClass());
   }
 
   @Test
-  public void testTableScopeUsingTableProperty() {
-    expect(tableConf.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER))
-        .andReturn(MockChooser1.class.getName());
-    replay(confFactory, tableConf, systemConf);
+  public void testTableScopeUsingTableProperty() throws Exception {
+    expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(MockChooser1.class.getName());
+    expect(serviceEnv.instantiate(TableId.of("testTable"), MockChooser1.class.getName(),
+        VolumeChooser.class)).andReturn(new MockChooser1());
+    replay(serviceEnv, tableConf, systemConf);
 
     VolumeChooser delegate = getTableDelegate();
     assertSame(MockChooser1.class, delegate.getClass());
   }
 
   @Test
-  public void testTableScopeUsingDefaultScopeProperty() {
-    expect(tableConf.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(null).once();
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
+  public void testTableScopeUsingDefaultScopeProperty() throws Exception {
+    expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT)))
         .andReturn(MockChooser2.class.getName()).once();
-    replay(confFactory, tableConf, systemConf);
+    expect(serviceEnv.instantiate(TableId.of("testTable"), MockChooser2.class.getName(),
+        VolumeChooser.class)).andReturn(new MockChooser2());
+    replay(serviceEnv, tableConf, systemConf);
 
     VolumeChooser delegate = getTableDelegate();
     assertSame(MockChooser2.class, delegate.getClass());
@@ -118,10 +126,10 @@ public class PerTableVolumeChooserTest {
 
   @Test
   public void testTableScopeWithNoConfig() {
-    expect(tableConf.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(null).once();
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
-        .andReturn(null).once();
-    replay(confFactory, tableConf, systemConf);
+    expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn(null)
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     getTableDelegate();
@@ -129,11 +137,13 @@ public class PerTableVolumeChooserTest {
   }
 
   @Test
-  public void testTableScopeWithBadDelegate() {
-    expect(tableConf.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(null).once();
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
+  public void testTableScopeWithBadDelegate() throws Exception {
+    expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT)))
         .andReturn("not a valid class name").once();
-    replay(confFactory, tableConf, systemConf);
+    expect(serviceEnv.instantiate(TableId.of("testTable"), "not a valid class name",
+        VolumeChooser.class)).andThrow(new RuntimeException());
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     getTableDelegate();
@@ -141,22 +151,26 @@ public class PerTableVolumeChooserTest {
   }
 
   @Test
-  public void testLoggerScopeUsingLoggerProperty() {
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER)))
+  public void testLoggerScopeUsingLoggerProperty() throws Exception {
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER)))
         .andReturn(MockChooser1.class.getName()).once();
-    replay(confFactory, tableConf, systemConf);
+    expect(serviceEnv.instantiate(MockChooser1.class.getName(), VolumeChooser.class))
+        .andReturn(new MockChooser1());
+    replay(serviceEnv, tableConf, systemConf);
 
     VolumeChooser delegate = getDelegate(ChooserScope.LOGGER);
     assertSame(MockChooser1.class, delegate.getClass());
   }
 
   @Test
-  public void testLoggerScopeUsingDefaultProperty() {
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER)))
-        .andReturn(null).once();
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
+  public void testLoggerScopeUsingDefaultProperty() throws Exception {
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
+        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT)))
         .andReturn(MockChooser2.class.getName()).once();
-    replay(confFactory, tableConf, systemConf);
+    expect(serviceEnv.instantiate(MockChooser2.class.getName(), VolumeChooser.class))
+        .andReturn(new MockChooser2());
+    replay(serviceEnv, tableConf, systemConf);
 
     VolumeChooser delegate = getDelegate(ChooserScope.LOGGER);
     assertSame(MockChooser2.class, delegate.getClass());
@@ -164,11 +178,11 @@ public class PerTableVolumeChooserTest {
 
   @Test
   public void testLoggerScopeWithNoConfig() {
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER)))
-        .andReturn(null).once();
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
-        .andReturn(null).once();
-    replay(confFactory, tableConf, systemConf);
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
+        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn(null)
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     getDelegate(ChooserScope.LOGGER);
@@ -176,12 +190,14 @@ public class PerTableVolumeChooserTest {
   }
 
   @Test
-  public void testLoggerScopeWithBadDelegate() {
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER)))
-        .andReturn(null).once();
-    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
+  public void testLoggerScopeWithBadDelegate() throws Exception {
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
+        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT)))
         .andReturn("not a valid class name").once();
-    replay(confFactory, tableConf, systemConf);
+    expect(serviceEnv.instantiate("not a valid class name", VolumeChooser.class))
+        .andThrow(new RuntimeException());
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     getDelegate(ChooserScope.LOGGER);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
index 18ad583..e85db46 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
@@ -27,10 +27,9 @@ import static org.junit.Assert.fail;
 
 import java.util.Arrays;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
 import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.junit.After;
@@ -41,60 +40,71 @@ import org.junit.rules.ExpectedException;
 
 public class PreferredVolumeChooserTest {
 
+  private static final String TABLE_CUSTOM_SUFFIX = "volume.preferred";
+
+  private static final String getCustomPropertySuffix(ChooserScope scope) {
+    return "volume.preferred." + scope.name().toLowerCase();
+  }
+
   private static final String[] ALL_OPTIONS = {"1", "2", "3"};
 
-  private ServerConfigurationFactory confFactory;
-  private TableConfiguration tableConf;
+  private ServiceEnvironment serviceEnv;
+  private Configuration tableConf;
+  private Configuration systemConf;
   private PreferredVolumeChooser chooser;
-  private AccumuloConfiguration systemConf;
 
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @Before
   public void before() {
-    confFactory = createStrictMock(ServerConfigurationFactory.class);
+    serviceEnv = createStrictMock(ServiceEnvironment.class);
 
-    chooser = new PreferredVolumeChooser() {
-      @Override
-      ServerConfigurationFactory loadConfFactory(VolumeChooserEnvironment env) {
-        return confFactory;
-      }
-    };
+    chooser = new PreferredVolumeChooser();
 
-    tableConf = createStrictMock(TableConfiguration.class);
-    systemConf = createStrictMock(AccumuloConfiguration.class);
-    expect(confFactory.getTableConfiguration(anyObject())).andReturn(tableConf).anyTimes();
-    expect(confFactory.getSystemConfiguration()).andReturn(systemConf).anyTimes();
+    tableConf = createStrictMock(Configuration.class);
+    systemConf = createStrictMock(Configuration.class);
+    expect(serviceEnv.getConfiguration(anyObject())).andReturn(tableConf).anyTimes();
+    expect(serviceEnv.getConfiguration()).andReturn(systemConf).anyTimes();
   }
 
   @After
   public void after() {
-    verify(confFactory, tableConf, systemConf);
+    verify(serviceEnv, tableConf, systemConf);
   }
 
   private String[] chooseForTable() {
-    VolumeChooserEnvironment env = new VolumeChooserEnvironment(TableId.of("testTable"), null,
-        null);
+    VolumeChooserEnvironment env = new VolumeChooserEnvironmentImpl(TableId.of("testTable"), null,
+        null) {
+      @Override
+      public ServiceEnvironment getServiceEnv() {
+        return serviceEnv;
+      }
+    };
     return chooser.getPreferredVolumes(env, ALL_OPTIONS);
   }
 
   private String[] choose(ChooserScope scope) {
-    VolumeChooserEnvironment env = new VolumeChooserEnvironment(scope, null);
+    VolumeChooserEnvironment env = new VolumeChooserEnvironmentImpl(scope, null) {
+      @Override
+      public ServiceEnvironment getServiceEnv() {
+        return serviceEnv;
+      }
+    };
     return chooser.getPreferredVolumes(env, ALL_OPTIONS);
   }
 
   @Test
   public void testInitScopeSelectsRandomlyFromAll() {
-    replay(confFactory, tableConf, systemConf);
+    replay(serviceEnv, tableConf, systemConf);
     String[] volumes = choose(ChooserScope.INIT);
     assertSame(ALL_OPTIONS, volumes);
   }
 
   @Test
   public void testTableScopeUsingTableProperty() {
-    expect(tableConf.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn("2,1");
-    replay(confFactory, tableConf, systemConf);
+    expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn("2,1");
+    replay(serviceEnv, tableConf, systemConf);
 
     String[] volumes = chooseForTable();
     Arrays.sort(volumes);
@@ -103,10 +113,10 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testTableScopeUsingDefaultScopeProperty() {
-    expect(tableConf.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(null).once();
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
-        .andReturn("3,2").once();
-    replay(confFactory, tableConf, systemConf);
+    expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn("3,2")
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     String[] volumes = chooseForTable();
     Arrays.sort(volumes);
@@ -115,10 +125,10 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testTableScopeWithNoConfig() {
-    expect(tableConf.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(null).once();
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
-        .andReturn(null).once();
-    replay(confFactory, tableConf, systemConf);
+    expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn(null)
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     chooseForTable();
@@ -127,8 +137,8 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testTableScopeWithEmptySet() {
-    expect(tableConf.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(",").once();
-    replay(confFactory, tableConf, systemConf);
+    expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(",").once();
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     chooseForTable();
@@ -137,10 +147,10 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testTableScopeWithUnrecognizedVolumes() {
-    expect(tableConf.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(null).once();
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
-        .andReturn("4").once();
-    replay(confFactory, tableConf, systemConf);
+    expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn("4")
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     chooseForTable();
@@ -149,9 +159,9 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testLoggerScopeUsingLoggerProperty() {
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER)))
-        .andReturn("2,1").once();
-    replay(confFactory, tableConf, systemConf);
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn("2,1")
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     String[] volumes = choose(ChooserScope.LOGGER);
     Arrays.sort(volumes);
@@ -160,11 +170,11 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testLoggerScopeUsingDefaultProperty() {
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER)))
-        .andReturn(null).once();
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
-        .andReturn("3,2").once();
-    replay(confFactory, tableConf, systemConf);
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
+        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn("3,2")
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     String[] volumes = choose(ChooserScope.LOGGER);
     Arrays.sort(volumes);
@@ -173,11 +183,11 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testLoggerScopeWithNoConfig() {
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER)))
-        .andReturn(null).once();
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
-        .andReturn(null).once();
-    replay(confFactory, tableConf, systemConf);
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
+        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn(null)
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     choose(ChooserScope.LOGGER);
@@ -186,9 +196,9 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testLoggerScopeWithEmptySet() {
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER)))
-        .andReturn(",").once();
-    replay(confFactory, tableConf, systemConf);
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(",")
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     choose(ChooserScope.LOGGER);
@@ -197,11 +207,11 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testLoggerScopeWithUnrecognizedVolumes() {
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER)))
-        .andReturn(null).once();
-    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT)))
-        .andReturn("4").once();
-    replay(confFactory, tableConf, systemConf);
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
+        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn("4")
+        .once();
+    replay(serviceEnv, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
     choose(ChooserScope.LOGGER);
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooserTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooserTest.java
index 1b4f032..5de7bd9 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooserTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooserTest.java
@@ -20,13 +20,11 @@ import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.volume.Volume;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsStatus;
-import org.apache.hadoop.fs.Path;
 import org.easymock.EasyMock;
 import org.junit.After;
 import org.junit.Before;
@@ -35,13 +33,10 @@ import org.junit.Test;
 import com.google.common.util.concurrent.UncheckedExecutionException;
 
 public class SpaceAwareVolumeChooserTest {
-  VolumeManager volumeManager = null;
+
   VolumeChooserEnvironment chooserEnv = null;
-  ServerContext serverContext = null;
-  ServerConfigurationFactory serverConfigurationFactory = null;
-  AccumuloConfiguration sysConfig = null;
-  Volume vol1 = null;
-  Volume vol2 = null;
+  ServiceEnvironment serviceEnv = null;
+  Configuration sysConfig = null;
   FileSystem fs1 = null;
   FileSystem fs2 = null;
   FsStatus status1 = null;
@@ -60,19 +55,13 @@ public class SpaceAwareVolumeChooserTest {
 
   @Before
   public void beforeTest() {
-    volumeManager = EasyMock.createMock(VolumeManager.class);
-    serverContext = EasyMock.createMock(ServerContext.class);
-    serverConfigurationFactory = EasyMock.createMock(ServerConfigurationFactory.class);
-    sysConfig = EasyMock.createMock(AccumuloConfiguration.class);
-    vol1 = EasyMock.createMock(Volume.class);
-    vol2 = EasyMock.createMock(Volume.class);
+    serviceEnv = EasyMock.createMock(ServiceEnvironment.class);
+    sysConfig = EasyMock.createMock(Configuration.class);
     fs1 = EasyMock.createMock(FileSystem.class);
     fs2 = EasyMock.createMock(FileSystem.class);
     status1 = EasyMock.createMock(FsStatus.class);
     status2 = EasyMock.createMock(FsStatus.class);
-    chooserEnv = new VolumeChooserEnvironment(VolumeChooserEnvironment.ChooserScope.DEFAULT,
-        serverContext);
-
+    chooserEnv = EasyMock.createMock(VolumeChooserEnvironment.class);
   }
 
   private void testSpecificSetup(long percentage1, long percentage2, String cacheDuration,
@@ -84,6 +73,7 @@ public class SpaceAwareVolumeChooserTest {
       max = iterations + 1;
       updatePropertyMax = max + 1;
     }
+
     // Volume 1 is percentage1 full
     EasyMock.expect(status1.getRemaining()).andReturn(percentage1).times(min, max);
     EasyMock.expect(status1.getCapacity()).andReturn(100L).times(min, max);
@@ -92,43 +82,33 @@ public class SpaceAwareVolumeChooserTest {
     EasyMock.expect(status2.getRemaining()).andReturn(percentage2).times(min, max);
     EasyMock.expect(status2.getCapacity()).andReturn(100L).times(min, max);
 
-    EasyMock.expect(sysConfig.get(SpaceAwareVolumeChooser.HDFS_SPACE_RECOMPUTE_INTERVAL))
+    EasyMock.expect(sysConfig.getCustom(SpaceAwareVolumeChooser.RECOMPUTE_INTERVAL))
         .andReturn(cacheDuration).times(1);
+
     EasyMock
-        .expect(sysConfig.get(PreferredVolumeChooser
-            .getPropertyNameForScope(VolumeChooserEnvironment.ChooserScope.DEFAULT)))
+        .expect(
+            sysConfig.getCustom("volume.preferred." + ChooserScope.DEFAULT.name().toLowerCase()))
         .andReturn(String.join(",", tableDirs)).times(timesToCallPreferredVolumeChooser);
 
-    EasyMock.expect(serverContext.getVolumeManager()).andReturn(volumeManager).times(min,
-        Math.max(max, updatePropertyMax));
-    EasyMock.expect(serverContext.getServerConfFactory()).andReturn(serverConfigurationFactory)
-        .times(min, updatePropertyMax);
-    EasyMock.expect(serverConfigurationFactory.getSystemConfiguration()).andReturn(sysConfig)
-        .times(1, updatePropertyMax);
-
-    EasyMock.expect(volumeManager.getVolumeByPath(new Path(volumeOne))).andReturn(vol1).times(min,
-        max);
-    EasyMock.expect(volumeManager.getVolumeByPath(new Path(volumeTwo))).andReturn(vol2).times(min,
-        max);
-    EasyMock.expect(vol1.getFileSystem()).andReturn(fs1).times(min, max);
-    EasyMock.expect(vol2.getFileSystem()).andReturn(fs2).times(min, max);
+    EasyMock.expect(serviceEnv.getConfiguration()).andReturn(sysConfig).times(1, updatePropertyMax);
+
     EasyMock.expect(fs1.getStatus()).andReturn(status1).times(min, max);
     EasyMock.expect(fs2.getStatus()).andReturn(status2).times(min, max);
 
-    EasyMock.replay(serverContext, vol1, vol2, fs1, fs2, status1, status2, volumeManager,
-        serverConfigurationFactory, sysConfig);
+    EasyMock.expect(chooserEnv.getFileSystem(volumeOne)).andReturn(fs1).times(min, max);
+    EasyMock.expect(chooserEnv.getFileSystem(volumeTwo)).andReturn(fs2).times(min, max);
+    EasyMock.expect(chooserEnv.getScope()).andReturn(ChooserScope.DEFAULT).times(min, max * 2);
+    EasyMock.expect(chooserEnv.getServiceEnv()).andReturn(serviceEnv).times(min, max);
+
+    EasyMock.replay(serviceEnv, fs1, fs2, status1, status2, sysConfig, chooserEnv);
   }
 
   @After
   public void afterTest() {
 
-    EasyMock.verify(serverContext, vol1, vol2, fs1, fs2, status1, status2, volumeManager,
-        serverConfigurationFactory, sysConfig);
+    EasyMock.verify(serviceEnv, fs1, fs2, status1, status2, sysConfig, chooserEnv);
 
-    volumeManager = null;
-    serverContext = null;
-    vol1 = null;
-    vol2 = null;
+    serviceEnv = null;
     fs1 = null;
     fs2 = null;
     status1 = null;
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
index 429adad..c9d5929 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
@@ -137,7 +137,7 @@ public class VolumeManagerImplTest {
     conf.set(Property.GENERAL_VOLUME_CHOOSER, WrongVolumeChooser.class.getName());
     thrown.expect(RuntimeException.class);
     VolumeManager vm = VolumeManagerImpl.get(conf, hadoopConf);
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(TableId.of("sometable"),
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(TableId.of("sometable"),
         null, null);
     String choice = vm.choose(chooserEnv, volumes.toArray(new String[0]));
     assertTrue("shouldn't see invalid options from misbehaving chooser.", volumes.contains(choice));
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 0f6dddd..2c5bc57 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
@@ -103,6 +103,7 @@ import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.ServerUtil;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.init.Initialize;
@@ -287,7 +288,7 @@ public class Master
     if (!zoo.exists(dirZPath)) {
       Path oldPath = fs.getFullPath(FileType.TABLE, "/" + MetadataTable.ID + "/root_tablet");
       if (fs.exists(oldPath)) {
-        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(RootTable.ID,
+        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(RootTable.ID,
             RootTable.EXTENT.getEndRow(), context);
         String newPath = fs.choose(chooserEnv, ServerConstants.getBaseUris(context))
             + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + RootTable.ID;
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 ef512cd..b547b4a 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
@@ -72,6 +72,7 @@ import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.log.WalStateManager;
 import org.apache.accumulo.server.log.WalStateManager.WalMarkerException;
@@ -659,7 +660,7 @@ abstract class TabletGroupWatcher extends Daemon {
       } else {
         // Recreate the default tablet to hold the end of the table
         Master.log.debug("Recreating the last tablet to point to {}", extent.getPrevEndRow());
-        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(extent.getTableId(),
+        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(extent.getTableId(),
             extent.getEndRow(), master.getContext());
 
         String tdir = master.getFileSystem().choose(chooserEnv,
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/ChooseDir.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/ChooseDir.java
index 0c401d9..18b3310 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/ChooseDir.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/create/ChooseDir.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.master.tableOps.TableInfo;
 import org.apache.accumulo.master.tableOps.Utils;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.tablets.UniqueNameAllocator;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -54,8 +55,8 @@ class ChooseDir extends MasterRepo {
     // Constants.DEFAULT_TABLET_LOCATION has a leading slash prepended to it so we don't need to add
     // one here
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableInfo.getTableId(), null,
-        master.getContext());
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(tableInfo.getTableId(),
+        null, master.getContext());
 
     String baseDir = master.getFileSystem().choose(chooserEnv,
         ServerConstants.getBaseUris(master.getContext())) + Constants.HDFS_TABLES_DIR
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/tableImport/PopulateMetadataTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/tableImport/PopulateMetadataTable.java
index fd21cf5..09b926f 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/tableImport/PopulateMetadataTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/tableImport/PopulateMetadataTable.java
@@ -47,6 +47,7 @@ import org.apache.accumulo.master.Master;
 import org.apache.accumulo.master.tableOps.MasterRepo;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.hadoop.fs.Path;
@@ -220,8 +221,8 @@ class PopulateMetadataTable extends MasterRepo {
   protected String getClonedTabletDir(Master master, Text endRow, String[] tableDirs,
       String tabletDir) {
     // We can try to spread out the tablet dirs across all volumes
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableInfo.tableId, endRow,
-        master.getContext());
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(tableInfo.tableId,
+        endRow, master.getContext());
     String tableDir = master.getFileSystem().choose(chooserEnv, tableDirs);
 
     // Build up a full hdfs://localhost:8020/accumulo/tables/$id/c-XXXXXXX
diff --git a/server/master/src/test/java/org/apache/accumulo/master/tableOps/tableImport/ImportTableTest.java b/server/master/src/test/java/org/apache/accumulo/master/tableOps/tableImport/ImportTableTest.java
index 07a3a4f..100ffe2 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/tableOps/tableImport/ImportTableTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/tableOps/tableImport/ImportTableTest.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.easymock.EasyMock;
 import org.junit.Test;
@@ -43,7 +44,7 @@ public class ImportTableTest {
     EasyMock.expect(master.getContext()).andReturn(null);
     EasyMock.expect(master.getFileSystem()).andReturn(volumeManager);
     // Choose the 2nd element
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(iti.tableId, null, null);
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(iti.tableId, null, null);
     EasyMock.expect(volumeManager.choose(EasyMock.eq(chooserEnv), EasyMock.eq(tableDirs)))
         .andReturn(tableDirs[1]);
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java
index 9a4b8a6..b715179 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ConditionCheckerContext.java
@@ -76,7 +76,7 @@ public class ConditionCheckerContext {
 
     tableIters = pic.getIterInfo();
     tableIterOpts = pic.getOpts();
-    context = pic.getContext();
+    context = pic.getServiceEnv();
 
     classCache = new HashMap<>();
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index 38806f6..2206716 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -55,6 +55,7 @@ import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.spi.cache.BlockCache;
 import org.apache.accumulo.core.spi.cache.BlockCacheManager;
 import org.apache.accumulo.core.spi.cache.CacheType;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.scan.ScanDispatcher;
 import org.apache.accumulo.core.spi.scan.ScanDispatcher.DispatchParmaters;
 import org.apache.accumulo.core.spi.scan.ScanExecutor;
@@ -65,6 +66,7 @@ import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.fate.util.LoggingRunnable;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServiceEnvironmentImpl;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -205,7 +207,19 @@ public class TabletServerResourceManager {
       if (factory == null) {
         queue = new LinkedBlockingQueue<>();
       } else {
-        Comparator<ScanInfo> comparator = factory.createComparator(() -> sec.prioritizerOpts);
+        Comparator<ScanInfo> comparator = factory
+            .createComparator(new ScanPrioritizer.CreateParameters() {
+
+              @Override
+              public Map<String,String> getOptions() {
+                return sec.prioritizerOpts;
+              }
+
+              @Override
+              public ServiceEnvironment getServiceEnv() {
+                return new ServiceEnvironmentImpl(context);
+              }
+            });
 
         // function to extract scan scan session from runnable
         Function<Runnable,ScanInfo> extractor = r -> ((ScanSession.ScanMeasurer) ((TraceRunnable) r)
@@ -933,6 +947,11 @@ public class TabletServerResourceManager {
         public Map<String,ScanExecutor> getScanExecutors() {
           return scanExecutorChoices;
         }
+
+        @Override
+        public ServiceEnvironment getServiceEnv() {
+          return new ServiceEnvironmentImpl(context);
+        }
       });
       ExecutorService executor = scanExecutors.get(scanExecutorName);
       if (executor == null) {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
index df6e37a..5384d87 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/compaction/MajorCompactionRequest.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.function.Predicate;
 
 import org.apache.accumulo.core.client.admin.TableOperations;
@@ -51,7 +52,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.cache.Cache;
 
 /**
@@ -149,9 +149,9 @@ public class MajorCompactionRequest implements Cloneable {
    */
   public List<Summary> getSummaries(Collection<FileRef> files,
       Predicate<SummarizerConfiguration> summarySelector) {
-    Preconditions.checkState(volumeManager != null,
-        "Getting summaries is not" + " supported at this time. It's only supported when"
-            + " CompactionStrategy.gatherInformation() is called.");
+    Objects.requireNonNull(volumeManager,
+        "Getting summaries is not  supported at this time. It's only supported when "
+            + "CompactionStrategy.gatherInformation() is called.");
     SummaryCollection sc = new SummaryCollection();
     SummarizerFactory factory = new SummarizerFactory(tableConfig);
     for (FileRef file : files) {
@@ -172,9 +172,9 @@ public class MajorCompactionRequest implements Cloneable {
   }
 
   public FileSKVIterator openReader(FileRef ref) throws IOException {
-    Preconditions.checkState(volumeManager != null,
-        "Opening files is not" + " supported at this time. It's only supported when"
-            + " CompactionStrategy.gatherInformation() is called.");
+    Objects.requireNonNull(volumeManager,
+        "Opening files is not supported at this time. It's only supported when "
+            + "CompactionStrategy.gatherInformation() is called.");
     // @TODO verify the file isn't some random file in HDFS
     // @TODO ensure these files are always closed?
     FileOperations fileFactory = FileOperations.getInstance();
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 5df5114..58b251e 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
@@ -62,6 +62,7 @@ import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.logger.LogFileKey;
@@ -406,7 +407,7 @@ public class DfsLogger implements Comparable<DfsLogger> {
     log.debug("DfsLogger.open() begin");
     VolumeManager fs = conf.getFileSystem();
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ChooserScope.LOGGER,
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(ChooserScope.LOGGER,
         context);
     logPath = fs.choose(chooserEnv, ServerConstants.getBaseUris(context)) + Path.SEPARATOR
         + ServerConstants.WAL_DIR + Path.SEPARATOR + logger + Path.SEPARATOR + filename;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java
index d5dc4d9..28f1d5a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java
@@ -22,6 +22,7 @@ import java.util.Map;
 
 import org.apache.accumulo.core.client.sample.SamplerConfiguration;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
 import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
@@ -63,8 +64,8 @@ public class MultiScanSession extends ScanSession {
   }
 
   @Override
-  public String getTableId() {
-    return threadPoolExtent.getTableId().canonical();
+  public TableId getTableId() {
+    return threadPoolExtent.getTableId();
   }
 
   @Override
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SingleScanSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SingleScanSession.java
index 367429b..692a79a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SingleScanSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SingleScanSession.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
 import org.apache.accumulo.core.security.Authorizations;
@@ -58,8 +59,8 @@ public class SingleScanSession extends ScanSession {
   }
 
   @Override
-  public String getTableId() {
-    return extent.getTableId().canonical();
+  public TableId getTableId() {
+    return extent.getTableId();
   }
 
   @Override
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
index 047069c..34b1130 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
@@ -230,7 +230,7 @@ class ScanDataSource implements DataSource {
             options.getClassLoaderContext());
         context = options.getClassLoaderContext();
       } else {
-        context = pic.getContext();
+        context = pic.getServiceEnv();
         if (context != null) {
           log.trace("Loading iterators for scan with table context: {}",
               options.getClassLoaderContext());
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 a31ca63..9b15378 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
@@ -98,6 +98,7 @@ import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.fs.VolumeUtil;
@@ -2861,7 +2862,8 @@ public class Tablet {
     String lowDirectory;
 
     UniqueNameAllocator namer = context.getUniqueNameAllocator();
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableId, endRow, context);
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(tableId, endRow,
+        context);
     String volume = fs.choose(chooserEnv, ServerConstants.getBaseUris(context))
         + Constants.HDFS_TABLES_DIR + Path.SEPARATOR;
 
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java b/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java
index d13dc31..d481956 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.test;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.test.VolumeChooserIT.PERTABLE_CHOOSER_PROP;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -32,13 +33,13 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.harness.conf.StandaloneAccumuloClusterConfiguration;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloClusterImpl;
-import org.apache.accumulo.server.fs.PerTableVolumeChooser;
 import org.apache.accumulo.test.ShellServerIT.TestShell;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 public class ShellConfigIT extends AccumuloClusterHarness {
+
   @Override
   public int defaultTimeoutSeconds() {
     return 30;
@@ -54,8 +55,8 @@ public class ShellConfigIT extends AccumuloClusterHarness {
       // This lets us run this test more generically rather than forcibly needing to update some
       // property in accumulo.properties
       origPropValue = client.instanceOperations().getSystemConfiguration()
-          .get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER);
-      client.instanceOperations().setProperty(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER,
+          .get(PERTABLE_CHOOSER_PROP);
+      client.instanceOperations().setProperty(PERTABLE_CHOOSER_PROP,
           FairVolumeChooser.class.getName());
     }
   }
@@ -64,8 +65,7 @@ public class ShellConfigIT extends AccumuloClusterHarness {
   public void resetProperty() throws Exception {
     if (origPropValue != null) {
       try (AccumuloClient client = createAccumuloClient()) {
-        client.instanceOperations().setProperty(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER,
-            origPropValue);
+        client.instanceOperations().setProperty(PERTABLE_CHOOSER_PROP, origPropValue);
       }
     }
   }
@@ -111,7 +111,7 @@ public class ShellConfigIT extends AccumuloClusterHarness {
 
     String configOutput = ts.exec("config");
 
-    assertTrue(configOutput.contains(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER));
+    assertTrue(configOutput.contains(PERTABLE_CHOOSER_PROP));
     assertFalse(configOutput.contains(Property.INSTANCE_CRYPTO_SERVICE.getKey()));
   }
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java b/test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java
index ce464dc..7ba2cde 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java
@@ -16,6 +16,9 @@
  */
 package org.apache.accumulo.test;
 
+import static org.apache.accumulo.test.VolumeChooserIT.PERTABLE_CHOOSER_PROP;
+import static org.apache.accumulo.test.VolumeChooserIT.PREFERRED_CHOOSER_PROP;
+
 import java.io.File;
 import java.util.HashMap;
 import java.util.Map;
@@ -66,8 +69,7 @@ public class VolumeChooserFailureIT extends ConfigurableMacBase {
     Map<String,String> siteConfig = new HashMap<>();
     siteConfig.put(Property.GENERAL_VOLUME_CHOOSER.getKey(), PerTableVolumeChooser.class.getName());
     // if a table doesn't have a volume chooser, use the preferred volume chooser
-    siteConfig.put(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER,
-        PreferredVolumeChooser.class.getName());
+    siteConfig.put(VolumeChooserIT.PERTABLE_CHOOSER_PROP, PreferredVolumeChooser.class.getName());
 
     // Set up 4 different volume paths
     File baseDir = cfg.getDir();
@@ -83,7 +85,7 @@ public class VolumeChooserFailureIT extends ConfigurableMacBase {
 
     cfg.setSiteConfig(siteConfig);
 
-    siteConfig.put(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER),
+    siteConfig.put(VolumeChooserIT.getPerTableProp(ChooserScope.LOGGER),
         PreferredVolumeChooser.class.getName());
     // do not set preferred volumes
     cfg.setSiteConfig(siteConfig);
@@ -121,8 +123,8 @@ public class VolumeChooserFailureIT extends ConfigurableMacBase {
       accumuloClient.namespaceOperations().create(namespace1);
 
       // Set properties on the namespace
-      accumuloClient.namespaceOperations().setProperty(namespace1,
-          PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, PreferredVolumeChooser.class.getName());
+      accumuloClient.namespaceOperations().setProperty(namespace1, PERTABLE_CHOOSER_PROP,
+          PreferredVolumeChooser.class.getName());
       // deliberately do not set preferred volumes
 
       // Create table1 on namespace1 (will fail)
@@ -144,13 +146,13 @@ public class VolumeChooserFailureIT extends ConfigurableMacBase {
       accumuloClient.namespaceOperations().create(namespace1);
 
       // Set properties on the namespace
-      String propertyName = PerTableVolumeChooser.TABLE_VOLUME_CHOOSER;
+      String propertyName = PERTABLE_CHOOSER_PROP;
       String volume = PreferredVolumeChooser.class.getName();
       accumuloClient.namespaceOperations().setProperty(namespace1, propertyName, volume);
 
       // set to v3 which is not included in the list of instance volumes, so it should go to the
       // system default preferred volumes
-      propertyName = PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES;
+      propertyName = PREFERRED_CHOOSER_PROP;
       volume = v3.toString();
       accumuloClient.namespaceOperations().setProperty(namespace1, propertyName, volume);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
index 96ba261..931d0f0 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
@@ -59,6 +59,20 @@ import org.junit.Test;
 
 public class VolumeChooserIT extends ConfigurableMacBase {
 
+  private static final String TP = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey();
+  static final String PREFERRED_CHOOSER_PROP = TP + "volume.preferred";
+  static final String PERTABLE_CHOOSER_PROP = TP + "volume.chooser";
+
+  private static final String GP = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey();
+
+  static final String getPreferredProp(ChooserScope scope) {
+    return GP + "volume.preferred." + scope.name().toLowerCase();
+  }
+
+  static final String getPerTableProp(ChooserScope scope) {
+    return GP + "volume.chooser." + scope.name().toLowerCase();
+  }
+
   private static final Text EMPTY = new Text();
   private static final Value EMPTY_VALUE = new Value(new byte[] {});
   private File volDirBase;
@@ -87,8 +101,7 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     Map<String,String> siteConfig = new HashMap<>();
     siteConfig.put(Property.GENERAL_VOLUME_CHOOSER.getKey(), PerTableVolumeChooser.class.getName());
     // if a table doesn't have a volume chooser, use the preferred volume chooser
-    siteConfig.put(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER,
-        PreferredVolumeChooser.class.getName());
+    siteConfig.put(PERTABLE_CHOOSER_PROP, PreferredVolumeChooser.class.getName());
 
     // Set up 4 different volume paths
     File baseDir = cfg.getDir();
@@ -104,13 +117,11 @@ public class VolumeChooserIT extends ConfigurableMacBase {
 
     systemPreferredVolumes = v1 + "," + v2;
     // exclude v4
-    siteConfig.put(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES, systemPreferredVolumes);
+    siteConfig.put(PREFERRED_CHOOSER_PROP, systemPreferredVolumes);
     cfg.setSiteConfig(siteConfig);
 
-    siteConfig.put(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER),
-        PreferredVolumeChooser.class.getName());
-    siteConfig.put(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER),
-        v2.toString());
+    siteConfig.put(getPerTableProp(ChooserScope.LOGGER), PreferredVolumeChooser.class.getName());
+    siteConfig.put(getPreferredProp(ChooserScope.LOGGER), v2.toString());
     cfg.setSiteConfig(siteConfig);
 
     // Only add volumes 1, 2, and 4 to the list of instance volumes to have one volume that isn't in
@@ -207,10 +218,10 @@ public class VolumeChooserIT extends ConfigurableMacBase {
       String configuredVolumes, String namespace) throws Exception {
     accumuloClient.namespaceOperations().create(namespace);
     // Set properties on the namespace
-    accumuloClient.namespaceOperations().setProperty(namespace,
-        PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, volumeChooserClassName);
-    accumuloClient.namespaceOperations().setProperty(namespace,
-        PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES, configuredVolumes);
+    accumuloClient.namespaceOperations().setProperty(namespace, PERTABLE_CHOOSER_PROP,
+        volumeChooserClassName);
+    accumuloClient.namespaceOperations().setProperty(namespace, PREFERRED_CHOOSER_PROP,
+        configuredVolumes);
   }
 
   private void verifyVolumesForWritesToNewTable(AccumuloClient accumuloClient, String myNamespace,
@@ -293,7 +304,7 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     accumuloClient.namespaceOperations().create(ns);
 
     // Set properties on the namespace
-    accumuloClient.namespaceOperations().setProperty(ns, PerTableVolumeChooser.TABLE_VOLUME_CHOOSER,
+    accumuloClient.namespaceOperations().setProperty(ns, PERTABLE_CHOOSER_PROP,
         RandomVolumeChooser.class.getName());
 
     verifyVolumesForWritesToNewTable(accumuloClient, ns, expectedVolumes);