You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2017/09/12 02:24:46 UTC

[accumulo] branch master updated (953c559 -> 21fb477)

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

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


    from 953c559  ACCUMULO-4686 Formatting update from verify
     new 513d1f5  ACCUMULO-4086 Improve volume chooser fallback
     new a496bc6  ACCUMULO-4086 Enum for chooser scopes and more
     new 21fb477  ACCUMULO-4086 Make chooser classes more testable

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../core/conf/ConfigurationTypeHelper.java         |  67 ++++-
 .../org/apache/accumulo/core/conf/Property.java    |  42 +---
 .../accumulo/server/fs/PerTableVolumeChooser.java  | 173 ++++++++++---
 .../accumulo/server/fs/PreferredVolumeChooser.java | 156 +++++++-----
 .../accumulo/server/fs/RandomVolumeChooser.java    |   2 +-
 .../apache/accumulo/server/fs/VolumeChooser.java   |  29 ++-
 .../server/fs/VolumeChooserEnvironment.java        |  59 ++---
 .../accumulo/server/fs/VolumeManagerImpl.java      |  30 ++-
 .../org/apache/accumulo/server/fs/VolumeUtil.java  |   3 +-
 .../apache/accumulo/server/init/Initialize.java    |   6 +-
 .../accumulo/server/util/MetadataTableUtil.java    |   5 +-
 .../accumulo/server/util/RandomizeVolumes.java     |   3 +-
 .../server/fs/PerTableVolumeChooserTest.java       | 179 +++++++++++++
 .../server/fs/PreferredVolumeChooserTest.java      | 199 +++++++++++++++
 .../accumulo/server/fs/VolumeManagerImplTest.java  |  36 ++-
 .../java/org/apache/accumulo/master/Master.java    |   3 +-
 .../apache/accumulo/master/TabletGroupWatcher.java |   3 +-
 .../apache/accumulo/master/tableOps/ChooseDir.java |   4 +-
 .../master/tableOps/PopulateMetadataTable.java     |   3 +-
 .../accumulo/master/tableOps/ImportTableTest.java  |   4 +-
 .../org/apache/accumulo/tserver/log/DfsLogger.java |   5 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |   3 +-
 .../accumulo/tserver/tablet/RootFilesTest.java     |   2 +
 .../org/apache/accumulo/test/ShellConfigIT.java    |  10 +-
 .../accumulo/test/VolumeChooserFailureIT.java      | 154 ++++++++++++
 .../org/apache/accumulo/test/VolumeChooserIT.java  | 277 +++++++++------------
 26 files changed, 1074 insertions(+), 383 deletions(-)
 create mode 100644 server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
 create mode 100644 server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
 create mode 100644 test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java

-- 
To stop receiving notification emails like this one, please contact
['"commits@accumulo.apache.org" <co...@accumulo.apache.org>'].

[accumulo] 02/03: ACCUMULO-4086 Enum for chooser scopes and more

Posted by ct...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a496bc6c42f6f3c81f03c4246a71f3008adad359
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Fri Aug 4 00:03:25 2017 -0400

    ACCUMULO-4086 Enum for chooser scopes and more
    
    Added enum for VolumeChooser scopes:
    
    * Added enum to VolumeChooserEnvironment
    * Simplified equals and hashCode in VolumeChooserEnvironment
    * All environment now has a scope, either DEFAULT, TABLE, INIT, or LOGGER
    * If scope is TABLE, then it also has a tableId
    * Other scopes (currently) do not have tableId, but may change,
    depending on future changes, and may be scope-dependent
    
    Additional improvements:
    
    * Move Class loading code to ConfigurationTypeHelper for easier reuse
    loading from configuration
    * Use per-table context for class loading in PerTableVolumeChooser
    * Use Table.ID and ChooserScope types for caching VolumeChoosers in
    PerTableVolumeChooser
    * Fall back to default scope if per-table setting is not set in
    PerTableVolumeChooser and PreferredVolumeChooser
    * Create explicit VolumeChooserException (RuntimeException)
    * Dramatically simplify retrieving from and updating cache in
    PerTableVolumeChooser
    * Provide check for ensuring preferred volumes configuration contains
    volumes which exist in instance.volumes
    * Clean up mocking code in PerTableVolumeChooserTest and
    PreferredVolumeChooserTest
---
 .../core/conf/ConfigurationTypeHelper.java         |  67 ++++++++-
 .../org/apache/accumulo/core/conf/Property.java    |  37 +----
 .../accumulo/server/fs/PerTableVolumeChooser.java  | 157 ++++++++++-----------
 .../accumulo/server/fs/PreferredVolumeChooser.java | 153 +++++++++-----------
 .../accumulo/server/fs/RandomVolumeChooser.java    |   2 +-
 .../apache/accumulo/server/fs/VolumeChooser.java   |  29 +++-
 .../server/fs/VolumeChooserEnvironment.java        |  59 +++-----
 .../accumulo/server/fs/VolumeManagerImpl.java      |   6 +-
 .../org/apache/accumulo/server/fs/VolumeUtil.java  |   3 +-
 .../apache/accumulo/server/init/Initialize.java    |   6 +-
 .../accumulo/server/util/MetadataTableUtil.java    |   5 +-
 .../accumulo/server/util/RandomizeVolumes.java     |   3 +-
 .../server/fs/PerTableVolumeChooserTest.java       | 139 +++++++++---------
 .../server/fs/PreferredVolumeChooserTest.java      | 134 +++++++++---------
 .../accumulo/server/fs/VolumeManagerImplTest.java  |   3 +-
 .../java/org/apache/accumulo/master/Master.java    |   3 +-
 .../apache/accumulo/master/TabletGroupWatcher.java |   3 +-
 .../apache/accumulo/master/tableOps/ChooseDir.java |   4 +-
 .../master/tableOps/PopulateMetadataTable.java     |   3 +-
 .../accumulo/master/tableOps/ImportTableTest.java  |   4 +-
 .../org/apache/accumulo/tserver/log/DfsLogger.java |   5 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |   3 +-
 .../accumulo/test/VolumeChooserFailureIT.java      |   3 +-
 .../org/apache/accumulo/test/VolumeChooserIT.java  |   5 +-
 24 files changed, 416 insertions(+), 420 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
