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

[accumulo] branch main updated: fixes #1900 Moves volume choosers to SPI (#1905)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 833d56a  fixes #1900 Moves volume choosers to SPI (#1905)
833d56a is described below

commit 833d56acb389c50c5483e23e0242651f13650288
Author: Keith Turner <kt...@apache.org>
AuthorDate: Thu Feb 4 17:57:53 2021 -0500

    fixes #1900 Moves volume choosers to SPI (#1905)
---
 .../org/apache/accumulo/core/conf/Property.java    |   5 +-
 .../core/spi}/fs/PerTableVolumeChooser.java        |  38 ++---
 .../core/spi}/fs/PreferredVolumeChooser.java       |  36 ++---
 .../accumulo/core/spi}/fs/RandomVolumeChooser.java |  11 +-
 .../core/spi}/fs/SpaceAwareVolumeChooser.java      |  23 ++-
 .../apache/accumulo/core/spi/fs/VolumeChooser.java |  62 ++++++++
 .../core/spi}/fs/VolumeChooserEnvironment.java     |  36 ++---
 .../core/spi}/fs/PerTableVolumeChooserTest.java    | 109 ++++++++------
 .../core/spi}/fs/PreferredVolumeChooserTest.java   | 120 +++++++++-------
 .../core/spi}/fs/SpaceAwareVolumeChooserTest.java  |  56 +++-----
 .../accumulo/server/fs/PerTableVolumeChooser.java  | 156 ++-------------------
 .../accumulo/server/fs/PreferredVolumeChooser.java | 136 ++----------------
 .../accumulo/server/fs/RandomVolumeChooser.java    |  29 ++--
 .../server/fs/SpaceAwareVolumeChooser.java         | 122 ++--------------
 .../apache/accumulo/server/fs/VolumeChooser.java   |  55 +++-----
 .../server/fs/VolumeChooserEnvironment.java        |  27 +++-
 .../server/fs/VolumeChooserEnvironmentImpl.java    |  39 +++---
 .../apache/accumulo/server/fs/VolumeManager.java   |   5 +-
 .../accumulo/server/fs/VolumeManagerImpl.java      |  14 +-
 .../apache/accumulo/server/init/Initialize.java    |  16 +--
 .../accumulo/server/fs/VolumeManagerImplTest.java  |   9 +-
 .../manager/upgrade/RootFilesUpgradeTest.java      |   2 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |   4 +-
 .../org/apache/accumulo/tserver/log/DfsLogger.java |   4 +-
 .../org/apache/accumulo/tserver/tablet/Tablet.java |   2 +-
 .../tserver/TabletServerSyncCheckTest.java         |   2 +-
 .../apache/accumulo/test/FairVolumeChooser.java    |   4 +-
 .../org/apache/accumulo/test/VolumeChooserIT.java  |  20 +--
 28 files changed, 454 insertions(+), 688 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 212a1b0..04015e3 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
@@ -35,6 +35,7 @@ import org.apache.accumulo.core.iteratorsImpl.system.DeletingIterator;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
 import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
+import org.apache.accumulo.core.spi.fs.RandomVolumeChooser;
 import org.apache.accumulo.core.spi.scan.ScanDispatcher;
 import org.apache.accumulo.core.spi.scan.ScanPrioritizer;
 import org.apache.accumulo.core.spi.scan.SimpleScanDispatcher;
