You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/03/21 03:13:07 UTC

[04/25] git commit: ACCUMULO-2061 Initial implementation to remove instance.dfs.dir from usage of instance.volumes

ACCUMULO-2061 Initial implementation to remove instance.dfs.dir from usage of instance.volumes

The value of instance.volumes, when provided, appended the value of
instance.dfs.dir to each configured volume which was unintuitive. It is more
straightforward to use each volume provided as-is instead of appending
on some more cruft.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/7c94c086
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/7c94c086
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/7c94c086

Branch: refs/heads/master
Commit: 7c94c086de2945d698d0907c9dc5ec6959042286
Parents: ef5dc4a
Author: Josh Elser <el...@apache.org>
Authored: Thu Feb 27 00:51:29 2014 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Mar 20 18:55:47 2014 -0400

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  |   4 +-
 .../core/client/impl/OfflineScanner.java        |   4 +-
 .../org/apache/accumulo/core/conf/Property.java |   2 +
 .../accumulo/core/file/VolumeConfiguration.java | 112 ------------
 .../accumulo/core/file/rfile/PrintInfo.java     |   4 +-
 .../core/file/rfile/bcfile/PrintInfo.java       |   4 +-
 .../apache/accumulo/core/util/shell/Shell.java  |   4 +-
 .../org/apache/accumulo/core/volume/Volume.java |  60 +++++++
 .../core/volume/VolumeConfiguration.java        | 151 +++++++++++++++++
 .../apache/accumulo/core/volume/VolumeImpl.java |  93 ++++++++++
 .../apache/accumulo/core/zookeeper/ZooUtil.java |   4 +-
 .../org/apache/accumulo/server/Accumulo.java    |  13 +-
 .../apache/accumulo/server/ServerConstants.java |  48 +++---
 .../accumulo/server/client/BulkImporter.java    |   4 +-
 .../accumulo/server/client/HdfsZooInstance.java |  16 +-
 .../accumulo/server/conf/ZooConfiguration.java  |  16 +-
 .../accumulo/server/fs/VolumeManager.java       |  21 ++-
 .../accumulo/server/fs/VolumeManagerImpl.java   | 169 +++++++++++++------
 .../apache/accumulo/server/fs/VolumeUtil.java   |  27 ++-
 .../apache/accumulo/server/init/Initialize.java |  18 +-
 .../server/master/recovery/HadoopLogCloser.java |   2 +-
 .../server/master/recovery/MapRLogCloser.java   |   2 +-
 .../accumulo/server/util/ChangeSecret.java      |  18 +-
 .../apache/accumulo/server/util/FileUtil.java   |  12 +-
 .../accumulo/server/util/LocalityCheck.java     |   2 +-
 .../accumulo/server/util/TabletOperations.java  |   4 +-
 .../accumulo/server/util/ZooKeeperMain.java     |   2 +-
 .../accumulo/server/ServerConstantsTest.java    |  12 +-
 .../apache/accumulo/server/fs/FileTypeTest.java |  48 ++++--
 .../accumulo/server/fs/VolumeUtilTest.java      |  89 +++++++++-
 .../accumulo/gc/GarbageCollectionTest.java      |  65 +++++++
 .../accumulo/master/tableOps/ExportTable.java   |   2 +-
 .../accumulo/master/tableOps/ImportTable.java   |   2 +-
 .../monitor/servlets/DefaultServlet.java        |   6 +-
 .../tserver/BulkFailedCopyProcessor.java        |  27 +--
 .../org/apache/accumulo/tserver/Compactor.java  |   4 +-
 .../apache/accumulo/tserver/FileManager.java    |   2 +-
 .../org/apache/accumulo/tserver/Tablet.java     |   4 +-
 .../apache/accumulo/tserver/TabletServer.java   |   2 +-
 .../compaction/MajorCompactionRequest.java      |   2 +-
 .../apache/accumulo/tserver/log/LogSorter.java  |   2 +-
 .../accumulo/tserver/log/MultiReader.java       |   2 +-
 .../apache/accumulo/tserver/RootFilesTest.java  |   1 +
 .../tserver/TabletServerSyncCheckTest.java      |  14 +-
 .../accumulo/tserver/log/MultiReaderTest.java   |   8 +-
 .../tserver/log/SortedLogRecoveryTest.java      |   9 +-
 .../tserver/log/TestUpgradePathForWALogs.java   |   6 +-
 .../performance/scan/CollectTabletStats.java    |   6 +-
 .../java/org/apache/accumulo/test/VolumeIT.java |  25 +--
 .../accumulo/test/functional/BulkFileIT.java    |   4 +-
 50 files changed, 822 insertions(+), 336 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index 0b2f10e..9d033e2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -82,7 +82,6 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.file.VolumeConfiguration;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