index 0ada9eb..1ad7965 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationTypeHelper.java
@@ -16,8 +16,13 @@
  */
 package org.apache.accumulo.core.conf;
 
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -36,7 +41,7 @@ public class ConfigurationTypeHelper {
     char lastChar = str.charAt(str.length() - 1);
 
     if (lastChar == 'b') {
-      log.warn("The 'b' in " + str + " is being considered as bytes. " + "Setting memory by bits is not supported");
+      log.warn("The 'b' in {} is being considered as bytes. Setting memory by bits is not supported", str);
     }
     try {
       int multiplier;
@@ -136,4 +141,64 @@ public class ConfigurationTypeHelper {
     return Double.parseDouble(str);
   }
 
+  // This is not a cache for loaded classes, just a way to avoid spamming the debug log
+  private static Map<String,Class<?>> loaded = Collections.synchronizedMap(new HashMap<String,Class<?>>());
+
+  /**
+   * Loads a class in the given classloader context, suppressing any exceptions, and optionally providing a default instance to use.
+   *
+   * @param context
+   *          the per-table context, can be null
+   * @param clazzName
+   *          the name of the class to load
+   * @param base
+   *          the type of the class
+   * @param defaultInstance
+   *          a default instance if the class cannot be loaded
+   * @return a new instance of the class, or the defaultInstance
+   */
+  public static <T> T getClassInstance(String context, String clazzName, Class<T> base, T defaultInstance) {
+    T instance = null;
+
+    try {
+      instance = getClassInstance(context, clazzName, base);
+    } catch (RuntimeException | ClassNotFoundException | IOException | InstantiationException | IllegalAccessException e) {
+      log.warn("Failed to load class {}", clazzName, e);
+    }
+
+    if (instance == null && defaultInstance != null) {
+      log.info("Using default class {}", defaultInstance.getClass().getName());
+      instance = defaultInstance;
+    }
+    return instance;
+  }
+
+  /**
+   * Loads a class in the given classloader context.
+   *
+   * @param context
+   *          the per-table context, can be null
+   * @param clazzName
+   *          the name of the class to load
+   * @param base
+   *          the type of the class
+   * @return a new instance of the class
+   */
+  public static <T> T getClassInstance(String context, String clazzName, Class<T> base) throws ClassNotFoundException, IOException, InstantiationException,
+      IllegalAccessException {
+    T instance;
+
+    Class<? extends T> clazz;
+    if (context != null && !context.isEmpty()) {
+      clazz = AccumuloVFSClassLoader.getContextManager().loadClass(context, clazzName, base);
+    } else {
+      clazz = AccumuloVFSClassLoader.loadClass(clazzName, base);
+    }
+
+    instance = clazz.newInstance();
+    if (loaded.put(clazzName, clazz) != clazz)
+      log.debug("Loaded class : {}", clazzName);
+
+    return instance;
+  }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index d0dffa4..e08df9e 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.core.conf;
 
 import java.io.File;
 import java.lang.annotation.Annotation;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -632,7 +631,6 @@ public enum Property {
 
   private String key, defaultValue, description;
   private PropertyType type;
-  private static final Logger log = LoggerFactory.getLogger(Property.class);
 
   private Property(String name, String defaultValue, PropertyType type, String description) {
     this.key = name;
@@ -909,35 +907,6 @@ public enum Property {
         || key.equals(Property.TABLE_LOAD_BALANCER.getKey());
   }
 
-  // This is not a cache for loaded classes, just a way to avoid spamming the debug log
-  static Map<String,Class<?>> loaded = Collections.synchronizedMap(new HashMap<String,Class<?>>());
-
-  private static <T> T createInstance(String context, String clazzName, Class<T> base, T defaultInstance) {
-    T instance = null;
-
-    try {
-
-      Class<? extends T> clazz;
-      if (context != null && !context.equals("")) {
-        clazz = AccumuloVFSClassLoader.getContextManager().loadClass(context, clazzName, base);
-      } else {
-        clazz = AccumuloVFSClassLoader.loadClass(clazzName, base);
-      }
-
-      instance = clazz.newInstance();
-      if (loaded.put(clazzName, clazz) != clazz)
-        log.debug("Loaded class : {}", clazzName);
-    } catch (Exception e) {
-      log.warn("Failed to load class ", e);
-    }
-
-    if (instance == null) {
-      log.info("Using default class {}", defaultInstance.getClass().getName());
-      instance = defaultInstance;
-    }
-    return instance;
-  }
-
   /**
    * Creates a new instance of a class specified in a configuration property. The table classpath context is used if set.
    *
@@ -955,8 +924,7 @@ public enum Property {
   public static <T> T createTableInstanceFromPropertyName(AccumuloConfiguration conf, Property property, Class<T> base, T defaultInstance) {
     String clazzName = conf.get(property);
     String context = conf.get(TABLE_CLASSPATH);
-
-    return createInstance(context, clazzName, base, defaultInstance);
+    return ConfigurationTypeHelper.getClassInstance(context, clazzName, base, defaultInstance);
   }
 
   /**
@@ -975,8 +943,7 @@ public enum Property {
    */
   public static <T> T createInstanceFromPropertyName(AccumuloConfiguration conf, Property property, Class<T> base, T defaultInstance) {
     String clazzName = conf.get(property);
-
-    return createInstance(null, clazzName, base, defaultInstance);
+    return ConfigurationTypeHelper.getClassInstance(null, clazzName, base, defaultInstance);
   }
 
   /**
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 4f630b1..6eba22c 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,14 +16,17 @@
  */
 package org.apache.accumulo.server.fs;
 
+import java.io.IOException;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.accumulo.core.client.impl.Table;
 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.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,84 +36,90 @@ import org.slf4j.LoggerFactory;
  * general.custom.scoped.volume.chooser. At the this this was written, the only known scope was "logger".
  */
 public class PerTableVolumeChooser implements VolumeChooser {
+  // TODO rename this class to DelegatingChooser? It delegates for more than just per-table scope
   private static final Logger log = LoggerFactory.getLogger(PerTableVolumeChooser.class);
   // TODO Add hint of expected size to construction, see ACCUMULO-3410
   /* Track VolumeChooser instances so they can keep state. */
-  private final ConcurrentHashMap<String,VolumeChooser> tableSpecificChooser = new ConcurrentHashMap<>();
-  private final ConcurrentHashMap<String,VolumeChooser> scopeSpecificChooser = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<Table.ID,VolumeChooser> tableSpecificChooserCache = new ConcurrentHashMap<>();
+  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 serverConfs;
-
-  public static final String INIT_SCOPE = "init";
+  private volatile ServerConfigurationFactory lazyConfFactory = null;
 
   public static final String TABLE_VOLUME_CHOOSER = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "volume.chooser";
 
-  public static final String SCOPED_VOLUME_CHOOSER(String scope) {
-    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope + ".volume.chooser";
+  public static final String getPropertyNameForScope(ChooserScope scope) {
+    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope.name().toLowerCase() + ".volume.chooser";
   }
 
-  public static final String DEFAULT_SCOPED_VOLUME_CHOOSER = SCOPED_VOLUME_CHOOSER("scoped");
+  private static final String DEFAULT_SCOPED_VOLUME_CHOOSER = getPropertyNameForScope(ChooserScope.DEFAULT);
 
   @Override
-  public String choose(VolumeChooserEnvironment env, String[] options) {
-    log.trace("PerTableVolumeChooser.choose");
-
-    VolumeChooser chooser;
-    if (!env.hasTableId() && (!env.hasScope() || env.getScope().equals(INIT_SCOPE))) {
-      // Should only get here during Initialize. Configurations are not yet available.
-      return randomChooser.choose(env, options);
-    }
-
-    ServerConfigurationFactory localConf = loadConf();
-    if (env.hasTableId()) {
-      // use the table configuration
-      chooser = getVolumeChooserForTable(env, localConf);
-    } else {
-      // use the system configuration
-      chooser = getVolumeChooserForNonTable(env, localConf);
+  public String choose(VolumeChooserEnvironment env, String[] options) throws VolumeChooserException {
+    log.trace("{}.choose", getClass().getSimpleName());
+
+    VolumeChooser delegateChooser;
+    switch (env.getScope()) {
+      case INIT:
+        // TODO should be possible to read from SiteConfiguration during init
+        log.warn("Not possible to determine delegate chooser at '{}' scope. Using all volumes.", ChooserScope.INIT);
+        delegateChooser = randomChooser;
+        break;
+      case TABLE:
+        delegateChooser = getVolumeChooserForTable(env, loadConfFactory());
+        break;
+      default:
+        delegateChooser = getVolumeChooserForScope(env, loadConfFactory());
+        break;
     }
-
-    return chooser.choose(env, options);
+    return delegateChooser.choose(env, options);
   }
 
-  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
-    log.trace("Looking up property {} for Table id: {}", TABLE_VOLUME_CHOOSER, env.getTableId());
-    final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
+  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);
 
+    // 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 (null == clazz || clazz.isEmpty()) {
-      String msg = "Property " + TABLE_VOLUME_CHOOSER + " must be set" + (null == clazz ? " " : " properly ") + "to use the " + getClass().getSimpleName();
-      throw new RuntimeException(msg);
+      clazz = confFactory.getSystemConfiguration().get(DEFAULT_SCOPED_VOLUME_CHOOSER);
     }
 
-    return createVolumeChooser(clazz, TABLE_VOLUME_CHOOSER, env.getTableId().canonicalID(), tableSpecificChooser);
-  }
+    if (null == clazz || clazz.isEmpty()) {
+      String msg = "Property " + TABLE_VOLUME_CHOOSER + " or " + DEFAULT_SCOPED_VOLUME_CHOOSER + " must be a valid " + VolumeChooser.class.getSimpleName()
+          + " to use the " + getClass().getSimpleName();
+      throw new VolumeChooserException(msg);
+    }
 
-  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
-    String scope = env.getScope();
-    String property = SCOPED_VOLUME_CHOOSER(scope);
+    String context = tableConf.get(Property.TABLE_CLASSPATH); // can be null
+    return createVolumeChooser(context, clazz, TABLE_VOLUME_CHOOSER, env.getTableId(), tableSpecificChooserCache);
+  }
 
-    log.trace("Looking up property: {}", property);
+  private VolumeChooser getVolumeChooserForScope(VolumeChooserEnvironment env, ServerConfigurationFactory confFactory) {
+    ChooserScope scope = env.getScope();
+    String property = getPropertyNameForScope(scope);
+    log.trace("Looking up property {} for scope: {}", property, scope);
 
-    AccumuloConfiguration systemConfiguration = localConf.getSystemConfiguration();
+    AccumuloConfiguration systemConfiguration = confFactory.getSystemConfiguration();
     String clazz = systemConfiguration.get(property);
-    // only if the custom property is not set do we fallback to the default scope volume chooser setting
-    if (null == clazz) {
-      log.debug("Property not found: {} using {}", property, DEFAULT_SCOPED_VOLUME_CHOOSER);
+
+    // fall back to global default scope if this scope isn't configured (and not already default scope)
+    if ((null == clazz || clazz.isEmpty()) && scope != ChooserScope.DEFAULT) {
+      log.debug("{} not found; using {}", property, DEFAULT_SCOPED_VOLUME_CHOOSER);
       clazz = systemConfiguration.get(DEFAULT_SCOPED_VOLUME_CHOOSER);
 
       if (null == clazz || clazz.isEmpty()) {
-        String msg = "Property " + property + " or " + DEFAULT_SCOPED_VOLUME_CHOOSER + " must be set" + (null == clazz ? " " : " properly ") + "to use the "
-            + getClass().getSimpleName();
-        throw new RuntimeException(msg);
+        String msg = "Property " + property + " or " + DEFAULT_SCOPED_VOLUME_CHOOSER + " must be a valid " + VolumeChooser.class.getSimpleName()
+            + " to use the " + getClass().getSimpleName();
+        throw new VolumeChooserException(msg);
       }
 
       property = DEFAULT_SCOPED_VOLUME_CHOOSER;
     }
 
-    return createVolumeChooser(clazz, property, scope, scopeSpecificChooser);
+    String context = null;
+    return createVolumeChooser(context, clazz, property, scope, scopeSpecificChooserCache);
   }
 
   /**
@@ -126,47 +135,35 @@ public class PerTableVolumeChooser implements VolumeChooser {
    *          The cache
    * @return The volume chooser instance
    */
-  private VolumeChooser createVolumeChooser(String clazz, String property, String key, ConcurrentHashMap<String,VolumeChooser> cache) {
-    VolumeChooser chooser = cache.get(key);
-    // if we do not have a chooser or the class has changed, then create a new one
-    if (chooser == null || !(chooser.getClass().getName().equals(clazz))) {
-      if (chooser != null) {
-        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
-        log.trace("Change detected for {} for {}", property, key);
-      }
-      // create a new volume chooser instance
-      VolumeChooser temp;
-      try {
-        temp = loadClass(clazz);
-      } catch (Exception e) {
-        String msg = "Failed to create instance for " + key + " configured to use " + clazz + " via " + property;
-        throw new RuntimeException(msg, e);
-      }
-      if (chooser == null) {
-        // if we did not have one previously, then put this one in the cache
-        // but use the one already in the cache if another thread beat us here
-        chooser = cache.computeIfAbsent(key, k -> temp);
-      } else {
-        // otherwise the class has changed, so replace the one in the cache
-        // unless another thread beat us here
-        chooser = cache.computeIfPresent(key, (k, v) -> (v.getClass().getName().equals(clazz) ? v : temp));
+  private <T> VolumeChooser createVolumeChooser(String context, 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
+    return cache.compute(key, (k, previousChooser) -> {
+      if (previousChooser != null && previousChooser.getClass().getName().equals(className)) {
+        // no change; return the old one
+        return previousChooser;
       }
-    }
-    return chooser;
-  }
-
-  private VolumeChooser loadClass(String className) throws ClassNotFoundException, IllegalAccessException, InstantiationException {
-    // not attempting to load context because this approach to loading the class is for non-tables only
-    return AccumuloVFSClassLoader.loadClass(className, VolumeChooser.class).newInstance();
+      // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
+        if (previousChooser == null) {
+          // don't log change if this is the first use
+          log.trace("Change detected for {} for {}", property, key);
+        }
+        try {
+          return ConfigurationTypeHelper.getClassInstance(context, className, VolumeChooser.class);
+        } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | IOException e) {
+          String msg = "Failed to create instance for " + key + " configured to use " + className + " via " + property;
+          throw new VolumeChooserException(msg, e);
+        }
+      });
   }
 
-  private ServerConfigurationFactory loadConf() {
+  private ServerConfigurationFactory loadConfFactory() {
     // This local variable is an intentional component of the single-check idiom.
-    ServerConfigurationFactory localConf = serverConfs;
+    ServerConfigurationFactory localConf = lazyConfFactory;
     if (localConf == null) {
       // If we're under contention when first getting here we'll throw away some initializations.
       localConf = new ServerConfigurationFactory(HdfsZooInstance.getInstance());
-      serverConfs = localConf;
+      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 edd9931..8bf5f2a 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
@@ -16,15 +16,9 @@
  */
 package org.apache.accumulo.server.fs;
 
-import static org.apache.commons.lang.ArrayUtils.EMPTY_STRING_ARRAY;
-
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -32,7 +26,7 @@ import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.commons.collections.map.LRUMap;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -45,130 +39,115 @@ import org.slf4j.LoggerFactory;
 public class PreferredVolumeChooser extends RandomVolumeChooser {
   private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
 
-  public static final String INIT_SCOPE = "init";
-
   public static final String TABLE_PREFERRED_VOLUMES = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "preferred.volumes";
 
-  public static final String SCOPED_PREFERRED_VOLUMES(String scope) {
-    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope + ".preferred.volumes";
+  public static final String getPropertyNameForScope(ChooserScope scope) {
+    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope.name().toLowerCase() + ".preferred.volumes";
   }
 
-  public static final String DEFAULT_SCOPED_PREFERRED_VOLUMES = SCOPED_PREFERRED_VOLUMES("scoped");
+  private static final String DEFAULT_SCOPED_PREFERRED_VOLUMES = getPropertyNameForScope(ChooserScope.DEFAULT);
 
-  @SuppressWarnings("unchecked")
-  private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
   // TODO has to be lazily initialized currently because of the reliance on HdfsZooInstance. see ACCUMULO-3411
-  private volatile ServerConfigurationFactory serverConfs;
+  private volatile ServerConfigurationFactory lazyConfFactory = null;
 
   @Override
-  public String choose(VolumeChooserEnvironment env, String[] options) {
-    if (!env.hasTableId() && (!env.hasScope() || env.getScope().equals(INIT_SCOPE))) {
-      // this should only happen during initialize
-      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
-      return super.choose(env, options);
-    }
-
-    // get the volumes property
-    ServerConfigurationFactory localConf = loadConf();
-    List<String> volumes;
-    if (env.hasTableId()) {
-      volumes = getPreferredVolumesForTable(env, localConf, options);
-    } else {
-      volumes = getPreferredVolumesForNonTable(env, localConf, options);
+  public String choose(VolumeChooserEnvironment env, String[] options) throws VolumeChooserException {
+    log.trace("{}.choose", getClass().getSimpleName());
+
+    Set<String> preferredVolumes;
+    switch (env.getScope()) {
+      case INIT:
+        // TODO should be possible to read from SiteConfiguration during init
+        log.warn("Not possible to determine preferred volumes at '{}' scope. Using all volumes.", ChooserScope.INIT);
+        return super.choose(env, options);
+      case TABLE:
+        preferredVolumes = getPreferredVolumesForTable(env, loadConfFactory(), options);
+        break;
+      default:
+        preferredVolumes = getPreferredVolumesForScope(env, loadConfFactory(), options);
+        break;
     }
 
     // Randomly choose the volume from the preferred volumes
-    String choice = super.choose(env, volumes.toArray(EMPTY_STRING_ARRAY));
+    String choice = super.choose(env, preferredVolumes.toArray(new String[preferredVolumes.size()]));
     log.trace("Choice = {}", choice);
 
     return choice;
   }
 
-  private List<String> getPreferredVolumesForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf, String[] options) {
+  private Set<String> getPreferredVolumesForTable(VolumeChooserEnvironment env, ServerConfigurationFactory confFactory, String[] options) {
     log.trace("Looking up property {} + for Table id: {}", TABLE_PREFERRED_VOLUMES, env.getTableId());
 
-    final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
-    String volumes = tableConf.get(TABLE_PREFERRED_VOLUMES);
+    final TableConfiguration tableConf = confFactory.getTableConfiguration(env.getTableId());
+    String preferredVolumes = tableConf.get(TABLE_PREFERRED_VOLUMES);
+
+    // 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 (null == preferredVolumes || preferredVolumes.isEmpty()) {
+      preferredVolumes = confFactory.getSystemConfiguration().get(DEFAULT_SCOPED_PREFERRED_VOLUMES);
+    }
 
     // throw an error if volumes not specified or empty
-    if (null == volumes || volumes.isEmpty()) {
-      String logmsg = "Missing or empty " + TABLE_PREFERRED_VOLUMES + " property";
-      throw new RuntimeException(logmsg);
+    if (null == preferredVolumes || preferredVolumes.isEmpty()) {
+      String msg = "Property " + TABLE_PREFERRED_VOLUMES + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES + " must be a subset of " + Arrays.toString(options)
+          + " to use the " + getClass().getSimpleName();
+      throw new VolumeChooserException(msg);
     }
 
-    return getFilteredOptions(TABLE_PREFERRED_VOLUMES, volumes, options);
+    return parsePreferred(TABLE_PREFERRED_VOLUMES, preferredVolumes, options);
   }
 
-  private List<String> getPreferredVolumesForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf, String[] options) {
-    String scope = env.getScope();
-    String property = SCOPED_PREFERRED_VOLUMES(scope);
+  private Set<String> getPreferredVolumesForScope(VolumeChooserEnvironment env, ServerConfigurationFactory confFactory, String[] options) {
+    ChooserScope scope = env.getScope();
+    String property = getPropertyNameForScope(scope);
+    log.trace("Looking up property {} for scope: {}", property, scope);
 
-    log.trace("Looking up property: {}", property);
+    AccumuloConfiguration systemConfiguration = confFactory.getSystemConfiguration();
+    String preferredVolumes = systemConfiguration.get(property);
 
-    AccumuloConfiguration systemConfiguration = localConf.getSystemConfiguration();
-    String volumes = systemConfiguration.get(property);
+    // fall back to global default scope if this scope isn't configured (and not already default scope)
+    if ((null == preferredVolumes || preferredVolumes.isEmpty()) && scope != ChooserScope.DEFAULT) {
+      log.debug("{} not found; using {}", property, DEFAULT_SCOPED_PREFERRED_VOLUMES);
+      preferredVolumes = systemConfiguration.get(DEFAULT_SCOPED_PREFERRED_VOLUMES);
 
-    // only if the custom property is not set to we fallback to the default scoped preferred volumes
-    if (null == volumes) {
-      log.debug("Property not found: {} using {}", property, DEFAULT_SCOPED_PREFERRED_VOLUMES);
-      volumes = systemConfiguration.get(DEFAULT_SCOPED_PREFERRED_VOLUMES);
-
-      if (null == volumes || volumes.isEmpty()) {
-        String logmsg = "Missing or empty " + property + " and " + DEFAULT_SCOPED_PREFERRED_VOLUMES + " properties";
-        throw new RuntimeException(logmsg);
+      // only if the custom property is not set to we fall back to the default scoped preferred volumes
+      if (null == preferredVolumes || preferredVolumes.isEmpty()) {
+        String msg = "Property " + property + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES + " must be a subset of " + Arrays.toString(options) + " to use the "
+            + getClass().getSimpleName();
+        throw new VolumeChooserException(msg);
       }
 
       property = DEFAULT_SCOPED_PREFERRED_VOLUMES;
     }
 
-    return getFilteredOptions(property, volumes, options);
+    return parsePreferred(property, preferredVolumes, options);
   }
 
-  private List<String> getFilteredOptions(String property, String volumes, String[] options) {
-    log.trace("Found {} = {}", property, volumes);
-
-    ArrayList<String> filteredOptions = getIntersection(options, volumes);
+  private Set<String> parsePreferred(String property, String preferredVolumes, String[] options) {
+    log.trace("Found {} = {}", property, preferredVolumes);
 
-    // throw error if intersecting with preferred volumes resulted in the empty set
-    if (filteredOptions.isEmpty()) {
-      String logMessage = "After filtering preferred volumes, there are no valid instance volumes.";
-      log.error(logMessage);
-      throw new RuntimeException(logMessage);
+    Set<String> preferred = Arrays.stream(StringUtils.split(preferredVolumes, ',')).map(String::trim).collect(Collectors.toSet());
+    if (preferred.isEmpty()) {
+      String msg = "No volumes could be parsed from '" + property + "', which had a value of '" + preferredVolumes + "'";
+      throw new VolumeChooserException(msg);
     }
-
-    return filteredOptions;
-  }
-
-  private ArrayList<String> getIntersection(String[] options, String volumes) {
-    Set<String> preferred = parseVolumes(volumes);
-    return filterWithPreferred(options, preferred);
-  }
-
-  private ArrayList<String> filterWithPreferred(String[] options, Set<String> preferred) {
-    // Only keep the options that are in the preferred set
-    final ArrayList<String> filteredOptions = new ArrayList<>(Arrays.asList(options));
-    filteredOptions.retainAll(preferred);
-    return filteredOptions;
-  }
-
-  private Set<String> parseVolumes(String volumes) {
-    // If the preferred volumes property was specified, split the returned string by the comma and add use it to filter the given options.
-    Set<String> preferred = parsedPreferredVolumes.get(volumes);
-    if (preferred == null) {
-      preferred = new HashSet<>(Arrays.asList(StringUtils.split(volumes, ',')));
-      parsedPreferredVolumes.put(volumes, preferred);
+    // preferred volumes should also exist in the original options (typically, from instance.volumes)
+    Set<String> optionsList = Arrays.stream(options).collect(Collectors.toSet());
+    if (!preferred.stream().allMatch(optionsList::contains)) {
+      String msg = "Some volumes in " + preferred + " are not valid volumes from " + optionsList;
+      throw new VolumeChooserException(msg);
     }
+
     return preferred;
   }
 
-  private ServerConfigurationFactory loadConf() {
+  private ServerConfigurationFactory loadConfFactory() {
     // 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 = serverConfs;
+    ServerConfigurationFactory localConf = lazyConfFactory;
     if (localConf == null) {
       // If we're under contention when first getting here we'll throw away some initializations.
       localConf = new ServerConfigurationFactory(HdfsZooInstance.getInstance());
-      serverConfs = localConf;
+      lazyConfFactory = localConf;
     }
     return localConf;
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java
index 766d4c7..1e74704 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java
@@ -22,7 +22,7 @@ public class RandomVolumeChooser implements VolumeChooser {
   private final Random random = new Random();
 
   @Override
-  public String choose(VolumeChooserEnvironment env, String[] options) {
+  public String choose(VolumeChooserEnvironment env, String[] options) throws VolumeChooserException {
     return options[random.nextInt(options.length)];
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java
index e8c1049..41bcc8c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooser.java
@@ -28,5 +28,32 @@ import org.apache.accumulo.core.volume.Volume;
  * benefit from using per-table configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
  */
 public interface VolumeChooser {
-  String choose(VolumeChooserEnvironment env, String[] options);
+
+  /**
+   * Choose a volume from the provided options.
+   *
+   * @param env
+   *          the server environment provided by the calling framework
+   * @param options
+   *          the list of volumes to choose from
+   * @return one of the options
+   * @throws VolumeChooserException
+   *           if there is an error choosing (this is a RuntimeException); this does not preclude other RuntimeExceptions from occurring
+   */
+  String choose(VolumeChooserEnvironment env, String[] options) throws VolumeChooserException;
+
+  public static class VolumeChooserException extends RuntimeException {
+
+    private static final long serialVersionUID = 1L;
+
+    public VolumeChooserException(String message) {
+      super(message);
+    }
+
+    public VolumeChooserException(String message, Throwable cause) {
+      super(message, cause);
+    }
+
+  }
+
 }
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 0747284..cdf838a 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
@@ -16,38 +16,40 @@
  */
 package org.apache.accumulo.server.fs;
 
-import java.util.Optional;
+import java.util.Objects;
 
 import org.apache.accumulo.core.client.impl.Table;
 
 public class VolumeChooserEnvironment {
 
-  private final Optional<Table.ID> tableId;
-  // scope is meant for non-table identifiers
-  private String scope;
-
-  public VolumeChooserEnvironment(Optional<Table.ID> tableId) {
-    this.tableId = tableId;
+  /**
+   * 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
   }
 
-  public boolean hasTableId() {
-    return tableId.isPresent();
-  }
+  private final ChooserScope scope;
+  private final Table.ID tableId;
 
-  public Table.ID getTableId() {
-    return tableId.get();
+  public VolumeChooserEnvironment(ChooserScope scope) {
+    this.scope = Objects.requireNonNull(scope);
+    this.tableId = null;
   }
 
-  public void setScope(String scope) {
-    this.scope = scope;
+  public VolumeChooserEnvironment(Table.ID tableId) {
+    this.scope = ChooserScope.TABLE;
+    this.tableId = Objects.requireNonNull(tableId);
   }
 
-  public String getScope() {
-    return this.scope;
+  public Table.ID getTableId() {
+    return tableId;
   }
 
-  public boolean hasScope() {
-    return this.scope != null;
+  public ChooserScope getScope() {
+    return this.scope;
   }
 
   @Override
@@ -55,30 +57,15 @@ public class VolumeChooserEnvironment {
     if (obj == this) {
       return true;
     }
-    if (obj == null) {
-      return false;
-    }
-    if (!(obj instanceof VolumeChooserEnvironment)) {
+    if (obj == null || !(obj instanceof VolumeChooserEnvironment)) {
       return false;
     }
     VolumeChooserEnvironment other = (VolumeChooserEnvironment) obj;
-    if (other.hasTableId() != this.hasTableId()) {
-      return false;
-    }
-    if (!other.getTableId().equals(this.getTableId())) {
-      return false;
-    }
-    if (other.hasScope() != this.hasScope()) {
-      return false;
-    }
-    if (other.hasScope() && !other.getScope().equals(this.getScope())) {
-      return false;
-    }
-    return true;
+    return getScope() == other.getScope() && Objects.equals(getTableId(), other.getTableId());
   }
 
   @Override
   public int hashCode() {
-    return tableId.hashCode() * 31 + (this.scope == null ? 17 : this.scope.hashCode());
+    return Objects.hashCode(scope) * 31 + Objects.hashCode(tableId);
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index 04e86fc..b7892ce 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -40,6 +40,7 @@ import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.volume.NonConfiguredVolume;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
+import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.lang.StringUtils;
@@ -88,7 +89,7 @@ public class VolumeManagerImpl implements VolumeManager {
       // null chooser handled below
     }
     if (chooser1 == null) {
-      throw new RuntimeException("Failed to load volume chooser specified by " + Property.GENERAL_VOLUME_CHOOSER);
+      throw new VolumeChooserException("Failed to load volume chooser specified by " + Property.GENERAL_VOLUME_CHOOSER);
     }
     chooser = chooser1;
   }
@@ -480,9 +481,8 @@ public class VolumeManagerImpl implements VolumeManager {
     choice = chooser.choose(env, options);
 
     if (!(ArrayUtils.contains(options, choice))) {
-      // we may want to go with random if they chooser was not overridden
       String msg = "The configured volume chooser, '" + chooser.getClass() + "', or one of its delegates returned a volume not in the set of options provided";
-      throw new RuntimeException(msg);
+      throw new VolumeChooserException(msg);
     }
 
     return choice;
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 d009d1e..ab48edf 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
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map.Entry;
-import java.util.Optional;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
@@ -262,7 +261,7 @@ public class VolumeUtil {
       throw new IllegalArgumentException("Unexpected table dir " + dir);
     }
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(extent.getTableId()));
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(extent.getTableId());
     Path newDir = new Path(vm.choose(chooserEnv, ServerConstants.getBaseUris()) + Path.SEPARATOR + ServerConstants.TABLE_DIR + Path.SEPARATOR
         + dir.getParent().getName() + Path.SEPARATOR + dir.getName());
 
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 50f901e..0abe4c8 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
@@ -30,7 +30,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Locale;
 import java.util.Map.Entry;
-import java.util.Optional;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.UUID;
@@ -89,6 +88,7 @@ import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 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.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.iterators.MetadataBulkLoadFilter;
@@ -324,7 +324,7 @@ public class Initialize implements KeywordExecutable {
     UUID uuid = UUID.randomUUID();
     // the actual disk locations of the root table and tablets
     String[] configuredVolumes = VolumeConfiguration.getVolumeUris(SiteConfiguration.getInstance());
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.empty());
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ChooserScope.INIT);
     final String rootTabletDir = new Path(fs.choose(chooserEnv, configuredVolumes) + Path.SEPARATOR + ServerConstants.TABLE_DIR + Path.SEPARATOR + RootTable.ID
         + RootTable.ROOT_TABLET_LOCATION).toString();
 
@@ -425,7 +425,7 @@ public class Initialize implements KeywordExecutable {
     // initialize initial system tables config in zookeeper
     initSystemTablesConfig();
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.empty());
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ChooserScope.INIT);
     String tableMetadataTabletDir = fs.choose(chooserEnv, ServerConstants.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + MetadataTable.ID
         + TABLE_TABLETS_TABLET_DIR;
     String replicationTableDefaultTabletDir = fs.choose(chooserEnv, ServerConstants.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
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 a9c3287..7e4da6c 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
@@ -32,7 +32,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -877,7 +876,7 @@ public class MetadataTableUtil {
         Key k = entry.getKey();
         Mutation m = new Mutation(k.getRow());
         m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
-        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(tableId));
+        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableId);
         String dir = volumeManager.choose(chooserEnv, ServerConstants.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + tableId + Path.SEPARATOR
             + new String(FastFormat.toZeroPaddedString(dirCount++, 8, 16, Constants.CLONE_PREFIX_BYTES));
         TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(dir.getBytes(UTF_8)));
@@ -976,7 +975,7 @@ public class MetadataTableUtil {
    */
   public static void createReplicationTable(ClientContext context) throws IOException {
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(ReplicationTable.ID));
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ReplicationTable.ID);
     String dir = VolumeManagerImpl.get().choose(chooserEnv, ServerConstants.getBaseUris()) + 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 1ee74b4..29b13ec 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
@@ -21,7 +21,6 @@ import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSec
 
 import java.io.IOException;
 import java.util.Map.Entry;
-import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.cli.ClientOnRequiredTable;
@@ -117,7 +116,7 @@ public class RandomizeVolumes {
       Key key = entry.getKey();
       Mutation m = new Mutation(key.getRow());
 
-      VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(tableId));
+      VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableId);
       final String newLocation = vm.choose(chooserEnv, ServerConstants.getBaseUris()) + Path.SEPARATOR + ServerConstants.TABLE_DIR + Path.SEPARATOR + tableId
           + Path.SEPARATOR + directory;
       m.put(key.getColumnFamily(), key.getColumnQualifier(), new Value(newLocation.getBytes(UTF_8)));
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 f8946c4..3d1a3f1 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
@@ -19,7 +19,6 @@ package org.apache.accumulo.server.fs;
 import java.lang.reflect.Field;
 import java.util.Arrays;
 import java.util.HashSet;
-import java.util.Optional;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.impl.Table;
@@ -27,7 +26,10 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.easymock.EasyMock;
+import org.easymock.IExpectationSetters;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -53,40 +55,35 @@ public class PerTableVolumeChooserTest {
     perTableVolumeChooser = new PerTableVolumeChooser();
 
     mockedServerConfigurationFactory = EasyMock.createMock(ServerConfigurationFactory.class);
-    Field field = perTableVolumeChooser.getClass().getDeclaredField("serverConfs");
+    Field field = perTableVolumeChooser.getClass().getDeclaredField("lazyConfFactory");
     field.setAccessible(true);
     field.set(perTableVolumeChooser, mockedServerConfigurationFactory);
 
     mockedTableConfiguration = EasyMock.createMock(TableConfiguration.class);
     mockedAccumuloConfiguration = EasyMock.createMock(AccumuloConfiguration.class);
-  }
-
-  private void configureDefaultVolumeChooser(String className) {
+    EasyMock.expect(mockedServerConfigurationFactory.getTableConfiguration(EasyMock.<Table.ID> anyObject())).andReturn(mockedTableConfiguration).anyTimes();
     EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
-    EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(className).anyTimes();
+    EasyMock.expect(mockedTableConfiguration.get(Property.TABLE_CLASSPATH)).andReturn(null).anyTimes();
   }
 
-  private void configureScopedVolumeChooser(String className, String scope) {
-    EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
-    EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.SCOPED_VOLUME_CHOOSER(scope))).andReturn(className).anyTimes();
+  private IExpectationSetters<String> expectDefaultScope(String className) {
+    return expectScope(ChooserScope.DEFAULT, className);
   }
 
-  private void configureChooserForTable(String className) {
-    EasyMock.expect(mockedServerConfigurationFactory.getTableConfiguration(EasyMock.<Table.ID> anyObject())).andReturn(mockedTableConfiguration).anyTimes();
-    EasyMock.expect(mockedTableConfiguration.get(Property.TABLE_CLASSPATH)).andReturn(null).anyTimes();
-    EasyMock.expect(mockedTableConfiguration.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(className).anyTimes();
+  private IExpectationSetters<String> expectLoggerScope(String className) {
+    return expectScope(ChooserScope.LOGGER, className);
   }
 
-  private void configureDefaultContextVolumeChooser(String className) {
-    EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.DEFAULT_SCOPED_VOLUME_CHOOSER)).andReturn(className).anyTimes();
+  private IExpectationSetters<String> expectScope(ChooserScope scope, String className) {
+    return EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.getPropertyNameForScope(scope))).andReturn(className);
   }
 
-  private void configureContextVolumeChooser(String className) {
-    EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.SCOPED_VOLUME_CHOOSER("logger"))).andReturn(className).anyTimes();
+  private IExpectationSetters<String> expectTableChooser(String className) {
+    return EasyMock.expect(mockedTableConfiguration.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(className);
   }
 
   private Set<String> chooseRepeatedlyForTable() {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.of(Table.ID.of("h")));
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Table.ID.of("h"));
     Set<String> results = new HashSet<>();
     for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
       results.add(perTableVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
@@ -130,9 +127,12 @@ public class PerTableVolumeChooserTest {
     }
   }
 
-  private Set<String> chooseRepeatedlyForContext() {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.empty());
-    volumeChooserEnvironment.setScope("logger");
+  private Set<String> chooseRepeatedlyForLogger() {
+    return chooseRepeatedlyForScope(ChooserScope.LOGGER);
+  }
+
+  private Set<String> chooseRepeatedlyForScope(ChooserScope scope) {
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(scope);
     Set<String> results = new HashSet<>();
 
     for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
@@ -142,8 +142,8 @@ public class PerTableVolumeChooserTest {
   }
 
   @Test
-  public void testEmptyEnvUsesRandomChooser() throws Exception {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.empty());
+  public void testInitScope() throws Exception {
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(ChooserScope.INIT);
 
     Set<String> results = new HashSet<>();
     for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
@@ -155,8 +155,7 @@ public class PerTableVolumeChooserTest {
 
   @Test
   public void testTableConfig() throws Exception {
-    configureDefaultVolumeChooser(VolumeChooserAlwaysOne.class.getName());
-    configureChooserForTable(VolumeChooserAlwaysTwo.class.getName());
+    expectTableChooser(VolumeChooserAlwaysTwo.class.getName()).atLeastOnce();
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
@@ -168,126 +167,114 @@ public class PerTableVolumeChooserTest {
 
   @Test
   public void testTableMisconfigured() throws Exception {
-    configureDefaultVolumeChooser(VolumeChooserAlwaysOne.class.getName());
-    configureChooserForTable(INVALID_CHOOSER_CLASSNAME);
+    expectDefaultScope(VolumeChooserAlwaysOne.class.getName());
+    expectTableChooser(INVALID_CHOOSER_CLASSNAME);
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
   public void testTableMissing() throws Exception {
-    configureDefaultVolumeChooser(VolumeChooserAlwaysOne.class.getName());
-    configureChooserForTable(null);
+    expectDefaultScope(null);
+    expectTableChooser(null);
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
   public void testTableMisconfiguredAndDefaultEmpty() throws Exception {
-    configureDefaultVolumeChooser("");
-    configureChooserForTable(INVALID_CHOOSER_CLASSNAME);
+    expectDefaultScope("");
+    expectTableChooser(INVALID_CHOOSER_CLASSNAME);
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
   public void testTableEmptyConfig() throws Exception {
-    configureDefaultVolumeChooser(VolumeChooserAlwaysThree.class.getName());
-    configureChooserForTable("");
+    expectDefaultScope("");
+    expectTableChooser("");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
   public void testTableAndDefaultEmpty() throws Exception {
-    configureDefaultVolumeChooser("");
-    configureChooserForTable("");
+    expectDefaultScope("");
+    expectTableChooser("");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
-  public void testContextConfig() throws Exception {
-    configureDefaultVolumeChooser(VolumeChooserAlwaysThree.class.getName());
-    configureContextVolumeChooser(VolumeChooserAlwaysOne.class.getName());
+  public void testScopeConfig() throws Exception {
+    expectLoggerScope(VolumeChooserAlwaysOne.class.getName()).atLeastOnce();
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    Set<String> results = chooseRepeatedlyForContext();
+    Set<String> results = chooseRepeatedlyForLogger();
 
     EasyMock.verify(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
     Assert.assertEquals(Sets.newHashSet(Arrays.asList("1")), results);
   }
 
   @Test
-  public void testContextMisconfigured() throws Exception {
-    configureDefaultVolumeChooser(VolumeChooserAlwaysThree.class.getName());
-    configureContextVolumeChooser(INVALID_CHOOSER_CLASSNAME);
+  public void testScopeMisconfigured() throws Exception {
+    expectDefaultScope(VolumeChooserAlwaysThree.class.getName());
+    expectLoggerScope(INVALID_CHOOSER_CLASSNAME);
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
+    thrown.expect(VolumeChooserException.class);
+    chooseRepeatedlyForLogger();
   }
 
   @Test
-  public void testContextMissing() throws Exception {
-    configureDefaultVolumeChooser(VolumeChooserAlwaysTwo.class.getName());
-    configureContextVolumeChooser(null);
-    configureDefaultContextVolumeChooser(null);
+  public void testScopeMissing() throws Exception {
+    expectLoggerScope(null);
+    expectDefaultScope(null);
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
+    thrown.expect(VolumeChooserException.class);
+    chooseRepeatedlyForLogger();
   }
 
   @Test
-  public void testContextMisconfiguredAndDefaultEmpty() throws Exception {
-    configureDefaultVolumeChooser("");
-    configureChooserForTable("");
-    configureContextVolumeChooser(INVALID_CHOOSER_CLASSNAME);
+  public void testScopeMisconfiguredAndDefaultEmpty() throws Exception {
+    expectDefaultScope("");
+    expectTableChooser("");
+    expectLoggerScope(INVALID_CHOOSER_CLASSNAME);
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
+    thrown.expect(VolumeChooserException.class);
+    chooseRepeatedlyForLogger();
   }
 
   @Test
-  public void testContextAndDefaultBothEmpty() throws Exception {
-    this.configureDefaultVolumeChooser("");
-    configureContextVolumeChooser("");
+  public void testScopeAndDefaultBothEmpty() throws Exception {
+    expectDefaultScope("");
+    expectLoggerScope("");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
+    thrown.expect(VolumeChooserException.class);
+    chooseRepeatedlyForLogger();
   }
 
-  @Test
-  public void testContextEmptyConfig() throws Exception {
-    configureDefaultVolumeChooser(VolumeChooserAlwaysTwo.class.getName());
-    configureContextVolumeChooser("");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
-
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
-  }
 }
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 fa3f672..440c628 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
@@ -19,14 +19,16 @@ package org.apache.accumulo.server.fs;
 import java.lang.reflect.Field;
 import java.util.Arrays;
 import java.util.HashSet;
-import java.util.Optional;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.easymock.EasyMock;
+import org.easymock.IExpectationSetters;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
@@ -51,39 +53,34 @@ public class PreferredVolumeChooserTest {
     preferredVolumeChooser = new PreferredVolumeChooser();
 
     mockedServerConfigurationFactory = EasyMock.createMock(ServerConfigurationFactory.class);
-    Field field = preferredVolumeChooser.getClass().getDeclaredField("serverConfs");
+    Field field = preferredVolumeChooser.getClass().getDeclaredField("lazyConfFactory");
     field.setAccessible(true);
     field.set(preferredVolumeChooser, mockedServerConfigurationFactory);
 
     mockedTableConfiguration = EasyMock.createMock(TableConfiguration.class);
     mockedAccumuloConfiguration = EasyMock.createMock(AccumuloConfiguration.class);
-  }
-
-  private void configureDefaultVolumes(String configuredVolumes) {
+    EasyMock.expect(mockedServerConfigurationFactory.getTableConfiguration(EasyMock.<Table.ID> anyObject())).andReturn(mockedTableConfiguration).anyTimes();
     EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
-    EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(configuredVolumes).anyTimes();
   }
 
-  private void configureTableVolumes(String configuredVolumes) {
-    EasyMock.expect(mockedServerConfigurationFactory.getTableConfiguration(EasyMock.<Table.ID> anyObject())).andReturn(mockedTableConfiguration).anyTimes();
-    EasyMock.expect(mockedTableConfiguration.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(configuredVolumes).anyTimes();
+  private IExpectationSetters<String> expectTableVolumes(String configuredVolumes) {
+    return EasyMock.expect(mockedTableConfiguration.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(configuredVolumes);
   }
 
-  private void configureScopedVolumes(String configuredVolumes, String scope) {
-    EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
-    EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.SCOPED_PREFERRED_VOLUMES(scope))).andReturn(configuredVolumes).anyTimes();
+  private IExpectationSetters<String> expectDefaultScope(String configuredVolumes) {
+    return expectScope(ChooserScope.DEFAULT, configuredVolumes);
   }
 
-  private void configureDefaultContextVolumes(String configuredVolumes) {
-    EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.DEFAULT_SCOPED_PREFERRED_VOLUMES)).andReturn(configuredVolumes).anyTimes();
+  private IExpectationSetters<String> expectLoggerScope(String configuredVolumes) {
+    return expectScope(ChooserScope.LOGGER, configuredVolumes);
   }
 
-  private void configureContextVolumes(String configuredVolumes) {
-    EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.SCOPED_PREFERRED_VOLUMES("logger"))).andReturn(configuredVolumes).anyTimes();
+  private IExpectationSetters<String> expectScope(ChooserScope scope, String configuredVolumes) {
+    return EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.getPropertyNameForScope(scope))).andReturn(configuredVolumes);
   }
 
   private Set<String> chooseRepeatedlyForTable() {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.of(Table.ID.of("h")));
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Table.ID.of("h"));
     Set<String> results = new HashSet<>();
     for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
       results.add(preferredVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
@@ -91,9 +88,12 @@ public class PreferredVolumeChooserTest {
     return results;
   }
 
-  private Set<String> chooseRepeatedlyForContext() {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.empty());
-    volumeChooserEnvironment.setScope("logger");
+  private Set<String> chooseRepeatedlyForLogger() {
+    return chooseRepeatedlyForScope(ChooserScope.LOGGER);
+  }
+
+  private Set<String> chooseRepeatedlyForScope(ChooserScope scope) {
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(scope);
     Set<String> results = new HashSet<>();
 
     for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
@@ -104,7 +104,7 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testEmptyEnvUsesRandomChooser() throws Exception {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.empty());
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(ChooserScope.INIT);
     Set<String> results = new HashSet<>();
     for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
       results.add(preferredVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
@@ -115,8 +115,8 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testTableConfig() throws Exception {
-    configureDefaultVolumes("1,3");
-    configureTableVolumes("1,2");
+    expectDefaultScope(null).anyTimes();
+    expectTableVolumes("1,2").atLeastOnce();
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
@@ -128,125 +128,123 @@ public class PreferredVolumeChooserTest {
 
   @Test
   public void testTableMisconfigured() throws Exception {
-    configureDefaultVolumes("1,3");
-    configureTableVolumes("4");
+    expectDefaultScope("1,3");
+    expectTableVolumes("4");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
   public void testTableMissing() throws Exception {
-    configureDefaultVolumes("1,3");
-    configureTableVolumes(null);
+    expectDefaultScope("");
+    expectTableVolumes(null);
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
   public void testTableEmptyConfig() throws Exception {
-    configureDefaultVolumes("1,3");
-    configureTableVolumes("");
+    expectDefaultScope(null);
+    expectTableVolumes("");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
   public void testTableMisconfiguredAndDefaultEmpty() throws Exception {
-    configureDefaultVolumes("");
-    configureTableVolumes("4");
+    expectDefaultScope("");
+    expectTableVolumes("4");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
   public void testTableAndDefaultEmpty() throws Exception {
-    configureDefaultVolumes("");
-    configureTableVolumes("");
+    expectDefaultScope("");
+    expectTableVolumes("");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
+    thrown.expect(VolumeChooserException.class);
     chooseRepeatedlyForTable();
   }
 
   @Test
-  public void testContextConfig() throws Exception {
-    configureDefaultVolumes("1,3");
-    configureContextVolumes("1,2");
+  public void testScopeConfig() throws Exception {
+    expectLoggerScope("1,2").atLeastOnce();
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    Set<String> results = chooseRepeatedlyForContext();
+    Set<String> results = chooseRepeatedlyForLogger();
 
     EasyMock.verify(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
     Assert.assertEquals(Sets.newHashSet(Arrays.asList("1", "2")), results);
   }
 
   @Test
-  public void testContextMisconfigured() throws Exception {
-    configureDefaultVolumes("1,3");
-    configureContextVolumes("4");
+  public void testScopeMisconfigured() throws Exception {
+    expectDefaultScope("1,3");
+    expectLoggerScope("4");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
+    thrown.expect(VolumeChooserException.class);
+    chooseRepeatedlyForLogger();
   }
 
   @Test
-  public void testContextMissing() throws Exception {
-    configureDefaultVolumes("1,3");
-    configureContextVolumes(null);
-    configureDefaultContextVolumes(null);
+  public void testScopeMissing() throws Exception {
+    expectDefaultScope("").atLeastOnce();
+    expectLoggerScope(null).atLeastOnce();
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
+    thrown.expect(VolumeChooserException.class);
+    chooseRepeatedlyForLogger();
   }
 
   @Test
-  public void testContextMisconfiguredAndDefaultEmpty() throws Exception {
-    configureDefaultVolumes("");
-    configureContextVolumes("4");
+  public void testScopeMisconfiguredAndDefaultEmpty() throws Exception {
+    expectDefaultScope("");
+    expectLoggerScope("4");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
+    thrown.expect(VolumeChooserException.class);
+    chooseRepeatedlyForLogger();
   }
 
   @Test
-  public void testContextAndDefaultBothEmpty() throws Exception {
-    this.configureDefaultVolumes("");
-    configureContextVolumes("");
+  public void testScopeAndDefaultBothEmpty() throws Exception {
+    expectDefaultScope("");
+    expectLoggerScope("");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
+    thrown.expect(VolumeChooserException.class);
+    chooseRepeatedlyForLogger();
   }
 
   @Test
-  public void testContextEmptyConfig() throws Exception {
-    configureDefaultVolumes("1,3");
-    configureContextVolumes("");
+  public void testScopeEmptyConfig() throws Exception {
+    expectDefaultScope("");
+    expectLoggerScope("");
 
     EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
 
-    thrown.expect(RuntimeException.class);
-    chooseRepeatedlyForContext();
+    thrown.expect(VolumeChooserException.class);
+    chooseRepeatedlyForLogger();
   }
 }
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 328f8f2..40467a3 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
@@ -18,7 +18,6 @@ package org.apache.accumulo.server.fs;
 
 import java.util.Arrays;
 import java.util.List;
-import java.util.Optional;
 
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
@@ -136,7 +135,7 @@ public class VolumeManagerImplTest {
     conf.set(Property.GENERAL_VOLUME_CHOOSER, WrongVolumeChooser.class.getName());
     thrown.expect(RuntimeException.class);
     VolumeManager vm = VolumeManagerImpl.get(conf);
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(Table.ID.of("sometable")));
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Table.ID.of("sometable"));
     String choice = vm.choose(chooserEnv, volumes.toArray(new String[0]));
     Assert.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 900081b..180acee 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
@@ -29,7 +29,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -283,7 +282,7 @@ public class Master extends AccumuloServerContext implements LiveTServerSet.List
     if (!zoo.exists(dirZPath)) {
       Path oldPath = fs.getFullPath(FileType.TABLE, "/" + MetadataTable.ID + "/root_tablet");
       if (fs.exists(oldPath)) {
-        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(RootTable.ID));
+        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(RootTable.ID);
         String newPath = fs.choose(chooserEnv, ServerConstants.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + RootTable.ID;
         fs.mkdirs(new Path(newPath));
         if (!fs.rename(oldPath, new Path(newPath))) {
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 a24e4f7..d6a13d9 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
@@ -27,7 +27,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
@@ -644,7 +643,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(Optional.of(extent.getTableId()));
+        VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(extent.getTableId());
         String tdir = master.getFileSystem().choose(chooserEnv, ServerConstants.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR
             + extent.getTableId() + Constants.DEFAULT_TABLET_LOCATION;
         MetadataTableUtil.addTablet(new KeyExtent(extent.getTableId(), null, extent.getPrevEndRow()), tdir, master, timeType, this.master.masterLock);
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChooseDir.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChooseDir.java
index 0a33bb3..3b8c5f5 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChooseDir.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChooseDir.java
@@ -16,8 +16,6 @@
  */
 package org.apache.accumulo.master.tableOps;
 
-import java.util.Optional;
-
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
@@ -42,7 +40,7 @@ class ChooseDir extends MasterRepo {
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
     // Constants.DEFAULT_TABLET_LOCATION has a leading slash prepended to it so we don't need to add one here
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(tableInfo.tableId));
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableInfo.tableId);
     tableInfo.dir = master.getFileSystem().choose(chooserEnv, ServerConstants.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + tableInfo.tableId
         + Constants.DEFAULT_TABLET_LOCATION;
     return new CreateDir(tableInfo);
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
index cfd61fe..896ac38 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Optional;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipInputStream;
 
@@ -207,7 +206,7 @@ class PopulateMetadataTable extends MasterRepo {
    */
   protected String getClonedTabletDir(Master master, String[] tableDirs, String tabletDir) {
     // We can try to spread out the tablet dirs across all volumes
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(tableInfo.tableId));
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableInfo.tableId);
     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/ImportTableTest.java b/server/master/src/test/java/org/apache/accumulo/master/tableOps/ImportTableTest.java
index 3ff5604..b1db1c3 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/tableOps/ImportTableTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/tableOps/ImportTableTest.java
@@ -16,8 +16,6 @@
  */
 package org.apache.accumulo.master.tableOps;
 
-import java.util.Optional;
-
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
@@ -45,7 +43,7 @@ public class ImportTableTest {
 
     EasyMock.expect(master.getFileSystem()).andReturn(volumeManager);
     // Choose the 2nd element
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(iti.tableId));
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(iti.tableId);
     EasyMock.expect(volumeManager.choose(EasyMock.eq(chooserEnv), EasyMock.eq(tableDirs))).andReturn(tableDirs[1]);
 
     EasyMock.replay(master, volumeManager);
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 1e20ed1..d336c3c 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
@@ -36,7 +36,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -57,6 +56,7 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.fate.util.LoggingRunnable;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.logger.LogFileKey;
@@ -439,8 +439,7 @@ public class DfsLogger implements Comparable<DfsLogger> {
     log.debug("DfsLogger.open() begin");
     VolumeManager fs = conf.getFileSystem();
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.empty());
-    chooserEnv.setScope("logger");
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(ChooserScope.LOGGER);
     logPath = fs.choose(chooserEnv, ServerConstants.getBaseUris()) + Path.SEPARATOR + ServerConstants.WAL_DIR + Path.SEPARATOR + logger + Path.SEPARATOR
         + filename;
 
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 e904cdd..fc496dd 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
@@ -35,7 +35,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Optional;
 import java.util.PriorityQueue;
 import java.util.Set;
 import java.util.SortedMap;
@@ -2668,7 +2667,7 @@ public class Tablet implements TabletCommitter {
     String lowDirectory;
 
     UniqueNameAllocator namer = UniqueNameAllocator.getInstance();
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(tableId));
+    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(tableId);
     String volume = fs.choose(chooserEnv, ServerConstants.getBaseUris()) + Constants.HDFS_TABLES_DIR + Path.SEPARATOR;
 
     while (true) {
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 dc0cb41..2d6236d 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.fs.PerTableVolumeChooser;
 import org.apache.accumulo.server.fs.PreferredVolumeChooser;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -80,7 +81,7 @@ public class VolumeChooserFailureIT extends ConfigurableMacBase {
 
     cfg.setSiteConfig(siteConfig);
 
-    siteConfig.put(PerTableVolumeChooser.SCOPED_VOLUME_CHOOSER("logger"), PreferredVolumeChooser.class.getName());
+    siteConfig.put(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER), PreferredVolumeChooser.class.getName());
     // do not set preferred volumes
     cfg.setSiteConfig(siteConfig);
 
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 b894681..2b72ed8 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.fs.PerTableVolumeChooser;
 import org.apache.accumulo.server.fs.PreferredVolumeChooser;
 import org.apache.accumulo.server.fs.RandomVolumeChooser;
+import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.apache.accumulo.test.functional.ConfigurableMacBase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -105,8 +106,8 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     siteConfig.put(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES, systemPreferredVolumes); // exclude v4
     cfg.setSiteConfig(siteConfig);
 
-    siteConfig.put(PerTableVolumeChooser.SCOPED_VOLUME_CHOOSER("logger"), PreferredVolumeChooser.class.getName());
-    siteConfig.put(PreferredVolumeChooser.SCOPED_PREFERRED_VOLUMES("logger"), v2.toString());
+    siteConfig.put(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER), PreferredVolumeChooser.class.getName());
+    siteConfig.put(PreferredVolumeChooser.getPropertyNameForScope(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 the options list when they are choosing

-- 
To stop receiving notification emails like this one, please contact
"commits@accumulo.apache.org" <co...@accumulo.apache.org>.

[accumulo] 03/03: ACCUMULO-4086 Make chooser classes more testable

Posted by ct...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 21fb47733416ec8690e364420c41d2e7b731cda7
Author: Christopher Tubbs <ct...@apache.org>
AuthorDate: Mon Sep 11 21:48:41 2017 -0400

    ACCUMULO-4086 Make chooser classes more testable
    
    Improve the testability of PerTableVolumeChooser and
    PreferredVolumeChooser, and dramatically simplify their unit tests.
    
    PerTableVolumeChooser.choose becomes:
      getDelegate(env).choose(env, options);
    And PerTableVolumeChooserTest tests getDelegate(env)
    
    PreferredVolumeChooser.choose becomes:
      super.choose(env, getPreferredVolumes(env, options));
    And PreferredVolumesChooserTest tests getPreferredVolumes(env, options)
---
 .../accumulo/server/fs/PerTableVolumeChooser.java  |  50 ++--
 .../accumulo/server/fs/PreferredVolumeChooser.java |  33 ++-
 .../server/fs/PerTableVolumeChooserTest.java       | 281 +++++++--------------
 .../server/fs/PreferredVolumeChooserTest.java      | 251 ++++++++----------
 4 files changed, 233 insertions(+), 382 deletions(-)

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 6eba22c..ab642d6 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
@@ -58,22 +58,21 @@ public class PerTableVolumeChooser implements VolumeChooser {
   @Override
   public String choose(VolumeChooserEnvironment env, String[] options) throws VolumeChooserException {
     log.trace("{}.choose", getClass().getSimpleName());
+    return getDelegateChooser(env).choose(env, options);
+  }
 
-    VolumeChooser delegateChooser;
+  // visible (not private) for testing
+  VolumeChooser getDelegateChooser(VolumeChooserEnvironment env) {
     switch (env.getScope()) {
       case INIT:
         // TODO should be possible to read from SiteConfiguration during init
-        log.warn("Not possible to determine delegate chooser at '{}' scope. Using all volumes.", ChooserScope.INIT);
-        delegateChooser = randomChooser;
-        break;
+        log.warn("Not possible to determine delegate chooser at '{}' scope. Using {}.", ChooserScope.INIT, RandomVolumeChooser.class.getName());
+        return randomChooser;
       case TABLE:
-        delegateChooser = getVolumeChooserForTable(env, loadConfFactory());
-        break;
+        return getVolumeChooserForTable(env, loadConfFactory());
       default:
-        delegateChooser = getVolumeChooserForScope(env, loadConfFactory());
-        break;
+        return getVolumeChooserForScope(env, loadConfFactory());
     }
-    return delegateChooser.choose(env, options);
   }
 
   private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory confFactory) {
@@ -92,10 +91,15 @@ public class PerTableVolumeChooser implements VolumeChooser {
       throw new VolumeChooserException(msg);
     }
 
-    String context = tableConf.get(Property.TABLE_CLASSPATH); // can be null
+    String context = getTableContext(tableConf); // can be null
     return createVolumeChooser(context, clazz, TABLE_VOLUME_CHOOSER, 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) {
     ChooserScope scope = env.getScope();
     String property = getPropertyNameForScope(scope);
@@ -142,22 +146,22 @@ public class PerTableVolumeChooser implements VolumeChooser {
       if (previousChooser != null && previousChooser.getClass().getName().equals(className)) {
         // no change; return the old one
         return previousChooser;
+      } else if (previousChooser == null) {
+        // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
+        // don't log change if this is the first use
+        log.trace("Change detected for {} for {}", property, key);
+      }
+      try {
+        return ConfigurationTypeHelper.getClassInstance(context, className, VolumeChooser.class);
+      } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | IOException e) {
+        String msg = "Failed to create instance for " + key + " configured to use " + className + " via " + property;
+        throw new VolumeChooserException(msg, e);
       }
-      // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
-        if (previousChooser == null) {
-          // don't log change if this is the first use
-          log.trace("Change detected for {} for {}", property, key);
-        }
-        try {
-          return ConfigurationTypeHelper.getClassInstance(context, className, VolumeChooser.class);
-        } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | IOException e) {
-          String msg = "Failed to create instance for " + key + " configured to use " + className + " via " + property;
-          throw new VolumeChooserException(msg, e);
-        }
-      });
+    });
   }
 
-  private ServerConfigurationFactory loadConfFactory() {
+  // visible (not private) for testing
+  ServerConfigurationFactory loadConfFactory() {
     // This local variable is an intentional component of the single-check idiom.
     ServerConfigurationFactory localConf = lazyConfFactory;
     if (localConf == null) {
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 8bf5f2a..e1a9b12 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
@@ -53,29 +53,27 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
   @Override
   public String choose(VolumeChooserEnvironment env, String[] options) throws VolumeChooserException {
     log.trace("{}.choose", getClass().getSimpleName());
+    // Randomly choose the volume from the preferred volumes
+    String choice = super.choose(env, getPreferredVolumes(env, options));
+    log.trace("Choice = {}", choice);
+    return choice;
+  }
 
-    Set<String> preferredVolumes;
+  // visible (not private) for testing
+  String[] getPreferredVolumes(VolumeChooserEnvironment env, String[] options) {
     switch (env.getScope()) {
       case INIT:
         // TODO should be possible to read from SiteConfiguration during init
         log.warn("Not possible to determine preferred volumes at '{}' scope. Using all volumes.", ChooserScope.INIT);
-        return super.choose(env, options);
+        return options;
       case TABLE:
-        preferredVolumes = getPreferredVolumesForTable(env, loadConfFactory(), options);
-        break;
+        return getPreferredVolumesForTable(env, loadConfFactory(), options);
       default:
-        preferredVolumes = getPreferredVolumesForScope(env, loadConfFactory(), options);
-        break;
+        return getPreferredVolumesForScope(env, loadConfFactory(), options);
     }
-
-    // Randomly choose the volume from the preferred volumes
-    String choice = super.choose(env, preferredVolumes.toArray(new String[preferredVolumes.size()]));
-    log.trace("Choice = {}", choice);
-
-    return choice;
   }
 
-  private Set<String> getPreferredVolumesForTable(VolumeChooserEnvironment env, ServerConfigurationFactory confFactory, String[] options) {
+  private String[] getPreferredVolumesForTable(VolumeChooserEnvironment env, ServerConfigurationFactory confFactory, String[] options) {
     log.trace("Looking up property {} + for Table id: {}", TABLE_PREFERRED_VOLUMES, env.getTableId());
 
     final TableConfiguration tableConf = confFactory.getTableConfiguration(env.getTableId());
@@ -96,7 +94,7 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
     return parsePreferred(TABLE_PREFERRED_VOLUMES, preferredVolumes, options);
   }
 
-  private Set<String> getPreferredVolumesForScope(VolumeChooserEnvironment env, ServerConfigurationFactory confFactory, String[] options) {
+  private String[] getPreferredVolumesForScope(VolumeChooserEnvironment env, ServerConfigurationFactory confFactory, String[] options) {
     ChooserScope scope = env.getScope();
     String property = getPropertyNameForScope(scope);
     log.trace("Looking up property {} for scope: {}", property, scope);
@@ -122,7 +120,7 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
     return parsePreferred(property, preferredVolumes, options);
   }
 
-  private Set<String> parsePreferred(String property, String preferredVolumes, String[] options) {
+  private String[] parsePreferred(String property, String preferredVolumes, String[] options) {
     log.trace("Found {} = {}", property, preferredVolumes);
 
     Set<String> preferred = Arrays.stream(StringUtils.split(preferredVolumes, ',')).map(String::trim).collect(Collectors.toSet());
@@ -137,10 +135,11 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
       throw new VolumeChooserException(msg);
     }
 
-    return preferred;
+    return preferred.toArray(new String[preferred.size()]);
   }
 
-  private ServerConfigurationFactory loadConfFactory() {
+  // visible (not private) for testing
+  ServerConfigurationFactory loadConfFactory() {
     // 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;
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 3d1a3f1..6f6ba52 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
@@ -16,265 +16,164 @@
  */
 package org.apache.accumulo.server.fs;
 
-import java.lang.reflect.Field;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Set;
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.createStrictMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.fail;
 
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
-import org.easymock.EasyMock;
-import org.easymock.IExpectationSetters;
-import org.junit.Assert;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import com.google.common.collect.Sets;
-
 public class PerTableVolumeChooserTest {
-  private static final int REQUIRED_NUMBER_TRIES = 20; // times to call choose for likely exercising of each preferred volume
-  private static final String[] ALL_OPTIONS = new String[] {"1", "2", "3"};
-  public static final String INVALID_CHOOSER_CLASSNAME = "MysteriousVolumeChooser";
-  private ServerConfigurationFactory mockedServerConfigurationFactory;
-  private TableConfiguration mockedTableConfiguration;
-  private PerTableVolumeChooser perTableVolumeChooser;
-  private AccumuloConfiguration mockedAccumuloConfiguration;
+
+  private ServerConfigurationFactory confFactory;
+  private TableConfiguration tableConf;
+  private PerTableVolumeChooser chooser;
+  private AccumuloConfiguration systemConf;
+
+  public static class MockChooser1 extends RandomVolumeChooser {}
+
+  public static class MockChooser2 extends RandomVolumeChooser {}
 
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @Before
   public void before() throws Exception {
-    perTableVolumeChooser = new PerTableVolumeChooser();
-
-    mockedServerConfigurationFactory = EasyMock.createMock(ServerConfigurationFactory.class);
-    Field field = perTableVolumeChooser.getClass().getDeclaredField("lazyConfFactory");
-    field.setAccessible(true);
-    field.set(perTableVolumeChooser, mockedServerConfigurationFactory);
-
-    mockedTableConfiguration = EasyMock.createMock(TableConfiguration.class);
-    mockedAccumuloConfiguration = EasyMock.createMock(AccumuloConfiguration.class);
-    EasyMock.expect(mockedServerConfigurationFactory.getTableConfiguration(EasyMock.<Table.ID> anyObject())).andReturn(mockedTableConfiguration).anyTimes();
-    EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
-    EasyMock.expect(mockedTableConfiguration.get(Property.TABLE_CLASSPATH)).andReturn(null).anyTimes();
-  }
-
-  private IExpectationSetters<String> expectDefaultScope(String className) {
-    return expectScope(ChooserScope.DEFAULT, className);
-  }
+    confFactory = createStrictMock(ServerConfigurationFactory.class);
 
-  private IExpectationSetters<String> expectLoggerScope(String className) {
-    return expectScope(ChooserScope.LOGGER, className);
-  }
-
-  private IExpectationSetters<String> expectScope(ChooserScope scope, String className) {
-    return EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.getPropertyNameForScope(scope))).andReturn(className);
-  }
-
-  private IExpectationSetters<String> expectTableChooser(String className) {
-    return EasyMock.expect(mockedTableConfiguration.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(className);
-  }
-
-  private Set<String> chooseRepeatedlyForTable() {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Table.ID.of("h"));
-    Set<String> results = new HashSet<>();
-    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
-      results.add(perTableVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
-    }
-    return results;
-  }
-
-  public static class VolumeChooserAlwaysOne extends VolumeChooserForFixedVolume {
-    public VolumeChooserAlwaysOne() {
-      super("1");
-    }
-  }
-
-  public static class VolumeChooserAlwaysTwo extends VolumeChooserForFixedVolume {
-    public VolumeChooserAlwaysTwo() {
-      super("2");
-    }
-  }
-
-  public static class VolumeChooserAlwaysThree extends VolumeChooserForFixedVolume {
-    public VolumeChooserAlwaysThree() {
-      super("3");
-    }
-  }
-
-  public static class VolumeChooserForFixedVolume implements VolumeChooser {
-    private final String onlyValidOption;
-
-    public VolumeChooserForFixedVolume(String fixedVolume) {
-      onlyValidOption = fixedVolume;
-    }
+    chooser = new PerTableVolumeChooser() {
+      @Override
+      ServerConfigurationFactory loadConfFactory() {
+        return confFactory;
+      }
 
-    @Override
-    public String choose(VolumeChooserEnvironment env, String[] options) {
-      for (String option : options) {
-        if (onlyValidOption.equals(option)) {
-          return onlyValidOption;
-        }
+      @Override
+      String getTableContext(TableConfiguration tableConf) {
+        return null;
       }
-      return null;
-    }
-  }
+    };
 
-  private Set<String> chooseRepeatedlyForLogger() {
-    return chooseRepeatedlyForScope(ChooserScope.LOGGER);
+    tableConf = createStrictMock(TableConfiguration.class);
+    systemConf = createStrictMock(AccumuloConfiguration.class);
+    expect(confFactory.getTableConfiguration(anyObject())).andReturn(tableConf).anyTimes();
+    expect(confFactory.getSystemConfiguration()).andReturn(systemConf).anyTimes();
   }
 
-  private Set<String> chooseRepeatedlyForScope(ChooserScope scope) {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(scope);
-    Set<String> results = new HashSet<>();
-
-    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
-      results.add(perTableVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
-    }
-    return results;
+  @After
+  public void after() throws Exception {
+    verify(confFactory, tableConf, systemConf);
   }
 
-  @Test
-  public void testInitScope() throws Exception {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(ChooserScope.INIT);
-
-    Set<String> results = new HashSet<>();
-    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
-      results.add(perTableVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
-    }
-
-    Assert.assertEquals(Sets.newHashSet(Arrays.asList(ALL_OPTIONS)), results);
+  private VolumeChooser getTableDelegate() {
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment(Table.ID.of("testTable"));
+    return chooser.getDelegateChooser(env);
   }
 
-  @Test
-  public void testTableConfig() throws Exception {
-    expectTableChooser(VolumeChooserAlwaysTwo.class.getName()).atLeastOnce();
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
-
-    Set<String> results = chooseRepeatedlyForTable();
-
-    EasyMock.verify(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
-    Assert.assertEquals(Sets.newHashSet(Arrays.asList("2")), results);
+  private VolumeChooser getDelegate(ChooserScope scope) {
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment(scope);
+    return chooser.getDelegateChooser(env);
   }
 
   @Test
-  public void testTableMisconfigured() throws Exception {
-    expectDefaultScope(VolumeChooserAlwaysOne.class.getName());
-    expectTableChooser(INVALID_CHOOSER_CLASSNAME);
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
-
-    thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+  public void testInitScopeSelectsRandomChooser() throws Exception {
+    replay(confFactory, tableConf, systemConf);
+    VolumeChooser delegate = getDelegate(ChooserScope.INIT);
+    assertSame(RandomVolumeChooser.class, delegate.getClass());
   }
 
   @Test
-  public void testTableMissing() throws Exception {
-    expectDefaultScope(null);
-    expectTableChooser(null);
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+  public void testTableScopeUsingTableProperty() throws Exception {
+    expect(tableConf.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(MockChooser1.class.getName());
+    replay(confFactory, tableConf, systemConf);
 
-    thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+    VolumeChooser delegate = getTableDelegate();
+    assertSame(MockChooser1.class, delegate.getClass());
   }
 
   @Test
-  public void testTableMisconfiguredAndDefaultEmpty() throws Exception {
-    expectDefaultScope("");
-    expectTableChooser(INVALID_CHOOSER_CLASSNAME);
+  public void testTableScopeUsingDefaultScopeProperty() throws Exception {
+    expect(tableConf.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(null).once();
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn(MockChooser2.class.getName()).once();
+    replay(confFactory, tableConf, systemConf);
 
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
-
-    thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+    VolumeChooser delegate = getTableDelegate();
+    assertSame(MockChooser2.class, delegate.getClass());
   }
 
   @Test
-  public void testTableEmptyConfig() throws Exception {
-    expectDefaultScope("");
-    expectTableChooser("");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+  public void testTableScopeWithNoConfig() throws Exception {
+    expect(tableConf.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(null).once();
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn(null).once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+    getTableDelegate();
+    fail("should not reach");
   }
 
   @Test
-  public void testTableAndDefaultEmpty() throws Exception {
-    expectDefaultScope("");
-    expectTableChooser("");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+  public void testTableScopeWithBadDelegate() throws Exception {
+    expect(tableConf.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(null).once();
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn("not a valid class name").once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+    getTableDelegate();
+    fail("should not reach");
   }
 
   @Test
-  public void testScopeConfig() throws Exception {
-    expectLoggerScope(VolumeChooserAlwaysOne.class.getName()).atLeastOnce();
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+  public void testLoggerScopeUsingLoggerProperty() throws Exception {
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER))).andReturn(MockChooser1.class.getName()).once();
+    replay(confFactory, tableConf, systemConf);
 
-    Set<String> results = chooseRepeatedlyForLogger();
-
-    EasyMock.verify(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
-    Assert.assertEquals(Sets.newHashSet(Arrays.asList("1")), results);
+    VolumeChooser delegate = getDelegate(ChooserScope.LOGGER);
+    assertSame(MockChooser1.class, delegate.getClass());
   }
 
   @Test
-  public void testScopeMisconfigured() throws Exception {
-    expectDefaultScope(VolumeChooserAlwaysThree.class.getName());
-    expectLoggerScope(INVALID_CHOOSER_CLASSNAME);
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+  public void testLoggerScopeUsingDefaultProperty() throws Exception {
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER))).andReturn(null).once();
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn(MockChooser2.class.getName()).once();
+    replay(confFactory, tableConf, systemConf);
 
-    thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForLogger();
+    VolumeChooser delegate = getDelegate(ChooserScope.LOGGER);
+    assertSame(MockChooser2.class, delegate.getClass());
   }
 
   @Test
-  public void testScopeMissing() throws Exception {
-    expectLoggerScope(null);
-    expectDefaultScope(null);
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+  public void testLoggerScopeWithNoConfig() throws Exception {
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER))).andReturn(null).once();
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn(null).once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForLogger();
+    getDelegate(ChooserScope.LOGGER);
+    fail("should not reach");
   }
 
   @Test
-  public void testScopeMisconfiguredAndDefaultEmpty() throws Exception {
-    expectDefaultScope("");
-    expectTableChooser("");
-    expectLoggerScope(INVALID_CHOOSER_CLASSNAME);
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
-
-    thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForLogger();
-  }
-
-  @Test
-  public void testScopeAndDefaultBothEmpty() throws Exception {
-    expectDefaultScope("");
-    expectLoggerScope("");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+  public void testLoggerScopeWithBadDelegate() throws Exception {
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER))).andReturn(null).once();
+    expect(systemConf.get(PerTableVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn("not a valid class name").once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForLogger();
+    getDelegate(ChooserScope.LOGGER);
+    fail("should not reach");
   }
 
 }
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 440c628..2110b40 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
@@ -16,10 +16,16 @@
  */
 package org.apache.accumulo.server.fs;
 
-import java.lang.reflect.Field;
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.createStrictMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.fail;
+
 import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Set;
 
 import org.apache.accumulo.core.client.impl.Table;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -27,224 +33,167 @@ import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
-import org.easymock.EasyMock;
-import org.easymock.IExpectationSetters;
-import org.junit.Assert;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import com.google.common.collect.Sets;
-
 public class PreferredVolumeChooserTest {
-  private static final int REQUIRED_NUMBER_TRIES = 20; // times to call choose for likely exercising of each preferred volume
+
   private static final String[] ALL_OPTIONS = new String[] {"1", "2", "3"};
-  private ServerConfigurationFactory mockedServerConfigurationFactory;
-  private TableConfiguration mockedTableConfiguration;
-  private PreferredVolumeChooser preferredVolumeChooser;
-  private AccumuloConfiguration mockedAccumuloConfiguration;
+
+  private ServerConfigurationFactory confFactory;
+  private TableConfiguration tableConf;
+  private PreferredVolumeChooser chooser;
+  private AccumuloConfiguration systemConf;
 
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @Before
   public void before() throws Exception {
-    preferredVolumeChooser = new PreferredVolumeChooser();
-
-    mockedServerConfigurationFactory = EasyMock.createMock(ServerConfigurationFactory.class);
-    Field field = preferredVolumeChooser.getClass().getDeclaredField("lazyConfFactory");
-    field.setAccessible(true);
-    field.set(preferredVolumeChooser, mockedServerConfigurationFactory);
+    confFactory = createStrictMock(ServerConfigurationFactory.class);
 
-    mockedTableConfiguration = EasyMock.createMock(TableConfiguration.class);
-    mockedAccumuloConfiguration = EasyMock.createMock(AccumuloConfiguration.class);
-    EasyMock.expect(mockedServerConfigurationFactory.getTableConfiguration(EasyMock.<Table.ID> anyObject())).andReturn(mockedTableConfiguration).anyTimes();
-    EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
-  }
-
-  private IExpectationSetters<String> expectTableVolumes(String configuredVolumes) {
-    return EasyMock.expect(mockedTableConfiguration.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(configuredVolumes);
-  }
-
-  private IExpectationSetters<String> expectDefaultScope(String configuredVolumes) {
-    return expectScope(ChooserScope.DEFAULT, configuredVolumes);
-  }
+    chooser = new PreferredVolumeChooser() {
+      @Override
+      ServerConfigurationFactory loadConfFactory() {
+        return confFactory;
+      }
+    };
 
-  private IExpectationSetters<String> expectLoggerScope(String configuredVolumes) {
-    return expectScope(ChooserScope.LOGGER, configuredVolumes);
+    tableConf = createStrictMock(TableConfiguration.class);
+    systemConf = createStrictMock(AccumuloConfiguration.class);
+    expect(confFactory.getTableConfiguration(anyObject())).andReturn(tableConf).anyTimes();
+    expect(confFactory.getSystemConfiguration()).andReturn(systemConf).anyTimes();
   }
 
-  private IExpectationSetters<String> expectScope(ChooserScope scope, String configuredVolumes) {
-    return EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.getPropertyNameForScope(scope))).andReturn(configuredVolumes);
+  @After
+  public void after() throws Exception {
+    verify(confFactory, tableConf, systemConf);
   }
 
-  private Set<String> chooseRepeatedlyForTable() {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Table.ID.of("h"));
-    Set<String> results = new HashSet<>();
-    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
-      results.add(preferredVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
-    }
-    return results;
+  private String[] chooseForTable() {
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment(Table.ID.of("testTable"));
+    return chooser.getPreferredVolumes(env, ALL_OPTIONS);
   }
 
-  private Set<String> chooseRepeatedlyForLogger() {
-    return chooseRepeatedlyForScope(ChooserScope.LOGGER);
-  }
-
-  private Set<String> chooseRepeatedlyForScope(ChooserScope scope) {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(scope);
-    Set<String> results = new HashSet<>();
-
-    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
-      results.add(preferredVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
-    }
-    return results;
+  private String[] choose(ChooserScope scope) {
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment(scope);
+    return chooser.getPreferredVolumes(env, ALL_OPTIONS);
   }
 
   @Test
-  public void testEmptyEnvUsesRandomChooser() throws Exception {
-    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(ChooserScope.INIT);
-    Set<String> results = new HashSet<>();
-    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
-      results.add(preferredVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
-    }
-
-    Assert.assertEquals(Sets.newHashSet(Arrays.asList(ALL_OPTIONS)), results);
-  }
-
-  @Test
-  public void testTableConfig() throws Exception {
-    expectDefaultScope(null).anyTimes();
-    expectTableVolumes("1,2").atLeastOnce();
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
-
-    Set<String> results = chooseRepeatedlyForTable();
-
-    EasyMock.verify(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
-    Assert.assertEquals(Sets.newHashSet(Arrays.asList("1", "2")), results);
+  public void testInitScopeSelectsRandomlyFromAll() throws Exception {
+    replay(confFactory, tableConf, systemConf);
+    String[] volumes = choose(ChooserScope.INIT);
+    assertSame(ALL_OPTIONS, volumes);
   }
 
   @Test
-  public void testTableMisconfigured() throws Exception {
-    expectDefaultScope("1,3");
-    expectTableVolumes("4");
+  public void testTableScopeUsingTableProperty() throws Exception {
+    expect(tableConf.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn("2,1");
+    replay(confFactory, tableConf, systemConf);
 
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
-
-    thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+    String[] volumes = chooseForTable();
+    Arrays.sort(volumes);
+    assertArrayEquals(new String[] {"1", "2"}, volumes);
   }
 
   @Test
-  public void testTableMissing() throws Exception {
-    expectDefaultScope("");
-    expectTableVolumes(null);
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+  public void testTableScopeUsingDefaultScopeProperty() throws Exception {
+    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);
 
-    thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+    String[] volumes = chooseForTable();
+    Arrays.sort(volumes);
+    assertArrayEquals(new String[] {"2", "3"}, volumes);
   }
 
   @Test
-  public void testTableEmptyConfig() throws Exception {
-    expectDefaultScope(null);
-    expectTableVolumes("");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+  public void testTableScopeWithNoConfig() throws Exception {
+    expect(tableConf.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(null).once();
+    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn(null).once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+    chooseForTable();
+    fail("should not reach");
   }
 
   @Test
-  public void testTableMisconfiguredAndDefaultEmpty() throws Exception {
-    expectDefaultScope("");
-    expectTableVolumes("4");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+  public void testTableScopeWithEmptySet() throws Exception {
+    expect(tableConf.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(",").once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+    chooseForTable();
+    fail("should not reach");
   }
 
   @Test
-  public void testTableAndDefaultEmpty() throws Exception {
-    expectDefaultScope("");
-    expectTableVolumes("");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+  public void testTableScopeWithUnrecognizedVolumes() throws Exception {
+    expect(tableConf.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(null).once();
+    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn("4").once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForTable();
+    chooseForTable();
+    fail("should not reach");
   }
 
   @Test
-  public void testScopeConfig() throws Exception {
-    expectLoggerScope("1,2").atLeastOnce();
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+  public void testLoggerScopeUsingLoggerProperty() throws Exception {
+    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER))).andReturn("2,1").once();
+    replay(confFactory, tableConf, systemConf);
 
-    Set<String> results = chooseRepeatedlyForLogger();
-
-    EasyMock.verify(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
-    Assert.assertEquals(Sets.newHashSet(Arrays.asList("1", "2")), results);
+    String[] volumes = choose(ChooserScope.LOGGER);
+    Arrays.sort(volumes);
+    assertArrayEquals(new String[] {"1", "2"}, volumes);
   }
 
   @Test
-  public void testScopeMisconfigured() throws Exception {
-    expectDefaultScope("1,3");
-    expectLoggerScope("4");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+  public void testLoggerScopeUsingDefaultProperty() throws Exception {
+    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);
 
-    thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForLogger();
+    String[] volumes = choose(ChooserScope.LOGGER);
+    Arrays.sort(volumes);
+    assertArrayEquals(new String[] {"2", "3"}, volumes);
   }
 
   @Test
-  public void testScopeMissing() throws Exception {
-    expectDefaultScope("").atLeastOnce();
-    expectLoggerScope(null).atLeastOnce();
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+  public void testLoggerScopeWithNoConfig() throws Exception {
+    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER))).andReturn(null).once();
+    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn(null).once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForLogger();
+    choose(ChooserScope.LOGGER);
+    fail("should not reach");
   }
 
   @Test
-  public void testScopeMisconfiguredAndDefaultEmpty() throws Exception {
-    expectDefaultScope("");
-    expectLoggerScope("4");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+  public void testLoggerScopeWithEmptySet() throws Exception {
+    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER))).andReturn(",").once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForLogger();
+    choose(ChooserScope.LOGGER);
+    fail("should not reach");
   }
 
   @Test
-  public void testScopeAndDefaultBothEmpty() throws Exception {
-    expectDefaultScope("");
-    expectLoggerScope("");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+  public void testLoggerScopeWithUnrecognizedVolumes() throws Exception {
+    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.LOGGER))).andReturn(null).once();
+    expect(systemConf.get(PreferredVolumeChooser.getPropertyNameForScope(ChooserScope.DEFAULT))).andReturn("4").once();
+    replay(confFactory, tableConf, systemConf);
 
     thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForLogger();
+    choose(ChooserScope.LOGGER);
+    fail("should not reach");
   }
 
-  @Test
-  public void testScopeEmptyConfig() throws Exception {
-    expectDefaultScope("");
-    expectLoggerScope("");
-
-    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
-
-    thrown.expect(VolumeChooserException.class);
-    chooseRepeatedlyForLogger();
-  }
 }

-- 
To stop receiving notification emails like this one, please contact
"commits@accumulo.apache.org" <co...@accumulo.apache.org>.

[accumulo] 01/03: ACCUMULO-4086 Improve volume chooser fallback

Posted by ct...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 513d1f500ca62374225008d1a2b93dd9384b9bf9
Author: Ivan Bella <iv...@bella.name>
AuthorDate: Wed Jul 26 21:28:05 2017 -0400

    ACCUMULO-4086 Improve volume chooser fallback
    
    Implement a better fallback mechanism when the user's selected
    VolumeChooser implementation cannot be loaded or is not specified.
    Handles all such cases, including non-table scopes (logger), and
    per-table scopes.
    
    * Modified to fail upon misconfiguration instead of falling back to an
    alternate chooser
    * Ensure we have an appropriate default
    * Changed the property names, and made the properties consistent between
    the PerTableVolumeChooser and the PreferredVolumeChooser
    * Use random volume chooser by default
    * Use random volume chooser on init scope for PerTableVolumeChooser
    
    This squashed commit contains contributions from the following authors:
    
      matthpeterson <mp...@appliedtech-group.com>
      Mike Walch <mw...@apache.org>
      Christopher Tubbs <ct...@apache.org>
      Ivan Bella <iv...@bella.name>
---
 .../org/apache/accumulo/core/conf/Property.java    |   5 +-
 .../accumulo/server/fs/PerTableVolumeChooser.java  | 166 +++++++++---
 .../accumulo/server/fs/PreferredVolumeChooser.java | 148 ++++++++---
 .../accumulo/server/fs/VolumeManagerImpl.java      |  30 ++-
 .../server/fs/PerTableVolumeChooserTest.java       | 293 +++++++++++++++++++++
 .../server/fs/PreferredVolumeChooserTest.java      | 252 ++++++++++++++++++
 .../accumulo/server/fs/VolumeManagerImplTest.java  |  33 ++-
 .../accumulo/tserver/tablet/RootFilesTest.java     |   2 +
 .../org/apache/accumulo/test/ShellConfigIT.java    |  10 +-
 .../accumulo/test/VolumeChooserFailureIT.java      | 153 +++++++++++
 .../org/apache/accumulo/test/VolumeChooserIT.java  | 276 ++++++++-----------
 11 files changed, 1106 insertions(+), 262 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 810c3fe..d0dffa4 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -191,7 +191,7 @@ public enum Property {
       + "server-internal scheduled tasks"),
   // If you update the default type, be sure to update the default used for initialization failures in VolumeManagerImpl
   @Experimental
-  GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.PerTableVolumeChooser", PropertyType.CLASSNAME,
+  GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.RandomVolumeChooser", PropertyType.CLASSNAME,
       "The class that will be used to select which volume will be used to create new files."),
   GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS("general.security.credential.provider.paths", "", PropertyType.STRING,
       "Comma-separated list of paths to CredentialProviders"),
@@ -554,9 +554,6 @@ public enum Property {
   TABLE_REPLICATION_TARGET("table.replication.target.", null, PropertyType.PREFIX, "Enumerate a mapping of other systems which this table should "
       + "replicate their data to. The key suffix is the identifying cluster name and the value is an identifier for a location on the target system, "
       + "e.g. the ID of the table on the target to replicate to"),
-  @Experimental
-  TABLE_VOLUME_CHOOSER("table.volume.chooser", "org.apache.accumulo.server.fs.RandomVolumeChooser", PropertyType.CLASSNAME,
-      "The class that will be used to select which volume will be used to create new files for this table."),
   TABLE_SAMPLER(
       "table.sampler",
       "",
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 4a57100..4f630b1 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
@@ -18,61 +18,157 @@ package org.apache.accumulo.server.fs;
 
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * A {@link VolumeChooser} that delegates to another volume chooser based on the presence of an experimental table property,
- * {@link Property#TABLE_VOLUME_CHOOSER}. If it isn't found, defaults back to {@link RandomVolumeChooser}.
+ * 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 "logger".
  */
 public class PerTableVolumeChooser implements VolumeChooser {
-
-  private final VolumeChooser fallbackVolumeChooser = new RandomVolumeChooser();
+  private static final Logger log = LoggerFactory.getLogger(PerTableVolumeChooser.class);
   // TODO Add hint of expected size to construction, see ACCUMULO-3410
   /* Track VolumeChooser instances so they can keep state. */
-  private final ConcurrentHashMap<Table.ID,VolumeChooser> tableSpecificChooser = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String,VolumeChooser> tableSpecificChooser = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<String,VolumeChooser> scopeSpecificChooser = 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 serverConfs;
 
+  public static final String INIT_SCOPE = "init";
+
+  public static final String TABLE_VOLUME_CHOOSER = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "volume.chooser";
+
+  public static final String SCOPED_VOLUME_CHOOSER(String scope) {
+    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope + ".volume.chooser";
+  }
+
+  public static final String DEFAULT_SCOPED_VOLUME_CHOOSER = SCOPED_VOLUME_CHOOSER("scoped");
+
   @Override
   public String choose(VolumeChooserEnvironment env, String[] options) {
-    VolumeChooser chooser = null;
+    log.trace("PerTableVolumeChooser.choose");
+
+    VolumeChooser chooser;
+    if (!env.hasTableId() && (!env.hasScope() || env.getScope().equals(INIT_SCOPE))) {
+      // Should only get here during Initialize. Configurations are not yet available.
+      return randomChooser.choose(env, options);
+    }
+
+    ServerConfigurationFactory localConf = loadConf();
     if (env.hasTableId()) {
-      // This local variable is an intentional component of the single-check idiom.
-      ServerConfigurationFactory localConf = serverConfs;
-      if (localConf == null) {
-        // If we're under contention when first getting here we'll throw away some initializations.
-        localConf = new ServerConfigurationFactory(HdfsZooInstance.getInstance());
-        serverConfs = localConf;
+      // use the table configuration
+      chooser = getVolumeChooserForTable(env, localConf);
+    } else {
+      // use the system configuration
+      chooser = getVolumeChooserForNonTable(env, localConf);
+    }
+
+    return chooser.choose(env, options);
+  }
+
+  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
+    log.trace("Looking up property {} for Table id: {}", TABLE_VOLUME_CHOOSER, env.getTableId());
+    final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
+    String clazz = tableConf.get(TABLE_VOLUME_CHOOSER);
+
+    if (null == clazz || clazz.isEmpty()) {
+      String msg = "Property " + TABLE_VOLUME_CHOOSER + " must be set" + (null == clazz ? " " : " properly ") + "to use the " + getClass().getSimpleName();
+      throw new RuntimeException(msg);
+    }
+
+    return createVolumeChooser(clazz, TABLE_VOLUME_CHOOSER, env.getTableId().canonicalID(), tableSpecificChooser);
+  }
+
+  private VolumeChooser getVolumeChooserForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf) {
+    String scope = env.getScope();
+    String property = SCOPED_VOLUME_CHOOSER(scope);
+
+    log.trace("Looking up property: {}", property);
+
+    AccumuloConfiguration systemConfiguration = localConf.getSystemConfiguration();
+    String clazz = systemConfiguration.get(property);
+    // only if the custom property is not set do we fallback to the default scope volume chooser setting
+    if (null == clazz) {
+      log.debug("Property not found: {} using {}", property, DEFAULT_SCOPED_VOLUME_CHOOSER);
+      clazz = systemConfiguration.get(DEFAULT_SCOPED_VOLUME_CHOOSER);
+
+      if (null == clazz || clazz.isEmpty()) {
+        String msg = "Property " + property + " or " + DEFAULT_SCOPED_VOLUME_CHOOSER + " must be set" + (null == clazz ? " " : " properly ") + "to use the "
+            + getClass().getSimpleName();
+        throw new RuntimeException(msg);
       }
-      final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
-      chooser = tableSpecificChooser.get(env.getTableId());
-      if (chooser == null) {
-        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
-        chooser = tableSpecificChooser.putIfAbsent(env.getTableId(), temp);
-        if (chooser == null) {
-          chooser = temp;
-          // Otherwise, someone else beat us to initializing; use theirs.
-        }
-      } else if (!(chooser.getClass().getName().equals(tableConf.get(Property.TABLE_VOLUME_CHOOSER)))) {
-        // the configuration for this table's chooser has been updated. In the case of failure to instantiate we'll repeat here next call.
+
+      property = DEFAULT_SCOPED_VOLUME_CHOOSER;
+    }
+
+    return createVolumeChooser(clazz, property, scope, scopeSpecificChooser);
+  }
+
+  /**
+   * Create a volume chooser, using the cached version if any. This will replace the cached version if the class name has changed.
+   *
+   * @param clazz
+   *          The volume chooser class name
+   * @param property
+   *          The property from which it was obtained
+   * @param key
+   *          The key to user in the cache
+   * @param cache
+   *          The cache
+   * @return The volume chooser instance
+   */
+  private VolumeChooser createVolumeChooser(String clazz, String property, String key, ConcurrentHashMap<String,VolumeChooser> cache) {
+    VolumeChooser chooser = cache.get(key);
+    // if we do not have a chooser or the class has changed, then create a new one
+    if (chooser == null || !(chooser.getClass().getName().equals(clazz))) {
+      if (chooser != null) {
         // TODO stricter definition of when the updated property is used, ref ACCUMULO-3412
-        VolumeChooser temp = Property.createTableInstanceFromPropertyName(tableConf, Property.TABLE_VOLUME_CHOOSER, VolumeChooser.class, fallbackVolumeChooser);
-        VolumeChooser last = tableSpecificChooser.replace(env.getTableId(), temp);
-        if (chooser.equals(last)) {
-          chooser = temp;
-        } else {
-          // Someone else beat us to updating; use theirs.
-          chooser = last;
-        }
+        log.trace("Change detected for {} for {}", property, key);
+      }
+      // create a new volume chooser instance
+      VolumeChooser temp;
+      try {
+        temp = loadClass(clazz);
+      } catch (Exception e) {
+        String msg = "Failed to create instance for " + key + " configured to use " + clazz + " via " + property;
+        throw new RuntimeException(msg, e);
+      }
+      if (chooser == null) {
+        // if we did not have one previously, then put this one in the cache
+        // but use the one already in the cache if another thread beat us here
+        chooser = cache.computeIfAbsent(key, k -> temp);
+      } else {
+        // otherwise the class has changed, so replace the one in the cache
+        // unless another thread beat us here
+        chooser = cache.computeIfPresent(key, (k, v) -> (v.getClass().getName().equals(clazz) ? v : temp));
       }
-    } else {
-      chooser = fallbackVolumeChooser;
     }
+    return chooser;
+  }
 
-    return chooser.choose(env, options);
+  private VolumeChooser loadClass(String className) throws ClassNotFoundException, IllegalAccessException, InstantiationException {
+    // not attempting to load context because this approach to loading the class is for non-tables only
+    return AccumuloVFSClassLoader.loadClass(className, VolumeChooser.class).newInstance();
   }
+
+  private ServerConfigurationFactory loadConf() {
+    // This local variable is an intentional component of the single-check idiom.
+    ServerConfigurationFactory localConf = serverConfs;
+    if (localConf == null) {
+      // If we're under contention when first getting here we'll throw away some initializations.
+      localConf = new ServerConfigurationFactory(HdfsZooInstance.getInstance());
+      serverConfs = 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 0143f24..edd9931 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
@@ -21,12 +21,12 @@ import static org.apache.commons.lang.ArrayUtils.EMPTY_STRING_ARRAY;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.function.Predicate;
 
+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.client.HdfsZooInstance;
@@ -39,18 +39,21 @@ 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 {@value #PREFERRED_VOLUMES_CUSTOM_KEY}, which should contain a comma
+ * to selecting from all of the options presented. Can be customized via the table property table.custom.preferredVolumes, 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);
 
-  /**
-   * This should match {@link Property#TABLE_ARBITRARY_PROP_PREFIX}
-   */
-  public static final String PREFERRED_VOLUMES_CUSTOM_KEY = "table.custom.preferredVolumes";
-  // TODO ACCUMULO-3417 replace this with the ability to retrieve by String key.
-  private static final Predicate<String> PREFERRED_VOLUMES_FILTER = key -> PREFERRED_VOLUMES_CUSTOM_KEY.equals(key);
+  public static final String INIT_SCOPE = "init";
+
+  public static final String TABLE_PREFERRED_VOLUMES = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "preferred.volumes";
+
+  public static final String SCOPED_PREFERRED_VOLUMES(String scope) {
+    return Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + scope + ".preferred.volumes";
+  }
+
+  public static final String DEFAULT_SCOPED_PREFERRED_VOLUMES = SCOPED_PREFERRED_VOLUMES("scoped");
 
   @SuppressWarnings("unchecked")
   private final Map<String,Set<String>> parsedPreferredVolumes = Collections.synchronizedMap(new LRUMap(1000));
@@ -59,53 +62,114 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
 
   @Override
   public String choose(VolumeChooserEnvironment env, String[] options) {
-    if (!env.hasTableId())
+    if (!env.hasTableId() && (!env.hasScope() || env.getScope().equals(INIT_SCOPE))) {
+      // this should only happen during initialize
+      log.warn("No table id or scope, so it's not possible to determine preferred volumes.  Using all volumes.");
       return super.choose(env, options);
+    }
 
-    // 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 = serverConfs;
-    if (localConf == null) {
-      // If we're under contention when first getting here we'll throw away some initializations.
-      localConf = new ServerConfigurationFactory(HdfsZooInstance.getInstance());
-      serverConfs = localConf;
+    // get the volumes property
+    ServerConfigurationFactory localConf = loadConf();
+    List<String> volumes;
+    if (env.hasTableId()) {
+      volumes = getPreferredVolumesForTable(env, localConf, options);
+    } else {
+      volumes = getPreferredVolumesForNonTable(env, localConf, options);
     }
-    TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
-    final Map<String,String> props = new HashMap<>();
-    tableConf.getProperties(props, PREFERRED_VOLUMES_FILTER);
-    if (props.isEmpty()) {
-      log.warn("No preferred volumes specified. Defaulting to randomly choosing from instance volumes");
-      return super.choose(env, options);
+
+    // Randomly choose the volume from the preferred volumes
+    String choice = super.choose(env, volumes.toArray(EMPTY_STRING_ARRAY));
+    log.trace("Choice = {}", choice);
+
+    return choice;
+  }
+
+  private List<String> getPreferredVolumesForTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf, String[] options) {
+    log.trace("Looking up property {} + for Table id: {}", TABLE_PREFERRED_VOLUMES, env.getTableId());
+
+    final TableConfiguration tableConf = localConf.getTableConfiguration(env.getTableId());
+    String volumes = tableConf.get(TABLE_PREFERRED_VOLUMES);
+
+    // throw an error if volumes not specified or empty
+    if (null == volumes || volumes.isEmpty()) {
+      String logmsg = "Missing or empty " + TABLE_PREFERRED_VOLUMES + " property";
+      throw new RuntimeException(logmsg);
     }
-    String volumes = props.get(PREFERRED_VOLUMES_CUSTOM_KEY);
 
-    if (log.isTraceEnabled()) {
-      log.trace("In custom chooser");
-      log.trace("Volumes: {}", volumes);
-      log.trace("TableID: {}", env.getTableId());
+    return getFilteredOptions(TABLE_PREFERRED_VOLUMES, volumes, options);
+  }
+
+  private List<String> getPreferredVolumesForNonTable(VolumeChooserEnvironment env, ServerConfigurationFactory localConf, String[] options) {
+    String scope = env.getScope();
+    String property = SCOPED_PREFERRED_VOLUMES(scope);
+
+    log.trace("Looking up property: {}", property);
+
+    AccumuloConfiguration systemConfiguration = localConf.getSystemConfiguration();
+    String volumes = systemConfiguration.get(property);
+
+    // only if the custom property is not set to we fallback to the default scoped preferred volumes
+    if (null == volumes) {
+      log.debug("Property not found: {} using {}", property, DEFAULT_SCOPED_PREFERRED_VOLUMES);
+      volumes = systemConfiguration.get(DEFAULT_SCOPED_PREFERRED_VOLUMES);
+
+      if (null == volumes || volumes.isEmpty()) {
+        String logmsg = "Missing or empty " + property + " and " + DEFAULT_SCOPED_PREFERRED_VOLUMES + " properties";
+        throw new RuntimeException(logmsg);
+      }
+
+      property = DEFAULT_SCOPED_PREFERRED_VOLUMES;
     }
-    // If the preferred volumes property was specified, split the returned string by the comma and add use it to filter the given options.
-    Set<String> preferred = parsedPreferredVolumes.get(volumes);
-    if (preferred == null) {
-      preferred = new HashSet<>(Arrays.asList(StringUtils.split(volumes, ',')));
-      parsedPreferredVolumes.put(volumes, preferred);
+
+    return getFilteredOptions(property, volumes, options);
+  }
+
+  private List<String> getFilteredOptions(String property, String volumes, String[] options) {
+    log.trace("Found {} = {}", property, volumes);
+
+    ArrayList<String> filteredOptions = getIntersection(options, volumes);
+
+    // throw error if intersecting with preferred volumes resulted in the empty set
+    if (filteredOptions.isEmpty()) {
+      String logMessage = "After filtering preferred volumes, there are no valid instance volumes.";
+      log.error(logMessage);
+      throw new RuntimeException(logMessage);
     }
 
+    return filteredOptions;
+  }
+
+  private ArrayList<String> getIntersection(String[] options, String volumes) {
+    Set<String> preferred = parseVolumes(volumes);
+    return filterWithPreferred(options, preferred);
+  }
+
+  private ArrayList<String> filterWithPreferred(String[] options, Set<String> preferred) {
     // Only keep the options that are in the preferred set
     final ArrayList<String> filteredOptions = new ArrayList<>(Arrays.asList(options));
     filteredOptions.retainAll(preferred);
+    return filteredOptions;
+  }
 
-    // If there are no preferred volumes left, then warn the user and choose randomly from the instance volumes
-    if (filteredOptions.isEmpty()) {
-      log.warn("Preferred volumes are not instance volumes. Defaulting to randomly choosing from instance volumes");
-      return super.choose(env, options);
+  private Set<String> parseVolumes(String volumes) {
+    // If the preferred volumes property was specified, split the returned string by the comma and add use it to filter the given options.
+    Set<String> preferred = parsedPreferredVolumes.get(volumes);
+    if (preferred == null) {
+      preferred = new HashSet<>(Arrays.asList(StringUtils.split(volumes, ',')));
+      parsedPreferredVolumes.put(volumes, preferred);
     }
+    return preferred;
+  }
 
-    // Randomly choose the volume from the preferred volumes
-    String choice = super.choose(env, filteredOptions.toArray(EMPTY_STRING_ARRAY));
-    if (log.isTraceEnabled()) {
-      log.trace("Choice = {}", choice);
+  private ServerConfigurationFactory loadConf() {
+    // 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 = serverConfs;
+    if (localConf == null) {
+      // If we're under contention when first getting here we'll throw away some initializations.
+      localConf = new ServerConfigurationFactory(HdfsZooInstance.getInstance());
+      serverConfs = localConf;
     }
-    return choice;
+    return localConf;
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index 4495e79..04e86fc 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -79,8 +79,18 @@ public class VolumeManagerImpl implements VolumeManager {
     this.volumesByFileSystemUri = HashMultimap.create();
     invertVolumesByFileSystem(volumesByName, volumesByFileSystemUri);
     ensureSyncIsEnabled();
-    // Keep in sync with default type in the property definition.
-    chooser = Property.createInstanceFromPropertyName(conf, Property.GENERAL_VOLUME_CHOOSER, VolumeChooser.class, new PerTableVolumeChooser());
+    // if they supplied a property and we cannot load it, then fail hard
+    VolumeChooser chooser1;
+    try {
+      chooser1 = Property.createInstanceFromPropertyName(conf, Property.GENERAL_VOLUME_CHOOSER, VolumeChooser.class, null);
+    } catch (NullPointerException npe) {
+      chooser1 = null;
+      // null chooser handled below
+    }
+    if (chooser1 == null) {
+      throw new RuntimeException("Failed to load volume chooser specified by " + Property.GENERAL_VOLUME_CHOOSER);
+    }
+    chooser = chooser1;
   }
 
   private void invertVolumesByFileSystem(Map<String,Volume> forward, Multimap<URI,Volume> inverted) {
@@ -408,7 +418,7 @@ public class VolumeManagerImpl implements VolumeManager {
     if (path.startsWith("../"))
       path = path.substring(2);
     else if (path.startsWith("/"))
-      path = "/" + tableId + path;
+      path = "/" + tableId.canonicalID() + path;
     else
       throw new IllegalArgumentException("Unexpected path prefix " + path);
 
@@ -464,17 +474,17 @@ public class VolumeManagerImpl implements VolumeManager {
     return getVolumeByPath(dir).getFileSystem().getContentSummary(dir);
   }
 
-  // Only used as a fall back if the configured chooser misbehaves.
-  private final VolumeChooser failsafeChooser = new RandomVolumeChooser();
-
   @Override
   public String choose(VolumeChooserEnvironment env, String[] options) {
-    final String choice = chooser.choose(env, options);
+    final String choice;
+    choice = chooser.choose(env, options);
+
     if (!(ArrayUtils.contains(options, choice))) {
-      log.error("The configured volume chooser, '{}', or one of its delegates returned a volume not in the set of options provided; "
-          + "will continue by relying on a RandomVolumeChooser. You should investigate and correct the named chooser.", chooser.getClass());
-      return failsafeChooser.choose(env, options);
+      // we may want to go with random if they chooser was not overridden
+      String msg = "The configured volume chooser, '" + chooser.getClass() + "', or one of its delegates returned a volume not in the set of options provided";
+      throw new RuntimeException(msg);
     }
+
     return choice;
   }
 
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
new file mode 100644
index 0000000..f8946c4
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
@@ -0,0 +1,293 @@
+/*
+ * 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.fs;
+
+import java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.conf.ServerConfigurationFactory;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import com.google.common.collect.Sets;
+
+public class PerTableVolumeChooserTest {
+  private static final int REQUIRED_NUMBER_TRIES = 20; // times to call choose for likely exercising of each preferred volume
+  private static final String[] ALL_OPTIONS = new String[] {"1", "2", "3"};
+  public static final String INVALID_CHOOSER_CLASSNAME = "MysteriousVolumeChooser";
+  private ServerConfigurationFactory mockedServerConfigurationFactory;
+  private TableConfiguration mockedTableConfiguration;
+  private PerTableVolumeChooser perTableVolumeChooser;
+  private AccumuloConfiguration mockedAccumuloConfiguration;
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Before
+  public void before() throws Exception {
+    perTableVolumeChooser = new PerTableVolumeChooser();
+
+    mockedServerConfigurationFactory = EasyMock.createMock(ServerConfigurationFactory.class);
+    Field field = perTableVolumeChooser.getClass().getDeclaredField("serverConfs");
+    field.setAccessible(true);
+    field.set(perTableVolumeChooser, mockedServerConfigurationFactory);
+
+    mockedTableConfiguration = EasyMock.createMock(TableConfiguration.class);
+    mockedAccumuloConfiguration = EasyMock.createMock(AccumuloConfiguration.class);
+  }
+
+  private void configureDefaultVolumeChooser(String className) {
+    EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
+    EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(className).anyTimes();
+  }
+
+  private void configureScopedVolumeChooser(String className, String scope) {
+    EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
+    EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.SCOPED_VOLUME_CHOOSER(scope))).andReturn(className).anyTimes();
+  }
+
+  private void configureChooserForTable(String className) {
+    EasyMock.expect(mockedServerConfigurationFactory.getTableConfiguration(EasyMock.<Table.ID> anyObject())).andReturn(mockedTableConfiguration).anyTimes();
+    EasyMock.expect(mockedTableConfiguration.get(Property.TABLE_CLASSPATH)).andReturn(null).anyTimes();
+    EasyMock.expect(mockedTableConfiguration.get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER)).andReturn(className).anyTimes();
+  }
+
+  private void configureDefaultContextVolumeChooser(String className) {
+    EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.DEFAULT_SCOPED_VOLUME_CHOOSER)).andReturn(className).anyTimes();
+  }
+
+  private void configureContextVolumeChooser(String className) {
+    EasyMock.expect(mockedAccumuloConfiguration.get(PerTableVolumeChooser.SCOPED_VOLUME_CHOOSER("logger"))).andReturn(className).anyTimes();
+  }
+
+  private Set<String> chooseRepeatedlyForTable() {
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.of(Table.ID.of("h")));
+    Set<String> results = new HashSet<>();
+    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
+      results.add(perTableVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
+    }
+    return results;
+  }
+
+  public static class VolumeChooserAlwaysOne extends VolumeChooserForFixedVolume {
+    public VolumeChooserAlwaysOne() {
+      super("1");
+    }
+  }
+
+  public static class VolumeChooserAlwaysTwo extends VolumeChooserForFixedVolume {
+    public VolumeChooserAlwaysTwo() {
+      super("2");
+    }
+  }
+
+  public static class VolumeChooserAlwaysThree extends VolumeChooserForFixedVolume {
+    public VolumeChooserAlwaysThree() {
+      super("3");
+    }
+  }
+
+  public static class VolumeChooserForFixedVolume implements VolumeChooser {
+    private final String onlyValidOption;
+
+    public VolumeChooserForFixedVolume(String fixedVolume) {
+      onlyValidOption = fixedVolume;
+    }
+
+    @Override
+    public String choose(VolumeChooserEnvironment env, String[] options) {
+      for (String option : options) {
+        if (onlyValidOption.equals(option)) {
+          return onlyValidOption;
+        }
+      }
+      return null;
+    }
+  }
+
+  private Set<String> chooseRepeatedlyForContext() {
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.empty());
+    volumeChooserEnvironment.setScope("logger");
+    Set<String> results = new HashSet<>();
+
+    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
+      results.add(perTableVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
+    }
+    return results;
+  }
+
+  @Test
+  public void testEmptyEnvUsesRandomChooser() throws Exception {
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.empty());
+
+    Set<String> results = new HashSet<>();
+    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
+      results.add(perTableVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
+    }
+
+    Assert.assertEquals(Sets.newHashSet(Arrays.asList(ALL_OPTIONS)), results);
+  }
+
+  @Test
+  public void testTableConfig() throws Exception {
+    configureDefaultVolumeChooser(VolumeChooserAlwaysOne.class.getName());
+    configureChooserForTable(VolumeChooserAlwaysTwo.class.getName());
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    Set<String> results = chooseRepeatedlyForTable();
+
+    EasyMock.verify(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+    Assert.assertEquals(Sets.newHashSet(Arrays.asList("2")), results);
+  }
+
+  @Test
+  public void testTableMisconfigured() throws Exception {
+    configureDefaultVolumeChooser(VolumeChooserAlwaysOne.class.getName());
+    configureChooserForTable(INVALID_CHOOSER_CLASSNAME);
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testTableMissing() throws Exception {
+    configureDefaultVolumeChooser(VolumeChooserAlwaysOne.class.getName());
+    configureChooserForTable(null);
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testTableMisconfiguredAndDefaultEmpty() throws Exception {
+    configureDefaultVolumeChooser("");
+    configureChooserForTable(INVALID_CHOOSER_CLASSNAME);
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testTableEmptyConfig() throws Exception {
+    configureDefaultVolumeChooser(VolumeChooserAlwaysThree.class.getName());
+    configureChooserForTable("");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testTableAndDefaultEmpty() throws Exception {
+    configureDefaultVolumeChooser("");
+    configureChooserForTable("");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testContextConfig() throws Exception {
+    configureDefaultVolumeChooser(VolumeChooserAlwaysThree.class.getName());
+    configureContextVolumeChooser(VolumeChooserAlwaysOne.class.getName());
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    Set<String> results = chooseRepeatedlyForContext();
+
+    EasyMock.verify(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+    Assert.assertEquals(Sets.newHashSet(Arrays.asList("1")), results);
+  }
+
+  @Test
+  public void testContextMisconfigured() throws Exception {
+    configureDefaultVolumeChooser(VolumeChooserAlwaysThree.class.getName());
+    configureContextVolumeChooser(INVALID_CHOOSER_CLASSNAME);
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+
+  @Test
+  public void testContextMissing() throws Exception {
+    configureDefaultVolumeChooser(VolumeChooserAlwaysTwo.class.getName());
+    configureContextVolumeChooser(null);
+    configureDefaultContextVolumeChooser(null);
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+
+  @Test
+  public void testContextMisconfiguredAndDefaultEmpty() throws Exception {
+    configureDefaultVolumeChooser("");
+    configureChooserForTable("");
+    configureContextVolumeChooser(INVALID_CHOOSER_CLASSNAME);
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+
+  @Test
+  public void testContextAndDefaultBothEmpty() throws Exception {
+    this.configureDefaultVolumeChooser("");
+    configureContextVolumeChooser("");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+
+  @Test
+  public void testContextEmptyConfig() throws Exception {
+    configureDefaultVolumeChooser(VolumeChooserAlwaysTwo.class.getName());
+    configureContextVolumeChooser("");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+}
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
new file mode 100644
index 0000000..fa3f672
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
@@ -0,0 +1,252 @@
+/*
+ * 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.fs;
+
+import java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.impl.Table;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.server.conf.ServerConfigurationFactory;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import com.google.common.collect.Sets;
+
+public class PreferredVolumeChooserTest {
+  private static final int REQUIRED_NUMBER_TRIES = 20; // times to call choose for likely exercising of each preferred volume
+  private static final String[] ALL_OPTIONS = new String[] {"1", "2", "3"};
+  private ServerConfigurationFactory mockedServerConfigurationFactory;
+  private TableConfiguration mockedTableConfiguration;
+  private PreferredVolumeChooser preferredVolumeChooser;
+  private AccumuloConfiguration mockedAccumuloConfiguration;
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Before
+  public void before() throws Exception {
+    preferredVolumeChooser = new PreferredVolumeChooser();
+
+    mockedServerConfigurationFactory = EasyMock.createMock(ServerConfigurationFactory.class);
+    Field field = preferredVolumeChooser.getClass().getDeclaredField("serverConfs");
+    field.setAccessible(true);
+    field.set(preferredVolumeChooser, mockedServerConfigurationFactory);
+
+    mockedTableConfiguration = EasyMock.createMock(TableConfiguration.class);
+    mockedAccumuloConfiguration = EasyMock.createMock(AccumuloConfiguration.class);
+  }
+
+  private void configureDefaultVolumes(String configuredVolumes) {
+    EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
+    EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(configuredVolumes).anyTimes();
+  }
+
+  private void configureTableVolumes(String configuredVolumes) {
+    EasyMock.expect(mockedServerConfigurationFactory.getTableConfiguration(EasyMock.<Table.ID> anyObject())).andReturn(mockedTableConfiguration).anyTimes();
+    EasyMock.expect(mockedTableConfiguration.get(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES)).andReturn(configuredVolumes).anyTimes();
+  }
+
+  private void configureScopedVolumes(String configuredVolumes, String scope) {
+    EasyMock.expect(mockedServerConfigurationFactory.getSystemConfiguration()).andReturn(mockedAccumuloConfiguration).anyTimes();
+    EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.SCOPED_PREFERRED_VOLUMES(scope))).andReturn(configuredVolumes).anyTimes();
+  }
+
+  private void configureDefaultContextVolumes(String configuredVolumes) {
+    EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.DEFAULT_SCOPED_PREFERRED_VOLUMES)).andReturn(configuredVolumes).anyTimes();
+  }
+
+  private void configureContextVolumes(String configuredVolumes) {
+    EasyMock.expect(mockedAccumuloConfiguration.get(PreferredVolumeChooser.SCOPED_PREFERRED_VOLUMES("logger"))).andReturn(configuredVolumes).anyTimes();
+  }
+
+  private Set<String> chooseRepeatedlyForTable() {
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.of(Table.ID.of("h")));
+    Set<String> results = new HashSet<>();
+    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
+      results.add(preferredVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
+    }
+    return results;
+  }
+
+  private Set<String> chooseRepeatedlyForContext() {
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.empty());
+    volumeChooserEnvironment.setScope("logger");
+    Set<String> results = new HashSet<>();
+
+    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
+      results.add(preferredVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
+    }
+    return results;
+  }
+
+  @Test
+  public void testEmptyEnvUsesRandomChooser() throws Exception {
+    VolumeChooserEnvironment volumeChooserEnvironment = new VolumeChooserEnvironment(Optional.empty());
+    Set<String> results = new HashSet<>();
+    for (int i = 0; i < REQUIRED_NUMBER_TRIES; i++) {
+      results.add(preferredVolumeChooser.choose(volumeChooserEnvironment, ALL_OPTIONS));
+    }
+
+    Assert.assertEquals(Sets.newHashSet(Arrays.asList(ALL_OPTIONS)), results);
+  }
+
+  @Test
+  public void testTableConfig() throws Exception {
+    configureDefaultVolumes("1,3");
+    configureTableVolumes("1,2");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    Set<String> results = chooseRepeatedlyForTable();
+
+    EasyMock.verify(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+    Assert.assertEquals(Sets.newHashSet(Arrays.asList("1", "2")), results);
+  }
+
+  @Test
+  public void testTableMisconfigured() throws Exception {
+    configureDefaultVolumes("1,3");
+    configureTableVolumes("4");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testTableMissing() throws Exception {
+    configureDefaultVolumes("1,3");
+    configureTableVolumes(null);
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testTableEmptyConfig() throws Exception {
+    configureDefaultVolumes("1,3");
+    configureTableVolumes("");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testTableMisconfiguredAndDefaultEmpty() throws Exception {
+    configureDefaultVolumes("");
+    configureTableVolumes("4");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testTableAndDefaultEmpty() throws Exception {
+    configureDefaultVolumes("");
+    configureTableVolumes("");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedTableConfiguration, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForTable();
+  }
+
+  @Test
+  public void testContextConfig() throws Exception {
+    configureDefaultVolumes("1,3");
+    configureContextVolumes("1,2");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    Set<String> results = chooseRepeatedlyForContext();
+
+    EasyMock.verify(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+    Assert.assertEquals(Sets.newHashSet(Arrays.asList("1", "2")), results);
+  }
+
+  @Test
+  public void testContextMisconfigured() throws Exception {
+    configureDefaultVolumes("1,3");
+    configureContextVolumes("4");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+
+  @Test
+  public void testContextMissing() throws Exception {
+    configureDefaultVolumes("1,3");
+    configureContextVolumes(null);
+    configureDefaultContextVolumes(null);
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+
+  @Test
+  public void testContextMisconfiguredAndDefaultEmpty() throws Exception {
+    configureDefaultVolumes("");
+    configureContextVolumes("4");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+
+  @Test
+  public void testContextAndDefaultBothEmpty() throws Exception {
+    this.configureDefaultVolumes("");
+    configureContextVolumes("");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+
+  @Test
+  public void testContextEmptyConfig() throws Exception {
+    configureDefaultVolumes("1,3");
+    configureContextVolumes("");
+
+    EasyMock.replay(mockedServerConfigurationFactory, mockedAccumuloConfiguration);
+
+    thrown.expect(RuntimeException.class);
+    chooseRepeatedlyForContext();
+  }
+}
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 13557bf..328f8f2 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
@@ -28,7 +28,9 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 /**
  *
@@ -37,32 +39,50 @@ public class VolumeManagerImplTest {
 
   protected VolumeManager fs;
 
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
   @Before
   public void setup() throws Exception {
     fs = VolumeManagerImpl.getLocal(System.getProperty("user.dir"));
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void defaultTabletDirWithoutTableId() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
     fs.getFullPath(FileType.TABLE, "/default_tablet/");
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void tabletDirWithoutTableId() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
     fs.getFullPath(FileType.TABLE, "/t-0000001/");
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void defaultTabletFileWithoutTableId() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
     fs.getFullPath(FileType.TABLE, "/default_tablet/C0000001.rf");
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void tabletFileWithoutTableId() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
     fs.getFullPath(FileType.TABLE, "/t-0000001/C0000001.rf");
   }
 
   @Test
+  public void invalidChooserConfigured() throws Exception {
+    List<String> volumes = Arrays.asList("file://one/", "file://two/", "file://three/");
+    ConfigurationCopy conf = new ConfigurationCopy();
+    conf.set(INSTANCE_DFS_URI, volumes.get(0));
+    conf.set(Property.INSTANCE_VOLUMES, StringUtils.join(volumes, ","));
+    conf.set(Property.GENERAL_VOLUME_CHOOSER, "org.apache.accumulo.server.fs.ChooserThatDoesntExist");
+    thrown.expect(RuntimeException.class);
+    VolumeManagerImpl.get(conf);
+  }
+
+  @Test
   public void tabletDirWithTableId() throws Exception {
     String basePath = fs.getDefaultVolume().getBasePath();
     String scheme = fs.getDefaultVolume().getFileSystem().getUri().toURL().getProtocol();
@@ -88,10 +108,11 @@ public class VolumeManagerImplTest {
     }
   }
 
-  @Test(expected = IllegalArgumentException.class)
+  @Test
   public void noViewFS() throws Exception {
     ConfigurationCopy conf = new ConfigurationCopy();
     conf.set(Property.INSTANCE_VOLUMES, "viewfs://dummy");
+    thrown.expect(IllegalArgumentException.class);
     VolumeManagerImpl.get(conf);
   }
 
@@ -105,6 +126,7 @@ public class VolumeManagerImplTest {
   @SuppressWarnings("deprecation")
   private static final Property INSTANCE_DFS_URI = Property.INSTANCE_DFS_URI;
 
+  // Expected to throw a runtime exception when the WrongVolumeChooser picks an invalid volume.
   @Test
   public void chooseFromOptions() throws Exception {
     List<String> volumes = Arrays.asList("file://one/", "file://two/", "file://three/");
@@ -112,6 +134,7 @@ public class VolumeManagerImplTest {
     conf.set(INSTANCE_DFS_URI, volumes.get(0));
     conf.set(Property.INSTANCE_VOLUMES, StringUtils.join(volumes, ","));
     conf.set(Property.GENERAL_VOLUME_CHOOSER, WrongVolumeChooser.class.getName());
+    thrown.expect(RuntimeException.class);
     VolumeManager vm = VolumeManagerImpl.get(conf);
     VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironment(Optional.of(Table.ID.of("sometable")));
     String choice = vm.choose(chooserEnv, volumes.toArray(new String[0]));
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/RootFilesTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/RootFilesTest.java
index d9c6862..fe87344 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/RootFilesTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/RootFilesTest.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.fs.RandomVolumeChooser;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.fs.Path;
@@ -123,6 +124,7 @@ public class RootFilesTest {
     ConfigurationCopy conf = new ConfigurationCopy();
     conf.set(Property.INSTANCE_DFS_URI, "file:///");
     conf.set(Property.INSTANCE_DFS_DIR, "/");
+    conf.set(Property.GENERAL_VOLUME_CHOOSER, RandomVolumeChooser.class.getName());
 
     VolumeManager vm = VolumeManagerImpl.get(conf);
 
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 ae2e4cc..977cc36 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellConfigIT.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.harness.conf.StandaloneAccumuloClusterConfiguration;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.server.fs.PerTableVolumeChooser;
 import org.apache.accumulo.test.ShellServerIT.TestShell;
 import org.junit.After;
 import org.junit.Assert;
@@ -49,15 +50,15 @@ public class ShellConfigIT extends AccumuloClusterHarness {
     Connector conn = getConnector();
     // TABLE_VOLUME_CHOOSER is a valid property that can be updated in ZK, whereas the crypto properties are not.
     // This lets us run this test more generically rather than forcibly needing to update some property in accumulo-site.xml
-    origPropValue = conn.instanceOperations().getSystemConfiguration().get(Property.TABLE_VOLUME_CHOOSER.getKey());
-    conn.instanceOperations().setProperty(Property.TABLE_VOLUME_CHOOSER.getKey(), FairVolumeChooser.class.getName());
+    origPropValue = conn.instanceOperations().getSystemConfiguration().get(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER);
+    conn.instanceOperations().setProperty(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, FairVolumeChooser.class.getName());
   }
 
   @After
   public void resetProperty() throws Exception {
     if (null != origPropValue) {
       Connector conn = getConnector();
-      conn.instanceOperations().setProperty(Property.TABLE_VOLUME_CHOOSER.getKey(), origPropValue);
+      conn.instanceOperations().setProperty(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, origPropValue);
     }
   }
 
@@ -92,12 +93,11 @@ public class ShellConfigIT extends AccumuloClusterHarness {
       Assert.fail("Unknown token type");
     }
 
-    assertTrue(Property.TABLE_VOLUME_CHOOSER.isExperimental());
     assertTrue(Property.CRYPTO_CIPHER_ALGORITHM_NAME.isExperimental());
 
     String configOutput = ts.exec("config");
 
-    assertTrue(configOutput.contains(Property.TABLE_VOLUME_CHOOSER.getKey()));
+    assertTrue(configOutput.contains(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER));
     assertFalse(configOutput.contains(Property.CRYPTO_CIPHER_ALGORITHM_NAME.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
new file mode 100644
index 0000000..dc0cb41
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserFailureIT.java
@@ -0,0 +1,153 @@
+/*
+ * 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.test;
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.fs.PerTableVolumeChooser;
+import org.apache.accumulo.server.fs.PreferredVolumeChooser;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.Text;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class VolumeChooserFailureIT extends ConfigurableMacBase {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private File volDirBase;
+  private Path v1, v2, v3, v4;
+  private static String[] rows = "a,b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u,v,w,x,y,z".split(",");
+  private String namespace1;
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 60;
+  }
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    // Get 2 tablet servers
+    cfg.setNumTservers(2);
+    namespace1 = "ns_" + getUniqueNames(1)[0];
+
+    // Set the general volume chooser to the PerTableVolumeChooser so that different choosers can be specified
+    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());
+
+    // Set up 4 different volume paths
+    File baseDir = cfg.getDir();
+    volDirBase = new File(baseDir, "volumes");
+    File v1f = new File(volDirBase, "v1");
+    File v2f = new File(volDirBase, "v2");
+    File v3f = new File(volDirBase, "v3");
+    File v4f = new File(volDirBase, "v4");
+    v1 = new Path("file://" + v1f.getAbsolutePath());
+    v2 = new Path("file://" + v2f.getAbsolutePath());
+    v3 = new Path("file://" + v3f.getAbsolutePath());
+    v4 = new Path("file://" + v4f.getAbsolutePath());
+
+    cfg.setSiteConfig(siteConfig);
+
+    siteConfig.put(PerTableVolumeChooser.SCOPED_VOLUME_CHOOSER("logger"), PreferredVolumeChooser.class.getName());
+    // do not set preferred volumes
+    cfg.setSiteConfig(siteConfig);
+
+    // Only add volumes 1, 2, and 4 to the list of instance volumes to have one volume that isn't in the options list when they are choosing
+    cfg.setProperty(Property.INSTANCE_VOLUMES, v1.toString() + "," + v2.toString() + "," + v4.toString());
+    // no not set preferred volumes
+
+    // use raw local file system so walogs sync and flush will work
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+
+    super.configure(cfg, hadoopCoreSite);
+
+  }
+
+  public static void addSplits(Connector connector, String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    // Add 10 splits to the table
+    SortedSet<Text> partitions = new TreeSet<>();
+    for (String s : rows)
+      partitions.add(new Text(s));
+    connector.tableOperations().addSplits(tableName, partitions);
+  }
+
+  // Test that uses one table with 10 split points each. It uses the PreferredVolumeChooser, but no preferred volume is specified.
+  // This means that the volume chooser will fail and no instance volumes will be assigned.
+  @Test
+  public void missingVolumePreferredVolumeChooser() throws Exception {
+    log.info("Starting missingVolumePreferredVolumeChooser");
+
+    // Create namespace
+    Connector connector = getConnector();
+    connector.namespaceOperations().create(namespace1);
+
+    // Set properties on the namespace
+    connector.namespaceOperations().setProperty(namespace1, PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, PreferredVolumeChooser.class.getName());
+    // deliberately do not set preferred volumes
+
+    // Create table1 on namespace1 (will fail)
+    String tableName = namespace1 + ".1";
+    thrown.expect(AccumuloException.class);
+    connector.tableOperations().create(tableName);
+  }
+
+  // Test that uses one table with 10 split points each. It uses the PreferredVolumeChooser, but preferred volume is not an instance volume.
+  // This should fail.
+  @Test
+  public void notInstancePreferredVolumeChooser() throws Exception {
+    log.info("Starting notInstancePreferredVolumeChooser");
+
+    // Create namespace
+    Connector connector = getConnector();
+    connector.namespaceOperations().create(namespace1);
+
+    // Set properties on the namespace
+    String propertyName = PerTableVolumeChooser.TABLE_VOLUME_CHOOSER;
+    String volume = PreferredVolumeChooser.class.getName();
+    connector.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;
+    volume = v3.toString();
+    connector.namespaceOperations().setProperty(namespace1, propertyName, volume);
+
+    // Create table1 on namespace1 (will fail)
+    String tableName = namespace1 + ".1";
+    thrown.expect(AccumuloException.class);
+    connector.tableOperations().create(tableName);
+  }
+
+}
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 4d940a8..b894681 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.util.ArrayList;
@@ -63,14 +64,16 @@ public class VolumeChooserIT extends ConfigurableMacBase {
   private static final Text EMPTY = new Text();
   private static final Value EMPTY_VALUE = new Value(new byte[] {});
   private File volDirBase;
+  @SuppressWarnings("unused")
   private Path v1, v2, v3, v4;
-  private String[] rows = "a,b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u,v,w,x,y,z".split(",");
+  private static String[] rows = "a,b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u,v,w,x,y,z".split(",");
   private String namespace1;
   private String namespace2;
+  private String systemPreferredVolumes;
 
   @Override
   protected int defaultTimeoutSeconds() {
-    return 30;
+    return 60;
   }
 
   @Override
@@ -83,7 +86,8 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     // Set the general volume chooser to the PerTableVolumeChooser so that different choosers can be specified
     Map<String,String> siteConfig = new HashMap<>();
     siteConfig.put(Property.GENERAL_VOLUME_CHOOSER.getKey(), PerTableVolumeChooser.class.getName());
-    cfg.setSiteConfig(siteConfig);
+    // if a table doesn't have a volume chooser, use the preferred volume chooser
+    siteConfig.put(PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, PreferredVolumeChooser.class.getName());
 
     // Set up 4 different volume paths
     File baseDir = cfg.getDir();
@@ -97,6 +101,14 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     v3 = new Path("file://" + v3f.getAbsolutePath());
     v4 = new Path("file://" + v4f.getAbsolutePath());
 
+    systemPreferredVolumes = v1.toString() + "," + v2.toString();
+    siteConfig.put(PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES, systemPreferredVolumes); // exclude v4
+    cfg.setSiteConfig(siteConfig);
+
+    siteConfig.put(PerTableVolumeChooser.SCOPED_VOLUME_CHOOSER("logger"), PreferredVolumeChooser.class.getName());
+    siteConfig.put(PreferredVolumeChooser.SCOPED_PREFERRED_VOLUMES("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 the options list when they are choosing
     cfg.setProperty(Property.INSTANCE_VOLUMES, v1.toString() + "," + v2.toString() + "," + v4.toString());
 
@@ -107,23 +119,16 @@ public class VolumeChooserIT extends ConfigurableMacBase {
 
   }
 
-  public void addSplits(Connector connector, String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+  public static void addSplits(Connector connector, String tableName) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     // Add 10 splits to the table
     SortedSet<Text> partitions = new TreeSet<>();
-    for (String s : "b,e,g,j,l,o,q,t,v,y".split(","))
+    for (String s : rows)
       partitions.add(new Text(s));
     connector.tableOperations().addSplits(tableName, partitions);
   }
 
-  public void writeAndReadData(Connector connector, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    // Write some data to the table
-    BatchWriter bw = connector.createBatchWriter(tableName, new BatchWriterConfig());
-    for (String s : rows) {
-      Mutation m = new Mutation(new Text(s));
-      m.put(EMPTY, EMPTY, EMPTY_VALUE);
-      bw.addMutation(m);
-    }
-    bw.close();
+  public static void writeAndReadData(Connector connector, String tableName) throws Exception {
+    writeDataToTable(connector, tableName);
 
     // Write the data to disk, read it back
     connector.tableOperations().flush(tableName, null, null, true);
@@ -134,12 +139,24 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     }
   }
 
-  public void verifyVolumes(Connector connector, String tableName, Range tableRange, String vol) throws TableNotFoundException {
+  public static void writeDataToTable(Connector connector, String tableName) throws Exception {
+    // Write some data to the table
+    BatchWriter bw = connector.createBatchWriter(tableName, new BatchWriterConfig());
+    for (String s : rows) {
+      Mutation m = new Mutation(new Text(s));
+      m.put(EMPTY, EMPTY, EMPTY_VALUE);
+      bw.addMutation(m);
+    }
+    bw.close();
+  }
+
+  public static void verifyVolumes(Connector connector, String tableName, Range tableRange, String vol) throws Exception {
     // Verify the new files are written to the Volumes specified
     ArrayList<String> volumes = new ArrayList<>();
     for (String s : vol.split(","))
       volumes.add(s);
 
+    TreeSet<String> volumesSeen = new TreeSet<>();
     Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     scanner.setRange(tableRange);
     scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
@@ -147,35 +164,37 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     for (Entry<Key,Value> entry : scanner) {
       boolean inVolume = false;
       for (String volume : volumes) {
-        if (entry.getKey().getColumnQualifier().toString().contains(volume))
+        if (entry.getKey().getColumnQualifier().toString().contains(volume)) {
+          volumesSeen.add(volume);
           inVolume = true;
+        }
       }
-      assertTrue("Data not written to the correct volumes", inVolume);
+      assertTrue("Data not written to the correct volumes.  " + entry.getKey().getColumnQualifier().toString(), inVolume);
       fileCount++;
     }
-    assertEquals("Wrong number of files", 11, fileCount);
+    assertEquals("Did not see all the volumes. volumes: " + volumes.toString() + " volumes seen: " + volumesSeen.toString(), volumes.size(), volumesSeen.size());
+    assertEquals("Wrong number of files", 26, fileCount);
   }
 
-  // Test that uses two tables with 10 split points each. They each use the PreferredVolumeChooser to choose volumes.
-  @Test
-  public void twoTablesPreferredVolumeChooser() throws Exception {
-    log.info("Starting twoTablesPreferredVolumeChooser");
-
-    // Create namespace
-    Connector connector = getConnector();
-    connector.namespaceOperations().create(namespace1);
+  public static void verifyNoVolumes(Connector connector, String tableName, Range tableRange) throws Exception {
+    Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    scanner.setRange(tableRange);
+    scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
+    for (Entry<Key,Value> entry : scanner) {
+      fail("Data incorrectly written to " + entry.getKey().getColumnQualifier().toString());
+    }
+  }
 
+  private void configureNamespace(Connector connector, String volumeChooserClassName, String configuredVolumes, String namespace) throws Exception {
+    connector.namespaceOperations().create(namespace);
     // Set properties on the namespace
-    String propertyName = Property.TABLE_VOLUME_CHOOSER.getKey();
-    String volume = PreferredVolumeChooser.class.getName();
-    connector.namespaceOperations().setProperty(namespace1, propertyName, volume);
+    connector.namespaceOperations().setProperty(namespace, PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, volumeChooserClassName);
+    connector.namespaceOperations().setProperty(namespace, PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES, configuredVolumes);
+  }
 
-    propertyName = "table.custom.preferredVolumes";
-    volume = v2.toString();
-    connector.namespaceOperations().setProperty(namespace1, propertyName, volume);
+  private void verifyVolumesForWritesToNewTable(Connector connector, String myNamespace, String expectedVolumes) throws Exception {
+    String tableName = myNamespace + ".1";
 
-    // Create table1 on namespace1
-    String tableName = namespace1 + ".1";
     connector.tableOperations().create(tableName);
     Table.ID tableID = Table.ID.of(connector.tableOperations().tableIdMap().get(tableName));
 
@@ -184,94 +203,67 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     // Write some data to the table
     writeAndReadData(connector, tableName);
     // Verify the new files are written to the Volumes specified
-    verifyVolumes(connector, tableName, TabletsSection.getRange(tableID), volume);
-
-    connector.namespaceOperations().create(namespace2);
-
-    // Set properties on the namespace
-    propertyName = Property.TABLE_VOLUME_CHOOSER.getKey();
-    volume = PreferredVolumeChooser.class.getName();
-    connector.namespaceOperations().setProperty(namespace2, propertyName, volume);
-
-    propertyName = "table.custom.preferredVolumes";
-    volume = v1.toString();
-    connector.namespaceOperations().setProperty(namespace2, propertyName, volume);
-
-    // Create table2 on namespace2
-    String tableName2 = namespace2 + ".1";
-
-    connector.tableOperations().create(tableName2);
-    Table.ID tableID2 = Table.ID.of(connector.tableOperations().tableIdMap().get(tableName2));
+    verifyVolumes(connector, tableName, TabletsSection.getRange(tableID), expectedVolumes);
+  }
 
-    // Add 10 splits to the table
-    addSplits(connector, tableName2);
-    // Write some data to the table
-    writeAndReadData(connector, tableName2);
+  public static void verifyWaLogVolumes(Connector connector, Range tableRange, String vol) throws TableNotFoundException {
     // Verify the new files are written to the Volumes specified
-    verifyVolumes(connector, tableName2, TabletsSection.getRange(tableID2), volume);
+    ArrayList<String> volumes = new ArrayList<>();
+    for (String s : vol.split(","))
+      volumes.add(s);
+
+    TreeSet<String> volumesSeen = new TreeSet<>();
+    Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    scanner.setRange(tableRange);
+    scanner.fetchColumnFamily(TabletsSection.LogColumnFamily.NAME);
+    for (Entry<Key,Value> entry : scanner) {
+      boolean inVolume = false;
+      for (String volume : volumes) {
+        if (entry.getKey().getColumnQualifier().toString().contains(volume))
+          volumesSeen.add(volume);
+        inVolume = true;
+      }
+      assertTrue("Data not written to the correct volumes.  " + entry.getKey().getColumnQualifier().toString(), inVolume);
+    }
   }
 
-  // Test that uses two tables with 10 split points each. They each use the RandomVolumeChooser to choose volumes.
+  // Test that uses two tables with 10 split points each. They each use the PreferredVolumeChooser to choose volumes.
   @Test
-  public void twoTablesRandomVolumeChooser() throws Exception {
-    log.info("Starting twoTablesRandomVolumeChooser()");
+  public void twoTablesPreferredVolumeChooser() throws Exception {
+    log.info("Starting twoTablesPreferredVolumeChooser");
 
     // Create namespace
     Connector connector = getConnector();
     connector.namespaceOperations().create(namespace1);
 
     // Set properties on the namespace
-    String propertyName = Property.TABLE_VOLUME_CHOOSER.getKey();
-    String volume = RandomVolumeChooser.class.getName();
-    connector.namespaceOperations().setProperty(namespace1, propertyName, volume);
+    // namespace 1 -> v2
+    connector.namespaceOperations().setProperty(namespace1, PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, PreferredVolumeChooser.class.getName());
+    connector.namespaceOperations().setProperty(namespace1, PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES, v2.toString());
 
     // Create table1 on namespace1
-    String tableName = namespace1 + ".1";
-    connector.tableOperations().create(tableName);
-    Table.ID tableID = Table.ID.of(connector.tableOperations().tableIdMap().get(tableName));
-
-    // Add 10 splits to the table
-    addSplits(connector, tableName);
-    // Write some data to the table
-    writeAndReadData(connector, tableName);
-    // Verify the new files are written to the Volumes specified
-
-    verifyVolumes(connector, tableName, TabletsSection.getRange(tableID), v1.toString() + "," + v2.toString() + "," + v4.toString());
+    verifyVolumesForWritesToNewTable(connector, namespace1, v2.toString());
 
     connector.namespaceOperations().create(namespace2);
-
     // Set properties on the namespace
-    propertyName = Property.TABLE_VOLUME_CHOOSER.getKey();
-    volume = RandomVolumeChooser.class.getName();
-    connector.namespaceOperations().setProperty(namespace2, propertyName, volume);
+    connector.namespaceOperations().setProperty(namespace2, PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, PreferredVolumeChooser.class.getName());
+    connector.namespaceOperations().setProperty(namespace2, PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES, v1.toString());
 
     // Create table2 on namespace2
-    String tableName2 = namespace2 + ".1";
-    connector.tableOperations().create(tableName2);
-    Table.ID tableID2 = Table.ID.of(connector.tableOperations().tableIdMap().get(tableName2));
-
-    // / Add 10 splits to the table
-    addSplits(connector, tableName2);
-    // Write some data to the table
-    writeAndReadData(connector, tableName2);
-    // Verify the new files are written to the Volumes specified
-    verifyVolumes(connector, tableName2, TabletsSection.getRange(tableID2), v1.toString() + "," + v2.toString() + "," + v4.toString());
+    verifyVolumesForWritesToNewTable(connector, namespace2, v1.toString());
   }
 
-  // Test that uses two tables with 10 split points each. The first uses the RandomVolumeChooser and the second uses the
-  // StaticVolumeChooser to choose volumes.
+  // Test that uses two tables with 10 split points each. They each use the RandomVolumeChooser to choose volumes.
   @Test
-  public void twoTablesDiffChoosers() throws Exception {
-    log.info("Starting twoTablesDiffChoosers");
+  public void twoTablesRandomVolumeChooser() throws Exception {
+    log.info("Starting twoTablesRandomVolumeChooser()");
 
     // Create namespace
     Connector connector = getConnector();
     connector.namespaceOperations().create(namespace1);
 
     // Set properties on the namespace
-    String propertyName = Property.TABLE_VOLUME_CHOOSER.getKey();
-    String volume = RandomVolumeChooser.class.getName();
-    connector.namespaceOperations().setProperty(namespace1, propertyName, volume);
+    connector.namespaceOperations().setProperty(namespace1, PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, RandomVolumeChooser.class.getName());
 
     // Create table1 on namespace1
     String tableName = namespace1 + ".1";
@@ -289,105 +281,67 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     connector.namespaceOperations().create(namespace2);
 
     // Set properties on the namespace
-    propertyName = Property.TABLE_VOLUME_CHOOSER.getKey();
-    volume = PreferredVolumeChooser.class.getName();
-    connector.namespaceOperations().setProperty(namespace2, propertyName, volume);
-
-    propertyName = "table.custom.preferredVolumes";
-    volume = v1.toString();
-    connector.namespaceOperations().setProperty(namespace2, propertyName, volume);
+    connector.namespaceOperations().setProperty(namespace2, PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, RandomVolumeChooser.class.getName());
 
     // Create table2 on namespace2
     String tableName2 = namespace2 + ".1";
     connector.tableOperations().create(tableName2);
     Table.ID tableID2 = Table.ID.of(connector.tableOperations().tableIdMap().get(tableName2));
 
-    // Add 10 splits to the table
+    // / Add 10 splits to the table
     addSplits(connector, tableName2);
     // Write some data to the table
     writeAndReadData(connector, tableName2);
     // Verify the new files are written to the Volumes specified
-    verifyVolumes(connector, tableName2, TabletsSection.getRange(tableID2), volume);
+    verifyVolumes(connector, tableName2, TabletsSection.getRange(tableID2), v1.toString() + "," + v2.toString() + "," + v4.toString());
   }
 
-  // Test that uses one table with 10 split points each. It uses the StaticVolumeChooser, but no preferred volume is specified. This means that the volume
-  // is chosen randomly from all instance volumes.
+  // Test that uses two tables with 10 split points each. The first uses the RandomVolumeChooser and the second uses the
+  // StaticVolumeChooser to choose volumes.
   @Test
-  public void missingVolumePreferredVolumeChooser() throws Exception {
-    log.info("Starting missingVolumePreferredVolumeChooser");
+  public void twoTablesDiffChoosers() throws Exception {
+    log.info("Starting twoTablesDiffChoosers");
 
     // Create namespace
     Connector connector = getConnector();
     connector.namespaceOperations().create(namespace1);
 
     // Set properties on the namespace
-    String propertyName = Property.TABLE_VOLUME_CHOOSER.getKey();
-    String volume = PreferredVolumeChooser.class.getName();
-    connector.namespaceOperations().setProperty(namespace1, propertyName, volume);
+    connector.namespaceOperations().setProperty(namespace1, PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, RandomVolumeChooser.class.getName());
 
     // Create table1 on namespace1
-    String tableName = namespace1 + ".1";
-    connector.tableOperations().create(tableName);
-    Table.ID tableID = Table.ID.of(connector.tableOperations().tableIdMap().get(tableName));
+    verifyVolumesForWritesToNewTable(connector, namespace1, v1.toString() + "," + v2.toString() + "," + v4.toString());
+    connector.namespaceOperations().create(namespace2);
 
-    // Add 10 splits to the table
-    addSplits(connector, tableName);
-    // Write some data to the table
-    writeAndReadData(connector, tableName);
-    // Verify the new files are written to the Volumes specified
-    verifyVolumes(connector, tableName, TabletsSection.getRange(tableID), v1.toString() + "," + v2.toString() + "," + v4.toString());
+    connector.namespaceOperations().setProperty(namespace2, PerTableVolumeChooser.TABLE_VOLUME_CHOOSER, PreferredVolumeChooser.class.getName());
+    connector.namespaceOperations().setProperty(namespace2, PreferredVolumeChooser.TABLE_PREFERRED_VOLUMES, v1.toString());
+
+    // Create table2 on namespace2
+    verifyVolumesForWritesToNewTable(connector, namespace2, v1.toString());
   }
 
-  // Test that uses one table with 10 split points each. It uses the PreferredVolumeChooser, but preferred volume is not an instance volume. This means that the
-  // volume is chosen randomly from all instance volumes
   @Test
-  public void notInstancePreferredVolumeChooser() throws Exception {
-    log.info("Starting notInstancePreferredVolumeChooser");
-
-    // Create namespace
+  public void includeSpecialVolumeForTable() throws Exception {
+    log.info("Starting includeSpecialVolumeForTable");
     Connector connector = getConnector();
-    connector.namespaceOperations().create(namespace1);
 
-    // Set properties on the namespace
-    String propertyName = Property.TABLE_VOLUME_CHOOSER.getKey();
-    String volume = PreferredVolumeChooser.class.getName();
-    connector.namespaceOperations().setProperty(namespace1, propertyName, volume);
-
-    propertyName = "table.custom.preferredVolumes";
-    volume = v3.toString();
-    connector.namespaceOperations().setProperty(namespace1, propertyName, volume);
-
-    // Create table1 on namespace1
-    String tableName = namespace1 + ".1";
-    connector.tableOperations().create(tableName);
-    Table.ID tableID = Table.ID.of(connector.tableOperations().tableIdMap().get(tableName));
-
-    // Add 10 splits to the table
-    addSplits(connector, tableName);
-    // Write some data to the table
-    writeAndReadData(connector, tableName);
-    // Verify the new files are written to the Volumes specified
-    verifyVolumes(connector, tableName, TabletsSection.getRange(tableID), v1.toString() + "," + v2.toString() + "," + v4.toString());
+    // the following table will be configured to go to the excluded volume
+    String configuredVolumes = v4.toString();
+    configureNamespace(connector, PreferredVolumeChooser.class.getName(), configuredVolumes, namespace2);
+    verifyVolumesForWritesToNewTable(connector, namespace2, configuredVolumes);
   }
 
-  // Test that uses one table with 10 split points each. It does not specify a specific chooser, so the volume is chosen randomly from all instance volumes.
   @Test
-  public void chooserNotSpecified() throws Exception {
-    log.info("Starting chooserNotSpecified");
+  public void waLogsSentToConfiguredVolumes() throws Exception {
+    log.info("Starting waLogsSentToConfiguredVolumes");
 
-    // Create a table
     Connector connector = getConnector();
-    String tableName = getUniqueNames(2)[0];
+    String tableName = "anotherTable";
     connector.tableOperations().create(tableName);
-    Table.ID tableID = Table.ID.of(connector.tableOperations().tableIdMap().get(tableName));
 
-    // Add 10 splits to the table
-    addSplits(connector, tableName);
-    // Write some data to the table
-    writeAndReadData(connector, tableName);
-
-    // Verify the new files are written to the Volumes specified
-    verifyVolumes(connector, tableName, TabletsSection.getRange(tableID), v1.toString() + "," + v2.toString() + "," + v4.toString());
+    VolumeChooserIT.addSplits(connector, tableName);
+    VolumeChooserIT.writeDataToTable(connector, tableName);
+    // should only go to v2 as per configuration in configure()
+    VolumeChooserIT.verifyWaLogVolumes(connector, new Range(), v2.toString());
   }
-
 }

-- 
To stop receiving notification emails like this one, please contact
"commits@accumulo.apache.org" <co...@accumulo.apache.org>.