@@ -226,8 +227,8 @@ public enum Property {
   // 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.RandomVolumeChooser", PropertyType.CLASSNAME,
+  GENERAL_VOLUME_CHOOSER("general.volume.chooser", RandomVolumeChooser.class.getName(),
+      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"),
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/PerTableVolumeChooser.java
similarity index 87%
copy from server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
copy to core/src/main/java/org/apache/accumulo/core/spi/fs/PerTableVolumeChooser.java
index 784ed79..ac9ab06 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/PerTableVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/PerTableVolumeChooser.java
@@ -16,13 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.fs;
+package org.apache.accumulo.core.spi.fs;
 
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -32,6 +32,8 @@ import org.slf4j.LoggerFactory;
  * general.custom.volume.chooser.{scope} can override the system wide setting for
  * general.custom.volume.chooser.scoped. At the this this was written, the only known scope was
  * "logger".
+ *
+ * @since 2.1.0
  */
 public class PerTableVolumeChooser implements VolumeChooser {
   // TODO rename this class to DelegatingChooser? It delegates for more than just per-table scope
@@ -40,44 +42,42 @@ public class PerTableVolumeChooser implements VolumeChooser {
   /* Track VolumeChooser instances so they can keep state. */
   private final ConcurrentHashMap<TableId,VolumeChooser> tableSpecificChooserCache =
       new ConcurrentHashMap<>();
-  private final ConcurrentHashMap<ChooserScope,VolumeChooser> scopeSpecificChooserCache =
+  private final ConcurrentHashMap<Scope,VolumeChooser> scopeSpecificChooserCache =
       new ConcurrentHashMap<>();
 
   private static final String TABLE_CUSTOM_SUFFIX = "volume.chooser";
 
-  private static final String getCustomPropertySuffix(ChooserScope scope) {
+  private static final String getCustomPropertySuffix(Scope scope) {
     return "volume.chooser." + scope.name().toLowerCase();
   }
 
   private static final String DEFAULT_SCOPED_VOLUME_CHOOSER =
-      getCustomPropertySuffix(ChooserScope.DEFAULT);
+      getCustomPropertySuffix(Scope.DEFAULT);
 
   @Override
-  public String choose(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
+  public String choose(VolumeChooserEnvironment env, Set<String> options) {
     log.trace("{}.choose", getClass().getSimpleName());
     return getDelegateChooser(env).choose(env, options);
   }
 
   @Override
-  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
+  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options) {
     return getDelegateChooser(env).choosable(env, options);
   }
 
   // visible (not private) for testing
   VolumeChooser getDelegateChooser(VolumeChooserEnvironment env) {
-    if (env.getScope() == ChooserScope.TABLE) {
+    if (env.getChooserScope() == Scope.TABLE) {
       return getVolumeChooserForTable(env);
     }
     return getVolumeChooserForScope(env);
   }
 
   private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env) {
-    log.trace("Looking up property {} for table id: {}", TABLE_CUSTOM_SUFFIX, env.getTableId());
+    log.trace("Looking up property {} for table id: {}", TABLE_CUSTOM_SUFFIX, env.getTable());
 
-    String clazz =
-        env.getServiceEnv().getConfiguration(env.getTableId()).getTableCustom(TABLE_CUSTOM_SUFFIX);
+    String clazz = env.getServiceEnv().getConfiguration(env.getTable().get())
+        .getTableCustom(TABLE_CUSTOM_SUFFIX);
 
     // fall back to global default scope, so setting only one default is necessary, rather than a
     // separate default for TABLE scope than other scopes
@@ -89,15 +89,15 @@ public class PerTableVolumeChooser implements VolumeChooser {
       String msg = "Property " + TABLE_CUSTOM_SUFFIX + " or " + DEFAULT_SCOPED_VOLUME_CHOOSER
           + " must be a valid " + VolumeChooser.class.getSimpleName() + " to use the "
           + getClass().getSimpleName();
-      throw new VolumeChooserException(msg);
+      throw new RuntimeException(msg);
     }
 
-    return createVolumeChooser(env, clazz, TABLE_CUSTOM_SUFFIX, env.getTableId(),
+    return createVolumeChooser(env, clazz, TABLE_CUSTOM_SUFFIX, env.getTable().get(),
         tableSpecificChooserCache);
   }
 
   private VolumeChooser getVolumeChooserForScope(VolumeChooserEnvironment env) {
-    ChooserScope scope = env.getScope();
+    Scope scope = env.getChooserScope();
     String property = getCustomPropertySuffix(scope);
     log.trace("Looking up property {} for scope: {}", property, scope);
 
@@ -105,7 +105,7 @@ public class PerTableVolumeChooser implements VolumeChooser {
 
     // fall back to global default scope if this scope isn't configured (and not already default
     // scope)
-    if ((clazz == null || clazz.isEmpty()) && scope != ChooserScope.DEFAULT) {
+    if ((clazz == null || clazz.isEmpty()) && scope != Scope.DEFAULT) {
       log.debug("{} not found; using {}", property, DEFAULT_SCOPED_VOLUME_CHOOSER);
       clazz = env.getServiceEnv().getConfiguration().getCustom(DEFAULT_SCOPED_VOLUME_CHOOSER);
 
@@ -113,7 +113,7 @@ public class PerTableVolumeChooser implements VolumeChooser {
         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);
+        throw new RuntimeException(msg);
       }
 
       property = DEFAULT_SCOPED_VOLUME_CHOOSER;
@@ -160,7 +160,7 @@ public class PerTableVolumeChooser implements VolumeChooser {
       } catch (Exception e) {
         String msg = "Failed to create instance for " + key + " configured to use " + className
             + " via " + property;
-        throw new VolumeChooserException(msg, e);
+        throw new RuntimeException(msg, e);
       }
     });
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java
similarity index 86%
copy from server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java
copy to core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java
index b246a6e..aba6d57 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/PreferredVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooser.java
@@ -16,15 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.fs;
+package org.apache.accumulo.core.spi.fs;
 
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
 import org.apache.accumulo.core.volume.Volume;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,22 +34,23 @@ import org.slf4j.LoggerFactory;
  * presented. Can be customized via the table property table.custom.volume.preferred, which should
  * contain a comma separated list of {@link Volume} URIs. Note that both the property name and the
  * format of its value are specific to this particular implementation.
+ *
+ * @since 2.1.0
  */
 public class PreferredVolumeChooser extends RandomVolumeChooser {
   private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
 
   private static final String TABLE_CUSTOM_SUFFIX = "volume.preferred";
 
-  private static final String getCustomPropertySuffix(ChooserScope scope) {
+  private static final String getCustomPropertySuffix(Scope scope) {
     return "volume.preferred." + scope.name().toLowerCase();
   }
 
   private static final String DEFAULT_SCOPED_PREFERRED_VOLUMES =
-      getCustomPropertySuffix(ChooserScope.DEFAULT);
+      getCustomPropertySuffix(Scope.DEFAULT);
 
   @Override
-  public String choose(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
+  public String choose(VolumeChooserEnvironment env, Set<String> options) {
     log.trace("{}.choose", getClass().getSimpleName());
     // Randomly choose the volume from the preferred volumes
     String choice = super.choose(env, getPreferredVolumes(env, options));
@@ -58,14 +59,13 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
   }
 
   @Override
-  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
+  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options) {
     return getPreferredVolumes(env, options);
   }
 
   // visible (not private) for testing
   Set<String> getPreferredVolumes(VolumeChooserEnvironment env, Set<String> options) {
-    if (env.getScope() == ChooserScope.TABLE) {
+    if (env.getChooserScope() == Scope.TABLE) {
       return getPreferredVolumesForTable(env, options);
     }
     return getPreferredVolumesForScope(env, options);
@@ -73,10 +73,10 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
 
   private Set<String> getPreferredVolumesForTable(VolumeChooserEnvironment env,
       Set<String> options) {
-    log.trace("Looking up property {} + for Table id: {}", TABLE_CUSTOM_SUFFIX, env.getTableId());
+    log.trace("Looking up property {} + for Table id: {}", TABLE_CUSTOM_SUFFIX, env.getTable());
 
-    String preferredVolumes =
-        env.getServiceEnv().getConfiguration(env.getTableId()).getTableCustom(TABLE_CUSTOM_SUFFIX);
+    String preferredVolumes = env.getServiceEnv().getConfiguration(env.getTable().get())
+        .getTableCustom(TABLE_CUSTOM_SUFFIX);
 
     // fall back to global default scope, so setting only one default is necessary, rather than a
     // separate default for TABLE scope than other scopes
@@ -89,7 +89,7 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
     if (preferredVolumes == null || preferredVolumes.isEmpty()) {
       String msg = "Property " + TABLE_CUSTOM_SUFFIX + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
           + " must be a subset of " + options + " to use the " + getClass().getSimpleName();
-      throw new VolumeChooserException(msg);
+      throw new RuntimeException(msg);
     }
 
     return parsePreferred(TABLE_CUSTOM_SUFFIX, preferredVolumes, options);
@@ -97,7 +97,7 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
 
   private Set<String> getPreferredVolumesForScope(VolumeChooserEnvironment env,
       Set<String> options) {
-    ChooserScope scope = env.getScope();
+    Scope scope = env.getChooserScope();
     String property = getCustomPropertySuffix(scope);
     log.trace("Looking up property {} for scope: {}", property, scope);
 
@@ -105,7 +105,7 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
 
     // fall back to global default scope if this scope isn't configured (and not already default
     // scope)
-    if ((preferredVolumes == null || preferredVolumes.isEmpty()) && scope != ChooserScope.DEFAULT) {
+    if ((preferredVolumes == null || preferredVolumes.isEmpty()) && scope != Scope.DEFAULT) {
       log.debug("{} not found; using {}", property, DEFAULT_SCOPED_PREFERRED_VOLUMES);
       preferredVolumes =
           env.getServiceEnv().getConfiguration().getCustom(DEFAULT_SCOPED_PREFERRED_VOLUMES);
@@ -115,7 +115,7 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
       if (preferredVolumes == null || preferredVolumes.isEmpty()) {
         String msg = "Property " + property + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
             + " must be a subset of " + options + " to use the " + getClass().getSimpleName();
-        throw new VolumeChooserException(msg);
+        throw new RuntimeException(msg);
       }
 
       property = DEFAULT_SCOPED_PREFERRED_VOLUMES;
@@ -133,13 +133,13 @@ public class PreferredVolumeChooser extends RandomVolumeChooser {
     if (preferred.isEmpty()) {
       String msg = "No volumes could be parsed from '" + property + "', which had a value of '"
           + preferredVolumes + "'";
-      throw new VolumeChooserException(msg);
+      throw new RuntimeException(msg);
     }
     // preferred volumes should also exist in the original options (typically, from
     // instance.volumes)
     if (Collections.disjoint(preferred, options)) {
       String msg = "Some volumes in " + preferred + " are not valid volumes from " + options;
-      throw new VolumeChooserException(msg);
+      throw new RuntimeException(msg);
     }
 
     return preferred;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java
similarity index 90%
copy from server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java
copy to core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java
index 8628e33..544eb2a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/RandomVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/RandomVolumeChooser.java
@@ -16,25 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.fs;
+package org.apache.accumulo.core.spi.fs;
 
 import java.security.SecureRandom;
 import java.util.Random;
 import java.util.Set;
 
+/**
+ * @since 2.1.0
+ */
 public class RandomVolumeChooser implements VolumeChooser {
   protected final Random random = new SecureRandom();
 
   @Override
-  public String choose(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
+  public String choose(VolumeChooserEnvironment env, Set<String> options) {
     String[] optionsArray = options.toArray(new String[0]);
     return optionsArray[random.nextInt(optionsArray.length)];
   }
 
   @Override
-  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
+  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options) {
     return options;
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java
similarity index 88%
copy from server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
copy to core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java
index b17c47b..8c3b5cb 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooser.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.fs;
+package org.apache.accumulo.core.spi.fs;
 
 import java.io.IOException;
 import java.util.NavigableMap;
@@ -27,8 +27,10 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -40,6 +42,8 @@ import com.google.common.cache.LoadingCache;
  * A {@link PreferredVolumeChooser} that takes remaining HDFS space into account when making a
  * volume choice rather than a simpler round robin. The list of volumes to use can be limited using
  * the same properties as {@link PreferredVolumeChooser}
+ *
+ * @since 2.1.0
  */
 public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
 
@@ -51,9 +55,16 @@ public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
 
   private static final Logger log = LoggerFactory.getLogger(SpaceAwareVolumeChooser.class);
 
+  private Configuration conf = new Configuration();
+
+  protected double getFreeSpace(String uri) throws IOException {
+    FileSystem pathFs = new Path(uri).getFileSystem(conf);
+    FsStatus optionStatus = pathFs.getStatus();
+    return ((double) optionStatus.getRemaining() / optionStatus.getCapacity());
+  }
+
   @Override
-  public String choose(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
+  public String choose(VolumeChooserEnvironment env, Set<String> options) {
     try {
       return getCache(env).get(getPreferredVolumes(env, options)).next();
     } catch (ExecutionException e) {
@@ -83,7 +94,7 @@ public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
     return choiceCache;
   }
 
-  private static class WeightedRandomCollection {
+  private class WeightedRandomCollection {
     private final NavigableMap<Double,String> map = new TreeMap<>();
     private final Random random;
     private double total = 0;
@@ -98,10 +109,8 @@ public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
 
       // Compute percentage space available on each volume
       for (String option : options) {
-        FileSystem pathFs = env.getFileSystem(option);
         try {
-          FsStatus optionStatus = pathFs.getStatus();
-          double percentFree = ((double) optionStatus.getRemaining() / optionStatus.getCapacity());
+          double percentFree = getFreeSpace(option);
           add(percentFree, option);
         } catch (IOException e) {
           log.error("Unable to get file system status for" + option, e);
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/fs/VolumeChooser.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/VolumeChooser.java
new file mode 100644
index 0000000..5801512
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/VolumeChooser.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.fs;
+
+import java.util.Set;
+
+import org.apache.accumulo.core.conf.Property;
+
+/**
+ * Helper used to select from a set of Volume URIs. N.B. implementations must be threadsafe.
+ * VolumeChooser.equals will be used for internal caching.
+ *
+ * <p>
+ * Implementations may wish to store configuration in Accumulo's system configuration using the
+ * {@link Property#GENERAL_ARBITRARY_PROP_PREFIX}. They may also benefit from using per-table
+ * configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
+ *
+ * @since 2.1.0
+ */
+public interface VolumeChooser {
+
+  /**
+   * 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
+   */
+  String choose(VolumeChooserEnvironment env, Set<String> options);
+
+  /**
+   * Return the subset of volumes that could possibly be chosen by this chooser across all
+   * invocations of {@link #choose(VolumeChooserEnvironment, Set)}. Currently this is used to
+   * determine if all of the volumes that could be chosen for write ahead logs support the needed
+   * filesystem operations. There may be other use cases in the future.
+   *
+   * @param env
+   *          the server environment provided by the calling framework
+   * @param options
+   *          the subset of volumes to choose from
+   * @return array of valid options
+   */
+  Set<String> choosable(VolumeChooserEnvironment env, Set<String> options);
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java b/core/src/main/java/org/apache/accumulo/core/spi/fs/VolumeChooserEnvironment.java
similarity index 70%
copy from server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
copy to core/src/main/java/org/apache/accumulo/core/spi/fs/VolumeChooserEnvironment.java
index bcedd4d..63f8889 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironment.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/fs/VolumeChooserEnvironment.java
@@ -16,48 +16,32 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.fs;
+package org.apache.accumulo.core.spi.fs;
+
+import java.util.Optional;
 
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
 
+/**
+ * @since 2.1.0
+ */
 public interface VolumeChooserEnvironment {
-
   /**
    * A scope the volume chooser environment; a TABLE scope should be accompanied by a tableId.
    *
-   * @since 2.0.0
+   * @since 2.1.0
    */
-  public static enum ChooserScope {
+  public static enum Scope {
     DEFAULT, TABLE, INIT, LOGGER
   }
 
-  /**
-   * The end row of the tablet for which a volume is being chosen. Only call this when the scope is
-   * TABLE
-   *
-   * @since 2.0.0
-   */
   public Text getEndRow();
 
-  public boolean hasTableId();
-
-  public TableId getTableId();
+  public Optional<TableId> getTable();
 
-  /**
-   * @since 2.0.0
-   */
-  public ChooserScope getScope();
+  public Scope getChooserScope();
 
-  /**
-   * @since 2.0.0
-   */
   public ServiceEnvironment getServiceEnv();
-
-  /**
-   * @since 2.0.0
-   */
-  public FileSystem getFileSystem(String option);
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java b/core/src/test/java/org/apache/accumulo/core/spi/fs/PerTableVolumeChooserTest.java
similarity index 69%
rename from server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
rename to core/src/test/java/org/apache/accumulo/core/spi/fs/PerTableVolumeChooserTest.java
index 328cf67..3ab3ccf 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/PerTableVolumeChooserTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/fs/PerTableVolumeChooserTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.fs;
+package org.apache.accumulo.core.spi.fs;
 
 import static org.easymock.EasyMock.anyObject;
 import static org.easymock.EasyMock.createStrictMock;
@@ -26,11 +26,13 @@ import static org.easymock.EasyMock.verify;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThrows;
 
+import java.util.Optional;
+
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
-import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
+import org.apache.hadoop.io.Text;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -39,7 +41,7 @@ public class PerTableVolumeChooserTest {
 
   private static final String TABLE_CUSTOM_SUFFIX = "volume.chooser";
 
-  private static final String getCustomPropertySuffix(ChooserScope scope) {
+  private static final String getCustomPropertySuffix(Scope scope) {
     return "volume.chooser." + scope.name().toLowerCase();
   }
 
@@ -70,22 +72,56 @@ public class PerTableVolumeChooserTest {
   }
 
   private VolumeChooser getTableDelegate() {
-    VolumeChooserEnvironment env =
-        new VolumeChooserEnvironmentImpl(TableId.of("testTable"), null, null) {
-          @Override
-          public ServiceEnvironment getServiceEnv() {
-            return serviceEnv;
-          }
-        };
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment() {
+
+      @Override
+      public Text getEndRow() {
+        return null;
+      }
+
+      @Override
+      public Optional<TableId> getTable() {
+        return Optional.of(TableId.of("testTable"));
+      }
+
+      @Override
+      public Scope getChooserScope() {
+        // TODO Auto-generated method stub
+        return Scope.TABLE;
+      }
+
+      @Override
+      public ServiceEnvironment getServiceEnv() {
+        return serviceEnv;
+      }
+    };
+
     return chooser.getDelegateChooser(env);
   }
 
-  private VolumeChooser getDelegate(ChooserScope scope) {
-    VolumeChooserEnvironment env = new VolumeChooserEnvironmentImpl(scope, null) {
+  private VolumeChooser getDelegate(Scope scope) {
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment() {
+
+      @Override
+      public Text getEndRow() {
+        return null;
+      }
+
+      @Override
+      public Optional<TableId> getTable() {
+        return Optional.empty();
+      }
+
+      @Override
+      public Scope getChooserScope() {
+        return scope;
+      }
+
       @Override
       public ServiceEnvironment getServiceEnv() {
         return serviceEnv;
       }
+
     };
     return chooser.getDelegateChooser(env);
   }
@@ -104,7 +140,7 @@ public class PerTableVolumeChooserTest {
   @Test
   public void testTableScopeUsingDefaultScopeProperty() throws Exception {
     expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT)))
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT)))
         .andReturn(MockChooser2.class.getName()).once();
     expect(serviceEnv.instantiate(TableId.of("testTable"), MockChooser2.class.getName(),
         VolumeChooser.class)).andReturn(new MockChooser2());
@@ -117,97 +153,92 @@ public class PerTableVolumeChooserTest {
   @Test
   public void testTableScopeWithNoConfig() {
     expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn(null)
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT))).andReturn(null).once();
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, this::getTableDelegate);
+    assertThrows(RuntimeException.class, this::getTableDelegate);
   }
 
   @Test
   public void testTableScopeWithBadDelegate() throws Exception {
     expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT)))
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT)))
         .andReturn("not a valid class name").once();
     expect(serviceEnv.instantiate(TableId.of("testTable"), "not a valid class name",
         VolumeChooser.class)).andThrow(new RuntimeException());
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, this::getTableDelegate);
+    assertThrows(RuntimeException.class, this::getTableDelegate);
   }
 
   @Test
   public void testLoggerScopeUsingLoggerProperty() throws Exception {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER)))
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.LOGGER)))
         .andReturn(MockChooser1.class.getName()).once();
     expect(serviceEnv.instantiate(MockChooser1.class.getName(), VolumeChooser.class))
         .andReturn(new MockChooser1());
     replay(serviceEnv, tableConf, systemConf);
 
-    VolumeChooser delegate = getDelegate(ChooserScope.LOGGER);
+    VolumeChooser delegate = getDelegate(Scope.LOGGER);
     assertSame(MockChooser1.class, delegate.getClass());
   }
 
   @Test
   public void testLoggerScopeUsingDefaultProperty() throws Exception {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
-        .once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT)))
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.LOGGER))).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT)))
         .andReturn(MockChooser2.class.getName()).once();
     expect(serviceEnv.instantiate(MockChooser2.class.getName(), VolumeChooser.class))
         .andReturn(new MockChooser2());
     replay(serviceEnv, tableConf, systemConf);
 
-    VolumeChooser delegate = getDelegate(ChooserScope.LOGGER);
+    VolumeChooser delegate = getDelegate(Scope.LOGGER);
     assertSame(MockChooser2.class, delegate.getClass());
   }
 
   @Test
   public void testLoggerScopeWithNoConfig() {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
-        .once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn(null)
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.LOGGER))).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT))).andReturn(null).once();
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, () -> getDelegate(ChooserScope.LOGGER));
+    assertThrows(RuntimeException.class, () -> getDelegate(Scope.LOGGER));
   }
 
   @Test
   public void testLoggerScopeWithBadDelegate() throws Exception {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
-        .once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT)))
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.LOGGER))).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT)))
         .andReturn("not a valid class name").once();
     expect(serviceEnv.instantiate("not a valid class name", VolumeChooser.class))
         .andThrow(new RuntimeException());
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, () -> getDelegate(ChooserScope.LOGGER));
+    assertThrows(RuntimeException.class, () -> getDelegate(Scope.LOGGER));
   }
 
   @Test
   public void testInitScopeUsingInitProperty() throws Exception {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.INIT)))
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.INIT)))
         .andReturn(MockChooser1.class.getName()).once();
     expect(serviceEnv.instantiate(MockChooser1.class.getName(), VolumeChooser.class))
         .andReturn(new MockChooser1());
     replay(serviceEnv, tableConf, systemConf);
 