@@ -109,6 +108,7 @@ import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -1139,7 +1139,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @SuppressWarnings("deprecation")
   private Path checkPath(String dir, String kind, String type) throws IOException, AccumuloException {
     Path ret;
-    FileSystem fs = VolumeConfiguration.getFileSystem(dir, CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance));
+    FileSystem fs = VolumeConfiguration.getVolume(dir, CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance)).getFileSystem();
 
     if (dir.contains(":")) {
       ret = new Path(dir);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
index c90d380..20228d2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
@@ -43,7 +43,6 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.VolumeConfiguration;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
@@ -65,6 +64,7 @@ import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -306,7 +306,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
     
     // TODO need to close files - ACCUMULO-1303
     for (String file : absFiles) {
-      FileSystem fs = VolumeConfiguration.getFileSystem(file, conf, config);
+      FileSystem fs = VolumeConfiguration.getVolume(file, conf, config).getFileSystem();
       FileSKVIterator reader = FileOperations.getInstance().openReader(file, false, fs, conf, acuTableConf, null, null);
       readers.add(reader);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
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 fc4d012..795a250 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
@@ -103,6 +103,7 @@ public enum Property {
   INSTANCE_ZK_HOST("instance.zookeeper.host", "localhost:2181", PropertyType.HOSTLIST, "Comma separated list of zookeeper servers"),
   INSTANCE_ZK_TIMEOUT("instance.zookeeper.timeout", "30s", PropertyType.TIMEDURATION,
       "Zookeeper session timeout; max value when represented as milliseconds should be no larger than " + Integer.MAX_VALUE),
+  @Deprecated
   INSTANCE_DFS_URI(
       "instance.dfs.uri",
       "",
@@ -111,6 +112,7 @@ public enum Property {
           + "will only be used when creating new files if instance.volumes is empty.  After an upgrade to 1.6.0 Accumulo will start using absolute paths to "
           + "reference files.  Files created before a 1.6.0 upgrade are referenced via relative paths.  Relative paths will always be resolved using this config "
           + "(if empty using the hadoop config)."),
+  @Deprecated
   INSTANCE_DFS_DIR("instance.dfs.dir", "/accumulo", PropertyType.ABSOLUTEPATH,
       "HDFS directory in which accumulo instance will run.  Do not change after accumulo is initialized."),
   @Sensitive

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/file/VolumeConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/VolumeConfiguration.java b/core/src/main/java/org/apache/accumulo/core/file/VolumeConfiguration.java
deleted file mode 100644
index fb8c6c8..0000000
--- a/core/src/main/java/org/apache/accumulo/core/file/VolumeConfiguration.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.file;
-
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-public class VolumeConfiguration {
-
-  public static FileSystem getFileSystem(String path, Configuration conf, AccumuloConfiguration acuconf) throws IOException {
-    if (path.contains(":"))
-      return new Path(path).getFileSystem(conf);
-    else
-      return getDefaultFilesystem(conf, acuconf);
-  }
-
-  public static FileSystem getDefaultFilesystem(Configuration conf, AccumuloConfiguration acuconf) throws IOException {
-    String uri = acuconf.get(Property.INSTANCE_DFS_URI);
-    if ("".equals(uri))
-      return FileSystem.get(conf);
-    else
-      try {
-        return FileSystem.get(new URI(uri), conf);
-      } catch (URISyntaxException e) {
-        throw new IOException(e);
-      }
-  }
-
-  public static String getConfiguredBaseDir(AccumuloConfiguration conf) {
-    String singleNamespace = conf.get(Property.INSTANCE_DFS_DIR);
-    String dfsUri = conf.get(Property.INSTANCE_DFS_URI);
-    String baseDir;
-  
-    if (dfsUri == null || dfsUri.isEmpty()) {
-      Configuration hadoopConfig = CachedConfiguration.getInstance();
-      try {
-        baseDir = FileSystem.get(hadoopConfig).getUri().toString() + singleNamespace;
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    } else {
-      if (!dfsUri.contains(":"))
-        throw new IllegalArgumentException("Expected fully qualified URI for " + Property.INSTANCE_DFS_URI.getKey() + " got " + dfsUri);
-      baseDir = dfsUri + singleNamespace;
-    }
-    return baseDir;
-  }
-
-  public static String[] getConfiguredBaseDirs(AccumuloConfiguration conf) {
-    String singleNamespace = conf.get(Property.INSTANCE_DFS_DIR);
-    String ns = conf.get(Property.INSTANCE_VOLUMES);
-  
-    String configuredBaseDirs[];
-  
-    if (ns == null || ns.isEmpty()) {
-      configuredBaseDirs = new String[] {getConfiguredBaseDir(conf)};
-    } else {
-      String namespaces[] = ns.split(",");
-      String unescapedNamespaces[] = new String[namespaces.length];
-      int i = 0;
-      for (String namespace : namespaces) {
-        if (!namespace.contains(":")) {
-          throw new IllegalArgumentException("Expected fully qualified URI for " + Property.INSTANCE_VOLUMES.getKey() + " got " + namespace);
-        }
-  
-        try {
-          // pass through URI to unescape hex encoded chars (e.g. convert %2C to "," char)
-          unescapedNamespaces[i++] = new Path(new URI(namespace)).toString();
-        } catch (URISyntaxException e) {
-          throw new IllegalArgumentException(Property.INSTANCE_VOLUMES.getKey() + " contains " + namespace + " which has a syntax error", e);
-        }
-      }
-  
-      configuredBaseDirs = prefix(unescapedNamespaces, singleNamespace);
-    }
-  
-    return configuredBaseDirs;
-  }
-
-  public static String[] prefix(String bases[], String suffix) {
-    if (suffix.startsWith("/"))
-      suffix = suffix.substring(1);
-    String result[] = new String[bases.length];
-    for (int i = 0; i < bases.length; i++) {
-      result[i] = bases[i] + "/" + suffix;
-    }
-    return result;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
index 4cfefad..4e39fc7 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java
@@ -25,9 +25,9 @@ import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.file.VolumeConfiguration;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
 import org.apache.accumulo.core.file.rfile.RFile.Reader;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -50,7 +50,7 @@ public class PrintInfo {
 
     @SuppressWarnings("deprecation")
     AccumuloConfiguration aconf = AccumuloConfiguration.getSiteConfiguration();
-    FileSystem hadoopFs = VolumeConfiguration.getDefaultFilesystem(conf, aconf);
+    FileSystem hadoopFs = VolumeConfiguration.getDefaultVolume(conf, aconf).getFileSystem();
     FileSystem localFs  = FileSystem.getLocal(conf);
     Opts opts = new Opts();
     opts.parseArgs(PrintInfo.class.getName(), args);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
index f21190e..e2c8a6d 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/PrintInfo.java
@@ -22,8 +22,8 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.file.VolumeConfiguration;
 import org.apache.accumulo.core.file.rfile.bcfile.BCFile.MetaIndexEntry;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -57,7 +57,7 @@ public class PrintInfo {
     Configuration conf = new Configuration();
     @SuppressWarnings("deprecation")
     AccumuloConfiguration siteConf = AccumuloConfiguration.getSiteConfiguration();
-    FileSystem hadoopFs = VolumeConfiguration.getDefaultFilesystem(conf, siteConf);
+    FileSystem hadoopFs = VolumeConfiguration.getDefaultVolume(conf, siteConf).getFileSystem();
     FileSystem localFs = FileSystem.getLocal(conf);
     Path path = new Path(args[0]);
     FileSystem fs;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
index 0c2cf3c..ff6ba09 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
@@ -63,7 +63,6 @@ import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
-import org.apache.accumulo.core.file.VolumeConfiguration;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.util.BadArgumentException;
@@ -156,6 +155,7 @@ import org.apache.accumulo.core.util.shell.commands.UserCommand;
 import org.apache.accumulo.core.util.shell.commands.UserPermissionsCommand;
 import org.apache.accumulo.core.util.shell.commands.UsersCommand;
 import org.apache.accumulo.core.util.shell.commands.WhoAmICommand;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
 import org.apache.commons.cli.BasicParser;
@@ -432,7 +432,7 @@ public class Shell extends ShellOptions {
     if (instanceName == null || keepers == null) {
       AccumuloConfiguration conf = SiteConfiguration.getInstance(ServerConfigurationUtil.convertClientConfig(DefaultConfiguration.getInstance(), clientConfig));
       if (instanceName == null) {
-        Path instanceDir = new Path(VolumeConfiguration.getConfiguredBaseDirs(conf)[0], "instance_id");
+        Path instanceDir = new Path(VolumeConfiguration.getVolumeUris(conf)[0], "instance_id");
         instanceId = UUID.fromString(ZooUtil.getInstanceIDFromHdfs(instanceDir, conf));
       }
       if (keepers == null) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/volume/Volume.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/volume/Volume.java b/core/src/main/java/org/apache/accumulo/core/volume/Volume.java
new file mode 100644
index 0000000..08f61d4
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/volume/Volume.java
@@ -0,0 +1,60 @@
+/*
+ * 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.volume;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Encapsulates a {@link FileSystem} and a base {@link Path} within that filesystem. This
+ * also avoid the necessity to pass around a Configuration. 
+ */
+public interface Volume {
+
+  /**
+   * A {@link FileSystem} that Accumulo will use
+   * @return
+   */
+  public FileSystem getFileSystem();
+
+  /**
+   * The base path which Accumulo will use within the given {@link FileSystem}
+   * @return
+   */
+  public String getBasePath();
+  
+  /**
+   * Convert the given Path into a Path that is relative to the base path for this Volume
+   * @param p
+   * @return
+   */
+  public Path prefixChild(Path p);
+
+  /**
+   * Convert the given child path into a Path that is relative to the base path for this Volume
+   * @param p
+   * @return
+   */
+  public Path prefixChild(String p);
+
+  /**
+   * Determine if the Path is valid on this Volume (contained by the basePath)
+   * @param p
+   * @return True if path is contained within the basePath, false otherwise
+   */
+  public boolean isValidPath(Path p);
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java b/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
new file mode 100644
index 0000000..3005174
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.volume;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import com.google.common.base.Preconditions;
+
+public class VolumeConfiguration {
+  
+  public static Volume getVolume(String path, Configuration conf, AccumuloConfiguration acuconf) throws IOException {
+    Preconditions.checkNotNull(path);
+    
+    if (path.contains(":")) {
+      // An absolute path
+      return create(new Path(path), conf);
+    } else {
+      // A relative path
+      return getDefaultVolume(conf, acuconf);
+    }
+  }
+
+  public static Volume getDefaultVolume(Configuration conf, AccumuloConfiguration acuconf) throws IOException {
+    @SuppressWarnings("deprecation")
+    String uri = acuconf.get(Property.INSTANCE_DFS_URI);
+
+    // By default pull from INSTANCE_DFS_URI, falling back to the Hadoop defined
+    // default filesystem (fs.defaultFS or the deprecated fs.default.name)
+    if ("".equals(uri))
+      return create(FileSystem.get(conf), acuconf);
+    else
+      try {
+        return create(FileSystem.get(new URI(uri), conf), acuconf);
+      } catch (URISyntaxException e) {
+        throw new IOException(e);
+      }
+  }
+
+  @Deprecated
+  public static String getConfiguredBaseDir(AccumuloConfiguration conf) {
+    String singleNamespace = conf.get(Property.INSTANCE_DFS_DIR);
+    String dfsUri = conf.get(Property.INSTANCE_DFS_URI);
+    String baseDir;
+  
+    if (dfsUri == null || dfsUri.isEmpty()) {
+      Configuration hadoopConfig = CachedConfiguration.getInstance();
+      try {
+        baseDir = FileSystem.get(hadoopConfig).getUri().toString() + singleNamespace;
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+      if (!dfsUri.contains(":"))
+        throw new IllegalArgumentException("Expected fully qualified URI for " + Property.INSTANCE_DFS_URI.getKey() + " got " + dfsUri);
+      baseDir = dfsUri + singleNamespace;
+    }
+    return baseDir;
+  }
+
+  /**
+   * Compute the URIs to be used by Accumulo
+   * @param conf
+   * @return
+   */
+  public static String[] getVolumeUris(AccumuloConfiguration conf) {
+    String ns = conf.get(Property.INSTANCE_VOLUMES);
+  
+    String configuredBaseDirs[];
+  
+    if (ns == null || ns.isEmpty()) {
+      // Fall back to using the old config values
+      configuredBaseDirs = new String[] {getConfiguredBaseDir(conf)};
+    } else {
+      String namespaces[] = ns.split(",");
+      configuredBaseDirs = new String[namespaces.length];
+      int i = 0;
+      for (String namespace : namespaces) {
+        if (!namespace.contains(":")) {
+          throw new IllegalArgumentException("Expected fully qualified URI for " + Property.INSTANCE_VOLUMES.getKey() + " got " + namespace);
+        }
+  
+        try {
+          // pass through URI to unescape hex encoded chars (e.g. convert %2C to "," char)
+          configuredBaseDirs[i++] = new Path(new URI(namespace)).toString();
+        } catch (URISyntaxException e) {
+          throw new IllegalArgumentException(Property.INSTANCE_VOLUMES.getKey() + " contains " + namespace + " which has a syntax error", e);
+        }
+      }
+    }
+  
+    return configuredBaseDirs;
+  }
+
+  public static String[] prefix(String bases[], String suffix) {
+    if (suffix.startsWith("/"))
+      suffix = suffix.substring(1);
+    String result[] = new String[bases.length];
+    for (int i = 0; i < bases.length; i++) {
+      result[i] = bases[i] + "/" + suffix;
+    }
+    return result;
+  }
+
+  /**
+   * Create a Volume with the given FileSystem that writes to the default path
+   * @param fs A FileSystem to write to
+   * @return A Volume instance writing to the given FileSystem in the default path 
+   */
+  @SuppressWarnings("deprecation")
+  public static <T extends FileSystem> Volume create(T fs, AccumuloConfiguration acuconf) {
+    String dfsDir = acuconf.get(Property.INSTANCE_DFS_DIR);
+    return new VolumeImpl(fs, null == dfsDir ? Property.INSTANCE_DFS_DIR.getDefaultValue() : dfsDir);
+  }
+  
+  public static <T extends FileSystem> Volume create(T fs, String basePath) {
+    return new VolumeImpl(fs, basePath);
+  }
+  
+  public static Volume create(String path, Configuration conf) throws IOException {
+    Preconditions.checkNotNull(path);
+    return create(new Path(path), conf);
+  }
+  
+  public static Volume create(Path path, Configuration conf) throws IOException {
+    return new VolumeImpl(path, conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/volume/VolumeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/volume/VolumeImpl.java b/core/src/main/java/org/apache/accumulo/core/volume/VolumeImpl.java
new file mode 100644
index 0000000..0aaf482
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/volume/VolumeImpl.java
@@ -0,0 +1,93 @@
+/*
+ * 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.volume;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+
+/**
+ * 
+ */
+public class VolumeImpl implements Volume {
+  protected final FileSystem fs;
+  protected final String basePath;
+  
+  public VolumeImpl(Path path, Configuration conf) throws IOException {
+    checkNotNull(path);
+    checkNotNull(conf);
+    
+    this.fs = path.getFileSystem(conf);
+    this.basePath = path.toUri().getPath();
+  }
+  
+  public VolumeImpl(FileSystem fs, String basePath) {
+    checkNotNull(fs);
+    checkNotNull(basePath);
+    
+    this.fs = fs;
+    this.basePath = basePath;
+  }
+  
+  @Override
+  public FileSystem getFileSystem() {
+    return fs;
+  }
+
+  @Override
+  public String getBasePath() {
+    return basePath;
+  }
+
+  @Override
+  public Path prefixChild(Path p) {
+    return new Path(basePath, p);
+  }
+
+  @Override
+  public boolean isValidPath(Path p) {
+    checkNotNull(p);
+    
+    return p.toUri().getPath().startsWith(basePath);
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof VolumeImpl) {
+      VolumeImpl other = (VolumeImpl) o;
+      return getFileSystem().equals(other.getFileSystem()) && getBasePath().equals(other.getBasePath());
+    }
+    
+    return false;
+  }
+  
+  @Override
+  public String toString() {
+    return getFileSystem() + " " + basePath;
+  }
+
+  @Override
+  public Path prefixChild(String p) {
+    return new Path(basePath, p);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
index de1b432..d536f42 100644
--- a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
@@ -23,8 +23,8 @@ import java.net.UnknownHostException;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.file.VolumeConfiguration;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -49,7 +49,7 @@ public class ZooUtil extends org.apache.accumulo.fate.zookeeper.ZooUtil {
   public static String getInstanceIDFromHdfs(Path instanceDirectory, AccumuloConfiguration conf) {
     try {
 
-      FileSystem fs = VolumeConfiguration.getFileSystem(instanceDirectory.toString(), CachedConfiguration.getInstance(), conf);
+      FileSystem fs = VolumeConfiguration.getVolume(instanceDirectory.toString(), CachedConfiguration.getInstance(), conf).getFileSystem();
       FileStatus[] files = null;
       try {
         files = fs.listStatus(instanceDirectory);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
index 2fa9051..48534f0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/Accumulo.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.Version;
+import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -50,10 +51,12 @@ public class Accumulo {
   private static final Logger log = Logger.getLogger(Accumulo.class);
   
   public static synchronized void updateAccumuloVersion(VolumeManager fs) {
+    // TODO ACCUMULO-2451 Should update all volumes, not one 
+    Volume volume = fs.getVolumes().iterator().next();
     try {
       if (getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
-        fs.create(new Path(ServerConstants.getDataVersionLocation() + "/" + ServerConstants.DATA_VERSION));
-        fs.delete(new Path(ServerConstants.getDataVersionLocation() + "/" + ServerConstants.PREV_DATA_VERSION));
+        fs.create(new Path(ServerConstants.getDataVersionLocation(volume), Integer.toString(ServerConstants.DATA_VERSION)));
+        fs.delete(new Path(ServerConstants.getDataVersionLocation(volume), Integer.toString(ServerConstants.PREV_DATA_VERSION)));
       }
     } catch (IOException e) {
       throw new RuntimeException("Unable to set accumulo version: an error occurred.", e);
@@ -76,8 +79,10 @@ public class Accumulo {
   }
   
   public static synchronized int getAccumuloPersistentVersion(VolumeManager fs) {
-    Path path = ServerConstants.getDataVersionLocation();
-    return getAccumuloPersistentVersion(fs.getFileSystemByPath(path), path);
+    // It doesn't matter which Volume is used as they should all have the data version stored
+    Volume v = fs.getVolumes().iterator().next();
+    Path path = ServerConstants.getDataVersionLocation(v);
+    return getAccumuloPersistentVersion(v.getFileSystem(), path);
   }
 
   public static void enableTracing(String address, String application) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java b/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
index 8983d08..7dd0a08 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerConstants.java
@@ -24,10 +24,11 @@ import java.util.HashSet;
 import java.util.List;
 
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.file.VolumeConfiguration;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeUtil;
@@ -51,29 +52,20 @@ public class ServerConstants {
   public static final int DATA_VERSION = 6;
   public static final int PREV_DATA_VERSION = 5;
 
-  private static String[] baseDirs = null;
-  private static String defaultBaseDir = null;
+  private static String[] baseUris = null;
 
   private static List<Pair<Path,Path>> replacementsList = null;
 
-  public static synchronized String getDefaultBaseDir() {
-    if (defaultBaseDir == null) {
-      defaultBaseDir = new Path(VolumeConfiguration.getConfiguredBaseDir(ServerConfiguration.getSiteConfiguration())).toString();
-    }
-
-    return defaultBaseDir;
-  }
-
   // these are functions to delay loading the Accumulo configuration unless we must
-  public static synchronized String[] getBaseDirs() {
-    if (baseDirs == null) {
-      baseDirs = checkBaseDirs(VolumeConfiguration.getConfiguredBaseDirs(ServerConfiguration.getSiteConfiguration()), false);
+  public static synchronized String[] getBaseUris() {
+    if (baseUris == null) {
+      baseUris = checkBaseUris(VolumeConfiguration.getVolumeUris(ServerConfiguration.getSiteConfiguration()), false);
     }
 
-    return baseDirs;
+    return baseUris;
   }
 
-  public static String[] checkBaseDirs(String[] configuredBaseDirs, boolean ignore) {
+  public static String[] checkBaseUris(String[] configuredBaseDirs, boolean ignore) {
     // all base dirs must have same instance id and data version, any dirs that have neither should be ignored
     String firstDir = null;
     String firstIid = null;
@@ -121,29 +113,29 @@ public class ServerConstants {
   public static final String WAL_DIR = "wal";
 
   public static String[] getTablesDirs() {
-    return VolumeConfiguration.prefix(getBaseDirs(), TABLE_DIR);
+    return VolumeConfiguration.prefix(getBaseUris(), TABLE_DIR);
   }
 
   public static String[] getRecoveryDirs() {
-    return VolumeConfiguration.prefix(getBaseDirs(), RECOVERY_DIR);
+    return VolumeConfiguration.prefix(getBaseUris(), RECOVERY_DIR);
   }
 
   public static String[] getWalDirs() {
-    return VolumeConfiguration.prefix(getBaseDirs(), WAL_DIR);
+    return VolumeConfiguration.prefix(getBaseUris(), WAL_DIR);
   }
 
   public static String[] getWalogArchives() {
-    return VolumeConfiguration.prefix(getBaseDirs(), "walogArchive");
+    return VolumeConfiguration.prefix(getBaseUris(), "walogArchive");
   }
 
-  public static Path getInstanceIdLocation() {
+  public static Path getInstanceIdLocation(Volume v) {
     // all base dirs should have the same instance id, so can choose any one
-    return new Path(getBaseDirs()[0], INSTANCE_ID_DIR);
+    return v.prefixChild(INSTANCE_ID_DIR);
   }
 
-  public static Path getDataVersionLocation() {
+  public static Path getDataVersionLocation(Volume v) {
     // all base dirs should have the same version, so can choose any one
-    return new Path(getBaseDirs()[0], VERSION_DIR);
+    return v.prefixChild(VERSION_DIR);
   }
 
   public static String[] getMetadataTableDirs() {
@@ -151,7 +143,7 @@ public class ServerConstants {
   }
 
   public static String[] getTemporaryDirs() {
-    return VolumeConfiguration.prefix(getBaseDirs(), "tmp");
+    return VolumeConfiguration.prefix(getBaseUris(), "tmp");
   }
 
   public static synchronized List<Pair<Path,Path>> getVolumeReplacements() {
@@ -194,9 +186,9 @@ public class ServerConstants {
       }
 
       HashSet<Path> baseDirs = new HashSet<Path>();
-      for (String baseDir : getBaseDirs()) {
-        // normalize using path and remove accumulo dir
-        baseDirs.add(new Path(baseDir).getParent());
+      for (String baseDir : getBaseUris()) {
+        // normalize using path
+        baseDirs.add(new Path(baseDir));
       }
 
       for (Pair<Path,Path> pair : ret)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
index dc9acf8..27ab078 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
@@ -339,7 +339,7 @@ public class BulkImporter {
     
     try {
       for (Path path : paths) {
-        FileSystem fs = vm.getFileSystemByPath(path);
+        FileSystem fs = vm.getVolumeByPath(path).getFileSystem();
         mapFileSizes.put(path, fs.getContentSummary(path).getLength());
       }
     } catch (IOException e) {
@@ -639,7 +639,7 @@ public class BulkImporter {
     Collection<ByteSequence> columnFamilies = Collections.emptyList();
     String filename = file.toString();
     // log.debug(filename + " finding overlapping tablets " + startRow + " -> " + endRow);
-    FileSystem fs = vm.getFileSystemByPath(file);
+    FileSystem fs = vm.getVolumeByPath(file).getFileSystem();
     FileSKVIterator reader = FileOperations.getInstance().openReader(filename, true, fs, fs.getConf(), acuConf);
     try {
       Text row = startRow;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index 6993a0a..ee928f3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.client;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
@@ -38,10 +39,13 @@ import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.TextUtil;
+import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.conf.ServerConfiguration;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
@@ -122,7 +126,17 @@ public class HdfsZooInstance implements Instance {
 
   private static synchronized void _getInstanceID() {
     if (instanceId == null) {
-      String instanceIdFromFile = ZooUtil.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation(), ServerConfiguration.getSiteConfiguration());
+      AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
+      // InstanceID should be the same across all volumes, so just choose one
+      VolumeManager fs;
+      try {
+        fs = VolumeManagerImpl.get();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      Volume randVolume = fs.getVolumes().iterator().next();
+      log.trace("Looking for instanceId from " + randVolume);
+      String instanceIdFromFile = ZooUtil.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation(randVolume), acuConf);
       instanceId = instanceIdFromFile;
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
index 32f6126..94e468b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.conf;
 
+import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -25,10 +26,15 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance.AccumuloNotInitializedException;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.log4j.Logger;
 
 public class ZooConfiguration extends AccumuloConfiguration {
@@ -57,7 +63,15 @@ public class ZooConfiguration extends AccumuloConfiguration {
     if (instance == null) {
       propCache = new ZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
       instance = new ZooConfiguration(parent);
-      String deprecatedInstanceIdFromHdfs = ZooUtil.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation(), parent);
+      // InstanceID should be the same across all volumes, so just choose one
+      VolumeManager fs;
+      try {
+        fs = VolumeManagerImpl.get();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      Volume randVolume = fs.getVolumes().iterator().next();
+      String deprecatedInstanceIdFromHdfs = ZooUtil.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation(randVolume), parent);
       instanceId = deprecatedInstanceIdFromHdfs;
     }
     return instance;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
----------------------------------------------------------------------
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 f0c7083..9b8fb98 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
@@ -17,14 +17,15 @@
 package org.apache.accumulo.server.fs;
 
 import java.io.IOException;
+import java.util.Collection;
 
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 /**
@@ -47,9 +48,10 @@ public interface VolumeManager {
     }
 
     private static int endOfVolumeIndex(String path, String dir) {
+      // Strip off the suffix that starts with the FileType (e.g. tables, wal, etc)
       int dirIndex = path.indexOf('/' + dir);
       if (dirIndex != -1) {
-        return path.lastIndexOf('/', dirIndex - 1);
+        return dirIndex;
       }
 
       if (path.contains(":"))
@@ -110,7 +112,7 @@ public interface VolumeManager {
   FileStatus getFileStatus(Path path) throws IOException;
 
   // find the appropriate FileSystem object given a path
-  FileSystem getFileSystemByPath(Path path);
+  Volume getVolumeByPath(Path path);
 
   // return the item in options that is in the same volume as source
   Path matchingFileSystem(Path source, String[] options);
@@ -155,4 +157,17 @@ public interface VolumeManager {
 
   // decide on which of the given locations to create a new file
   String choose(String[] options);
+
+  /**
+   * Fetch the default Volume
+   * @return
+   */
+  public Volume getDefaultVolume();
+
+  /**
+   * Fetch the configured Volumes, excluding the default Volume
+   * @return
+   */
+  public Collection<Volume> getVolumes();
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
----------------------------------------------------------------------
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 80301ef..ca5167d 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
@@ -16,12 +16,15 @@
  */
 package org.apache.accumulo.server.fs;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.URI;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -34,9 +37,9 @@ import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.file.VolumeConfiguration;
 import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.core.volume.Volume;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.commons.lang.NotImplementedException;
@@ -54,34 +57,49 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
 public class VolumeManagerImpl implements VolumeManager {
 
   private static final Logger log = Logger.getLogger(VolumeManagerImpl.class);
 
-  Map<String,? extends FileSystem> volumes;
-  FileSystem defaultVolume;
+  Map<String,Volume> volumesByName;
+  Multimap<FileSystem,Volume> volumesByFileSystem;
+  Volume defaultVolume;
   AccumuloConfiguration conf;
   VolumeChooser chooser;
 
-  protected VolumeManagerImpl(Map<String,? extends FileSystem> volumes, String defaultVolume, AccumuloConfiguration conf) {
-    this.volumes = volumes;
-    this.defaultVolume = volumes.get(defaultVolume);
+  protected VolumeManagerImpl(Map<String,Volume> volumes, Volume defaultVolume, AccumuloConfiguration conf) {
+    this.volumesByName = volumes;
+    this.defaultVolume = defaultVolume;
+    // We may have multiple directories used in a single FileSystem (e.g. testing)
+    this.volumesByFileSystem = HashMultimap.create();
+    invertVolumesByFileSystem(volumesByName, volumesByFileSystem);
     this.conf = conf;
     ensureSyncIsEnabled();
     chooser = Property.createInstanceFromPropertyName(conf, Property.GENERAL_VOLUME_CHOOSER, VolumeChooser.class, new RandomVolumeChooser());
   }
 
-  public static org.apache.accumulo.server.fs.VolumeManager getLocal() throws IOException {
-    return new VolumeManagerImpl(Collections.singletonMap("", FileSystem.getLocal(CachedConfiguration.getInstance())), "",
-        DefaultConfiguration.getDefaultConfiguration());
+  private void invertVolumesByFileSystem(Map<String,Volume> forward, Multimap<FileSystem,Volume> inverted) {
+    for (Volume volume : forward.values()) {
+      inverted.put(volume.getFileSystem(), volume);
+    }
+  }
+  
+  public static org.apache.accumulo.server.fs.VolumeManager getLocal(String localBasePath) throws IOException {
+    AccumuloConfiguration accConf = DefaultConfiguration.getDefaultConfiguration();
+    Volume defaultLocalVolume = VolumeConfiguration.create(FileSystem.getLocal(CachedConfiguration.getInstance()), localBasePath);
+    
+    return new VolumeManagerImpl(Collections.singletonMap(DEFAULT, defaultLocalVolume), defaultLocalVolume, accConf);
   }
 
   @Override
   public void close() throws IOException {
     IOException ex = null;
-    for (FileSystem fs : volumes.values()) {
+    for (Volume volume : volumesByName.values()) {
       try {
-        fs.close();
+        volume.getFileSystem().close();
       } catch (IOException e) {
         ex = e;
       }
@@ -93,14 +111,20 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @Override
   public FSDataOutputStream create(Path path) throws IOException {
-    FileSystem fs = getFileSystemByPath(path);
-    return fs.create(path);
+    checkNotNull(path);
+    
+    Volume v = getVolumeByPath(path);
+    
+    return v.getFileSystem().create(path);
   }
 
   @Override
   public FSDataOutputStream create(Path path, boolean overwrite) throws IOException {
-    FileSystem fs = getFileSystemByPath(path);
-    return fs.create(path, overwrite);
+    checkNotNull(path);
+    
+    Volume v = getVolumeByPath(path);
+
+    return v.getFileSystem().create(path, overwrite);
   }
 
   private static long correctBlockSize(Configuration conf, long blockSize) {
@@ -121,7 +145,11 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @Override
   public FSDataOutputStream create(Path path, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException {
-    FileSystem fs = getFileSystemByPath(path);
+    checkNotNull(path);
+
+    Volume v = getVolumeByPath(path);
+    FileSystem fs = v.getFileSystem();
+    
     if (bufferSize == 0) {
       fs.getConf().getInt("io.file.buffer.size", 4096);
     }
@@ -130,13 +158,16 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @Override
   public boolean createNewFile(Path path) throws IOException {
-    FileSystem fs = getFileSystemByPath(path);
-    return fs.createNewFile(path);
+    checkNotNull(path);
+
+    Volume v = getVolumeByPath(path);
+    return v.getFileSystem().createNewFile(path);
   }
 
   @Override
   public FSDataOutputStream createSyncable(Path logPath, int bufferSize, short replication, long blockSize) throws IOException {
-    FileSystem fs = getFileSystemByPath(logPath);
+    Volume v = getVolumeByPath(logPath);
+    FileSystem fs = v.getFileSystem();
     blockSize = correctBlockSize(fs.getConf(), blockSize);
     bufferSize = correctBufferSize(fs.getConf(), bufferSize);
     try {
@@ -174,18 +205,18 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @Override
   public boolean delete(Path path) throws IOException {
-    return getFileSystemByPath(path).delete(path, false);
+    return getVolumeByPath(path).getFileSystem().delete(path, false);
   }
 
   @Override
   public boolean deleteRecursively(Path path) throws IOException {
-    return getFileSystemByPath(path).delete(path, true);
+    return getVolumeByPath(path).getFileSystem().delete(path, true);
   }
 
   protected void ensureSyncIsEnabled() {
-    for (Entry<String,? extends FileSystem> entry : getFileSystems().entrySet()) {
+    for (Entry<String,Volume> entry : getFileSystems().entrySet()) {
       final String volumeName = entry.getKey();
-      FileSystem fs = entry.getValue();
+      FileSystem fs = entry.getValue().getFileSystem();
 
       if (ViewFSUtils.isViewFS(fs)) {
         try {
@@ -255,19 +286,36 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @Override
   public boolean exists(Path path) throws IOException {
-    return getFileSystemByPath(path).exists(path);
+    return getVolumeByPath(path).getFileSystem().exists(path);
   }
 
   @Override
   public FileStatus getFileStatus(Path path) throws IOException {
-    return getFileSystemByPath(path).getFileStatus(path);
+    return getVolumeByPath(path).getFileSystem().getFileStatus(path);
   }
 
   @Override
-  public FileSystem getFileSystemByPath(Path path) {
+  public Volume getVolumeByPath(Path path) {
     if (path.toString().contains(":")) {
       try {
-        return path.getFileSystem(CachedConfiguration.getInstance());
+        FileSystem pathFs = path.getFileSystem(CachedConfiguration.getInstance());
+        Collection<Volume> candidateVolumes = volumesByFileSystem.get(pathFs);
+        if (null != candidateVolumes) {
+          for (Volume candidateVolume : candidateVolumes) {
+            if (candidateVolume.isValidPath(path)) {
+              return candidateVolume;
+            }
+          }
+
+          // For the same reason as we can have multiple Volumes within a single filesystem
+          // we could also not find a matching one. We should defer back to the defaultVolume
+          // e.g. volume rename with old path references
+          log.debug("Defaulting to " + defaultVolume + " as a valid volume could not be determined for " + path);
+
+          return defaultVolume;
+        }
+        
+        log.debug("Could not determine volume for Path '" + path + "' from defined volumes");
       } catch (IOException ex) {
         throw new RuntimeException(ex);
       }
@@ -276,29 +324,31 @@ public class VolumeManagerImpl implements VolumeManager {
     return defaultVolume;
   }
 
-  private Map<String,? extends FileSystem> getFileSystems() {
-    return volumes;
+  private Map<String,Volume> getFileSystems() {
+    return volumesByName;
   }
 
   @Override
   public FileStatus[] listStatus(Path path) throws IOException {
-    return getFileSystemByPath(path).listStatus(path);
+    return getVolumeByPath(path).getFileSystem().listStatus(path);
   }
 
   @Override
   public boolean mkdirs(Path path) throws IOException {
-    return getFileSystemByPath(path).mkdirs(path);
+    return getVolumeByPath(path).getFileSystem().mkdirs(path);
   }
 
   @Override
   public FSDataInputStream open(Path path) throws IOException {
-    return getFileSystemByPath(path).open(path);
+    return getVolumeByPath(path).getFileSystem().open(path);
   }
 
   @Override
   public boolean rename(Path path, Path newPath) throws IOException {
-    FileSystem source = getFileSystemByPath(path);
-    FileSystem dest = getFileSystemByPath(newPath);
+    Volume srcVolume = getVolumeByPath(path);
+    Volume destVolume = getVolumeByPath(newPath);
+    FileSystem source = srcVolume.getFileSystem();
+    FileSystem dest = destVolume.getFileSystem();
     if (source != dest) {
       throw new NotImplementedException("Cannot rename files across volumes: " + path + " -> " + newPath);
     }
@@ -307,14 +357,15 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @Override
   public boolean moveToTrash(Path path) throws IOException {
-    FileSystem fs = getFileSystemByPath(path);
+    FileSystem fs = getVolumeByPath(path).getFileSystem();
     Trash trash = new Trash(fs, fs.getConf());
     return trash.moveToTrash(path);
   }
 
   @Override
   public short getDefaultReplication(Path path) {
-    FileSystem fs = getFileSystemByPath(path);
+    Volume v = getVolumeByPath(path);
+    FileSystem fs = v.getFileSystem();
     try {
       // try calling hadoop 2 method
       Method method = fs.getClass().getMethod("getDefaultReplication", Path.class);
@@ -336,7 +387,7 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @Override
   public boolean isFile(Path path) throws IOException {
-    return getFileSystemByPath(path).isFile(path);
+    return getVolumeByPath(path).getFileSystem().isFile(path);
   }
 
   public static VolumeManager get() throws IOException {
@@ -347,26 +398,30 @@ public class VolumeManagerImpl implements VolumeManager {
   static private final String DEFAULT = "";
 
   public static VolumeManager get(AccumuloConfiguration conf) throws IOException {
-    Map<String,FileSystem> fileSystems = new HashMap<String,FileSystem>();
-    Configuration hadoopConf = CachedConfiguration.getInstance();
-    fileSystems.put(DEFAULT, VolumeConfiguration.getDefaultFilesystem(hadoopConf, conf));
-    for (String space : VolumeConfiguration.getConfiguredBaseDirs(conf)) {
-      if (space.equals(DEFAULT))
+    final Map<String,Volume> volumes = new HashMap<String,Volume>();
+    final Configuration hadoopConf = CachedConfiguration.getInstance();
+
+    // The "default" Volume for Accumulo (in case no volumes are specified)
+    for (String volumeUriOrDir : VolumeConfiguration.getVolumeUris(conf)) {
+      if (volumeUriOrDir.equals(DEFAULT))
+      // Cannot re-define the default volume
         throw new IllegalArgumentException();
 
-      if (space.contains(":")) {
-        fileSystems.put(space, new Path(space).getFileSystem(hadoopConf));
+      // We require a URI here, fail if it doesn't look like one
+      if (volumeUriOrDir.contains(":")) {
+        volumes.put(volumeUriOrDir, VolumeConfiguration.create(new Path(volumeUriOrDir), hadoopConf));
       } else {
-        throw new IllegalArgumentException("Expected fully qualified URI for " + Property.INSTANCE_VOLUMES.getKey() + " got " + space);
+        throw new IllegalArgumentException("Expected fully qualified URI for " + Property.INSTANCE_VOLUMES.getKey() + " got " + volumeUriOrDir);
       }
     }
 
-    return new VolumeManagerImpl(fileSystems, DEFAULT, conf);
+    return new VolumeManagerImpl(volumes, VolumeConfiguration.getDefaultVolume(hadoopConf, conf), conf);
   }
 
   @Override
   public boolean isReady() throws IOException {
-    for (FileSystem fs : getFileSystems().values()) {
+    for (Volume volume : getFileSystems().values()) {
+      FileSystem fs = volume.getFileSystem();
 
       if (ViewFSUtils.isViewFS(fs)) {
         try {
@@ -421,7 +476,7 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @Override
   public FileStatus[] globStatus(Path pathPattern) throws IOException {
-    return getFileSystemByPath(pathPattern).globStatus(pathPattern);
+    return getVolumeByPath(pathPattern).getFileSystem().globStatus(pathPattern);
   }
 
   @Override
@@ -476,18 +531,18 @@ public class VolumeManagerImpl implements VolumeManager {
       return new Path(path);
 
     // normalize the path
-    Path fullPath = new Path(ServerConstants.getDefaultBaseDir(), fileType.getDirectory());
+    Path fullPath = new Path(defaultVolume.getBasePath(), fileType.getDirectory());
     if (path.startsWith("/"))
       path = path.substring(1);
     fullPath = new Path(fullPath, path);
 
-    FileSystem fs = getFileSystemByPath(fullPath);
+    FileSystem fs = getVolumeByPath(fullPath).getFileSystem();
     return fs.makeQualified(fullPath);
   }
 
   @Override
   public ContentSummary getContentSummary(Path dir) throws IOException {
-    return getFileSystemByPath(dir).getContentSummary(dir);
+    return getVolumeByPath(dir).getFileSystem().getContentSummary(dir);
   }
 
   @Override
@@ -495,4 +550,14 @@ public class VolumeManagerImpl implements VolumeManager {
     return chooser.choose(options);
   }
 
+  @Override
+  public Volume getDefaultVolume() {
+    return defaultVolume;
+  }
+  
+  @Override
+  public Collection<Volume> getVolumes() {
+    return volumesByName.values();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
index da3baa6..bcfb008 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
@@ -52,7 +52,7 @@ import org.apache.log4j.Logger;
 public class VolumeUtil {
 
   private static final Logger log = Logger.getLogger(VolumeUtil.class);
-
+  
   private static boolean isActiveVolume(Path dir) {
 
     // consider relative path as active and take no action
@@ -81,8 +81,10 @@ public class VolumeUtil {
   }
 
   public static String switchVolume(String path, FileType ft, List<Pair<Path,Path>> replacements) {
-    if (replacements.size() == 0)
+    if (replacements.size() == 0) {
+      log.trace("Not switching volume because there are no replacements");
       return null;
+    }
 
     Path p = new Path(path);
 
@@ -92,10 +94,15 @@ public class VolumeUtil {
     for (Pair<Path,Path> pair : replacements) {
       Path key = removeTrailingSlash(pair.getFirst());
 
-      if (key.equals(volume))
-        return new Path(pair.getSecond(), ft.removeVolume(p)).toString();
+      if (key.equals(volume)) {
+        String replacement = new Path(pair.getSecond(), ft.removeVolume(p)).toString();
+        log.trace("Replacing " + path + " with " + replacement);
+        return replacement;
+      }
     }
 
+    log.trace("Could not find replacement for " + ft + " at " + path);
+
     return null;
   }
 
@@ -119,13 +126,17 @@ public class VolumeUtil {
 
     }
 
-    if (numSwitched == 0)
+    if (numSwitched == 0) {
+      log.trace("Did not switch " + le);
       return null;
+    }
 
     LogEntry newLogEntry = new LogEntry(le);
     newLogEntry.filename = switchedPath;
     newLogEntry.logSet = switchedLogs;
 
+    log.trace("Switched " + le + " to " + newLogEntry);
+
     return newLogEntry;
   }
 
@@ -165,6 +176,7 @@ public class VolumeUtil {
    */
   public static TabletFiles updateTabletVolumes(ZooLock zooLock, VolumeManager vm, KeyExtent extent, TabletFiles tabletFiles) throws IOException {
     List<Pair<Path,Path>> replacements = ServerConstants.getVolumeReplacements();
+    log.trace("Using volume replacements: " + replacements);
 
     List<LogEntry> logsToRemove = new ArrayList<LogEntry>();
     List<LogEntry> logsToAdd = new ArrayList<LogEntry>();
@@ -239,8 +251,8 @@ public class VolumeUtil {
 
       // this code needs to be idempotent
 
-      FileSystem fs1 = vm.getFileSystemByPath(dir);
-      FileSystem fs2 = vm.getFileSystemByPath(newDir);
+      FileSystem fs1 = vm.getVolumeByPath(dir).getFileSystem();
+      FileSystem fs2 = vm.getVolumeByPath(newDir).getFileSystem();
 
       if (!same(fs1, dir, fs2, newDir)) {
         if (fs2.exists(newDir)) {
@@ -335,4 +347,5 @@ public class VolumeUtil {
     SecureRandom rand = new SecureRandom();
     return new Path(path.getParent(), path.getName() + "_" + System.currentTimeMillis() + "_" + Math.abs(rand.nextInt()) + ".bak");
   }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------
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 925f602..206a721 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
@@ -40,7 +40,6 @@ import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.file.VolumeConfiguration;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
@@ -52,6 +51,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Da
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
@@ -151,7 +151,7 @@ public class Initialize {
     else
       fsUri = FileSystem.getDefaultUri(conf).toString();
     log.info("Hadoop Filesystem is " + fsUri);
-    log.info("Accumulo data dirs are " + Arrays.asList(VolumeConfiguration.getConfiguredBaseDirs(ServerConfiguration.getSiteConfiguration())));
+    log.info("Accumulo data dirs are " + Arrays.asList(VolumeConfiguration.getVolumeUris(ServerConfiguration.getSiteConfiguration())));
     log.info("Zookeeper server is " + sconf.get(Property.INSTANCE_ZK_HOST));
     log.info("Checking if Zookeeper is available. If this hangs, then you need to make sure zookeeper is running");
     if (!zookeeperAvailable()) {
@@ -173,7 +173,7 @@ public class Initialize {
     try {
       if (isInitialized(fs)) {
         String instanceDfsDir = sconf.get(Property.INSTANCE_DFS_DIR);
-        log.fatal("It appears the directories " + Arrays.asList(VolumeConfiguration.getConfiguredBaseDirs(ServerConfiguration.getSiteConfiguration()))
+        log.fatal("It appears the directories " + Arrays.asList(VolumeConfiguration.getVolumeUris(ServerConfiguration.getSiteConfiguration()))
             + " were previously initialized.");
         String instanceVolumes = sconf.get(Property.INSTANCE_VOLUMES);
         String instanceDfsUri = sconf.get(Property.INSTANCE_DFS_URI);
@@ -220,7 +220,7 @@ public class Initialize {
 
     UUID uuid = UUID.randomUUID();
     // the actual disk locations of the root table and tablets
-    String[] configuredTableDirs = VolumeConfiguration.prefix(VolumeConfiguration.getConfiguredBaseDirs(ServerConfiguration.getSiteConfiguration()),
+    String[] configuredTableDirs = VolumeConfiguration.prefix(VolumeConfiguration.getVolumeUris(ServerConfiguration.getSiteConfiguration()),
         ServerConstants.TABLE_DIR);
     final Path rootTablet = new Path(fs.choose(configuredTableDirs) + "/" + RootTable.ID + RootTable.ROOT_TABLET_LOCATION);
     try {
@@ -296,7 +296,7 @@ public class Initialize {
   private static void initFileSystem(Opts opts, VolumeManager fs, UUID uuid, Path rootTablet) throws IOException {
     FileStatus fstat;
 
-    initDirs(fs, uuid, VolumeConfiguration.getConfiguredBaseDirs(ServerConfiguration.getSiteConfiguration()), false);
+    initDirs(fs, uuid, VolumeConfiguration.getVolumeUris(ServerConfiguration.getSiteConfiguration()), false);
 
     // the actual disk locations of the metadata table and tablets
     final Path[] metadataTableDirs = paths(ServerConstants.getMetadataTableDirs());
@@ -341,7 +341,7 @@ public class Initialize {
     // the root tablet contains the key extent and locations of all the
     // metadata tablets
     String initRootTabFile = rootTablet + "/00000_00000." + FileOperations.getNewFileExtension(AccumuloConfiguration.getDefaultConfiguration());
-    FileSystem ns = fs.getFileSystemByPath(new Path(initRootTabFile));
+    FileSystem ns = fs.getVolumeByPath(new Path(initRootTabFile)).getFileSystem();
     FileSKVWriter mfw = FileOperations.getInstance().openWriter(initRootTabFile, ns, ns.getConf(), AccumuloConfiguration.getDefaultConfiguration());
     mfw.startDefaultLocalityGroup();
 
@@ -553,7 +553,7 @@ public class Initialize {
   }
 
   public static boolean isInitialized(VolumeManager fs) throws IOException {
-    for (String baseDir : VolumeConfiguration.getConfiguredBaseDirs(ServerConfiguration.getSiteConfiguration())) {
+    for (String baseDir : VolumeConfiguration.getVolumeUris(ServerConfiguration.getSiteConfiguration())) {
       if (fs.exists(new Path(baseDir, ServerConstants.INSTANCE_ID_DIR)) || fs.exists(new Path(baseDir, ServerConstants.VERSION_DIR)))
         return true;
     }
@@ -564,10 +564,10 @@ public class Initialize {
   private static void addVolumes(VolumeManager fs) throws IOException {
     HashSet<String> initializedDirs = new HashSet<String>();
     initializedDirs
-        .addAll(Arrays.asList(ServerConstants.checkBaseDirs(VolumeConfiguration.getConfiguredBaseDirs(ServerConfiguration.getSiteConfiguration()), true)));
+        .addAll(Arrays.asList(ServerConstants.checkBaseUris(VolumeConfiguration.getVolumeUris(ServerConfiguration.getSiteConfiguration()), true)));
 
     HashSet<String> uinitializedDirs = new HashSet<String>();
-    uinitializedDirs.addAll(Arrays.asList(VolumeConfiguration.getConfiguredBaseDirs(ServerConfiguration.getSiteConfiguration())));
+    uinitializedDirs.addAll(Arrays.asList(VolumeConfiguration.getVolumeUris(ServerConfiguration.getSiteConfiguration())));
     uinitializedDirs.removeAll(initializedDirs);
 
     Path aBasePath = new Path(initializedDirs.iterator().next());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
index 7edc0cf..0e7f83a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
@@ -37,7 +37,7 @@ public class HadoopLogCloser implements LogCloser {
 
   @Override
   public long close(AccumuloConfiguration conf, VolumeManager fs, Path source) throws IOException {
-    FileSystem ns = fs.getFileSystemByPath(source);
+    FileSystem ns = fs.getVolumeByPath(source).getFileSystem();
 
     // if path points to a viewfs path, then resolve to underlying filesystem
     if (ViewFSUtils.isViewFS(ns)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java
index bba7ac5..d737109 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/MapRLogCloser.java
@@ -32,7 +32,7 @@ public class MapRLogCloser implements LogCloser {
   @Override
   public long close(AccumuloConfiguration conf, VolumeManager fs, Path path) throws IOException {
     log.info("Recovering file " + path.toString() + " by changing permission to readonly");
-    FileSystem ns = fs.getFileSystemByPath(path);
+    FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
     FsPermission roPerm = new FsPermission((short) 0444);
     try {
       ns.setPermission(path, roPerm);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java b/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
index ac13034..3f33a0e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
@@ -24,7 +24,7 @@ import java.util.UUID;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
@@ -32,8 +32,9 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.cli.ClientOpts;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.ACL;
@@ -57,7 +58,7 @@ public class ChangeSecret {
     argsList.add("--new");
     argsList.addAll(Arrays.asList(args));
     opts.parseArgs(ChangeSecret.class.getName(), argsList.toArray(new String[0]));
-    FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
+    VolumeManager fs = VolumeManagerImpl.get();
     Instance inst = opts.getInstance();
     if (!verifyAccumuloIsDown(inst, opts.oldPass))
       System.exit(-1);
@@ -142,10 +143,13 @@ public class ChangeSecret {
     return newInstanceId;
   }
   
-  private static void updateHdfs(FileSystem fs, Instance inst, String newInstanceId) throws IOException {
-    fs.delete(ServerConstants.getInstanceIdLocation(), true);
-    fs.mkdirs(ServerConstants.getInstanceIdLocation());
-    fs.create(new Path(ServerConstants.getInstanceIdLocation(), newInstanceId)).close();
+  private static void updateHdfs(VolumeManager fs, Instance inst, String newInstanceId) throws IOException {
+    // Need to recreate the instanceId on all of them to keep consistency
+    for (Volume v : fs.getVolumes()) {
+      v.getFileSystem().delete(ServerConstants.getInstanceIdLocation(v), true);
+      v.getFileSystem().mkdirs(ServerConstants.getInstanceIdLocation(v));
+      v.getFileSystem().create(new Path(ServerConstants.getInstanceIdLocation(v), newInstanceId)).close();
+    }
   }
   
   private static void deleteInstance(Instance origInstance, String oldPass) throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
index 8e38cbd..277a0c2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java
@@ -125,7 +125,7 @@ public class FileUtil {
       String newMapFile = String.format("%s/%04d.%s", newDir, count++, RFile.EXTENSION);
       
       outFiles.add(newMapFile);
-      FileSystem ns = fs.getFileSystemByPath(new Path(newMapFile));
+      FileSystem ns = fs.getVolumeByPath(new Path(newMapFile)).getFileSystem();
       FileSKVWriter writer = new RFileOperations().openWriter(newMapFile.toString(), ns, ns.getConf(), acuConf);
       writer.startDefaultLocalityGroup();
       List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<SortedKeyValueIterator<Key,Value>>(inFiles.size());
@@ -133,7 +133,7 @@ public class FileUtil {
       FileSKVIterator reader = null;
       try {
         for (String s : inFiles) {
-          ns = fs.getFileSystemByPath(new Path(s));
+          ns = fs.getVolumeByPath(new Path(s)).getFileSystem();
           reader = FileOperations.getInstance().openIndex(s, ns, ns.getConf(), acuConf);
           iters.add(reader);
         }
@@ -390,7 +390,7 @@ public class FileUtil {
     for (String ref : mapFiles) {
       FileSKVIterator reader = null;
       Path path = new Path(ref);
-      FileSystem ns = fs.getFileSystemByPath(path);
+      FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
       try {
         if (useIndex)
           reader = FileOperations.getInstance().openIndex(path.toString(), ns, ns.getConf(), acuConf);
@@ -435,7 +435,7 @@ public class FileUtil {
     for (FileRef mapfile : mapfiles) {
       
       FileSKVIterator reader = null;
-      FileSystem ns = fs.getFileSystemByPath(mapfile.path());
+      FileSystem ns = fs.getVolumeByPath(mapfile.path()).getFileSystem();
       try {
         reader = FileOperations.getInstance().openReader(mapfile.toString(), false, ns, ns.getConf(), acuConf);
         
@@ -470,7 +470,7 @@ public class FileUtil {
     
     for (FileRef ref : mapFiles) {
       Path path = ref.path();
-      FileSystem ns = fs.getFileSystemByPath(path);
+      FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
       FileSKVIterator reader = FileOperations.getInstance().openReader(path.toString(), true, ns, ns.getConf(), acuConf);
       
       try {
@@ -513,7 +513,7 @@ public class FileUtil {
       counts.put(keyExtent, new MLong(0));
     
     Text row = new Text();
-    FileSystem ns = fs.getFileSystemByPath(mapFile);
+    FileSystem ns = fs.getVolumeByPath(mapFile).getFileSystem();
     FileSKVIterator index = FileOperations.getInstance().openIndex(mapFile.toString(), ns, ns.getConf(), acuConf);
     
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java b/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
index a96e791..35bb8f5 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
@@ -88,7 +88,7 @@ public class LocalityCheck {
     }
     for (String file : files) {
       Path filePath = new Path(file);
-      FileSystem ns = fs.getFileSystemByPath(filePath);
+      FileSystem ns = fs.getVolumeByPath(filePath).getFileSystem();
       FileStatus fileStatus = ns.getFileStatus(filePath);
       BlockLocation[] fileBlockLocations = ns.getFileBlockLocations(fileStatus, 0, fileStatus.getLen());
       for (BlockLocation blockLocation : fileBlockLocations) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/util/TabletOperations.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TabletOperations.java b/server/base/src/main/java/org/apache/accumulo/server/util/TabletOperations.java
index b237cd0..b8e7113 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TabletOperations.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TabletOperations.java
@@ -46,7 +46,7 @@ public class TabletOperations {
           lowDirectory = Constants.DEFAULT_TABLET_LOCATION;
           Path lowDirectoryPath = new Path(volume + "/" + tableId + "/" + lowDirectory);
           if (fs.exists(lowDirectoryPath) || fs.mkdirs(lowDirectoryPath)) {
-            FileSystem pathFs = fs.getFileSystemByPath(lowDirectoryPath);
+            FileSystem pathFs = fs.getVolumeByPath(lowDirectoryPath).getFileSystem();
             return lowDirectoryPath.makeQualified(pathFs.getUri(), pathFs.getWorkingDirectory()).toString();
           }
           log.warn("Failed to create " + lowDirectoryPath + " for unknown reason");
@@ -56,7 +56,7 @@ public class TabletOperations {
           if (fs.exists(lowDirectoryPath))
             throw new IllegalStateException("Dir exist when it should not " + lowDirectoryPath);
           if (fs.mkdirs(lowDirectoryPath)) {
-            FileSystem lowDirectoryFs = fs.getFileSystemByPath(lowDirectoryPath);
+            FileSystem lowDirectoryFs = fs.getVolumeByPath(lowDirectoryPath).getFileSystem();
             return lowDirectoryPath.makeQualified(lowDirectoryFs.getUri(), lowDirectoryFs.getWorkingDirectory()).toString();
           }
         }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java b/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
index 37edb1a..7cac046 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
@@ -41,7 +41,7 @@ public class ZooKeeperMain {
     Opts opts = new Opts();
     opts.parseArgs(ZooKeeperMain.class.getName(), args);
     FileSystem fs = FileSystem.get(CachedConfiguration.getInstance());
-    String baseDir = ServerConstants.getBaseDirs()[0];
+    String baseDir = ServerConstants.getBaseUris()[0];
     System.out.println("Using " + fs.makeQualified(new Path(baseDir + "/instance_id")) + " to lookup accumulo instance");
     Instance instance = HdfsZooInstance.getInstance();
     if (opts.servers == null) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7c94c086/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java b/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java
index a316155..8c8b2f2 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java
@@ -59,26 +59,26 @@ public class ServerConstantsTest {
   }
 
   private void verifyAllPass(ArrayList<String> paths) {
-    Assert.assertEquals(paths, Arrays.asList(ServerConstants.checkBaseDirs(paths.toArray(new String[paths.size()]), true)));
-    Assert.assertEquals(paths, Arrays.asList(ServerConstants.checkBaseDirs(paths.toArray(new String[paths.size()]), false)));
+    Assert.assertEquals(paths, Arrays.asList(ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), true)));
+    Assert.assertEquals(paths, Arrays.asList(ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), false)));
   }
 
   private void verifySomePass(ArrayList<String> paths, int numExpected) {
-    Assert.assertEquals(paths.subList(0, 2), Arrays.asList(ServerConstants.checkBaseDirs(paths.toArray(new String[paths.size()]), true)));
+    Assert.assertEquals(paths.subList(0, 2), Arrays.asList(ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), true)));
     try {
-      ServerConstants.checkBaseDirs(paths.toArray(new String[paths.size()]), false);
+      ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), false);
       Assert.fail();
     } catch (Exception e) {}
   }
 
   private void verifyError(ArrayList<String> paths) {
     try {
-      ServerConstants.checkBaseDirs(paths.toArray(new String[paths.size()]), true);
+      ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), true);
       Assert.fail();
     } catch (Exception e) {}
 
     try {
-      ServerConstants.checkBaseDirs(paths.toArray(new String[paths.size()]), false);
+      ServerConstants.checkBaseUris(paths.toArray(new String[paths.size()]), false);
       Assert.fail();
     } catch (Exception e) {}
   }