You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2016/09/22 16:34:27 UTC

[39/50] [abbrv] hbase git commit: HBASE-14439 Move fs stuff out pt2 - region fs

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 85a8080..80fbb76 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -32,6 +32,7 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -72,6 +73,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
+import org.apache.hadoop.hbase.fs.RegionFileSystem;
+import org.apache.hadoop.hbase.fs.RegionFileSystem.StoreFileVisitor;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
@@ -275,21 +279,6 @@ public abstract class FSUtils {
   }
 
   /**
-   * Delete the region directory if exists.
-   * @param conf
-   * @param hri
-   * @return True if deleted the region directory.
-   * @throws IOException
-   */
-  public static boolean deleteRegionDir(final Configuration conf, final HRegionInfo hri)
-  throws IOException {
-    Path rootDir = getRootDir(conf);
-    FileSystem fs = rootDir.getFileSystem(conf);
-    return deleteDirectory(fs,
-      new Path(getTableDir(rootDir, hri.getTable()), hri.getEncodedName()));
-  }
-
-  /**
    * Return the number of bytes that large input files should be optimally
    * be split into to minimize i/o time.
    *
@@ -1078,7 +1067,7 @@ public abstract class FSUtils {
    * @throws IOException When scanning the directory fails.
    */
   public static int getTotalTableFragmentation(final HMaster master)