-    VolumeChooser delegate = getDelegate(ChooserScope.INIT);
+    VolumeChooser delegate = getDelegate(Scope.INIT);
     assertSame(MockChooser1.class, delegate.getClass());
   }
 
   @Test
   public void testInitScopeUsingDefaultProperty() throws Exception {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.INIT))).andReturn(null).once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT)))
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.INIT))).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT)))
         .andReturn(MockChooser2.class.getName()).once();
     expect(serviceEnv.instantiate(MockChooser2.class.getName(), VolumeChooser.class))
         .andReturn(new MockChooser2());
     replay(serviceEnv, tableConf, systemConf);
 
-    VolumeChooser delegate = getDelegate(ChooserScope.INIT);
+    VolumeChooser delegate = getDelegate(Scope.INIT);
     assertSame(MockChooser2.class, delegate.getClass());
   }
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java b/core/src/test/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooserTest.java
similarity index 59%
rename from server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
rename to core/src/test/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooserTest.java
index cfd653e..f67fb6a 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/PreferredVolumeChooserTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/fs/PreferredVolumeChooserTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.fs;
+package org.apache.accumulo.core.spi.fs;
 
 import static org.easymock.EasyMock.anyObject;
 import static org.easymock.EasyMock.createStrictMock;
@@ -26,13 +26,14 @@ import static org.easymock.EasyMock.verify;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThrows;
 
