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/07 21:54:03 UTC

[49/50] [abbrv] git commit: ACCUMULO-2061 Get VolumeManagerImpl compiling again

ACCUMULO-2061 Get VolumeManagerImpl compiling again


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

Branch: refs/heads/ACCUMULO-2061
Commit: ce32e3ed8e1801eb282ff0977e413d48b633f31a
Parents: 63037db
Author: Josh Elser <el...@apache.org>
Authored: Fri Mar 7 15:40:48 2014 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Mar 7 15:40:48 2014 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/util/shell/Shell.java  |  2 +-
 .../org/apache/accumulo/core/volume/Volume.java | 13 +++
 .../core/volume/VolumeConfiguration.java        |  7 +-
 .../apache/accumulo/core/volume/VolumeImpl.java | 18 ++++
 .../apache/accumulo/server/ServerConstants.java | 28 +++---
 .../accumulo/server/fs/VolumeManager.java       |  2 +-
 .../accumulo/server/fs/VolumeManagerImpl.java   | 94 +++++++++++++-------
 .../apache/accumulo/server/init/Initialize.java | 14 +--
 .../accumulo/server/util/ZooKeeperMain.java     |  2 +-
 .../accumulo/server/ServerConstantsTest.java    | 12 +--
 .../monitor/servlets/DefaultServlet.java        |  2 +-
 11 files changed, 131 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ce32e3ed/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 0f8e4d3..208a4cc 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
@@ -434,7 +434,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/ce32e3ed/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
index 3726518..9dd7b16 100644
--- a/core/src/main/java/org/apache/accumulo/core/volume/Volume.java
+++ b/core/src/main/java/org/apache/accumulo/core/volume/Volume.java
@@ -36,5 +36,18 @@ public interface Volume {
    * @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);
 
+  /**
+   * 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/ce32e3ed/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
index 59f3da7..460ffbe 100644
--- a/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java
@@ -77,7 +77,12 @@ public class VolumeConfiguration {
     return baseDir;
   }
 
-  public static String[] getConfiguredBaseDirs(AccumuloConfiguration conf) {
+  /**
+   * 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[];

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ce32e3ed/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
index bbdc166..65740ce 100644
--- a/core/src/main/java/org/apache/accumulo/core/volume/VolumeImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/volume/VolumeImpl.java
@@ -63,6 +63,24 @@ public class VolumeImpl implements Volume {
   public String getBasePath() {
     return basePath;
   }
+
+  /* (non-javadoc)
+   * @see org.apache.accumulo.core.volume.Volume#prefixChild(Path)
+   */
+  @Override
+  public Path prefixChild(Path p) {
+    return new Path(basePath, p);
+  }
+
+  /* (non-javadoc)
+   * @see org.apache.accumulo.core.volume.Volume#isValidPath(Path)
+   */
+  @Override
+  public boolean isValidPath(Path p) {
+    checkNotNull(p);
+    
+    return p.toUri().getPath().startsWith(basePath);
+  }
   
   @Override
   public boolean equals(Object o) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ce32e3ed/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 ac1ca7e..6ea3b91 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
@@ -51,20 +51,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[] baseUris = null;
 
   private static List<Pair<Path,Path>> replacementsList = null;
 
   // 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;