-  throws IOException {
+      throws IOException {
     Map<String, Integer> map = getTableFragmentation(master);
     return map != null && map.size() > 0 ? map.get("-TOTAL-") : -1;
   }
@@ -1093,56 +1082,29 @@ public abstract class FSUtils {
    *
    * @throws IOException When scanning the directory fails.
    */
-  public static Map<String, Integer> getTableFragmentation(
-    final HMaster master)
-  throws IOException {
-    Path path = getRootDir(master.getConfiguration());
-    // since HMaster.getFileSystem() is package private
-    FileSystem fs = path.getFileSystem(master.getConfiguration());
-    return getTableFragmentation(fs, path);
-  }
-
-  /**
-   * Runs through the HBase rootdir and checks how many stores for each table
-   * have more than one file in them. Checks -ROOT- and hbase:meta too. The total
-   * percentage across all tables is stored under the special key "-TOTAL-".
-   *
-   * @param fs  The file system to use.
-   * @param hbaseRootDir  The root directory to scan.
-   * @return A map for each table and its percentage.
-   * @throws IOException When scanning the directory fails.
-   */
-  public static Map<String, Integer> getTableFragmentation(
-    final FileSystem fs, final Path hbaseRootDir)
-  throws IOException {
-    Map<String, Integer> frags = new HashMap<String, Integer>();
+  public static Map<String, Integer> getTableFragmentation(final HMaster master)
+      throws IOException {
+    final Map<String, Integer> frags = new HashMap<String, Integer>();
     int cfCountTotal = 0;
     int cfFragTotal = 0;
-    PathFilter regionFilter = new RegionDirFilter(fs);
-    PathFilter familyFilter = new FamilyDirFilter(fs);
-    List<Path> tableDirs = getTableDirs(fs, hbaseRootDir);
-    for (Path d : tableDirs) {
+
+    MasterFileSystem mfs = master.getMasterFileSystem();
+    for (TableName table: mfs.getTables()) {
       int cfCount = 0;
       int cfFrag = 0;
-      FileStatus[] regionDirs = fs.listStatus(d, regionFilter);
-      for (FileStatus regionDir : regionDirs) {
-        Path dd = regionDir.getPath();
-        // else its a region name, now look in region for families
-        FileStatus[] familyDirs = fs.listStatus(dd, familyFilter);
-        for (FileStatus familyDir : familyDirs) {
+      for (HRegionInfo hri: mfs.getRegions(table)) {
+        RegionFileSystem rfs = mfs.getRegionFileSystem(hri);
+        for (String family: rfs.getFamilies()) {
           cfCount++;
           cfCountTotal++;
-          Path family = familyDir.getPath();
-          // now in family make sure only one file
-          FileStatus[] familyStatus = fs.listStatus(family);
-          if (familyStatus.length > 1) {
+          if (rfs.getStoreFiles(family).size() > 1) {
             cfFrag++;
             cfFragTotal++;
           }
         }
       }
       // compute percentage per table and store in result list
-      frags.put(FSUtils.getTableName(d).getNameAsString(),
+      frags.put(table.getNameAsString(),
         cfCount == 0? 0: Math.round((float) cfFrag / cfCount * 100));
     }
     // set overall percentage for all tables
@@ -1459,6 +1421,18 @@ public abstract class FSUtils {
     return referenceFiles;
   }
 
+  public static int getRegionReferenceFileCount(final FileSystem fs, final Path p) {
+    int result = 0;
+    try {
+      for (Path familyDir:getFamilyDirs(fs, p)){
+        result += getReferenceFilePaths(fs, familyDir).size();
+      }
+    } catch (IOException e) {
+      LOG.warn("Error Counting reference files.", e);
+    }
+    return result;
+  }
+
   /**
    * Filter for HFiles that excludes reference files.
    */
@@ -1521,232 +1495,6 @@ public abstract class FSUtils {
     return getRootDir(conf).getFileSystem(conf);
   }
 
-
-  /**
-   * Runs through the HBase rootdir/tablename and creates a reverse lookup map for
-   * table StoreFile names to the full Path.
-   * <br>
-   * Example...<br>
-   * Key = 3944417774205889744  <br>
-   * Value = hdfs://localhost:51169/user/userid/-ROOT-/70236052/info/3944417774205889744
-   *
-   * @param map map to add values.  If null, this method will create and populate one to return
-   * @param fs  The file system to use.
-   * @param hbaseRootDir  The root directory to scan.
-   * @param tableName name of the table to scan.
-   * @return Map keyed by StoreFile name with a value of the full Path.
-   * @throws IOException When scanning the directory fails.
-   * @throws InterruptedException
-   */
-  public static Map<String, Path> getTableStoreFilePathMap(Map<String, Path> map,
-  final FileSystem fs, final Path hbaseRootDir, TableName tableName)
-  throws IOException, InterruptedException {
-    return getTableStoreFilePathMap(map, fs, hbaseRootDir, tableName, null, null, null);
-  }
-
-  /**
-   * Runs through the HBase rootdir/tablename and creates a reverse lookup map for
-   * table StoreFile names to the full Path.  Note that because this method can be called
-   * on a 'live' HBase system that we will skip files that no longer exist by the time
-   * we traverse them and similarly the user of the result needs to consider that some
-   * entries in this map may not exist by the time this call completes.
-   * <br>
-   * Example...<br>
-   * Key = 3944417774205889744  <br>
-   * Value = hdfs://localhost:51169/user/userid/-ROOT-/70236052/info/3944417774205889744
-   *
-   * @param resultMap map to add values.  If null, this method will create and populate one to return
-   * @param fs  The file system to use.
-   * @param hbaseRootDir  The root directory to scan.
-   * @param tableName name of the table to scan.
-   * @param sfFilter optional path filter to apply to store files
-   * @param executor optional executor service to parallelize this operation
-   * @param errors ErrorReporter instance or null
-   * @return Map keyed by StoreFile name with a value of the full Path.
-   * @throws IOException When scanning the directory fails.
-   * @throws InterruptedException
-   */
-  public static Map<String, Path> getTableStoreFilePathMap(
-      Map<String, Path> resultMap,
-      final FileSystem fs, final Path hbaseRootDir, TableName tableName, final PathFilter sfFilter,
-      ExecutorService executor, final ErrorReporter errors) throws IOException, InterruptedException {
-
-    final Map<String, Path> finalResultMap =
-        resultMap == null ? new ConcurrentHashMap<String, Path>(128, 0.75f, 32) : resultMap;
-
-    // only include the directory paths to tables
-    Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
-    // Inside a table, there are compaction.dir directories to skip.  Otherwise, all else
-    // should be regions.
-    final FamilyDirFilter familyFilter = new FamilyDirFilter(fs);
-    final Vector<Exception> exceptions = new Vector<Exception>();
-
-    try {
-      List<FileStatus> regionDirs = FSUtils.listStatusWithStatusFilter(fs, tableDir, new RegionDirFilter(fs));
-      if (regionDirs == null) {
-        return finalResultMap;
-      }
-
-      final List<Future<?>> futures = new ArrayList<Future<?>>(regionDirs.size());
-
-      for (FileStatus regionDir : regionDirs) {
-        if (null != errors) {
-          errors.progress();
-        }
-        final Path dd = regionDir.getPath();
-
-        if (!exceptions.isEmpty()) {
-          break;
-        }
-
-        Runnable getRegionStoreFileMapCall = new Runnable() {
-          @Override
-          public void run() {
-            try {
-              HashMap<String,Path> regionStoreFileMap = new HashMap<String, Path>();
-              List<FileStatus> familyDirs = FSUtils.listStatusWithStatusFilter(fs, dd, familyFilter);
-              if (familyDirs == null) {
-                if (!fs.exists(dd)) {
-                  LOG.warn("Skipping region because it no longer exists: " + dd);
-                } else {
-                  LOG.warn("Skipping region because it has no family dirs: " + dd);
-                }
-                return;
-              }
-              for (FileStatus familyDir : familyDirs) {
-                if (null != errors) {
-                  errors.progress();
-                }
-                Path family = familyDir.getPath();
-                if (family.getName().equals(HConstants.RECOVERED_EDITS_DIR)) {
-                  continue;
-                }
-                // now in family, iterate over the StoreFiles and
-                // put in map
-                FileStatus[] familyStatus = fs.listStatus(family);
-                for (FileStatus sfStatus : familyStatus) {
-                  if (null != errors) {
-                    errors.progress();
-                  }
-                  Path sf = sfStatus.getPath();
-                  if (sfFilter == null || sfFilter.accept(sf)) {
-                    regionStoreFileMap.put( sf.getName(), sf);
-                  }
-                }
-              }
-              finalResultMap.putAll(regionStoreFileMap);
-            } catch (Exception e) {
-              LOG.error("Could not get region store file map for region: " + dd, e);
-              exceptions.add(e);
-            }
-          }
-        };
-
-        // If executor is available, submit async tasks to exec concurrently, otherwise
-        // just do serial sync execution
-        if (executor != null) {
-          Future<?> future = executor.submit(getRegionStoreFileMapCall);
-          futures.add(future);
-        } else {
-          FutureTask<?> future = new FutureTask<Object>(getRegionStoreFileMapCall, null);
-          future.run();
-          futures.add(future);
-        }
-      }
-
-      // Ensure all pending tasks are complete (or that we run into an exception)
-      for (Future<?> f : futures) {
-        if (!exceptions.isEmpty()) {
-          break;
-        }
-        try {
-          f.get();
-        } catch (ExecutionException e) {
-          LOG.error("Unexpected exec exception!  Should've been caught already.  (Bug?)", e);
-          // Shouldn't happen, we already logged/caught any exceptions in the Runnable
-        }
-      }
-    } catch (IOException e) {
-      LOG.error("Cannot execute getTableStoreFilePathMap for " + tableName, e);
-      exceptions.add(e);
-    } finally {
-      if (!exceptions.isEmpty()) {
-        // Just throw the first exception as an indication something bad happened
-        // Don't need to propagate all the exceptions, we already logged them all anyway
-        Throwables.propagateIfInstanceOf(exceptions.firstElement(), IOException.class);
-        throw Throwables.propagate(exceptions.firstElement());
-      }
-    }
-
-    return finalResultMap;
-  }
-
-  public static int getRegionReferenceFileCount(final FileSystem fs, final Path p) {
-    int result = 0;
-    try {
-      for (Path familyDir:getFamilyDirs(fs, p)){
-        result += getReferenceFilePaths(fs, familyDir).size();
-      }
-    } catch (IOException e) {
-      LOG.warn("Error Counting reference files.", e);
-    }
-    return result;
-  }
-
-  /**
-   * Runs through the HBase rootdir and creates a reverse lookup map for
-   * table StoreFile names to the full Path.
-   * <br>
-   * Example...<br>
-   * Key = 3944417774205889744  <br>
-   * Value = hdfs://localhost:51169/user/userid/-ROOT-/70236052/info/3944417774205889744
-   *
-   * @param fs  The file system to use.
-   * @param hbaseRootDir  The root directory to scan.
-   * @return Map keyed by StoreFile name with a value of the full Path.
-   * @throws IOException When scanning the directory fails.
-   * @throws InterruptedException
-   */
-  public static Map<String, Path> getTableStoreFilePathMap(
-    final FileSystem fs, final Path hbaseRootDir)
-  throws IOException, InterruptedException {
-    return getTableStoreFilePathMap(fs, hbaseRootDir, null, null, null);
-  }
-
-  /**
-   * Runs through the HBase rootdir and creates a reverse lookup map for
-   * table StoreFile names to the full Path.
-   * <br>
-   * Example...<br>
-   * Key = 3944417774205889744  <br>
-   * Value = hdfs://localhost:51169/user/userid/-ROOT-/70236052/info/3944417774205889744
-   *
-   * @param fs  The file system to use.
-   * @param hbaseRootDir  The root directory to scan.
-   * @param sfFilter optional path filter to apply to store files
-   * @param executor optional executor service to parallelize this operation
-   * @param errors ErrorReporter instance or null
-   * @return Map keyed by StoreFile name with a value of the full Path.
-   * @throws IOException When scanning the directory fails.
-   * @throws InterruptedException
-   */
-  public static Map<String, Path> getTableStoreFilePathMap(
-    final FileSystem fs, final Path hbaseRootDir, PathFilter sfFilter,
-    ExecutorService executor, ErrorReporter errors)
-  throws IOException, InterruptedException {
-    ConcurrentHashMap<String, Path> map = new ConcurrentHashMap<String, Path>(1024, 0.75f, 32);
-
-    // if this method looks similar to 'getTableFragmentation' that is because
-    // it was borrowed from it.
-
-    // only include the directory paths to tables
-    for (Path tableDir : FSUtils.getTableDirs(fs, hbaseRootDir)) {
-      getTableStoreFilePathMap(map, fs, hbaseRootDir,
-          FSUtils.getTableName(tableDir), sfFilter, executor, errors);
-    }
-    return map;
-  }
-
   /**
    * Filters FileStatuses in an array and returns a list
    *
@@ -1999,13 +1747,12 @@ public abstract class FSUtils {
    * @throws IOException
    *           in case of file system errors or interrupts
    */
-  public static Map<String, Map<String, Float>> getRegionDegreeLocalityMappingFromFS(
+  private static Map<String, Map<String, Float>> getRegionDegreeLocalityMappingFromFS(
       final Configuration conf, final String desiredTable, int threadPoolSize)
       throws IOException {
     Map<String, Map<String, Float>> regionDegreeLocalityMapping =
         new ConcurrentHashMap<String, Map<String, Float>>();
-    getRegionLocalityMappingFromFS(conf, desiredTable, threadPoolSize, null,
-        regionDegreeLocalityMapping);
+    getRegionLocalityMappingFromFS(conf, desiredTable, threadPoolSize, regionDegreeLocalityMapping);
     return regionDegreeLocalityMapping;
   }
 
@@ -2021,8 +1768,6 @@ public abstract class FSUtils {
    *          the table you wish to scan locality for
    * @param threadPoolSize
    *          the thread pool size to use
-   * @param regionToBestLocalityRSMapping
-   *          the map into which to put the best locality mapping or null
    * @param regionDegreeLocalityMapping
    *          the map into which to put the locality degree mapping or null,
    *          must be a thread-safe implementation
@@ -2032,81 +1777,36 @@ public abstract class FSUtils {
   private static void getRegionLocalityMappingFromFS(
       final Configuration conf, final String desiredTable,
       int threadPoolSize,
-      Map<String, String> regionToBestLocalityRSMapping,
       Map<String, Map<String, Float>> regionDegreeLocalityMapping)
       throws IOException {
-    FileSystem fs =  FileSystem.get(conf);
-    Path rootPath = FSUtils.getRootDir(conf);
     long startTime = EnvironmentEdgeManager.currentTime();
-    Path queryPath;
-    // The table files are in ${hbase.rootdir}/data/<namespace>/<table>/*
-    if (null == desiredTable) {
-      queryPath = new Path(new Path(rootPath, HConstants.BASE_NAMESPACE_DIR).toString() + "/*/*/*/");
-    } else {
-      queryPath = new Path(FSUtils.getTableDir(rootPath, TableName.valueOf(desiredTable)).toString() + "/*/");
-    }
-
-    // reject all paths that are not appropriate
-    PathFilter pathFilter = new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        // this is the region name; it may get some noise data
-        if (null == path) {
-          return false;
-        }
-
-        // no parent?
-        Path parent = path.getParent();
-        if (null == parent) {
-          return false;
-        }
-
-        String regionName = path.getName();
-        if (null == regionName) {
-          return false;
-        }
 
-        if (!regionName.toLowerCase(Locale.ROOT).matches("[0-9a-f]+")) {
-          return false;
-        }
-        return true;
+    MasterFileSystem mfs = MasterFileSystem.open(conf, false);
+    Collection<HRegionInfo> hris;
+    if (desiredTable != null) {
+      hris = mfs.getRegions(TableName.valueOf(desiredTable));
+    } else {
+      hris = new ArrayList<HRegionInfo>();
+      for (TableName tableName: mfs.getTables()) {
+        hris.addAll(mfs.getRegions(tableName));
       }
-    };
-
-    FileStatus[] statusList = fs.globStatus(queryPath, pathFilter);
+    }
 
-    if (null == statusList) {
+    if (hris.isEmpty()) {
       return;
-    } else {
-      LOG.debug("Query Path: " + queryPath + " ; # list of files: " +
-          statusList.length);
     }
 
     // lower the number of threads in case we have very few expected regions
-    threadPoolSize = Math.min(threadPoolSize, statusList.length);
+    threadPoolSize = Math.min(threadPoolSize, hris.size());
 
     // run in multiple threads
     ThreadPoolExecutor tpe = new ThreadPoolExecutor(threadPoolSize,
         threadPoolSize, 60, TimeUnit.SECONDS,
-        new ArrayBlockingQueue<Runnable>(statusList.length));
+        new ArrayBlockingQueue<Runnable>(hris.size()));
     try {
       // ignore all file status items that are not of interest
-      for (FileStatus regionStatus : statusList) {
-        if (null == regionStatus) {
-          continue;
-        }
-
-        if (!regionStatus.isDirectory()) {
-          continue;
-        }
-
-        Path regionPath = regionStatus.getPath();
-        if (null == regionPath) {
-          continue;
-        }
-
-        tpe.execute(new FSRegionScanner(fs, regionPath,
-            regionToBestLocalityRSMapping, regionDegreeLocalityMapping));
+      for (HRegionInfo hri: hris) {
+        tpe.execute(new FSRegionScanner(conf, hri, regionDegreeLocalityMapping));
       }
     } finally {
       tpe.shutdown();

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java
deleted file mode 100644
index d5197cf..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSVisitor.java
+++ /dev/null
@@ -1,109 +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.hadoop.hbase.util;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-
-/**
- * Utility methods for interacting with the hbase.root file system.
- */
-@InterfaceAudience.Private
-public final class FSVisitor {
-  private static final Log LOG = LogFactory.getLog(FSVisitor.class);
-
-  public interface StoreFileVisitor {
-    void storeFile(final String region, final String family, final String hfileName)
-       throws IOException;
-  }
-
-  private FSVisitor() {
-    // private constructor for utility class
-  }
-
-  /**
-   * Iterate over the table store files
-   *
-   * @param fs {@link FileSystem}
-   * @param tableDir {@link Path} to the table directory
-   * @param visitor callback object to get the store files
-   * @throws IOException if an error occurred while scanning the directory
-   */
-  public static void visitTableStoreFiles(final FileSystem fs, final Path tableDir,
-      final StoreFileVisitor visitor) throws IOException {
-    List<FileStatus> regions = FSUtils.listStatusWithStatusFilter(fs, tableDir, new FSUtils.RegionDirFilter(fs));
-    if (regions == null) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("No regions under directory:" + tableDir);
-      }
-      return;
-    }
-
-    for (FileStatus region: regions) {
-      visitRegionStoreFiles(fs, region.getPath(), visitor);
-    }
-  }
-
-  /**
-   * Iterate over the region store files
-   *
-   * @param fs {@link FileSystem}
-   * @param regionDir {@link Path} to the region directory
-   * @param visitor callback object to get the store files
-   * @throws IOException if an error occurred while scanning the directory
-   */
-  public static void visitRegionStoreFiles(final FileSystem fs, final Path regionDir,
-      final StoreFileVisitor visitor) throws IOException {
-    List<FileStatus> families = FSUtils.listStatusWithStatusFilter(fs, regionDir, new FSUtils.FamilyDirFilter(fs));
-    if (families == null) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("No families under region directory:" + regionDir);
-      }
-      return;
-    }
-
-    PathFilter fileFilter = new FSUtils.FileFilter(fs);
-    for (FileStatus family: families) {
-      Path familyDir = family.getPath();
-      String familyName = familyDir.getName();
-
-      // get all the storeFiles in the family
-      FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir, fileFilter);
-      if (storeFiles == null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("No hfiles found for family: " + familyDir + ", skipping.");
-        }
-        continue;
-      }
-
-      for (FileStatus hfile: storeFiles) {
-        Path hfilePath = hfile.getPath();
-        visitor.storeFile(regionDir.getName(), familyName, hfilePath.getName());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index a12425d..e1a3fa2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -110,6 +110,8 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
+import org.apache.hadoop.hbase.fs.RegionFileSystem.StoreFileVisitor;
 import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -1070,51 +1072,54 @@ public class HBaseFsck extends Configured implements Closeable {
    * Lingering reference file prevents a region from opening. It has to
    * be fixed before a cluster can start properly.
    */
-  private void offlineReferenceFileRepair() throws IOException, InterruptedException {
+  private void offlineReferenceFileRepair() throws IOException {
     clearState();
-    Configuration conf = getConf();
-    Path hbaseRoot = FSUtils.getRootDir(conf);
-    FileSystem fs = hbaseRoot.getFileSystem(conf);
-    LOG.info("Computing mapping of all store files");
-    Map<String, Path> allFiles = FSUtils.getTableStoreFilePathMap(fs, hbaseRoot,
-      new FSUtils.ReferenceFileFilter(fs), executor, errors);
-    errors.print("");
     LOG.info("Validating mapping using HDFS state");
-    for (Path path: allFiles.values()) {
-      Path referredToFile = StoreFileInfo.getReferredToFile(path);
-      if (fs.exists(referredToFile)) continue;  // good, expected
-
-      // Found a lingering reference file
-      errors.reportError(ERROR_CODE.LINGERING_REFERENCE_HFILE,
-        "Found lingering reference file " + path);
-      if (!shouldFixReferenceFiles()) continue;
-
-      // Now, trying to fix it since requested
-      boolean success = false;
-      String pathStr = path.toString();
-
-      // A reference file path should be like
-      // ${hbase.rootdir}/data/namespace/table_name/region_id/family_name/referred_file.region_name
-      // Up 5 directories to get the root folder.
-      // So the file will be sidelined to a similar folder structure.
-      int index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR);
-      for (int i = 0; index > 0 && i < 5; i++) {
-        index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR, index - 1);
-      }
-      if (index > 0) {
-        Path rootDir = getSidelineDir();
-        Path dst = new Path(rootDir, pathStr.substring(index + 1));
-        fs.mkdirs(dst.getParent());
-        LOG.info("Trying to sildeline reference file "
-          + path + " to " + dst);
-        setShouldRerun();
-
-        success = fs.rename(path, dst);
-      }
-      if (!success) {
-        LOG.error("Failed to sideline reference file " + path);
+    final MasterFileSystem mfs = MasterFileSystem.open(getConf(), false);
+    mfs.visitStoreFiles(new StoreFileVisitor() {
+      @Override
+      public void storeFile(HRegionInfo region, String family, StoreFileInfo storeFile)
+          throws IOException {
+        if (errors != null) errors.progress();
+        if (!storeFile.isReference()) return;
+
+        FileSystem fs = mfs.getFileSystem();
+        Path path = storeFile.getPath();
+        Path referredToFile = StoreFileInfo.getReferredToFile(path);
+        if (fs.exists(referredToFile)) return;  // good, expected
+
+        // Found a lingering reference file
+        errors.reportError(ERROR_CODE.LINGERING_REFERENCE_HFILE,
+          "Found lingering reference file " + path);
+        if (!shouldFixReferenceFiles()) return;
+
+        // Now, trying to fix it since requested
+        boolean success = false;
+        String pathStr = path.toString();
+
+        // A reference file path should be like
+        // ${hbase.rootdir}/data/namespace/table_name/region_id/family_name/referred_file.region_name
+        // Up 5 directories to get the root folder.
+        // So the file will be sidelined to a similar folder structure.
+        int index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR);
+        for (int i = 0; index > 0 && i < 5; i++) {
+          index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR, index - 1);
+        }
+        if (index > 0) {
+          Path rootDir = getSidelineDir();
+          Path dst = new Path(rootDir, pathStr.substring(index + 1));
+          fs.mkdirs(dst.getParent());
+          LOG.info("Trying to sildeline reference file "
+            + path + " to " + dst);
+          setShouldRerun();
+
+          success = fs.rename(path, dst);
+        }
+        if (!success) {
+          LOG.error("Failed to sideline reference file " + path);
+        }
       }
-    }
+    });
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
deleted file mode 100644
index 2fbaa92..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
+++ /dev/null
@@ -1,156 +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.hadoop.hbase.util;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HStore;
-
-/**
- * Helper class for all utilities related to archival/retrieval of HFiles
- */
-public class HFileArchiveUtil {
-  private HFileArchiveUtil() {
-    // non-external instantiation - util class
-  }
-
-  /**
-   * Get the directory to archive a store directory
-   * @param conf {@link Configuration} to read for the archive directory name
-   * @param tableName table name under which the store currently lives
-   * @param regionName region encoded name under which the store currently lives
-   * @param familyName name of the family in the store
-   * @return {@link Path} to the directory to archive the given store or
-   *         <tt>null</tt> if it should not be archived
-   */
-  public static Path getStoreArchivePath(final Configuration conf,
-                                         final TableName tableName,
-      final String regionName, final String familyName) throws IOException {
-    Path tableArchiveDir = getTableArchivePath(conf, tableName);
-    return HStore.getStoreHomedir(tableArchiveDir, regionName, Bytes.toBytes(familyName));
-  }
-
-  /**
-   * Get the directory to archive a store directory
-   * @param conf {@link Configuration} to read for the archive directory name.
-   * @param region parent region information under which the store currently lives
-   * @param tabledir directory for the table under which the store currently lives
-   * @param family name of the family in the store
-   * @return {@link Path} to the directory to archive the given store or <tt>null</tt> if it should
-   *         not be archived
-   */
-  public static Path getStoreArchivePath(Configuration conf,
-                                         HRegionInfo region,
-                                         Path tabledir,
-      byte[] family) throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
-    Path tableArchiveDir = getTableArchivePath(rootDir, region.getTable());
-    return HStore.getStoreHomedir(tableArchiveDir, region, family);
-  }
-
-  /**
-   * Get the archive directory for a given region under the specified table
-   * @param tableName the table name. Cannot be null.
-   * @param regiondir the path to the region directory. Cannot be null.
-   * @return {@link Path} to the directory to archive the given region, or <tt>null</tt> if it
-   *         should not be archived
-   */
-  public static Path getRegionArchiveDir(Path rootDir,
-                                         TableName tableName,
-                                         Path regiondir) {
-    // get the archive directory for a table
-    Path archiveDir = getTableArchivePath(rootDir, tableName);
-
-    // then add on the region path under the archive
-    String encodedRegionName = regiondir.getName();
-    return HRegion.getRegionDir(archiveDir, encodedRegionName);
-  }
-
-  /**
-   * Get the archive directory for a given region under the specified table
-   * @param rootDir {@link Path} to the root directory where hbase files are stored (for building
-   *          the archive path)
-   * @param tableName name of the table to archive. Cannot be null.
-   * @return {@link Path} to the directory to archive the given region, or <tt>null</tt> if it
-   *         should not be archived
-   */
-  public static Path getRegionArchiveDir(Path rootDir,
-                                         TableName tableName, String encodedRegionName) {
-    // get the archive directory for a table
-    Path archiveDir = getTableArchivePath(rootDir, tableName);
-    return HRegion.getRegionDir(archiveDir, encodedRegionName);
-  }
-
-  /**
-   * Get the path to the table archive directory based on the configured archive directory.
-   * <p>
-   * Get the path to the table's archive directory.
-   * <p>
-   * Generally of the form: /hbase/.archive/[tablename]
-   * @param rootdir {@link Path} to the root directory where hbase files are stored (for building
-   *          the archive path)
-   * @param tableName Name of the table to be archived. Cannot be null.
-   * @return {@link Path} to the archive directory for the table
-   */
-  public static Path getTableArchivePath(final Path rootdir, final TableName tableName) {
-    return FSUtils.getTableDir(getArchivePath(rootdir), tableName);
-  }
-
-  /**
-   * Get the path to the table archive directory based on the configured archive directory.
-   * <p>
-   * Assumed that the table should already be archived.
-   * @param conf {@link Configuration} to read the archive directory property. Can be null
-   * @param tableName Name of the table to be archived. Cannot be null.
-   * @return {@link Path} to the archive directory for the table
-   */
-  public static Path getTableArchivePath(final Configuration conf,
-                                         final TableName tableName)
-      throws IOException {
-    return FSUtils.getTableDir(getArchivePath(conf), tableName);
-  }
-
-  /**
-   * Get the full path to the archive directory on the configured 
-   * {@link org.apache.hadoop.hbase.fs.MasterFileSystem}
-   * @param conf to look for archive directory name and root directory. Cannot be null. Notes for
-   *          testing: requires a FileSystem root directory to be specified.
-   * @return the full {@link Path} to the archive directory, as defined by the configuration
-   * @throws IOException if an unexpected error occurs
-   */
-  public static Path getArchivePath(Configuration conf) throws IOException {
-    return getArchivePath(FSUtils.getRootDir(conf));
-  }
-
-  /**
-   * Get the full path to the archive directory on the configured 
-   * {@link org.apache.hadoop.hbase.fs.MasterFileSystem}
-   * @param rootdir {@link Path} to the root directory where hbase files are stored (for building
-   *          the archive path)
-   * @return the full {@link Path} to the archive directory, as defined by the configuration
-   */
-  private static Path getArchivePath(final Path rootdir) {
-    return new Path(rootdir, HConstants.HFILE_ARCHIVE_DIRECTORY);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
index 68b0ba3..1325c85 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
@@ -28,19 +28,22 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.fs.FsContext;
+import org.apache.hadoop.hbase.fs.RegionFileSystem.StoreFileVisitor;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.FSVisitor;
 import org.apache.hadoop.hbase.util.TestTableName;
 
 import org.junit.After;
@@ -103,13 +106,11 @@ public class TestCorruptedRegionStoreFile {
 
     // get the store file paths
     storeFiles.clear();
-    tableDir = FSUtils.getTableDir(getRootDir(), tableName);
-    FSVisitor.visitTableStoreFiles(getFileSystem(), tableDir, new FSVisitor.StoreFileVisitor() {
-      @Override
-      public void storeFile(final String region, final String family, final String hfile)
+    UTIL.getHBaseCluster().getMaster().getMasterFileSystem().visitStoreFiles(
+        FsContext.DATA, tableName, new StoreFileVisitor() {
+      public void storeFile(HRegionInfo region, String family, StoreFileInfo storeFile)
           throws IOException {
-        HFileLink link = HFileLink.build(UTIL.getConfiguration(), tableName, region, family, hfile);
-        storeFiles.add(link.getOriginPath());
+        storeFiles.add(storeFile.getPath());
       }
     });
     assertTrue("Expected at least " + NUM_FILES + " store files", storeFiles.size() >= NUM_FILES);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 7fbcfea..c318fe4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -29,6 +29,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.Collection;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -75,6 +76,7 @@ import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.fs.RegionFileSystem.StoreFileVisitor;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
@@ -767,12 +769,8 @@ public class TestSplitTransactionOnCluster {
     try {
       // Precondition: we created a table with no data, no store files.
       printOutRegions(regionServer, "Initial regions: ");
-      Configuration conf = cluster.getConfiguration();
-      HBaseFsck.debugLsr(conf, new Path("/"));
-      Path rootDir = FSUtils.getRootDir(conf);
-      FileSystem fs = TESTING_UTIL.getDFSCluster().getFileSystem();
-      Map<String, Path> storefiles =
-          FSUtils.getTableStoreFilePathMap(null, fs, rootDir, tableName);
+      cluster.getMaster().getMasterFileSystem().logFSTree(LOG);
+      List<StoreFileInfo> storefiles = getStoreFiles(tableName);
       assertEquals("Expected nothing but found " + storefiles.toString(), storefiles.size(), 0);
 
       // find a splittable region.  Refresh the regions list
@@ -795,9 +793,8 @@ public class TestSplitTransactionOnCluster {
       assertTrue(daughters.size() == 2);
 
       // check dirs
-      HBaseFsck.debugLsr(conf, new Path("/"));
-      Map<String, Path> storefilesAfter =
-          FSUtils.getTableStoreFilePathMap(null, fs, rootDir, tableName);
+      cluster.getMaster().getMasterFileSystem().logFSTree(LOG);
+      List<StoreFileInfo> storefilesAfter = getStoreFiles(tableName);
       assertEquals("Expected nothing but found " + storefilesAfter.toString(),
           storefilesAfter.size(), 0);
 
@@ -1370,5 +1367,17 @@ public class TestSplitTransactionOnCluster {
       }
     }
   }
+
+  private List<StoreFileInfo> getStoreFiles(TableName table) throws IOException {
+    final ArrayList<StoreFileInfo> storeFiles = new ArrayList<StoreFileInfo>();
+    cluster.getMaster().getMasterFileSystem().visitStoreFiles(table, new StoreFileVisitor() {
+      @Override
+      public void storeFile(HRegionInfo region, String family, StoreFileInfo storeFile)
+          throws IOException {
+        storeFiles.add(storeFile);
+      }
+    });
+    return storeFiles;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index 958278d..7ca5d36 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -70,7 +70,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSVisitor;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.MD5Hash;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java
deleted file mode 100644
index 4a870f8..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java
+++ /dev/null
@@ -1,123 +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.hadoop.hbase.util;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.util.UUID;
-import java.util.Set;
-import java.util.HashSet;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.junit.*;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test {@link FSUtils}.
- */
-@Category({MiscTests.class, MediumTests.class})
-public class TestFSVisitor {
-  private static final Log LOG = LogFactory.getLog(TestFSVisitor.class);
-
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  private final String TABLE_NAME = "testtb";
-
-  private Set<String> tableFamilies;
-  private Set<String> tableRegions;
-  private Set<String> tableHFiles;
-
-  private FileSystem fs;
-  private Path tableDir;
-  private Path rootDir;
-
-  @Before
-  public void setUp() throws Exception {
-    fs = FileSystem.get(TEST_UTIL.getConfiguration());
-    rootDir = TEST_UTIL.getDataTestDir("hbase");
-
-    tableFamilies = new HashSet<String>();
-    tableRegions = new HashSet<String>();
-    tableHFiles = new HashSet<String>();
-    tableDir = createTableFiles(rootDir, TABLE_NAME, tableRegions, tableFamilies, tableHFiles);
-    FSUtils.logFileSystemState(fs, rootDir, LOG);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    fs.delete(rootDir, true);
-  }
-
-  @Test
-  public void testVisitStoreFiles() throws IOException {
-    final Set<String> regions = new HashSet<String>();
-    final Set<String> families = new HashSet<String>();
-    final Set<String> hfiles = new HashSet<String>();
-    FSVisitor.visitTableStoreFiles(fs, tableDir, new FSVisitor.StoreFileVisitor() {
-      public void storeFile(final String region, final String family, final String hfileName)
-          throws IOException {
-        regions.add(region);
-        families.add(family);
-        hfiles.add(hfileName);
-      }
-    });
-    assertEquals(tableRegions, regions);
-    assertEquals(tableFamilies, families);
-    assertEquals(tableHFiles, hfiles);
-  }
-
-  /*
-   * |-testtb/
-   * |----f1d3ff8443297732862df21dc4e57262/
-   * |-------f1/
-   * |----------d0be84935ba84b66b1e866752ec5d663
-   * |----------9fc9d481718f4878b29aad0a597ecb94
-   * |-------f2/
-   * |----------4b0fe6068c564737946bcf4fd4ab8ae1
-   */
-  private Path createTableFiles(final Path rootDir, final String tableName,
-      final Set<String> tableRegions, final Set<String> tableFamilies,
-      final Set<String> tableHFiles) throws IOException {
-    Path tableDir = new Path(rootDir, tableName);
-    for (int r = 0; r < 10; ++r) {
-      String regionName = MD5Hash.getMD5AsHex(Bytes.toBytes(r));
-      tableRegions.add(regionName);
-      Path regionDir = new Path(tableDir, regionName);
-      for (int f = 0; f < 3; ++f) {
-        String familyName = "f" + f;
-        tableFamilies.add(familyName);
-        Path familyDir = new Path(regionDir, familyName);
-        fs.mkdirs(familyDir);
-        for (int h = 0; h < 5; ++h) {
-         String hfileName = UUID.randomUUID().toString().replaceAll("-", "");
-         tableHFiles.add(hfileName);
-         fs.createNewFile(new Path(familyDir, hfileName));
-        }
-      }
-    }
-    return tableDir;
-  }
-}