+import java.util.Optional;
 import java.util.Set;
 
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
-import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
+import org.apache.hadoop.io.Text;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -41,7 +42,7 @@ public class PreferredVolumeChooserTest {
 
   private static final String TABLE_CUSTOM_SUFFIX = "volume.preferred";
 
-  private static final String getCustomPropertySuffix(ChooserScope scope) {
+  private static final String getCustomPropertySuffix(Scope scope) {
     return "volume.preferred." + scope.name().toLowerCase();
   }
 
@@ -70,18 +71,50 @@ public class PreferredVolumeChooserTest {
   }
 
   private Set<String> chooseForTable() {
-    VolumeChooserEnvironment env =
-        new VolumeChooserEnvironmentImpl(TableId.of("testTable"), null, null) {
-          @Override
-          public ServiceEnvironment getServiceEnv() {
-            return serviceEnv;
-          }
-        };
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment() {
+
+      @Override
+      public Text getEndRow() {
+        return null;
+      }
+
+      @Override
+      public Optional<TableId> getTable() {
+        return Optional.of(TableId.of("testTable"));
+      }
+
+      @Override
+      public Scope getChooserScope() {
+        return Scope.TABLE;
+      }
+
+      @Override
+      public ServiceEnvironment getServiceEnv() {
+        return serviceEnv;
+      }
+
+    };
     return chooser.getPreferredVolumes(env, ALL_OPTIONS);
   }
 
-  private Set<String> choose(ChooserScope scope) {
-    VolumeChooserEnvironment env = new VolumeChooserEnvironmentImpl(scope, null) {
+  private Set<String> choose(Scope scope) {
+    VolumeChooserEnvironment env = new VolumeChooserEnvironment() {
+
+      @Override
+      public Text getEndRow() {
+        return null;
+      }
+
+      @Override
+      public Optional<TableId> getTable() {
+        return Optional.empty();
+      }
+
+      @Override
+      public Scope getChooserScope() {
+        return scope;
+      }
+
       @Override
       public ServiceEnvironment getServiceEnv() {
         return serviceEnv;
@@ -100,8 +133,7 @@ public class PreferredVolumeChooserTest {
   @Test
   public void testTableScopeUsingDefaultScopeProperty() {
     expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn("3,2")
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT))).andReturn("3,2").once();
     replay(serviceEnv, tableConf, systemConf);
     assertEquals(Set.of("2", "3"), chooseForTable());
   }
@@ -109,11 +141,10 @@ public class PreferredVolumeChooserTest {
   @Test
   public void testTableScopeWithNoConfig() {
     expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn(null)
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT))).andReturn(null).once();
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, this::chooseForTable);
+    assertThrows(RuntimeException.class, this::chooseForTable);
   }
 
   @Test
@@ -121,83 +152,72 @@ public class PreferredVolumeChooserTest {
     expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(",").once();
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, this::chooseForTable);
+    assertThrows(RuntimeException.class, this::chooseForTable);
   }
 
   @Test
   public void testTableScopeWithUnrecognizedVolumes() {
     expect(tableConf.getTableCustom(TABLE_CUSTOM_SUFFIX)).andReturn(null).once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn("4")
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT))).andReturn("4").once();
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, this::chooseForTable);
+    assertThrows(RuntimeException.class, this::chooseForTable);
   }
 
   @Test
   public void testLoggerScopeUsingLoggerProperty() {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn("2,1")
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.LOGGER))).andReturn("2,1").once();
     replay(serviceEnv, tableConf, systemConf);
-    assertEquals(Set.of("1", "2"), choose(ChooserScope.LOGGER));
+    assertEquals(Set.of("1", "2"), choose(Scope.LOGGER));
   }
 
   @Test
   public void testLoggerScopeUsingDefaultProperty() {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
-        .once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn("3,2")
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.LOGGER))).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT))).andReturn("3,2").once();
     replay(serviceEnv, tableConf, systemConf);
-    assertEquals(Set.of("2", "3"), choose(ChooserScope.LOGGER));
+    assertEquals(Set.of("2", "3"), choose(Scope.LOGGER));
   }
 
   @Test
   public void testLoggerScopeWithNoConfig() {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
-        .once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn(null)
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.LOGGER))).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT))).andReturn(null).once();
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, () -> choose(ChooserScope.LOGGER));
+    assertThrows(RuntimeException.class, () -> choose(Scope.LOGGER));
   }
 
   @Test
   public void testLoggerScopeWithEmptySet() {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(",")
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.LOGGER))).andReturn(",").once();
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, () -> choose(ChooserScope.LOGGER));
+    assertThrows(RuntimeException.class, () -> choose(Scope.LOGGER));
   }
 
   @Test
   public void testLoggerScopeWithUnrecognizedVolumes() {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.LOGGER))).andReturn(null)
-        .once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn("4")
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.LOGGER))).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT))).andReturn("4").once();
     replay(serviceEnv, tableConf, systemConf);
 
-    assertThrows(VolumeChooserException.class, () -> choose(ChooserScope.LOGGER));
+    assertThrows(RuntimeException.class, () -> choose(Scope.LOGGER));
   }
 
   @Test
   public void testInitScopeUsingInitProperty() {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.INIT))).andReturn("2,1")
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.INIT))).andReturn("2,1").once();
     replay(serviceEnv, tableConf, systemConf);
-    assertEquals(Set.of("1", "2"), choose(ChooserScope.INIT));
+    assertEquals(Set.of("1", "2"), choose(Scope.INIT));
   }
 
   @Test
   public void testInitScopeUsingDefaultProperty() {
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.INIT))).andReturn(null).once();
-    expect(systemConf.getCustom(getCustomPropertySuffix(ChooserScope.DEFAULT))).andReturn("3,2")
-        .once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.INIT))).andReturn(null).once();
+    expect(systemConf.getCustom(getCustomPropertySuffix(Scope.DEFAULT))).andReturn("3,2").once();
     replay(serviceEnv, tableConf, systemConf);
-    assertEquals(Set.of("2", "3"), choose(ChooserScope.INIT));
+    assertEquals(Set.of("2", "3"), choose(Scope.INIT));
   }
 
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooserTest.java b/core/src/test/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooserTest.java
similarity index 74%
rename from server/base/src/test/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooserTest.java
rename to core/src/test/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooserTest.java
index 3b789a3..ac50822 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooserTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/spi/fs/SpaceAwareVolumeChooserTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server.fs;
+package org.apache.accumulo.core.spi.fs;
 
 import static org.junit.Assert.assertEquals;
 
@@ -25,9 +25,7 @@ import java.util.Set;
 
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsStatus;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
 import org.easymock.EasyMock;
 import org.junit.After;
 import org.junit.Before;