@@ -112,29 +112,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() {
     // all base dirs should have the same instance id, so can choose any one
-    return new Path(getBaseDirs()[0], INSTANCE_ID_DIR);
+    return new Path(getBaseUris()[0], INSTANCE_ID_DIR);
   }
 
   public static Path getDataVersionLocation() {
     // all base dirs should have the same version, so can choose any one
-    return new Path(getBaseDirs()[0], VERSION_DIR);
+    return new Path(getBaseUris()[0], VERSION_DIR);
   }
 
   public static String[] getMetadataTableDirs() {
@@ -142,7 +142,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() {
@@ -185,7 +185,7 @@ public class ServerConstants {
       }
 
       HashSet<Path> baseDirs = new HashSet<Path>();
-      for (String baseDir : getBaseDirs()) {
+      for (String baseDir : getBaseUris()) {
         // normalize using path and remove accumulo dir
         baseDirs.add(new Path(baseDir).getParent());
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ce32e3ed/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 04632b9..ce37058 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
@@ -19,12 +19,12 @@ package org.apache.accumulo.server.fs;
 import java.io.IOException;
 
 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;
 
 /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ce32e3ed/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 e1f6aa3..9b62ce3 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;
@@ -37,7 +40,6 @@ import org.apache.accumulo.core.data.KeyExtent;
 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.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.commons.lang.NotImplementedException;
@@ -55,14 +57,15 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
 
-import com.google.common.collect.Maps;
+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,Volume> volumesByName;
-  Map<FileSystem,Volume> volumesByFileSystem;
+  Multimap<FileSystem,Volume> volumesByFileSystem;
   Volume defaultVolume;
   AccumuloConfiguration conf;
   VolumeChooser chooser;
@@ -70,14 +73,15 @@ public class VolumeManagerImpl implements VolumeManager {
   protected VolumeManagerImpl(Map<String,Volume> volumes, String defaultVolume, AccumuloConfiguration conf) {
     this.volumesByName = volumes;
     this.defaultVolume = volumes.get(defaultVolume);
-    this.volumesByFileSystem = Maps.newHashMapWithExpectedSize(volumesByName.size());
+    // 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());
   }
 
-  private void invertVolumesByFileSystem(Map<String,Volume> forward, Map<FileSystem,Volume> inverted) {
+  private void invertVolumesByFileSystem(Map<String,Volume> forward, Multimap<FileSystem,Volume> inverted) {
     for (Volume volume : forward.values()) {
       inverted.put(volume.getFileSystem(), volume);
     }
@@ -107,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) {
@@ -135,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);
     }
@@ -144,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 {
@@ -188,12 +205,12 @@ 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() {
@@ -269,19 +286,31 @@ 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 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;
+            }
+          }
+          
+          throw new RuntimeException("Could not determine valid Volume for Path '" + path + "' from Volumes " + candidateVolumes);
+        }
+        
+        log.info("Could not determine Volume for Path: " + path);
       } catch (IOException ex) {
         throw new RuntimeException(ex);
       }
@@ -296,23 +325,25 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @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);
     }
@@ -321,14 +352,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);
@@ -350,7 +382,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 {
@@ -366,7 +398,7 @@ public class VolumeManagerImpl implements VolumeManager {
 
     // The "default" Volume for Accumulo (in case no volumes are specified)
     volumes.put(DEFAULT, VolumeConfiguration.getDefaultVolume(hadoopConf, conf));
-    for (String volumeUriOrDir : VolumeConfiguration.getConfiguredBaseDirs(conf)) {
+    for (String volumeUriOrDir : VolumeConfiguration.getVolumeUris(conf)) {
       if (volumeUriOrDir.equals(DEFAULT))
       // Cannot re-define the default volume
         throw new IllegalArgumentException();
@@ -440,7 +472,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
@@ -495,18 +527,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

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ce32e3ed/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 3dc3665..59fad73 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
@@ -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());
@@ -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/ce32e3ed/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/ce32e3ed/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) {}
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ce32e3ed/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/DefaultServlet.java
----------------------------------------------------------------------
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/DefaultServlet.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/DefaultServlet.java
index 369949f..0da06cb 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/DefaultServlet.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/DefaultServlet.java
@@ -268,7 +268,7 @@ public class DefaultServlet extends BasicServlet {
       long totalHdfsBytesUsed = 0l;
       
       try {
-        for (String baseDir : VolumeConfiguration.getConfiguredBaseDirs(ServerConfiguration.getSiteConfiguration())) {
+        for (String baseDir : VolumeConfiguration.getVolumeUris(ServerConfiguration.getSiteConfiguration())) {
           final Path basePath = new Path(baseDir);
           final FileSystem fs = vm.getFileSystemByPath(basePath);