You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2013/06/18 19:45:02 UTC

svn commit: r1494226 [2/2] - in /accumulo/branches/ACCUMULO-118: core/src/main/java/org/apache/accumulo/core/conf/ server/src/main/java/org/apache/accumulo/server/ server/src/main/java/org/apache/accumulo/server/fs/ server/src/main/java/org/apache/accu...

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/FileUtil.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/FileUtil.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/FileUtil.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/FileUtil.java Tue Jun 18 17:45:01 2013
@@ -45,8 +45,9 @@ import org.apache.accumulo.core.iterator
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.FileSystem;
+import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
@@ -74,7 +75,7 @@ public class FileUtil {
   
   private static final Logger log = Logger.getLogger(FileUtil.class);
   
-  private static String createTmpDir(AccumuloConfiguration acuConf, FileSystem fs) throws IOException {
+  private static String createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs) throws IOException {
     String accumuloDir = acuConf.get(Property.INSTANCE_DFS_DIR);
     
     String tmpDir = null;
@@ -99,7 +100,7 @@ public class FileUtil {
     return tmpDir;
   }
   
-  public static Collection<FileRef> reduceFiles(AccumuloConfiguration acuConf, Configuration conf, FileSystem fs, Text prevEndRow, Text endRow,
+  public static Collection<FileRef> reduceFiles(AccumuloConfiguration acuConf, Configuration conf, VolumeManager fs, Text prevEndRow, Text endRow,
       Collection<FileRef> mapFiles, int maxFiles, String tmpDir, int pass) throws IOException {
     ArrayList<FileRef> paths = new ArrayList<FileRef>(mapFiles);
     
@@ -123,7 +124,7 @@ public class FileUtil {
       String newMapFile = String.format("%s/%04d." + RFile.EXTENSION, newDir, count++);
       
       outFiles.add(new FileRef(newMapFile));
-      org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(newMapFile);
+      FileSystem ns = fs.getFileSystemByPath(newMapFile);
       FileSKVWriter writer = new RFileOperations().openWriter(newMapFile, ns, ns.getConf(), acuConf);
       writer.startDefaultLocalityGroup();
       List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<SortedKeyValueIterator<Key,Value>>(inFiles.size());
@@ -181,12 +182,12 @@ public class FileUtil {
     return reduceFiles(acuConf, conf, fs, prevEndRow, endRow, outFiles, maxFiles, tmpDir, pass + 1);
   }
 
-  public static SortedMap<Double,Key> findMidPoint(FileSystem fs, AccumuloConfiguration acuConf, Text prevEndRow, Text endRow, Collection<FileRef> mapFiles,
+  public static SortedMap<Double,Key> findMidPoint(VolumeManager fs, AccumuloConfiguration acuConf, Text prevEndRow, Text endRow, Collection<FileRef> mapFiles,
       double minSplit) throws IOException {
     return findMidPoint(fs, acuConf, prevEndRow, endRow, mapFiles, minSplit, true);
   }
   
-  public static double estimatePercentageLTE(FileSystem fs, AccumuloConfiguration acuconf, Text prevEndRow, Text endRow, Collection<FileRef> mapFiles,
+  public static double estimatePercentageLTE(VolumeManager fs, AccumuloConfiguration acuconf, Text prevEndRow, Text endRow, Collection<FileRef> mapFiles,
       Text splitRow) throws IOException {
     
     Configuration conf = CachedConfiguration.getInstance();
@@ -258,7 +259,7 @@ public class FileUtil {
    *          ISSUES : This method used the index files to find the mid point. If the map files have different index intervals this method will not return an
    *          accurate mid point. Also, it would be tricky to use this method in conjunction with an in memory map because the indexing interval is unknown.
    */
-  public static SortedMap<Double,Key> findMidPoint(FileSystem fs, AccumuloConfiguration acuConf, Text prevEndRow, Text endRow, Collection<FileRef> mapFiles,
+  public static SortedMap<Double,Key> findMidPoint(VolumeManager fs, AccumuloConfiguration acuConf, Text prevEndRow, Text endRow, Collection<FileRef> mapFiles,
       double minSplit, boolean useIndex) throws IOException {
     Configuration conf = CachedConfiguration.getInstance();
     
@@ -358,7 +359,7 @@ public class FileUtil {
     }
   }
   
-  private static void cleanupIndexOp(AccumuloConfiguration acuConf, String tmpDir, FileSystem fs, ArrayList<FileSKVIterator> readers) throws IOException {
+  private static void cleanupIndexOp(AccumuloConfiguration acuConf, String tmpDir, VolumeManager fs, ArrayList<FileSKVIterator> readers) throws IOException {
     // close all of the index sequence files
     for (FileSKVIterator r : readers) {
       try {
@@ -380,7 +381,7 @@ public class FileUtil {
   }
   
   private static long countIndexEntries(AccumuloConfiguration acuConf, Text prevEndRow, Text endRow, Collection<FileRef> mapFiles, boolean useIndex,
-      Configuration conf, FileSystem fs, ArrayList<FileSKVIterator> readers) throws IOException {
+      Configuration conf, VolumeManager fs, ArrayList<FileSKVIterator> readers) throws IOException {
     
     long numKeys = 0;
     
@@ -388,7 +389,7 @@ public class FileUtil {
     for (FileRef ref : mapFiles) {
       FileSKVIterator reader = null;
       Path path = ref.path();
-      org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(path);
+      FileSystem ns = fs.getFileSystemByPath(path);
       try {
         if (useIndex)
           reader = FileOperations.getInstance().openIndex(path.toString(), ns, ns.getConf(), acuConf);
@@ -424,7 +425,7 @@ public class FileUtil {
     return numKeys;
   }
   
-  public static Map<FileRef,FileInfo> tryToGetFirstAndLastRows(FileSystem fs, AccumuloConfiguration acuConf, Set<FileRef> mapfiles) {
+  public static Map<FileRef,FileInfo> tryToGetFirstAndLastRows(VolumeManager fs, AccumuloConfiguration acuConf, Set<FileRef> mapfiles) {
     
     HashMap<FileRef,FileInfo> mapFilesInfo = new HashMap<FileRef,FileInfo>();
     
@@ -434,7 +435,7 @@ public class FileUtil {
       
       FileSKVIterator reader = null;
       String path = mapfile.path().toString();
-      org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(path);
+      FileSystem ns = fs.getFileSystemByPath(path);
       try {
         reader = FileOperations.getInstance().openReader(path, false, ns, ns.getConf(), acuConf);
         
@@ -464,12 +465,12 @@ public class FileUtil {
     return mapFilesInfo;
   }
   
-  public static WritableComparable<Key> findLastKey(FileSystem fs, AccumuloConfiguration acuConf, Collection<FileRef> mapFiles) throws IOException {
+  public static WritableComparable<Key> findLastKey(VolumeManager fs, AccumuloConfiguration acuConf, Collection<FileRef> mapFiles) throws IOException {
     Key lastKey = null;
     
     for (FileRef ref : mapFiles) {
       Path path = ref.path();
-      org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(path);
+      FileSystem ns = fs.getFileSystemByPath(path);
       FileSKVIterator reader = FileOperations.getInstance().openReader(path.toString(), true, ns, ns.getConf(), acuConf);
       
       try {
@@ -504,7 +505,7 @@ public class FileUtil {
   }
   
   public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path mapFile, long fileSize, List<KeyExtent> extents, Configuration conf,
-      FileSystem fs) throws IOException {
+      VolumeManager fs) throws IOException {
     
     long totalIndexEntries = 0;
     Map<KeyExtent,MLong> counts = new TreeMap<KeyExtent,MLong>();
@@ -512,7 +513,7 @@ public class FileUtil {
       counts.put(keyExtent, new MLong(0));
     
     Text row = new Text();
-    org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(mapFile);
+    FileSystem ns = fs.getFileSystemByPath(mapFile);
     FileSKVIterator index = FileOperations.getInstance().openIndex(mapFile.toString(), ns, ns.getConf(), acuConf);
     
     try {

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/Initialize.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/Initialize.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/Initialize.java Tue Jun 18 17:45:01 2013
@@ -53,8 +53,8 @@ import org.apache.accumulo.server.Server
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.constraints.MetadataConstraints;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.iterators.MetadataBulkLoadFilter;
 import org.apache.accumulo.server.master.state.tables.TableManager;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
@@ -63,6 +63,7 @@ import org.apache.accumulo.server.tablet
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -114,11 +115,11 @@ public class Initialize {
     initialMetadataConf.put(Property.TABLE_BLOCKCACHE_ENABLED.getKey(), "true");
   }
   
-  public static boolean doInit(Opts opts, Configuration conf, FileSystem fs) throws IOException {
+  public static boolean doInit(Opts opts, Configuration conf, VolumeManager fs) throws IOException {
     if (!ServerConfiguration.getSiteConfiguration().get(Property.INSTANCE_DFS_URI).equals(""))
       log.info("Hadoop Filesystem is " + ServerConfiguration.getSiteConfiguration().get(Property.INSTANCE_DFS_URI));
     else
-      log.info("Hadoop Filesystem is " + org.apache.hadoop.fs.FileSystem.getDefaultUri(conf));
+      log.info("Hadoop Filesystem is " + FileSystem.getDefaultUri(conf));
     
     log.info("Accumulo data dirs are " + Arrays.asList(ServerConstants.getBaseDirs()));
     log.info("Zookeeper server is " + ServerConfiguration.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST));
@@ -162,7 +163,7 @@ public class Initialize {
     return initialize(opts, instanceNamePath, fs);
   }
   
-  public static boolean initialize(Opts opts, String instanceNamePath, FileSystem fs) {
+  public static boolean initialize(Opts opts, String instanceNamePath, VolumeManager fs) {
     
     UUID uuid = UUID.randomUUID();
     try {
@@ -220,7 +221,7 @@ public class Initialize {
     return result.toArray(a);
   }
   
-  private static void initFileSystem(Opts opts, FileSystem fs, UUID uuid) throws IOException {
+  private static void initFileSystem(Opts opts, VolumeManager fs, UUID uuid) throws IOException {
     FileStatus fstat;
     
     // the actual disk location of the root tablet
@@ -275,7 +276,7 @@ public class Initialize {
     // metadata tablets
     String initRootTabFile = rootTablet + "/00000_00000."
         + FileOperations.getNewFileExtension(AccumuloConfiguration.getDefaultConfiguration());
-    org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(new Path(initRootTabFile));
+    FileSystem ns = fs.getFileSystemByPath(new Path(initRootTabFile));
     FileSKVWriter mfw = FileOperations.getInstance().openWriter(initRootTabFile, ns, ns.getConf(), AccumuloConfiguration.getDefaultConfiguration());
     mfw.startDefaultLocalityGroup();
     
@@ -477,7 +478,7 @@ public class Initialize {
     initialMetadataConf.put(Property.TABLE_FILE_REPLICATION.getKey(), rep);
   }
   
-  public static boolean isInitialized(FileSystem fs) throws IOException {
+  public static boolean isInitialized(VolumeManager fs) throws IOException {
     return (fs.exists(ServerConstants.getInstanceIdLocation()) || fs.exists(ServerConstants.getDataVersionLocation()));
   }
   
@@ -505,7 +506,7 @@ public class Initialize {
       Configuration conf = CachedConfiguration.getInstance();
       
       @SuppressWarnings("deprecation")
-      FileSystem fs = FileSystemImpl.get(SiteConfiguration.getSiteConfiguration());
+      VolumeManager fs = VolumeManagerImpl.get(SiteConfiguration.getSiteConfiguration());
       
       if (opts.resetSecurity) {
         if (isInitialized(fs)) {

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java Tue Jun 18 17:45:01 2013
@@ -29,10 +29,11 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.server.cli.ClientOpts;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 public class LocalityCheck {
@@ -41,7 +42,7 @@ public class LocalityCheck {
     ClientOpts opts = new ClientOpts();
     opts.parseArgs(LocalityCheck.class.getName(), args);
     
-    FileSystem fs = FileSystemImpl.get();
+    VolumeManager fs = VolumeManagerImpl.get();
     Connector connector = opts.getConnector();
     Scanner scanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY);
     scanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
@@ -72,7 +73,7 @@ public class LocalityCheck {
     return 0;
   }
   
-  private void addBlocks(FileSystem fs, String host, ArrayList<String> files, Map<String,Long> totalBlocks, Map<String,Long> localBlocks) throws Exception {
+  private void addBlocks(VolumeManager fs, String host, ArrayList<String> files, Map<String,Long> totalBlocks, Map<String,Long> localBlocks) throws Exception {
     long allBlocks = 0;
     long matchingBlocks = 0;
     if (!totalBlocks.containsKey(host)) {
@@ -81,7 +82,7 @@ public class LocalityCheck {
     }
     for (String file : files) {
       Path filePath = new Path(file);
-      org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(filePath);
+      FileSystem ns = fs.getFileSystemByPath(filePath);
       FileStatus fileStatus = ns.getFileStatus(filePath);
       BlockLocation[] fileBlockLocations = ns.getFileBlockLocations(fileStatus, 0, fileStatus.getLen());
       for (BlockLocation blockLocation : fileBlockLocations) {

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java Tue Jun 18 17:45:01 2013
@@ -70,8 +70,8 @@ import org.apache.accumulo.fate.zookeepe
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.accumulo.server.zookeeper.ZooLock;
@@ -341,7 +341,7 @@ public class MetadataTable extends org.a
     Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
     mdScanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
     Text row = extent.getMetadataEntry();
-    FileSystem fs = FileSystemImpl.get();
+    VolumeManager fs = VolumeManagerImpl.get();
     
     Key endKey = new Key(row, Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(""));
     endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
@@ -491,7 +491,7 @@ public class MetadataTable extends org.a
         pathToRemove = "/" + tableId + "/" + pathToRemove;
     }
     
-    Path path = FileSystemImpl.get().getFullPath(ServerConstants.getTablesDirs(), pathToRemove);
+    Path path = VolumeManagerImpl.get().getFullPath(ServerConstants.getTablesDirs(), pathToRemove);
     Mutation delFlag = new Mutation(new Text(prefix + path.toString()));
     delFlag.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
     return delFlag;
@@ -519,7 +519,7 @@ public class MetadataTable extends org.a
     ScannerImpl scanner2 = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
     scanner2.setRange(new Range(prevRowKey, prevRowKey.followingKey(PartialKey.ROW)));
     
-    FileSystem fs = FileSystemImpl.get();
+    VolumeManager fs = VolumeManagerImpl.get();
     if (!scanner2.iterator().hasNext()) {
       log.info("Rolling back incomplete split " + metadataEntry + " " + metadataPrevEndRow);
       rollBackSplit(metadataEntry, KeyExtent.decodePrevEndRow(oper), credentials, lock);
@@ -658,7 +658,7 @@ public class MetadataTable extends org.a
         Key key = cell.getKey();
         
         if (key.getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
-          FileRef ref = new FileRef(FileSystemImpl.get(), key);
+          FileRef ref = new FileRef(VolumeManagerImpl.get(), key);
           bw.addMutation(createDeleteMutation(tableId, ref.meta().toString()));
         }
         
@@ -796,11 +796,11 @@ public class MetadataTable extends org.a
     ArrayList<LogEntry> result = new ArrayList<LogEntry>();
     TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
     
-    FileSystem fs = FileSystemImpl.get();
+    VolumeManager fs = VolumeManagerImpl.get();
     if (extent.isRootTablet()) {
       getRootLogEntries(result);
       Path rootDir = new Path(ServerConstants.getRootTabletDir());
-      rootDir = rootDir.makeQualified(fs.getDefaultNamespace());
+      rootDir = rootDir.makeQualified(fs.getDefaultVolume());
       FileStatus[] files = fs.listStatus(rootDir);
       for (FileStatus fileStatus : files) {
         if (fileStatus.getPath().toString().endsWith("_tmp")) {
@@ -1191,7 +1191,7 @@ public class MetadataTable extends org.a
   public static List<FileRef> getBulkFilesLoaded(Connector conn, KeyExtent extent, long tid) throws IOException {
     List<FileRef> result = new ArrayList<FileRef>();
     try {
-      FileSystem fs = FileSystemImpl.get();
+      VolumeManager fs = VolumeManagerImpl.get();
       Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Authorizations.EMPTY));
       mscanner.setRange(extent.toMetadataRange());
       mscanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
@@ -1215,7 +1215,7 @@ public class MetadataTable extends org.a
     
     Map<FileRef,Long> ret = new HashMap<FileRef,Long>();
     
-    FileSystem fs = FileSystemImpl.get();
+    VolumeManager fs = VolumeManagerImpl.get();
     Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Authorizations.EMPTY);
     scanner.setRange(new Range(metadataRow));
     scanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java Tue Jun 18 17:45:01 2013
@@ -51,23 +51,24 @@ import org.apache.accumulo.core.util.Tex
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.util.MetadataTable.LogEntry;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 public class OfflineMetadataScanner extends ScannerOptions implements Scanner {
   
   private Set<String> allFiles = new HashSet<String>();
   private Range range = new Range();
-  private final FileSystem fs;
+  private final VolumeManager fs;
   private final AccumuloConfiguration conf;
   
-  private List<SortedKeyValueIterator<Key,Value>> openMapFiles(Collection<String> files, FileSystem fs, AccumuloConfiguration conf) throws IOException {
+  private List<SortedKeyValueIterator<Key,Value>> openMapFiles(Collection<String> files, VolumeManager fs, AccumuloConfiguration conf) throws IOException {
     List<SortedKeyValueIterator<Key,Value>> readers = new ArrayList<SortedKeyValueIterator<Key,Value>>();
     for (String file : files) {
-      org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(file);
+      FileSystem ns = fs.getFileSystemByPath(file);
       FileSKVIterator reader = FileOperations.getInstance().openReader(file, true, ns, ns.getConf(), conf);
       readers.add(reader);
     }
@@ -119,7 +120,7 @@ public class OfflineMetadataScanner exte
     
   }
   
-  public OfflineMetadataScanner(AccumuloConfiguration conf, FileSystem fs) throws IOException {
+  public OfflineMetadataScanner(AccumuloConfiguration conf, VolumeManager fs) throws IOException {
     super();
     this.fs = fs;
     this.conf = conf;
@@ -258,7 +259,7 @@ public class OfflineMetadataScanner exte
   
   public static void main(String[] args) throws IOException {
     ServerConfiguration conf = new ServerConfiguration(HdfsZooInstance.getInstance());
-    FileSystem fs = FileSystemImpl.get();
+    VolumeManager fs = VolumeManagerImpl.get();
     OfflineMetadataScanner scanner = new OfflineMetadataScanner(conf.getConfiguration(), fs);
     scanner.setRange(Constants.METADATA_KEYSPACE);
     for (Entry<Key,Value> entry : scanner)

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java Tue Jun 18 17:45:01 2013
@@ -28,8 +28,8 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.server.cli.ClientOpts;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
@@ -52,7 +52,7 @@ public class RemoveEntriesForMissingFile
     ScannerOpts scanOpts = new ScannerOpts();
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.parseArgs(RemoveEntriesForMissingFiles.class.getName(), args, scanOpts, bwOpts);
-    FileSystem fs = FileSystemImpl.get();
+    VolumeManager fs = VolumeManagerImpl.get();
     Connector connector = opts.getConnector();
     Scanner metadata = connector.createScanner(Constants.METADATA_TABLE_NAME, opts.auths);
     metadata.setBatchSize(scanOpts.scanBatchSize);

Modified: accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/TabletOperations.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/TabletOperations.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/TabletOperations.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/main/java/org/apache/accumulo/server/util/TabletOperations.java Tue Jun 18 17:45:01 2013
@@ -22,8 +22,8 @@ import java.util.Random;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.ServerConstants;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.tabletserver.UniqueNameAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -36,7 +36,7 @@ public class TabletOperations {
   private static final Random random = new Random();
   
   // TODO ACCUMULO-118 make the namespace selection pluggable
-  public static String createTabletDirectory(FileSystem fs, String tableId, Text endRow) {
+  public static String createTabletDirectory(VolumeManager fs, String tableId, Text endRow) {
     String lowDirectory;
     
     UniqueNameAllocator namer = UniqueNameAllocator.getInstance();
@@ -72,7 +72,7 @@ public class TabletOperations {
   public static String createTabletDirectory(String tableDir, Text endRow) {
     while (true) {
       try {
-        FileSystem fs = FileSystemImpl.get();
+        VolumeManager fs = VolumeManagerImpl.get();
         return createTabletDirectory(fs, tableDir, endRow);
       } catch (IOException e) {
         log.warn(e);

Modified: accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/gc/TestConfirmDeletes.java Tue Jun 18 17:45:01 2013
@@ -35,8 +35,8 @@ import org.apache.accumulo.core.data.Val
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.TCredentials;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.hadoop.io.Text;
 import org.junit.Assert;
 import org.junit.Test;
@@ -94,7 +94,7 @@ public class TestConfirmDeletes {
     TCredentials auth = CredentialHelper.create("root", new PasswordToken(new byte[0]), "instance");
     
     Instance instance = new MockInstance();
-    FileSystem fs = FileSystemImpl.getLocal();
+    VolumeManager fs = VolumeManagerImpl.getLocal();
     
     load(instance, metadata, deletes);
     

Modified: accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/tabletserver/log/MultiReaderTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/tabletserver/log/MultiReaderTest.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/tabletserver/log/MultiReaderTest.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/tabletserver/log/MultiReaderTest.java Tue Jun 18 17:45:01 2013
@@ -22,8 +22,9 @@ import static org.junit.Assert.assertTru
 
 import java.io.IOException;
 
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
@@ -37,20 +38,20 @@ import org.junit.rules.TemporaryFolder;
 
 public class MultiReaderTest {
   
-  FileSystem fs;
+  VolumeManager fs;
   TemporaryFolder root = new TemporaryFolder();
   
   @Before
   public void setUp() throws Exception {
     // quiet log messages about compress.CodecPool
     Logger.getRootLogger().setLevel(Level.ERROR);
-    fs = FileSystemImpl.getLocal();
+    fs = VolumeManagerImpl.getLocal();
     root.create();
     String path = root.getRoot().getAbsolutePath();
     Path root = new Path("file://" + path + "/manyMaps");
     fs.mkdirs(root);
     fs.create(new Path(root, "finished")).close();
-    org.apache.hadoop.fs.FileSystem ns = fs.getDefaultNamespace();
+    FileSystem ns = fs.getDefaultVolume();
     Writer writer = new Writer(ns.getConf(), ns, new Path(root, "odd").toString(), IntWritable.class, BytesWritable.class);
     BytesWritable value = new BytesWritable("someValue".getBytes());
     for (int i = 1; i < 1000; i += 2) {

Modified: accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/tabletserver/log/SortedLogRecoveryTest.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/tabletserver/log/SortedLogRecoveryTest.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/tabletserver/log/SortedLogRecoveryTest.java (original)
+++ accumulo/branches/ACCUMULO-118/server/src/test/java/org/apache/accumulo/server/tabletserver/log/SortedLogRecoveryTest.java Tue Jun 18 17:45:01 2013
@@ -37,11 +37,12 @@ import org.apache.accumulo.core.data.Key
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.server.data.ServerMutation;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.logger.LogEvents;
 import org.apache.accumulo.server.logger.LogFileKey;
 import org.apache.accumulo.server.logger.LogFileValue;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.MapFile.Writer;
@@ -116,13 +117,13 @@ public class SortedLogRecoveryTest {
     TemporaryFolder root = new TemporaryFolder();
     root.create();
     final String workdir = "file://" + root.getRoot().getAbsolutePath() + "/workdir";
-    FileSystem fs = FileSystemImpl.getLocal();
+    VolumeManager fs = VolumeManagerImpl.getLocal();
     fs.deleteRecursively(new Path(workdir));
     ArrayList<Path> dirs = new ArrayList<Path>();
     try {
       for (Entry<String,KeyValue[]> entry : logs.entrySet()) {
         String path = workdir + "/" + entry.getKey();
-        org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(path);
+        FileSystem ns = fs.getFileSystemByPath(path);
         Writer map = new MapFile.Writer(ns.getConf(), ns, path + "/log1", LogFileKey.class, LogFileValue.class);
         for (KeyValue lfe : entry.getValue()) {
           map.append(lfe.key, lfe.value);

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousStatsCollector.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousStatsCollector.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousStatsCollector.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousStatsCollector.java Tue Jun 18 17:45:01 2013
@@ -41,13 +41,14 @@ import org.apache.accumulo.core.util.Cac
 import org.apache.accumulo.core.util.Stat;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.cli.ClientOnRequiredTable;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.monitor.Monitor;
 import org.apache.accumulo.server.security.SecurityConstants;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.JobClient;
@@ -110,15 +111,15 @@ public class ContinuousStatsCollector {
     }
     
     private String getFSStats() throws Exception {
-      FileSystem fs = FileSystemImpl.get();
+      VolumeManager fs = VolumeManagerImpl.get();
       long length1 = 0, dcount1 = 0, fcount1 = 0;
       long length2 = 0, dcount2 = 0, fcount2 = 0;
       for (String dir : ServerConstants.getTablesDirs()) {
-        ContentSummary contentSummary = fs.getContentSummary(dir);
+        ContentSummary contentSummary = fs.getContentSummary(new Path(dir));
         length1 += contentSummary.getLength();
         dcount1 += contentSummary.getDirectoryCount();
         fcount1 += contentSummary.getFileCount();
-        contentSummary = fs.getContentSummary(dir + "/" + tableId);
+        contentSummary = fs.getContentSummary(new Path(dir, tableId));
         length2 += contentSummary.getLength();
         dcount2 += contentSummary.getDirectoryCount();
         fcount2 += contentSummary.getFileCount();

Modified: accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java
URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java?rev=1494226&r1=1494225&r2=1494226&view=diff
==============================================================================
--- accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java (original)
+++ accumulo/branches/ACCUMULO-118/test/src/main/java/org/apache/accumulo/test/performance/scan/CollectTabletStats.java Tue Jun 18 17:45:01 2013
@@ -68,11 +68,12 @@ import org.apache.accumulo.server.cli.Cl
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.FileSystem;
-import org.apache.accumulo.server.fs.FileSystemImpl;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.util.MetadataTable;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
@@ -104,7 +105,7 @@ public class CollectTabletStats {
       columnsTmp = opts.columns.split(",");
     final String columns[] = columnsTmp;
     
-    final FileSystem fs = FileSystemImpl.get();
+    final VolumeManager fs = VolumeManagerImpl.get();
 
     Instance instance = opts.getInstance();
     final ServerConfiguration sconf = new ServerConfiguration(instance);
@@ -376,7 +377,7 @@ public class CollectTabletStats {
   }
   
   private static void reportHdfsBlockLocations(List<FileRef> files) throws Exception {
-    FileSystem fs = FileSystemImpl.get();
+    VolumeManager fs = VolumeManagerImpl.get();
     
     System.out.println("\t\tFile block report : ");
     for (FileRef file : files) {
@@ -386,7 +387,7 @@ public class CollectTabletStats {
         // assume it is a map file
         status = fs.getFileStatus(new Path(file + "/data"));
       }
-      org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(file.path());
+      FileSystem ns = fs.getFileSystemByPath(file.path());
       BlockLocation[] locs = ns.getFileBlockLocations(status, 0, status.getLen());
       
       System.out.println("\t\t\tBlocks for : " + file);
@@ -426,14 +427,14 @@ public class CollectTabletStats {
     return visFilter;
   }
   
-  private static int readFiles(FileSystem fs, AccumuloConfiguration aconf, List<FileRef> files, KeyExtent ke, String[] columns) throws Exception {
+  private static int readFiles(VolumeManager fs, AccumuloConfiguration aconf, List<FileRef> files, KeyExtent ke, String[] columns) throws Exception {
     
     int count = 0;
     
     HashSet<ByteSequence> columnSet = createColumnBSS(columns);
     
     for (FileRef file : files) {
-      org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(file.path());
+      FileSystem ns = fs.getFileSystemByPath(file.path());
       FileSKVIterator reader = FileOperations.getInstance().openReader(file.path().toString(), false, ns, ns.getConf(), aconf);
       Range range = new Range(ke.getPrevEndRow(), false, ke.getEndRow(), true);
       reader.seek(range, columnSet, columnSet.size() == 0 ? false : true);
@@ -455,7 +456,7 @@ public class CollectTabletStats {
     return columnSet;
   }
   
-  private static int readFilesUsingIterStack(FileSystem fs, ServerConfiguration aconf, List<FileRef> files, Authorizations auths, KeyExtent ke, String[] columns,
+  private static int readFilesUsingIterStack(VolumeManager fs, ServerConfiguration aconf, List<FileRef> files, Authorizations auths, KeyExtent ke, String[] columns,
       boolean useTableIterators)
       throws Exception {
     
@@ -464,7 +465,7 @@ public class CollectTabletStats {
     List<SortedKeyValueIterator<Key,Value>> readers = new ArrayList<SortedKeyValueIterator<Key,Value>>(files.size());
     
     for (FileRef file : files) {
-      org.apache.hadoop.fs.FileSystem ns = fs.getFileSystemByPath(file.path());
+      FileSystem ns = fs.getFileSystemByPath(file.path());
       readers.add(FileOperations.getInstance().openReader(file.path().toString(), false, ns, ns.getConf(), aconf.getConfiguration()));
     }