@@ -40,10 +38,9 @@ public class SpaceAwareVolumeChooserTest {
   VolumeChooserEnvironment chooserEnv = null;
   ServiceEnvironment serviceEnv = null;
   Configuration sysConfig = null;
-  FileSystem fs1 = null;
-  FileSystem fs2 = null;
-  FsStatus status1 = null;
-  FsStatus status2 = null;
+
+  double free1;
+  double free2;
 
   int iterations = 1000;
 
@@ -60,10 +57,6 @@ public class SpaceAwareVolumeChooserTest {
   public void beforeTest() {
     serviceEnv = EasyMock.createMock(ServiceEnvironment.class);
     sysConfig = EasyMock.createMock(Configuration.class);
-    fs1 = EasyMock.createMock(FileSystem.class);
-    fs2 = EasyMock.createMock(FileSystem.class);
-    status1 = EasyMock.createMock(FsStatus.class);
-    status2 = EasyMock.createMock(FsStatus.class);
     chooserEnv = EasyMock.createMock(VolumeChooserEnvironment.class);
   }
 
@@ -77,45 +70,29 @@ public class SpaceAwareVolumeChooserTest {
       updatePropertyMax = max + 1;
     }
 
-    // Volume 1 is percentage1 full
-    EasyMock.expect(status1.getRemaining()).andReturn(percentage1).times(min, max);
-    EasyMock.expect(status1.getCapacity()).andReturn(100L).times(min, max);
-
-    // Volume 2 is percentage2 full
-    EasyMock.expect(status2.getRemaining()).andReturn(percentage2).times(min, max);
-    EasyMock.expect(status2.getCapacity()).andReturn(100L).times(min, max);
+    free1 = percentage1 / (double) 100;
+    free2 = percentage2 / (double) 100;
 
     EasyMock.expect(sysConfig.getCustom(SpaceAwareVolumeChooser.RECOMPUTE_INTERVAL))
         .andReturn(cacheDuration).times(1);
 
-    EasyMock
-        .expect(
-            sysConfig.getCustom("volume.preferred." + ChooserScope.DEFAULT.name().toLowerCase()))
+    EasyMock.expect(sysConfig.getCustom("volume.preferred." + Scope.DEFAULT.name().toLowerCase()))
         .andReturn(String.join(",", tableDirs)).times(timesToCallPreferredVolumeChooser);
 
     EasyMock.expect(serviceEnv.getConfiguration()).andReturn(sysConfig).times(1, updatePropertyMax);
 
-    EasyMock.expect(fs1.getStatus()).andReturn(status1).times(min, max);
-    EasyMock.expect(fs2.getStatus()).andReturn(status2).times(min, max);
-
-    EasyMock.expect(chooserEnv.getFileSystem(volumeOne)).andReturn(fs1).times(min, max);
-    EasyMock.expect(chooserEnv.getFileSystem(volumeTwo)).andReturn(fs2).times(min, max);
-    EasyMock.expect(chooserEnv.getScope()).andReturn(ChooserScope.DEFAULT).times(min, max * 2);
+    EasyMock.expect(chooserEnv.getChooserScope()).andReturn(Scope.DEFAULT).times(min, max * 2);
     EasyMock.expect(chooserEnv.getServiceEnv()).andReturn(serviceEnv).times(min, max);
 
-    EasyMock.replay(serviceEnv, fs1, fs2, status1, status2, sysConfig, chooserEnv);
+    EasyMock.replay(serviceEnv, sysConfig, chooserEnv);
   }
 
   @After
   public void afterTest() {
 
-    EasyMock.verify(serviceEnv, fs1, fs2, status1, status2, sysConfig, chooserEnv);
+    EasyMock.verify(serviceEnv, sysConfig, chooserEnv);
 
     serviceEnv = null;
-    fs1 = null;
-    fs2 = null;
-    status1 = null;
-    status2 = null;
     vol1Count = 0;
     vol2Count = 0;
   }
@@ -189,7 +166,16 @@ public class SpaceAwareVolumeChooserTest {
   }
 
   private void makeChoices() {
-    SpaceAwareVolumeChooser chooser = new SpaceAwareVolumeChooser();
+    SpaceAwareVolumeChooser chooser = new SpaceAwareVolumeChooser() {
+      @Override
+      protected double getFreeSpace(String uri) throws IOException {
+        if (uri.equals(volumeOne))
+          return free1;
+        if (uri.equals(volumeTwo))
+          return free2;
+        throw new IllegalArgumentException();
+      }
+    };
     for (int i = 0; i < iterations; i++) {
       String choice = chooser.choose(chooserEnv, tableDirs);
       if (choice.equals(volumeOne)) {
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 784ed79..079a3ea 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,150 +18,20 @@
  */
 package org.apache.accumulo.server.fs;
 
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
-import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * A {@link VolumeChooser} that delegates to another volume chooser based on other properties:
- * table.custom.volume.chooser for tables, and general.custom.volume.chooser.scoped for scopes.
- * general.custom.volume.chooser.{scope} can override the system wide setting for
- * general.custom.volume.chooser.scoped. At the this this was written, the only known scope was
- * "logger".
- */
-public class PerTableVolumeChooser implements VolumeChooser {
-  // 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<TableId,VolumeChooser> tableSpecificChooserCache =
-      new ConcurrentHashMap<>();
-  private final ConcurrentHashMap<ChooserScope,VolumeChooser> scopeSpecificChooserCache =
-      new ConcurrentHashMap<>();
-
-  private static final String TABLE_CUSTOM_SUFFIX = "volume.chooser";
-
-  private static final String getCustomPropertySuffix(ChooserScope scope) {
-    return "volume.chooser." + scope.name().toLowerCase();
-  }
-
-  private static final String DEFAULT_SCOPED_VOLUME_CHOOSER =
-      getCustomPropertySuffix(ChooserScope.DEFAULT);
-
-  @Override
-  public String choose(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
-    log.trace("{}.choose", getClass().getSimpleName());
-    return getDelegateChooser(env).choose(env, options);
-  }
-
-  @Override
-  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
-    return getDelegateChooser(env).choosable(env, options);
-  }
-
-  // visible (not private) for testing
-  VolumeChooser getDelegateChooser(VolumeChooserEnvironment env) {
-    if (env.getScope() == ChooserScope.TABLE) {
-      return getVolumeChooserForTable(env);
-    }
-    return getVolumeChooserForScope(env);
-  }
-
-  private VolumeChooser getVolumeChooserForTable(VolumeChooserEnvironment env) {
-    log.trace("Looking up property {} for table id: {}", TABLE_CUSTOM_SUFFIX, env.getTableId());
-
-    String clazz =
-        env.getServiceEnv().getConfiguration(env.getTableId()).getTableCustom(TABLE_CUSTOM_SUFFIX);
-
-    // fall back to global default scope, so setting only one default is necessary, rather than a
-    // separate default for TABLE scope than other scopes
-    if (clazz == null || clazz.isEmpty()) {
-      clazz = env.getServiceEnv().getConfiguration().getCustom(DEFAULT_SCOPED_VOLUME_CHOOSER);
-    }
-
-    if (clazz == null || clazz.isEmpty()) {
-      String msg = "Property " + TABLE_CUSTOM_SUFFIX + " or " + DEFAULT_SCOPED_VOLUME_CHOOSER
-          + " must be a valid " + VolumeChooser.class.getSimpleName() + " to use the "
-          + getClass().getSimpleName();
-      throw new VolumeChooserException(msg);
-    }
-
-    return createVolumeChooser(env, clazz, TABLE_CUSTOM_SUFFIX, env.getTableId(),
-        tableSpecificChooserCache);
-  }
-
-  private VolumeChooser getVolumeChooserForScope(VolumeChooserEnvironment env) {
-    ChooserScope scope = env.getScope();
-    String property = getCustomPropertySuffix(scope);
-    log.trace("Looking up property {} for scope: {}", property, scope);
-
-    String clazz = env.getServiceEnv().getConfiguration().getCustom(property);
-
-    // fall back to global default scope if this scope isn't configured (and not already default
-    // scope)
-    if ((clazz == null || clazz.isEmpty()) && scope != ChooserScope.DEFAULT) {
-      log.debug("{} not found; using {}", property, DEFAULT_SCOPED_VOLUME_CHOOSER);
-      clazz = env.getServiceEnv().getConfiguration().getCustom(DEFAULT_SCOPED_VOLUME_CHOOSER);
-
-      if (clazz == null || clazz.isEmpty()) {
-        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(env, clazz, property, scope, scopeSpecificChooserCache);
-  }
-
-  /**
-   * 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 <T> VolumeChooser createVolumeChooser(VolumeChooserEnvironment env, String clazz,
-      String property, T key, ConcurrentHashMap<T,VolumeChooser> cache) {
-    final String className = clazz.trim();
-    // create a new instance, unless another thread beat us with one of the same class name, then
-    // use theirs
-    return cache.compute(key, (k, previousChooser) -> {
-      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 {
-        if (key instanceof TableId) {
-          TableId tableId = (TableId) key;
-          return env.getServiceEnv().instantiate(tableId, className, VolumeChooser.class);
-        } else {
-          return env.getServiceEnv().instantiate(className, VolumeChooser.class);
-        }
-      } catch (Exception e) {
-        String msg = "Failed to create instance for " + key + " configured to use " + className
-            + " via " + property;
-        throw new VolumeChooserException(msg, e);
-      }
-    });
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+@Deprecated(since = "2.1.0")
+@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
+    justification = "Same name used for compatibility during deprecation cycle")
+public class PerTableVolumeChooser extends org.apache.accumulo.core.spi.fs.PerTableVolumeChooser
+    implements VolumeChooser {
+  public PerTableVolumeChooser() {
+    super();
+    LoggerFactory.getLogger(PerTableVolumeChooser.class).warn(
+        "The class {} is deprecated.  Please configure {} instead.",
+        PerTableVolumeChooser.class.getName(),
+        org.apache.accumulo.core.spi.fs.PerTableVolumeChooser.class.getName());
   }
 }
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 b246a6e..158afda 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
@@ -18,130 +18,20 @@
  */
 package org.apache.accumulo.server.fs;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.accumulo.core.volume.Volume;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
-import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * A {@link RandomVolumeChooser} that limits its choices from a given set of options to the subset
- * of those options preferred for a particular table. Defaults to selecting from all of the options
- * presented. Can be customized via the table property table.custom.volume.preferred, which should
- * contain a comma separated list of {@link Volume} URIs. Note that both the property name and the
- * format of its value are specific to this particular implementation.
- */
-public class PreferredVolumeChooser extends RandomVolumeChooser {
-  private static final Logger log = LoggerFactory.getLogger(PreferredVolumeChooser.class);
-
-  private static final String TABLE_CUSTOM_SUFFIX = "volume.preferred";
-
-  private static final String getCustomPropertySuffix(ChooserScope scope) {
-    return "volume.preferred." + scope.name().toLowerCase();
-  }
-
-  private static final String DEFAULT_SCOPED_PREFERRED_VOLUMES =
-      getCustomPropertySuffix(ChooserScope.DEFAULT);
-
-  @Override
-  public String choose(VolumeChooserEnvironment env, Set<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;
-  }
-
-  @Override
-  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
-    return getPreferredVolumes(env, options);
-  }
-
-  // visible (not private) for testing
-  Set<String> getPreferredVolumes(VolumeChooserEnvironment env, Set<String> options) {
-    if (env.getScope() == ChooserScope.TABLE) {
-      return getPreferredVolumesForTable(env, options);
-    }
-    return getPreferredVolumesForScope(env, options);
-  }
-
-  private Set<String> getPreferredVolumesForTable(VolumeChooserEnvironment env,
-      Set<String> options) {
-    log.trace("Looking up property {} + for Table id: {}", TABLE_CUSTOM_SUFFIX, env.getTableId());
-
-    String preferredVolumes =
-        env.getServiceEnv().getConfiguration(env.getTableId()).getTableCustom(TABLE_CUSTOM_SUFFIX);
-
-    // fall back to global default scope, so setting only one default is necessary, rather than a
-    // separate default for TABLE scope than other scopes
-    if (preferredVolumes == null || preferredVolumes.isEmpty()) {
-      preferredVolumes =
-          env.getServiceEnv().getConfiguration().getCustom(DEFAULT_SCOPED_PREFERRED_VOLUMES);
-    }
-
-    // throw an error if volumes not specified or empty
-    if (preferredVolumes == null || preferredVolumes.isEmpty()) {
-      String msg = "Property " + TABLE_CUSTOM_SUFFIX + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
-          + " must be a subset of " + options + " to use the " + getClass().getSimpleName();
-      throw new VolumeChooserException(msg);
-    }
-
-    return parsePreferred(TABLE_CUSTOM_SUFFIX, preferredVolumes, options);
-  }
-
-  private Set<String> getPreferredVolumesForScope(VolumeChooserEnvironment env,
-      Set<String> options) {
-    ChooserScope scope = env.getScope();
-    String property = getCustomPropertySuffix(scope);
-    log.trace("Looking up property {} for scope: {}", property, scope);
-
-    String preferredVolumes = env.getServiceEnv().getConfiguration().getCustom(property);
-
-    // fall back to global default scope if this scope isn't configured (and not already default
-    // scope)
-    if ((preferredVolumes == null || preferredVolumes.isEmpty()) && scope != ChooserScope.DEFAULT) {
-      log.debug("{} not found; using {}", property, DEFAULT_SCOPED_PREFERRED_VOLUMES);
-      preferredVolumes =
-          env.getServiceEnv().getConfiguration().getCustom(DEFAULT_SCOPED_PREFERRED_VOLUMES);
-
-      // only if the custom property is not set to we fall back to the default scoped preferred
-      // volumes
-      if (preferredVolumes == null || preferredVolumes.isEmpty()) {
-        String msg = "Property " + property + " or " + DEFAULT_SCOPED_PREFERRED_VOLUMES
-            + " must be a subset of " + options + " to use the " + getClass().getSimpleName();
-        throw new VolumeChooserException(msg);
-      }
-
-      property = DEFAULT_SCOPED_PREFERRED_VOLUMES;
-    }
-
-    return parsePreferred(property, preferredVolumes, options);
-  }
-
-  private Set<String> parsePreferred(String property, String preferredVolumes,
-      Set<String> options) {
-    log.trace("Found {} = {}", property, preferredVolumes);
-
-    Set<String> preferred =
-        Arrays.stream(preferredVolumes.split(",")).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);
-    }
-    // preferred volumes should also exist in the original options (typically, from
-    // instance.volumes)
-    if (Collections.disjoint(preferred, options)) {
-      String msg = "Some volumes in " + preferred + " are not valid volumes from " + options;
-      throw new VolumeChooserException(msg);
-    }
-
-    return preferred;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+@Deprecated(since = "2.1.0")
+@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
+    justification = "Same name used for compatibility during deprecation cycle")
+public class PreferredVolumeChooser extends org.apache.accumulo.core.spi.fs.PreferredVolumeChooser
+    implements VolumeChooser {
+  public PreferredVolumeChooser() {
+    super();
+    LoggerFactory.getLogger(PreferredVolumeChooser.class).warn(
+        "The class {} is deprecated.  Please configure {} instead.",
+        PreferredVolumeChooser.class.getName(),
+        org.apache.accumulo.core.spi.fs.PreferredVolumeChooser.class.getName());
   }
 }
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 8628e33..bd91eba 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
@@ -18,23 +18,20 @@
  */
 package org.apache.accumulo.server.fs;
 
-import java.security.SecureRandom;
-import java.util.Random;
-import java.util.Set;
+import org.slf4j.LoggerFactory;
 
-public class RandomVolumeChooser implements VolumeChooser {
-  protected final Random random = new SecureRandom();
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
-  @Override
-  public String choose(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
-    String[] optionsArray = options.toArray(new String[0]);
-    return optionsArray[random.nextInt(optionsArray.length)];
-  }
-
-  @Override
-  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
-    return options;
+@Deprecated(since = "2.1.0")
+@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
+    justification = "Same name used for compatibility during deprecation cycle")
+public class RandomVolumeChooser extends org.apache.accumulo.core.spi.fs.RandomVolumeChooser
+    implements VolumeChooser {
+  public RandomVolumeChooser() {
+    super();
+    LoggerFactory.getLogger(RandomVolumeChooser.class).warn(
+        "The class {} is deprecated.  Please configure {} instead.",
+        RandomVolumeChooser.class.getName(),
+        org.apache.accumulo.core.spi.fs.RandomVolumeChooser.class.getName());
   }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java b/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
index b17c47b..029465d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/SpaceAwareVolumeChooser.java
@@ -18,116 +18,20 @@
  */
 package org.apache.accumulo.server.fs;
 
-import java.io.IOException;
-import java.util.NavigableMap;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsStatus;
-import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-
-/**
- * A {@link PreferredVolumeChooser} that takes remaining HDFS space into account when making a
- * volume choice rather than a simpler round robin. The list of volumes to use can be limited using
- * the same properties as {@link PreferredVolumeChooser}
- */
-public class SpaceAwareVolumeChooser extends PreferredVolumeChooser {
-
-  public static final String RECOMPUTE_INTERVAL = "spaceaware.volume.chooser.recompute.interval";
-
-  // Default time to wait in ms. Defaults to 5 min
-  private long defaultComputationCacheDuration = 300000;
-  private LoadingCache<Set<String>,WeightedRandomCollection> choiceCache = null;
-
-  private static final Logger log = LoggerFactory.getLogger(SpaceAwareVolumeChooser.class);
-
-  @Override
-  public String choose(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
-    try {
-      return getCache(env).get(getPreferredVolumes(env, options)).next();
-    } catch (ExecutionException e) {
-      throw new IllegalStateException("Execution exception when attempting to cache choice", e);
-    }
-  }
-
-  private synchronized LoadingCache<Set<String>,WeightedRandomCollection>
-      getCache(VolumeChooserEnvironment env) {
-
-    if (choiceCache == null) {
-      String propertyValue = env.getServiceEnv().getConfiguration().getCustom(RECOMPUTE_INTERVAL);
-
-      long computationCacheDuration = StringUtils.isNotBlank(propertyValue)
-          ? Long.parseLong(propertyValue) : defaultComputationCacheDuration;
-
-      choiceCache = CacheBuilder.newBuilder()
-          .expireAfterWrite(computationCacheDuration, TimeUnit.MILLISECONDS)
-          .build(new CacheLoader<>() {
-            @Override
-            public WeightedRandomCollection load(Set<String> key) {
-              return new WeightedRandomCollection(key, env, random);
-            }
-          });
-    }
-
-    return choiceCache;
-  }
-
-  private static class WeightedRandomCollection {
-    private final NavigableMap<Double,String> map = new TreeMap<>();
-    private final Random random;
-    private double total = 0;
-
-    public WeightedRandomCollection(Set<String> options, VolumeChooserEnvironment env,
-        Random random) {
-      this.random = random;
-
-      if (options.size() < 1) {
-        throw new IllegalStateException("Options was empty! No valid volumes to choose from.");
-      }
-
-      // Compute percentage space available on each volume
-      for (String option : options) {
-        FileSystem pathFs = env.getFileSystem(option);
-        try {
-          FsStatus optionStatus = pathFs.getStatus();
-          double percentFree = ((double) optionStatus.getRemaining() / optionStatus.getCapacity());
-          add(percentFree, option);
-        } catch (IOException e) {
-          log.error("Unable to get file system status for" + option, e);
-        }
-      }
-
-      if (map.size() < 1) {
-        throw new IllegalStateException(
-            "Weighted options was empty! Could indicate an issue getting file system status or "
-                + "no free space on any volume");
-      }
-    }
-
-    public WeightedRandomCollection add(double weight, String result) {
-      if (weight <= 0) {
-        log.info("Weight was 0. Not adding " + result);
-        return this;
-      }
-      total += weight;
-      map.put(total, result);
-      return this;
-    }
-
-    public String next() {
-      double value = random.nextDouble() * total;
-      return map.higherEntry(value).getValue();
-    }
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+@Deprecated(since = "2.1.0")
+@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
+    justification = "Same name used for compatibility during deprecation cycle")
+public class SpaceAwareVolumeChooser extends org.apache.accumulo.core.spi.fs.SpaceAwareVolumeChooser
+    implements VolumeChooser {
+  public SpaceAwareVolumeChooser() {
+    super();
+    LoggerFactory.getLogger(SpaceAwareVolumeChooser.class).warn(
+        "The class {} is deprecated.  Please configure {} instead.",
+        SpaceAwareVolumeChooser.class.getName(),
+        org.apache.accumulo.core.spi.fs.SpaceAwareVolumeChooser.class.getName());
   }
 }
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 072b082..61c8d74 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
@@ -20,19 +20,15 @@ package org.apache.accumulo.server.fs;
 
 import java.util.Set;
 
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.volume.Volume;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 /**
- * Helper used by {@link VolumeManager}s to select from a set of {@link Volume} URIs. N.B.
- * implementations must be threadsafe. VolumeChooser.equals will be used for internal caching.
- *
- * <p>
- * Implementations may wish to store configuration in Accumulo's system configuration using the
- * {@link Property#GENERAL_ARBITRARY_PROP_PREFIX}. They may also benefit from using per-table
- * configuration using {@link Property#TABLE_ARBITRARY_PROP_PREFIX}.
+ * @deprecated since 2.1.0; implement {@link org.apache.accumulo.core.spi.fs.VolumeChooser} instead.
  */
-public interface VolumeChooser {
+@Deprecated(since = "2.1.0")
+@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_INTERFACE",
+    justification = "Same name used for compatibility during deprecation cycle")
+public interface VolumeChooser extends org.apache.accumulo.core.spi.fs.VolumeChooser {
 
   /**
    * Choose a volume from the provided options.
@@ -45,55 +41,38 @@ public interface VolumeChooser {
    * @throws VolumeChooserException
    *           if there is an error choosing (this is a RuntimeException); this does not preclude
    *           other RuntimeExceptions from occurring
-   * @deprecated since 2.1.0; override {@link #choose(VolumeChooserEnvironment, Set)} instead. This
-   *             method will be removed in 3.0
    */
-  @Deprecated(since = "2.1.0")
   default String choose(VolumeChooserEnvironment env, String[] options)
       throws VolumeChooserException {
     throw new UnsupportedOperationException("This method will be removed in 3.0");
   }
 
   /**
-   * Choose a volume from the provided options.
+   * Default method provided for compatibility with 2.0.0.
    *
-   * @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
+   * @since 2.1.0
    */
-  default String choose(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
+  @Override
+  default String choose(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env,
+      Set<String> options) {
     InterfaceEvolutionWarner.warnOnce(getClass(), VolumeChooser.class,
         "choose(VolumeChooserEnvironment,Set)", "3.0");
-    return choose(env, options.toArray(new String[0]));
+    return choose((VolumeChooserEnvironmentImpl) env, options.toArray(new String[0]));
   }
 
   /**
-   * Return the subset of volumes that could possibly be chosen by this chooser across all
-   * invocations of {@link #choose(VolumeChooserEnvironment, Set)}.
-   *
-   * @param env
-   *          the server environment provided by the calling framework
-   * @param options
-   *          the subset of volumes to choose from
-   * @return array of valid options
-   * @throws VolumeChooserException
-   *           if there is an error choosing (this is a RuntimeException); this does not preclude
-   *           other RuntimeExceptions from occurring
+   * Default method provided for compatibility with 2.0.0.
    *
    * @since 2.1.0
    */
-  default Set<String> choosable(VolumeChooserEnvironment env, Set<String> options)
-      throws VolumeChooserException {
+  @Override
+  default Set<String> choosable(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env,
+      Set<String> options) {
     // assume that all options are possible to be chosen by this chooser
     return options;
   }
 
+  @Deprecated(since = "2.1.0")
   class VolumeChooserException extends RuntimeException {
 
     private static final long serialVersionUID = 1L;
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 bcedd4d..8b0c4f4 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
@@ -23,7 +23,13 @@ import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
 
-public interface VolumeChooserEnvironment {
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+@Deprecated(since = "2.1.0")
+@SuppressFBWarnings(value = "NM_SAME_SIMPLE_NAME_AS_INTERFACE",
+    justification = "Same name used for compatibility during deprecation cycle")
+public interface VolumeChooserEnvironment
+    extends org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment {
 
   /**
    * A scope the volume chooser environment; a TABLE scope should be accompanied by a tableId.
@@ -40,6 +46,7 @@ public interface VolumeChooserEnvironment {
    *
    * @since 2.0.0
    */
+  @Override
   public Text getEndRow();
 
   public boolean hasTableId();
@@ -49,11 +56,27 @@ public interface VolumeChooserEnvironment {
   /**
    * @since 2.0.0
    */
-  public ChooserScope getScope();
+  public default ChooserScope getScope() {
+
+    var scope = getChooserScope();
+    switch (scope) {
+      case DEFAULT:
+        return ChooserScope.DEFAULT;
+      case INIT:
+        return ChooserScope.INIT;
+      case LOGGER:
+        return ChooserScope.LOGGER;
+      case TABLE:
+        return ChooserScope.TABLE;
+      default:
+        throw new IllegalArgumentException("Unknown chooser scope : " + scope);
+    }
+  }
 
   /**
    * @since 2.0.0
    */
+  @Override
   public ServiceEnvironment getServiceEnv();
 
   /**
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java
index ff93eda..ccfd2df 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeChooserEnvironmentImpl.java
@@ -19,6 +19,7 @@
 package org.apache.accumulo.server.fs;
 
 import java.util.Objects;
+import java.util.Optional;
 
 import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
@@ -34,32 +35,33 @@ import org.apache.hadoop.io.Text;
  * {@link VolumeChooserEnvironment} should result in more stable code over time than using this
  * class.
  */
+@SuppressWarnings("deprecation")
 public class VolumeChooserEnvironmentImpl implements VolumeChooserEnvironment {
 
   private final ServerContext context;
-  private final ChooserScope scope;
-  private final TableId tableId;
+  private final Scope scope;
+  private final Optional<TableId> tableId;
   private final Text endRow;
 
-  public VolumeChooserEnvironmentImpl(ChooserScope scope, ServerContext context) {
+  public VolumeChooserEnvironmentImpl(Scope scope, ServerContext context) {
     this.context = context;
     this.scope = Objects.requireNonNull(scope);
-    this.tableId = null;
+    this.tableId = Optional.empty();
     this.endRow = null;
   }
 
   public VolumeChooserEnvironmentImpl(TableId tableId, Text endRow, ServerContext context) {
     this.context = context;
-    this.scope = ChooserScope.TABLE;
-    this.tableId = Objects.requireNonNull(tableId);
+    this.scope = Scope.TABLE;
+    this.tableId = Optional.of(tableId);
     this.endRow = endRow;
   }
 
-  public VolumeChooserEnvironmentImpl(ChooserScope scope, TableId tableId, Text endRow,
+  public VolumeChooserEnvironmentImpl(Scope scope, TableId tableId, Text endRow,
       ServerContext context) {
     this.context = context;
     this.scope = Objects.requireNonNull(scope);
-    this.tableId = Objects.requireNonNull(tableId);
+    this.tableId = Optional.of(tableId);
     this.endRow = endRow;
   }
 
@@ -71,28 +73,28 @@ public class VolumeChooserEnvironmentImpl implements VolumeChooserEnvironment {
    */
   @Override
   public Text getEndRow() {
-    if (scope != ChooserScope.TABLE && scope != ChooserScope.INIT)
+    if (scope != Scope.TABLE && scope != Scope.INIT)
       throw new IllegalStateException("Can only request end row for tables, not for " + scope);
     return endRow;
   }
 
   @Override
+  public Optional<TableId> getTable() {
+    return tableId;
+  }
+
+  @Override
   public boolean hasTableId() {
-    return scope == ChooserScope.TABLE || scope == ChooserScope.INIT;
+    return tableId.isPresent();
   }
 
   @Override
   public TableId getTableId() {
-    if (scope != ChooserScope.TABLE && scope != ChooserScope.INIT)
-      throw new IllegalStateException("Can only request table id for tables, not for " + scope);
-    return tableId;
+    return tableId.get();
   }
 
-  /**
-   * @since 2.0.0
-   */
   @Override
-  public ChooserScope getScope() {
+  public Scope getChooserScope() {
     return this.scope;
   }
 
@@ -115,7 +117,8 @@ public class VolumeChooserEnvironmentImpl implements VolumeChooserEnvironment {
       return false;
     }
     VolumeChooserEnvironmentImpl other = (VolumeChooserEnvironmentImpl) obj;
-    return getScope() == other.getScope() && Objects.equals(getTableId(), other.getTableId());
+    return getChooserScope() == other.getChooserScope()
+        && Objects.equals(getTableId(), other.getTableId());
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
index 2a53e18..dffc151 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
@@ -170,10 +170,11 @@ public interface VolumeManager extends AutoCloseable {
   FileStatus[] globStatus(Path path) throws IOException;
 
   // decide on which of the given locations to create a new file
-  String choose(VolumeChooserEnvironment env, Set<String> options);
+  String choose(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env, Set<String> options);
 
   // return all valid locations to create a new file
-  Set<String> choosable(VolumeChooserEnvironment env, Set<String> options);
+  Set<String> choosable(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env,
+      Set<String> options);
 
   // are sync and flush supported for the given path
   boolean canSyncAndFlush(Path path);
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 789b5a7..8f2d58e 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
@@ -42,11 +42,11 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.spi.fs.VolumeChooser;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.volume.VolumeImpl;
-import org.apache.accumulo.server.fs.VolumeChooser.VolumeChooserException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -92,7 +92,7 @@ public class VolumeManagerImpl implements VolumeManager {
       // null chooser handled below
     }
     if (chooser1 == null) {
-      throw new VolumeChooserException(
+      throw new RuntimeException(
           "Failed to load volume chooser specified by " + Property.GENERAL_VOLUME_CHOOSER);
     }
     chooser = chooser1;
@@ -400,25 +400,27 @@ public class VolumeManagerImpl implements VolumeManager {
   }
 
   @Override
-  public String choose(VolumeChooserEnvironment env, Set<String> options) {
+  public String choose(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env,
+      Set<String> options) {
     final String choice;
     choice = chooser.choose(env, options);
     if (!options.contains(choice)) {
       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 VolumeChooserException(msg);
+      throw new RuntimeException(msg);
     }
     return choice;
   }
 
   @Override
-  public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options) {
+  public Set<String> choosable(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env,
+      Set<String> options) {
     final Set<String> choices = chooser.choosable(env, options);
     for (String choice : choices) {
       if (!options.contains(choice)) {
         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 VolumeChooserException(msg);
+        throw new RuntimeException(msg);
       }
     }
     return choices;
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 5eb6299..f077466 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
@@ -81,6 +81,8 @@ import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
 import org.apache.accumulo.core.spi.crypto.CryptoService;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.Pair;
@@ -92,8 +94,6 @@ import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServerUtil;
 import org.apache.accumulo.server.constraints.MetadataConstraints;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
@@ -350,7 +350,7 @@ public class Initialize implements KeywordExecutable {
     try (ServerContext context =
         ServerContext.initialize(siteConfig, instanceName, uuid.toString())) {
       VolumeChooserEnvironment chooserEnv =
-          new VolumeChooserEnvironmentImpl(ChooserScope.INIT, RootTable.ID, null, context);
+          new VolumeChooserEnvironmentImpl(Scope.INIT, RootTable.ID, null, context);
       String rootTabletDirName = RootTable.ROOT_TABLET_DIR_NAME;
       String ext = FileOperations.getNewFileExtension(DefaultConfiguration.getInstance());
       String rootTabletFileUri = new Path(fs.choose(chooserEnv, configuredVolumes) + Path.SEPARATOR
@@ -474,22 +474,22 @@ public class Initialize implements KeywordExecutable {
 
     Text splitPoint = TabletsSection.getRange().getEndKey().getRow();
 
-    VolumeChooserEnvironment chooserEnv = new VolumeChooserEnvironmentImpl(ChooserScope.INIT,
-        MetadataTable.ID, splitPoint, serverContext);
+    VolumeChooserEnvironment chooserEnv =
+        new VolumeChooserEnvironmentImpl(Scope.INIT, MetadataTable.ID, splitPoint, serverContext);
     String tableMetadataTabletDirName = TABLE_TABLETS_TABLET_DIR;
     String tableMetadataTabletDirUri =
         fs.choose(chooserEnv, ServerConstants.getBaseUris(siteConfig, hadoopConf))
             + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + MetadataTable.ID + Path.SEPARATOR
             + tableMetadataTabletDirName;
-    chooserEnv = new VolumeChooserEnvironmentImpl(ChooserScope.INIT, ReplicationTable.ID, null,
-        serverContext);
+    chooserEnv =
+        new VolumeChooserEnvironmentImpl(Scope.INIT, ReplicationTable.ID, null, serverContext);
     String replicationTableDefaultTabletDirName = ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
     String replicationTableDefaultTabletDirUri =
         fs.choose(chooserEnv, ServerConstants.getBaseUris(siteConfig, hadoopConf))
             + Constants.HDFS_TABLES_DIR + Path.SEPARATOR + ReplicationTable.ID + Path.SEPARATOR
             + replicationTableDefaultTabletDirName;
     chooserEnv =
-        new VolumeChooserEnvironmentImpl(ChooserScope.INIT, MetadataTable.ID, null, serverContext);
+        new VolumeChooserEnvironmentImpl(Scope.INIT, MetadataTable.ID, null, serverContext);
     String defaultMetadataTabletDirName = ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
     String defaultMetadataTabletDirUri =
         fs.choose(chooserEnv, ServerConstants.getBaseUris(siteConfig, hadoopConf))
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 1fbddeb..92e91c3 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
@@ -27,6 +27,7 @@ import java.util.Set;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.fs.VolumeChooser;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.Test;
 
@@ -53,12 +54,14 @@ public class VolumeManagerImplTest {
 
   public static class WrongVolumeChooser implements VolumeChooser {
     @Override
-    public String choose(VolumeChooserEnvironment env, Set<String> options) {
+    public String choose(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env,
+        Set<String> options) {
       return "file://totally-not-given/";
     }
 
     @Override
-    public Set<String> choosable(VolumeChooserEnvironment env, Set<String> options) {
+    public Set<String> choosable(org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment env,
+        Set<String> options) {
       return Set.of("file://totally-not-given");
     }
   }
@@ -71,7 +74,7 @@ public class VolumeManagerImplTest {
     conf.set(Property.INSTANCE_VOLUMES, String.join(",", volumes));
     conf.set(Property.GENERAL_VOLUME_CHOOSER, WrongVolumeChooser.class.getName());
     try (var vm = VolumeManagerImpl.get(conf, hadoopConf)) {
-      VolumeChooserEnvironment chooserEnv =
+      org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment chooserEnv =
           new VolumeChooserEnvironmentImpl(TableId.of("sometable"), null, null);
       assertThrows(RuntimeException.class, () -> vm.choose(chooserEnv, volumes));
     }
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java
index f095332..8686685 100644
--- a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java
@@ -31,7 +31,7 @@ import java.util.Set;
 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.RandomVolumeChooser;
+import org.apache.accumulo.core.spi.fs.RandomVolumeChooser;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.conf.Configuration;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 97f706c..e8c7200 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -76,6 +76,7 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.replication.ReplicationConstants;
 import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
 import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Iface;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Processor;
@@ -105,7 +106,6 @@ import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.TabletLevel;
 import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.log.SortedLogState;
@@ -963,7 +963,7 @@ public class TabletServer extends AbstractServer {
 
   private static void checkWalCanSync(ServerContext context) {
     VolumeChooserEnvironment chooserEnv =
-        new VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.ChooserScope.LOGGER, context);
+        new VolumeChooserEnvironmentImpl(VolumeChooserEnvironment.Scope.LOGGER, context);
     Set<String> prefixes;
     var options = ServerConstants.getBaseUris(context);
     try {
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 dd67d7a..9dd0252 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -62,7 +62,6 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment.ChooserScope;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.tserver.TabletMutations;
@@ -408,7 +407,8 @@ public class DfsLogger implements Comparable<DfsLogger> {
     log.debug("DfsLogger.open() begin");
     VolumeManager fs = conf.getVolumeManager();
 
-    var chooserEnv = new VolumeChooserEnvironmentImpl(ChooserScope.LOGGER, context);
+    var chooserEnv = new VolumeChooserEnvironmentImpl(
+        org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope.LOGGER, context);
     logPath = fs.choose(chooserEnv, ServerConstants.getBaseUris(context)) + Path.SEPARATOR
         + ServerConstants.WAL_DIR + Path.SEPARATOR + logger + Path.SEPARATOR + filename;
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index cf08e26..1744240 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
@@ -80,6 +80,7 @@ import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.core.spi.scan.ScanDirectives;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
@@ -90,7 +91,6 @@ import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeUtil;
 import org.apache.accumulo.server.fs.VolumeUtil.TabletFiles;
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java
index 9691d13..217f7b7 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/TabletServerSyncCheckTest.java
@@ -23,9 +23,9 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.volume.VolumeImpl;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
diff --git a/test/src/main/java/org/apache/accumulo/test/FairVolumeChooser.java b/test/src/main/java/org/apache/accumulo/test/FairVolumeChooser.java
index 0fa4c27..b0845c6 100644
--- a/test/src/main/java/org/apache/accumulo/test/FairVolumeChooser.java
+++ b/test/src/main/java/org/apache/accumulo/test/FairVolumeChooser.java
@@ -21,8 +21,8 @@ package org.apache.accumulo.test;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.accumulo.server.fs.VolumeChooser;
-import org.apache.accumulo.server.fs.VolumeChooserEnvironment;
+import org.apache.accumulo.core.spi.fs.VolumeChooser;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment;
 
 /**
  * Try to assign some fairness to choosing Volumes. Intended for tests, not for production
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 9bcb497..950b940 100644
--- a/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/VolumeChooserIT.java
@@ -49,11 +49,11 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.spi.fs.PerTableVolumeChooser;
+import org.apache.accumulo.core.spi.fs.PreferredVolumeChooser;
+import org.apache.accumulo.core.spi.fs.RandomVolumeChooser;
+import org.apache.accumulo.core.spi.fs.VolumeChooserEnvironment.Scope;
 import org.apache.accumulo.miniclusterImpl.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;
@@ -69,11 +69,11 @@ public class VolumeChooserIT extends ConfigurableMacBase {
 
   private static final String GP = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey();
 
-  static final String getPreferredProp(ChooserScope scope) {
+  static final String getPreferredProp(Scope scope) {
     return GP + "volume.preferred." + scope.name().toLowerCase();
   }
 
-  static final String getPerTableProp(ChooserScope scope) {
+  static final String getPerTableProp(Scope scope) {
     return GP + "volume.chooser." + scope.name().toLowerCase();
   }
 
@@ -121,10 +121,10 @@ public class VolumeChooserIT extends ConfigurableMacBase {
     siteConfig.put(PREFERRED_CHOOSER_PROP, systemPreferredVolumes);
     cfg.setSiteConfig(siteConfig);
 
-    siteConfig.put(getPerTableProp(ChooserScope.LOGGER), PreferredVolumeChooser.class.getName());
-    siteConfig.put(getPreferredProp(ChooserScope.LOGGER), v2.toString());
-    siteConfig.put(getPerTableProp(ChooserScope.INIT), PreferredVolumeChooser.class.getName());
-    siteConfig.put(getPreferredProp(ChooserScope.INIT), systemPreferredVolumes);
+    siteConfig.put(getPerTableProp(Scope.LOGGER), PreferredVolumeChooser.class.getName());
+    siteConfig.put(getPreferredProp(Scope.LOGGER), v2.toString());
+    siteConfig.put(getPerTableProp(Scope.INIT), PreferredVolumeChooser.class.getName());
+    siteConfig.put(getPreferredProp(Scope.INIT), systemPreferredVolumes);
     cfg.setSiteConfig(siteConfig);
 
     // Only add volumes 1, 2, and 4 to the list of instance volumes to have one volume that isn't in