HBASE-14439 Move fs stuff out pt2 - region fs

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

Branch: refs/heads/hbase-14439
Commit: fe2918ba148701ba7b166fd1da9b6a85cc645a8b
Parents: 6796171
Author: Matteo Bertozzi <matteo.berto...@cloudera.com>
Authored: Fri Oct 9 11:47:27 2015 -0700
Committer: Sean Busbey <bus...@apache.org>
Committed: Thu Sep 22 11:19:52 2016 -0500

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  54 +-
 .../hadoop/hbase/backup/HFileArchiver.java      | 690 -----------------
 .../hadoop/hbase/fs/FSUtilsWithRetries.java     | 178 +++++
 .../hadoop/hbase/fs/MasterFileSystem.java       | 102 ++-
 .../hadoop/hbase/fs/RegionFileSystem.java       | 183 +++++
 .../hbase/fs/legacy/HFileArchiveUtil.java       | 156 ++++
 .../hadoop/hbase/fs/legacy/HFileArchiver.java   | 690 +++++++++++++++++
 .../hadoop/hbase/fs/legacy/LegacyArchiver.java  |  29 +
 .../hadoop/hbase/fs/legacy/LegacyLayout.java    |  52 +-
 .../hbase/fs/legacy/LegacyMasterFileSystem.java |  29 +-
 .../hbase/fs/legacy/LegacyRegionFileSystem.java | 758 +++++++++++++++++++
 .../hadoop/hbase/master/RegionStates.java       |   5 +-
 .../hbase/regionserver/StoreFileInfo.java       | 104 ++-
 .../hadoop/hbase/util/FSRegionScanner.java      | 104 +--
 .../org/apache/hadoop/hbase/util/FSUtils.java   | 388 ++--------
 .../org/apache/hadoop/hbase/util/FSVisitor.java | 109 ---
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  89 ++-
 .../hadoop/hbase/util/HFileArchiveUtil.java     | 156 ----
 .../TestCorruptedRegionStoreFile.java           |  15 +-
 .../TestSplitTransactionOnCluster.java          |  27 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |   1 -
 .../apache/hadoop/hbase/util/TestFSVisitor.java | 123 ---
 22 files changed, 2384 insertions(+), 1658 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index 867dc0d..8897e35 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -1437,23 +1437,47 @@ public class HTableDescriptor implements 
Comparable<HTableDescriptor> {
       remove(match);
   }
 
-  /**
-   * Returns the {@link Path} object representing the table directory under
-   * path rootdir
-   *
-   * Deprecated use FSUtils.getTableDir() instead.
-   *
-   * @param rootdir qualified path of HBase root directory
-   * @param tableName name of table
-   * @return {@link Path} for table
+  /** Table descriptor for <code>hbase:meta</code> catalog table
+   * Deprecated, use TableDescriptors#get(TableName.META_TABLE) or
+   * Admin#getTableDescriptor(TableName.META_TABLE) instead.
    */
   @Deprecated
-  public static Path getTableDir(Path rootdir, final byte [] tableName) {
-    //This is bad I had to mirror code from FSUTils.getTableDir since
-    //there is no module dependency between hbase-client and hbase-server
-    TableName name = TableName.valueOf(tableName);
-    return new Path(rootdir, new Path(HConstants.BASE_NAMESPACE_DIR,
-              new Path(name.getNamespaceAsString(), new 
Path(name.getQualifierAsString()))));
+  public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor(
+      TableName.META_TABLE_NAME,
+      new HColumnDescriptor[] {
+          new HColumnDescriptor(HConstants.CATALOG_FAMILY)
+              // Ten is arbitrary number.  Keep versions to help debugging.
+              .setMaxVersions(10)
+              .setInMemory(true)
+              .setBlocksize(8 * 1024)
+              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+              // Disable blooms for meta.  Needs work.  Seems to mess w/ 
getClosestOrBefore.
+              .setBloomFilterType(BloomType.NONE)
+              // Enable cache of data blocks in L1 if more than one caching 
tier deployed:
+              // e.g. if using CombinedBlockCache (BucketCache).
+              .setCacheDataInL1(true),
+          new HColumnDescriptor(HConstants.TABLE_FAMILY)
+              // Ten is arbitrary number.  Keep versions to help debugging.
+              .setMaxVersions(10)
+              .setInMemory(true)
+              .setBlocksize(8 * 1024)
+              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                  // Disable blooms for meta.  Needs work.  Seems to mess w/ 
getClosestOrBefore.
+              .setBloomFilterType(BloomType.NONE)
+                  // Enable cache of data blocks in L1 if more than one 
caching tier deployed:
+                  // e.g. if using CombinedBlockCache (BucketCache).
+              .setCacheDataInL1(true)
+      });
+
+  static {
+    try {
+      META_TABLEDESC.addCoprocessor(
+          "org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
+          null, Coprocessor.PRIORITY_SYSTEM, null);
+    } catch (IOException ex) {
+      //LOG.warn("exception in loading coprocessor for the hbase:meta table");
+      throw new RuntimeException(ex);
+    }
   }
 
   public final static String NAMESPACE_FAMILY_INFO = "info";

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
deleted file mode 100644
index d682ccc..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
+++ /dev/null
@@ -1,690 +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.backup;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-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.fs.PathFilter;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.HFileArchiveUtil;
-import org.apache.hadoop.io.MultipleIOException;
-
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.Lists;
-
-/**
- * Utility class to handle the removal of HFiles (or the respective {@link 
StoreFile StoreFiles})
- * for a HRegion from the {@link FileSystem}. The hfiles will be archived or 
deleted, depending on
- * the state of the system.
- */
-public class HFileArchiver {
-  private static final Log LOG = LogFactory.getLog(HFileArchiver.class);
-  private static final String SEPARATOR = ".";
-
-  /** Number of retries in case of fs operation failure */
-  private static final int DEFAULT_RETRIES_NUMBER = 3;
-
-  private HFileArchiver() {
-    // hidden ctor since this is just a util
-  }
-
-  /**
-   * Cleans up all the files for a HRegion by archiving the HFiles to the
-   * archive directory
-   * @param conf the configuration to use
-   * @param fs the file system object
-   * @param info HRegionInfo for region to be deleted
-   * @throws IOException
-   */
-  public static void archiveRegion(Configuration conf, FileSystem fs, 
HRegionInfo info)
-      throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
-    archiveRegion(fs, rootDir, FSUtils.getTableDir(rootDir, info.getTable()),
-      HRegion.getRegionDir(rootDir, info));
-  }
-
-  /**
-   * Remove an entire region from the table directory via archiving the 
region's hfiles.
-   * @param fs {@link FileSystem} from which to remove the region
-   * @param rootdir {@link Path} to the root directory where hbase files are 
stored (for building
-   *          the archive path)
-   * @param tableDir {@link Path} to where the table is being stored (for 
building the archive path)
-   * @param regionDir {@link Path} to where a region is being stored (for 
building the archive path)
-   * @return <tt>true</tt> if the region was sucessfully deleted. 
<tt>false</tt> if the filesystem
-   *         operations could not complete.
-   * @throws IOException if the request cannot be completed
-   */
-  public static boolean archiveRegion(FileSystem fs, Path rootdir, Path 
tableDir, Path regionDir)
-      throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("ARCHIVING " + regionDir.toString());
-    }
-
-    // otherwise, we archive the files
-    // make sure we can archive
-    if (tableDir == null || regionDir == null) {
-      LOG.error("No archive directory could be found because tabledir (" + 
tableDir
-          + ") or regiondir (" + regionDir + "was null. Deleting files 
instead.");
-      deleteRegionWithoutArchiving(fs, regionDir);
-      // we should have archived, but failed to. Doesn't matter if we deleted
-      // the archived files correctly or not.
-      return false;
-    }
-
-    // make sure the regiondir lives under the tabledir
-    
Preconditions.checkArgument(regionDir.toString().startsWith(tableDir.toString()));
-    Path regionArchiveDir = HFileArchiveUtil.getRegionArchiveDir(rootdir,
-        FSUtils.getTableName(tableDir),
-        regionDir.getName());
-
-    FileStatusConverter getAsFile = new FileStatusConverter(fs);
-    // otherwise, we attempt to archive the store files
-
-    // build collection of just the store directories to archive
-    Collection<File> toArchive = new ArrayList<File>();
-    final PathFilter dirFilter = new FSUtils.DirFilter(fs);
-    PathFilter nonHidden = new PathFilter() {
-      @Override
-      public boolean accept(Path file) {
-        return dirFilter.accept(file) && 
!file.getName().toString().startsWith(".");
-      }
-    };
-    FileStatus[] storeDirs = FSUtils.listStatus(fs, regionDir, nonHidden);
-    // if there no files, we can just delete the directory and return;
-    if (storeDirs == null) {
-      LOG.debug("Region directory (" + regionDir + ") was empty, just deleting 
and returning!");
-      return deleteRegionWithoutArchiving(fs, regionDir);
-    }
-
-    // convert the files in the region to a File
-    toArchive.addAll(Lists.transform(Arrays.asList(storeDirs), getAsFile));
-    LOG.debug("Archiving " + toArchive);
-    boolean success = false;
-    try {
-      success = resolveAndArchive(fs, regionArchiveDir, toArchive);
-    } catch (IOException e) {
-      LOG.error("Failed to archive " + toArchive, e);
-      success = false;
-    }
-
-    // if that was successful, then we delete the region
-    if (success) {
-      return deleteRegionWithoutArchiving(fs, regionDir);
-    }
-
-    throw new IOException("Received error when attempting to archive files (" 
+ toArchive
-        + "), cannot delete region directory. ");
-  }
-
-  /**
-   * Remove from the specified region the store files of the specified column 
family,
-   * either by archiving them or outright deletion
-   * @param fs the filesystem where the store files live
-   * @param conf {@link Configuration} to examine to determine the archive 
directory
-   * @param parent Parent region hosting the store files
-   * @param tableDir {@link Path} to where the table is being stored (for 
building the archive path)
-   * @param family the family hosting the store files
-   * @throws IOException if the files could not be correctly disposed.
-   */
-  public static void archiveFamily(FileSystem fs, Configuration conf,
-      HRegionInfo parent, Path tableDir, byte[] family) throws IOException {
-    Path familyDir = new Path(tableDir, new Path(parent.getEncodedName(), 
Bytes.toString(family)));
-    FileStatus[] storeFiles = FSUtils.listStatus(fs, familyDir);
-    if (storeFiles == null) {
-      LOG.debug("No store files to dispose for region=" + 
parent.getRegionNameAsString() +
-          ", family=" + Bytes.toString(family));
-      return;
-    }
-
-    FileStatusConverter getAsFile = new FileStatusConverter(fs);
-    Collection<File> toArchive = Lists.transform(Arrays.asList(storeFiles), 
getAsFile);
-    Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, parent, 
tableDir, family);
-
-    // do the actual archive
-    if (!resolveAndArchive(fs, storeArchiveDir, toArchive)) {
-      throw new IOException("Failed to archive/delete all the files for 
region:"
-          + Bytes.toString(parent.getRegionName()) + ", family:" + 
Bytes.toString(family)
-          + " into " + storeArchiveDir + ". Something is probably awry on the 
filesystem.");
-    }
-  }
-
-  /**
-   * Remove the store files, either by archiving them or outright deletion
-   * @param conf {@link Configuration} to examine to determine the archive 
directory
-   * @param fs the filesystem where the store files live
-   * @param regionInfo {@link HRegionInfo} of the region hosting the store 
files
-   * @param family the family hosting the store files
-   * @param compactedFiles files to be disposed of. No further reading of 
these files should be
-   *          attempted; otherwise likely to cause an {@link IOException}
-   * @throws IOException if the files could not be correctly disposed.
-   */
-  public static void archiveStoreFiles(Configuration conf, FileSystem fs, 
HRegionInfo regionInfo,
-      Path tableDir, byte[] family, Collection<StoreFile> compactedFiles) 
throws IOException {
-
-    // sometimes in testing, we don't have rss, so we need to check for that
-    if (fs == null) {
-      LOG.warn("Passed filesystem is null, so just deleting the files without 
archiving for region:"
-          + Bytes.toString(regionInfo.getRegionName()) + ", family:" + 
Bytes.toString(family));
-      deleteStoreFilesWithoutArchiving(compactedFiles);
-      return;
-    }
-
-    // short circuit if we don't have any files to delete
-    if (compactedFiles.size() == 0) {
-      LOG.debug("No store files to dispose, done!");
-      return;
-    }
-
-    // build the archive path
-    if (regionInfo == null || family == null) throw new IOException(
-        "Need to have a region and a family to archive from.");
-
-    Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, 
regionInfo, tableDir, family);
-
-    // make sure we don't archive if we can't and that the archive dir exists
-    if (!fs.mkdirs(storeArchiveDir)) {
-      throw new IOException("Could not make archive directory (" + 
storeArchiveDir + ") for store:"
-          + Bytes.toString(family) + ", deleting compacted files instead.");
-    }
-
-    // otherwise we attempt to archive the store files
-    if (LOG.isDebugEnabled()) LOG.debug("Archiving compacted store files.");
-
-    // Wrap the storefile into a File
-    StoreToFile getStorePath = new StoreToFile(fs);
-    Collection<File> storeFiles = Collections2.transform(compactedFiles, 
getStorePath);
-
-    // do the actual archive
-    if (!resolveAndArchive(fs, storeArchiveDir, storeFiles)) {
-      throw new IOException("Failed to archive/delete all the files for 
region:"
-          + Bytes.toString(regionInfo.getRegionName()) + ", family:" + 
Bytes.toString(family)
-          + " into " + storeArchiveDir + ". Something is probably awry on the 
filesystem.");
-    }
-  }
-
-  /**
-   * Archive the store file
-   * @param fs the filesystem where the store files live
-   * @param regionInfo region hosting the store files
-   * @param conf {@link Configuration} to examine to determine the archive 
directory
-   * @param tableDir {@link Path} to where the table is being stored (for 
building the archive path)
-   * @param family the family hosting the store files
-   * @param storeFile file to be archived
-   * @throws IOException if the files could not be correctly disposed.
-   */
-  public static void archiveStoreFile(Configuration conf, FileSystem fs, 
HRegionInfo regionInfo,
-      Path tableDir, byte[] family, Path storeFile) throws IOException {
-    Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, 
regionInfo, tableDir, family);
-    // make sure we don't archive if we can't and that the archive dir exists
-    if (!fs.mkdirs(storeArchiveDir)) {
-      throw new IOException("Could not make archive directory (" + 
storeArchiveDir + ") for store:"
-          + Bytes.toString(family) + ", deleting compacted files instead.");
-    }
-
-    // do the actual archive
-    long start = EnvironmentEdgeManager.currentTime();
-    File file = new FileablePath(fs, storeFile);
-    if (!resolveAndArchiveFile(storeArchiveDir, file, Long.toString(start))) {
-      throw new IOException("Failed to archive/delete the file for region:"
-          + regionInfo.getRegionNameAsString() + ", family:" + 
Bytes.toString(family)
-          + " into " + storeArchiveDir + ". Something is probably awry on the 
filesystem.");
-    }
-  }
-
-  /**
-   * Archive the given files and resolve any conflicts with existing files via 
appending the time
-   * archiving started (so all conflicts in the same group have the same 
timestamp appended).
-   * <p>
-   * If any of the passed files to archive are directories, archives all the 
files under that
-   * directory. Archive directory structure for children is the base archive 
directory name + the
-   * parent directory and is built recursively is passed files are directories 
themselves.
-   * @param fs {@link FileSystem} on which to archive the files
-   * @param baseArchiveDir base archive directory to archive the given files
-   * @param toArchive files to be archived
-   * @return <tt>true</tt> on success, <tt>false</tt> otherwise
-   * @throws IOException on unexpected failure
-   */
-  private static boolean resolveAndArchive(FileSystem fs, Path baseArchiveDir,
-      Collection<File> toArchive) throws IOException {
-    if (LOG.isTraceEnabled()) LOG.trace("Starting to archive " + toArchive);
-    long start = EnvironmentEdgeManager.currentTime();
-    List<File> failures = resolveAndArchive(fs, baseArchiveDir, toArchive, 
start);
-
-    // notify that some files were not archived.
-    // We can't delete the files otherwise snapshots or other backup system
-    // that relies on the archiver end up with data loss.
-    if (failures.size() > 0) {
-      LOG.warn("Failed to complete archive of: " + failures +
-        ". Those files are still in the original location, and they may slow 
down reads.");
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * Resolve any conflict with an existing archive file via timestamp-append
-   * renaming of the existing file and then archive the passed in files.
-   * @param fs {@link FileSystem} on which to archive the files
-   * @param baseArchiveDir base archive directory to store the files. If any of
-   *          the files to archive are directories, will append the name of the
-   *          directory to the base archive directory name, creating a parallel
-   *          structure.
-   * @param toArchive files/directories that need to be archvied
-   * @param start time the archiving started - used for resolving archive
-   *          conflicts.
-   * @return the list of failed to archive files.
-   * @throws IOException if an unexpected file operation exception occured
-   */
-  private static List<File> resolveAndArchive(FileSystem fs, Path 
baseArchiveDir,
-      Collection<File> toArchive, long start) throws IOException {
-    // short circuit if no files to move
-    if (toArchive.size() == 0) return Collections.emptyList();
-
-    if (LOG.isTraceEnabled()) LOG.trace("moving files to the archive 
directory: " + baseArchiveDir);
-
-    // make sure the archive directory exists
-    if (!fs.exists(baseArchiveDir)) {
-      if (!fs.mkdirs(baseArchiveDir)) {
-        throw new IOException("Failed to create the archive directory:" + 
baseArchiveDir
-            + ", quitting archive attempt.");
-      }
-      if (LOG.isTraceEnabled()) LOG.trace("Created archive directory:" + 
baseArchiveDir);
-    }
-
-    List<File> failures = new ArrayList<File>();
-    String startTime = Long.toString(start);
-    for (File file : toArchive) {
-      // if its a file archive it
-      try {
-        if (LOG.isTraceEnabled()) LOG.trace("Archiving: " + file);
-        if (file.isFile()) {
-          // attempt to archive the file
-          if (!resolveAndArchiveFile(baseArchiveDir, file, startTime)) {
-            LOG.warn("Couldn't archive " + file + " into backup directory: " + 
baseArchiveDir);
-            failures.add(file);
-          }
-        } else {
-          // otherwise its a directory and we need to archive all files
-          if (LOG.isTraceEnabled()) LOG.trace(file + " is a directory, 
archiving children files");
-          // so we add the directory name to the one base archive
-          Path parentArchiveDir = new Path(baseArchiveDir, file.getName());
-          // and then get all the files from that directory and attempt to
-          // archive those too
-          Collection<File> children = file.getChildren();
-          failures.addAll(resolveAndArchive(fs, parentArchiveDir, children, 
start));
-        }
-      } catch (IOException e) {
-        LOG.warn("Failed to archive " + file, e);
-        failures.add(file);
-      }
-    }
-    return failures;
-  }
-
-  /**
-   * Attempt to archive the passed in file to the archive directory.
-   * <p>
-   * If the same file already exists in the archive, it is moved to a 
timestamped directory under
-   * the archive directory and the new file is put in its place.
-   * @param archiveDir {@link Path} to the directory that stores the archives 
of the hfiles
-   * @param currentFile {@link Path} to the original HFile that will be 
archived
-   * @param archiveStartTime time the archiving started, to resolve naming 
conflicts
-   * @return <tt>true</tt> if the file is successfully archived. 
<tt>false</tt> if there was a
-   *         problem, but the operation still completed.
-   * @throws IOException on failure to complete {@link FileSystem} operations.
-   */
-  private static boolean resolveAndArchiveFile(Path archiveDir, File 
currentFile,
-      String archiveStartTime) throws IOException {
-    // build path as it should be in the archive
-    String filename = currentFile.getName();
-    Path archiveFile = new Path(archiveDir, filename);
-    FileSystem fs = currentFile.getFileSystem();
-
-    // if the file already exists in the archive, move that one to a 
timestamped backup. This is a
-    // really, really unlikely situtation, where we get the same name for the 
existing file, but
-    // is included just for that 1 in trillion chance.
-    if (fs.exists(archiveFile)) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("File:" + archiveFile + " already exists in archive, moving 
to "
-            + "timestamped backup and overwriting current.");
-      }
-
-      // move the archive file to the stamped backup
-      Path backedupArchiveFile = new Path(archiveDir, filename + SEPARATOR + 
archiveStartTime);
-      if (!fs.rename(archiveFile, backedupArchiveFile)) {
-        LOG.error("Could not rename archive file to backup: " + 
backedupArchiveFile
-            + ", deleting existing file in favor of newer.");
-        // try to delete the exisiting file, if we can't rename it
-        if (!fs.delete(archiveFile, false)) {
-          throw new IOException("Couldn't delete existing archive file (" + 
archiveFile
-              + ") or rename it to the backup file (" + backedupArchiveFile
-              + ") to make room for similarly named file.");
-        }
-      }
-      LOG.debug("Backed up archive file from " + archiveFile);
-    }
-
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("No existing file in archive for: " + archiveFile +
-        ", free to archive original file.");
-    }
-
-    // at this point, we should have a free spot for the archive file
-    boolean success = false;
-    for (int i = 0; !success && i < DEFAULT_RETRIES_NUMBER; ++i) {
-      if (i > 0) {
-        // Ensure that the archive directory exists.
-        // The previous "move to archive" operation has failed probably because
-        // the cleaner has removed our archive directory (HBASE-7643).
-        // (we're in a retry loop, so don't worry too much about the exception)
-        try {
-          if (!fs.exists(archiveDir)) {
-            if (fs.mkdirs(archiveDir)) {
-              LOG.debug("Created archive directory:" + archiveDir);
-            }
-          }
-        } catch (IOException e) {
-          LOG.warn("Failed to create directory: " + archiveDir, e);
-        }
-      }
-
-      try {
-        success = currentFile.moveAndClose(archiveFile);
-      } catch (IOException e) {
-        LOG.warn("Failed to archive " + currentFile + " on try #" + i, e);
-        success = false;
-      }
-    }
-
-    if (!success) {
-      LOG.error("Failed to archive " + currentFile);
-      return false;
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Finished archiving from " + currentFile + ", to " + 
archiveFile);
-    }
-    return true;
-  }
-
-  /**
-   * Without regard for backup, delete a region. Should be used with caution.
-   * @param regionDir {@link Path} to the region to be deleted.
-   * @param fs FileSystem from which to delete the region
-   * @return <tt>true</tt> on successful deletion, <tt>false</tt> otherwise
-   * @throws IOException on filesystem operation failure
-   */
-  private static boolean deleteRegionWithoutArchiving(FileSystem fs, Path 
regionDir)
-      throws IOException {
-    if (fs.delete(regionDir, true)) {
-      LOG.debug("Deleted all region files in: " + regionDir);
-      return true;
-    }
-    LOG.debug("Failed to delete region directory:" + regionDir);
-    return false;
-  }
-
-  /**
-   * Just do a simple delete of the given store files
-   * <p>
-   * A best effort is made to delete each of the files, rather than bailing on 
the first failure.
-   * <p>
-   * This method is preferable to {@link 
#deleteFilesWithoutArchiving(Collection)} since it consumes
-   * less resources, but is limited in terms of usefulness
-   * @param compactedFiles store files to delete from the file system.
-   * @throws IOException if a file cannot be deleted. All files will be 
attempted to deleted before
-   *           throwing the exception, rather than failing at the first file.
-   */
-  private static void deleteStoreFilesWithoutArchiving(Collection<StoreFile> 
compactedFiles)
-      throws IOException {
-    LOG.debug("Deleting store files without archiving.");
-    List<IOException> errors = new ArrayList<IOException>(0);
-    for (StoreFile hsf : compactedFiles) {
-      try {
-        hsf.deleteReader();
-      } catch (IOException e) {
-        LOG.error("Failed to delete store file:" + hsf.getPath());
-        errors.add(e);
-      }
-    }
-    if (errors.size() > 0) {
-      throw MultipleIOException.createIOException(errors);
-    }
-  }
-
-  /**
-   * Adapt a type to match the {@link File} interface, which is used 
internally for handling
-   * archival/removal of files
-   * @param <T> type to adapt to the {@link File} interface
-   */
-  private static abstract class FileConverter<T> implements Function<T, File> {
-    protected final FileSystem fs;
-
-    public FileConverter(FileSystem fs) {
-      this.fs = fs;
-    }
-  }
-
-  /**
-   * Convert a FileStatus to something we can manage in the archiving
-   */
-  private static class FileStatusConverter extends FileConverter<FileStatus> {
-    public FileStatusConverter(FileSystem fs) {
-      super(fs);
-    }
-
-    @Override
-    public File apply(FileStatus input) {
-      return new FileablePath(fs, input.getPath());
-    }
-  }
-
-  /**
-   * Convert the {@link StoreFile} into something we can manage in the archive
-   * methods
-   */
-  private static class StoreToFile extends FileConverter<StoreFile> {
-    public StoreToFile(FileSystem fs) {
-      super(fs);
-    }
-
-    @Override
-    public File apply(StoreFile input) {
-      return new FileableStoreFile(fs, input);
-    }
-  }
-
-  /**
-   * Wrapper to handle file operations uniformly
-   */
-  private static abstract class File {
-    protected final FileSystem fs;
-
-    public File(FileSystem fs) {
-      this.fs = fs;
-    }
-
-    /**
-     * Delete the file
-     * @throws IOException on failure
-     */
-    abstract void delete() throws IOException;
-
-    /**
-     * Check to see if this is a file or a directory
-     * @return <tt>true</tt> if it is a file, <tt>false</tt> otherwise
-     * @throws IOException on {@link FileSystem} connection error
-     */
-    abstract boolean isFile() throws IOException;
-
-    /**
-     * @return if this is a directory, returns all the children in the
-     *         directory, otherwise returns an empty list
-     * @throws IOException
-     */
-    abstract Collection<File> getChildren() throws IOException;
-
-    /**
-     * close any outside readers of the file
-     * @throws IOException
-     */
-    abstract void close() throws IOException;
-
-    /**
-     * @return the name of the file (not the full fs path, just the individual
-     *         file name)
-     */
-    abstract String getName();
-
-    /**
-     * @return the path to this file
-     */
-    abstract Path getPath();
-
-    /**
-     * Move the file to the given destination
-     * @param dest
-     * @return <tt>true</tt> on success
-     * @throws IOException
-     */
-    public boolean moveAndClose(Path dest) throws IOException {
-      this.close();
-      Path p = this.getPath();
-      return FSUtils.renameAndSetModifyTime(fs, p, dest);
-    }
-
-    /**
-     * @return the {@link FileSystem} on which this file resides
-     */
-    public FileSystem getFileSystem() {
-      return this.fs;
-    }
-
-    @Override
-    public String toString() {
-      return this.getClass() + ", file:" + getPath().toString();
-    }
-  }
-
-  /**
-   * A {@link File} that wraps a simple {@link Path} on a {@link FileSystem}.
-   */
-  private static class FileablePath extends File {
-    private final Path file;
-    private final FileStatusConverter getAsFile;
-
-    public FileablePath(FileSystem fs, Path file) {
-      super(fs);
-      this.file = file;
-      this.getAsFile = new FileStatusConverter(fs);
-    }
-
-    @Override
-    public void delete() throws IOException {
-      if (!fs.delete(file, true)) throw new IOException("Failed to delete:" + 
this.file);
-    }
-
-    @Override
-    public String getName() {
-      return file.getName();
-    }
-
-    @Override
-    public Collection<File> getChildren() throws IOException {
-      if (fs.isFile(file)) return Collections.emptyList();
-      return Collections2.transform(Arrays.asList(fs.listStatus(file)), 
getAsFile);
-    }
-
-    @Override
-    public boolean isFile() throws IOException {
-      return fs.isFile(file);
-    }
-
-    @Override
-    public void close() throws IOException {
-      // NOOP - files are implicitly closed on removal
-    }
-
-    @Override
-    Path getPath() {
-      return file;
-    }
-  }
-
-  /**
-   * {@link File} adapter for a {@link StoreFile} living on a {@link 
FileSystem}
-   * .
-   */
-  private static class FileableStoreFile extends File {
-    StoreFile file;
-
-    public FileableStoreFile(FileSystem fs, StoreFile store) {
-      super(fs);
-      this.file = store;
-    }
-
-    @Override
-    public void delete() throws IOException {
-      file.deleteReader();
-    }
-
-    @Override
-    public String getName() {
-      return file.getPath().getName();
-    }
-
-    @Override
-    public boolean isFile() {
-      return true;
-    }
-
-    @Override
-    public Collection<File> getChildren() throws IOException {
-      // storefiles don't have children
-      return Collections.emptyList();
-    }
-
-    @Override
-    public void close() throws IOException {
-      file.closeReader(true);
-    }
-
-    @Override
-    Path getPath() {
-      return file.getPath();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/FSUtilsWithRetries.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/FSUtilsWithRetries.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/FSUtilsWithRetries.java
new file mode 100644
index 0000000..5dad2fb
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/FSUtilsWithRetries.java
@@ -0,0 +1,178 @@
+/**
+ *
+ * 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.fs;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class FSUtilsWithRetries {
+  private static final Log LOG = LogFactory.getLog(FSUtilsWithRetries.class);
+
+  /**
+   * In order to handle NN connectivity hiccups, one need to retry 
non-idempotent operation at the
+   * client level.
+   */
+  private static final int DEFAULT_HDFS_CLIENT_RETRIES_NUMBER = 10;
+  private static final int DEFAULT_BASE_SLEEP_BEFORE_RETRIES = 1000;
+  private final int hdfsClientRetriesNumber;
+  private final int baseSleepBeforeRetries;
+
+  private final FileSystem fs;
+
+  public FSUtilsWithRetries(final Configuration conf, final FileSystem fs) {
+    this.fs = fs;
+
+    this.hdfsClientRetriesNumber = conf.getInt("hdfs.client.retries.number",
+      DEFAULT_HDFS_CLIENT_RETRIES_NUMBER);
+    this.baseSleepBeforeRetries = 
conf.getInt("hdfs.client.sleep.before.retries",
+      DEFAULT_BASE_SLEEP_BEFORE_RETRIES);
+  }
+
+  public FileSystem getFileSystem() {
+    return fs;
+  }
+
+  public boolean exists(Path path) throws IOException {
+    IOException lastIOE = null;
+    int i = 0;
+    do {
+      try {
+        return fs.exists(path);
+      } catch (IOException ioe) {
+        lastIOE = ioe;
+        if (fs.exists(path)) return true; // directory is present
+        try {
+          sleepBeforeRetry("Check existence", i+1);
+        } catch (InterruptedException e) {
+          throw (InterruptedIOException)new 
InterruptedIOException().initCause(e);
+        }
+      }
+    } while (++i <= hdfsClientRetriesNumber);
+    throw new IOException("Exception in check existence", lastIOE);
+  }
+
+  /**
+   * Creates a directory. Assumes the user has already checked for this 
directory existence.
+   * @param dir
+   * @return the result of fs.mkdirs(). In case underlying fs throws an 
IOException, it checks
+   *         whether the directory exists or not, and returns true if it 
exists.
+   * @throws IOException
+   */
+  public boolean createDir(Path dir) throws IOException {
+    IOException lastIOE = null;
+    int i = 0;
+    do {
+      try {
+        return fs.mkdirs(dir);
+      } catch (IOException ioe) {
+        lastIOE = ioe;
+        if (fs.exists(dir)) return true; // directory is present
+        try {
+          sleepBeforeRetry("Create Directory", i+1);
+        } catch (InterruptedException e) {
+          throw (InterruptedIOException)new 
InterruptedIOException().initCause(e);
+        }
+      }
+    } while (++i <= hdfsClientRetriesNumber);
+    throw new IOException("Exception in createDir", lastIOE);
+  }
+
+  /**
+   * Renames a directory. Assumes the user has already checked for this 
directory existence.
+   * @param srcpath
+   * @param dstPath
+   * @return true if rename is successful.
+   * @throws IOException
+   */
+  public boolean rename(Path srcpath, Path dstPath) throws IOException {
+    IOException lastIOE = null;
+    int i = 0;
+    do {
+      try {
+        return fs.rename(srcpath, dstPath);
+      } catch (IOException ioe) {
+        lastIOE = ioe;
+        if (!fs.exists(srcpath) && fs.exists(dstPath)) return true; // 
successful move
+        // dir is not there, retry after some time.
+        try {
+          sleepBeforeRetry("Rename Directory", i+1);
+        } catch (InterruptedException e) {
+          throw (InterruptedIOException)new 
InterruptedIOException().initCause(e);
+        }
+      }
+    } while (++i <= hdfsClientRetriesNumber);
+
+    throw new IOException("Exception in rename", lastIOE);
+  }
+
+  /**
+   * Deletes a directory. Assumes the user has already checked for this 
directory existence.
+   * @param dir
+   * @return true if the directory is deleted.
+   * @throws IOException
+   */
+  public boolean deleteDir(Path dir) throws IOException {
+    IOException lastIOE = null;
+    int i = 0;
+    do {
+      try {
+        return fs.delete(dir, true);
+      } catch (IOException ioe) {
+        lastIOE = ioe;
+        if (!fs.exists(dir)) return true;
+        // dir is there, retry deleting after some time.
+        try {
+          sleepBeforeRetry("Delete Directory", i+1);
+        } catch (InterruptedException e) {
+          throw (InterruptedIOException)new 
InterruptedIOException().initCause(e);
+        }
+      }
+    } while (++i <= hdfsClientRetriesNumber);
+
+    throw new IOException("Exception in DeleteDir", lastIOE);
+  }
+
+  /**
+   * sleeping logic; handles the interrupt exception.
+   */
+  private void sleepBeforeRetry(String msg, int sleepMultiplier) throws 
InterruptedException {
+    sleepBeforeRetry(msg, sleepMultiplier, baseSleepBeforeRetries, 
hdfsClientRetriesNumber);
+  }
+
+  /**
+   * sleeping logic for static methods; handles the interrupt exception. 
Keeping a static version
+   * for this to avoid re-looking for the integer values.
+   */
+  private static void sleepBeforeRetry(String msg, int sleepMultiplier, int 
baseSleepBeforeRetries,
+      int hdfsClientRetriesNumber) throws InterruptedException {
+    if (sleepMultiplier > hdfsClientRetriesNumber) {
+      LOG.debug(msg + ", retries exhausted");
+      return;
+    }
+    LOG.debug(msg + ", sleeping " + baseSleepBeforeRetries + " times " + 
sleepMultiplier);
+    Thread.sleep((long)baseSleepBeforeRetries * sleepMultiplier);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterFileSystem.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterFileSystem.java
index 2a58b9e..020917a 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterFileSystem.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterFileSystem.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.fs.legacy.LegacyMasterFileSystem;
+import org.apache.hadoop.hbase.fs.RegionFileSystem.StoreFileVisitor;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 
@@ -142,6 +143,69 @@ public abstract class MasterFileSystem {
   }
 
   // ==========================================================================
+  //  PUBLIC Methods - Table Region related
+  // ==========================================================================
+  public void deleteRegion(HRegionInfo regionInfo) throws IOException {
+    RegionFileSystem.destroy(conf, fs, rootDir, regionInfo);
+  }
+
+  public Collection<HRegionInfo> getRegions(TableName tableName) throws 
IOException {
+    return getRegions(FsContext.DATA, tableName);
+  }
+
+  public abstract Collection<HRegionInfo> getRegions(FsContext ctx, TableName 
tableName)
+    throws IOException;
+
+  // TODO: Move in HRegionFileSystem
+  public void deleteFamilyFromFS(HRegionInfo regionInfo, byte[] familyName, 
boolean hasMob)
+      throws IOException {
+    getRegionFileSystem(regionInfo).deleteFamily(Bytes.toString(familyName), 
hasMob);
+  }
+
+  public RegionFileSystem getRegionFileSystem(HRegionInfo regionInfo) throws 
IOException {
+    return RegionFileSystem.open(conf, fs, rootDir, regionInfo, false);
+  }
+
+  // ==========================================================================
+  //  PUBLIC Methods - visitors
+  // ==========================================================================
+  public void visitStoreFiles(StoreFileVisitor visitor)
+      throws IOException {
+    visitStoreFiles(FsContext.DATA, visitor);
+  }
+
+  public void visitStoreFiles(String namespace, StoreFileVisitor visitor)
+      throws IOException {
+    visitStoreFiles(FsContext.DATA, namespace, visitor);
+  }
+
+  public void visitStoreFiles(TableName table, StoreFileVisitor visitor)
+      throws IOException {
+    visitStoreFiles(FsContext.DATA, table, visitor);
+  }
+
+  public void visitStoreFiles(FsContext ctx, StoreFileVisitor visitor)
+      throws IOException {
+    for (String namespace: getNamespaces()) {
+      visitStoreFiles(ctx, namespace, visitor);
+    }
+  }
+
+  public void visitStoreFiles(FsContext ctx, String namespace, 
StoreFileVisitor visitor)
+      throws IOException {
+    for (TableName tableName: getTables(namespace)) {
+      visitStoreFiles(ctx, tableName, visitor);
+    }
+  }
+
+  public void visitStoreFiles(FsContext ctx, TableName table, StoreFileVisitor 
visitor)
+      throws IOException {
+    for (HRegionInfo hri: getRegions(ctx, table)) {
+      RegionFileSystem.open(conf, fs, rootDir, hri, 
false).visitStoreFiles(visitor);
+    }
+  }
+
+  // ==========================================================================
   //  PUBLIC Methods - bootstrap
   // ==========================================================================
   public abstract Path getTempDir();
@@ -244,44 +308,6 @@ public abstract class MasterFileSystem {
     bootstrapMeta();
   }
 
-  // TODO: Move in HRegionFileSystem
-  public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName, 
boolean hasMob)
-      throws IOException {
-    // archive family store files
-    Path tableDir = FSUtils.getTableDir(getRootDir(), region.getTable());
-    HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
-
-    // delete the family folder
-    Path familyDir = new Path(tableDir,
-      new Path(region.getEncodedName(), Bytes.toString(familyName)));
-    if (fs.delete(familyDir, true) == false) {
-      if (fs.exists(familyDir)) {
-        throw new IOException("Could not delete family "
-            + Bytes.toString(familyName) + " from FileSystem for region "
-            + region.getRegionNameAsString() + "(" + region.getEncodedName()
-            + ")");
-      }
-    }
-
-    // archive and delete mob files
-    if (hasMob) {
-      Path mobTableDir =
-          FSUtils.getTableDir(new Path(getRootDir(), 
MobConstants.MOB_DIR_NAME), region.getTable());
-      HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(region.getTable());
-      Path mobFamilyDir =
-          new Path(mobTableDir,
-              new Path(mobRegionInfo.getEncodedName(), 
Bytes.toString(familyName)));
-      // archive mob family store files
-      MobUtils.archiveMobStoreFiles(conf, fs, mobRegionInfo, mobFamilyDir, 
familyName);
-
-      if (!fs.delete(mobFamilyDir, true)) {
-        throw new IOException("Could not delete mob store files for family "
-            + Bytes.toString(familyName) + " from FileSystem region "
-            + mobRegionInfo.getRegionNameAsString() + "(" + 
mobRegionInfo.getEncodedName() + ")");
-      }
-    }
-  }
-
   // ==========================================================================
   //  PUBLIC
   // ==========================================================================

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/RegionFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/RegionFileSystem.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/RegionFileSystem.java
new file mode 100644
index 0000000..3b94355
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/RegionFileSystem.java
@@ -0,0 +1,183 @@
+/**
+ *
+ * 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.fs;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ClusterId;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.fs.legacy.LegacyRegionFileSystem;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.mob.MobUtils;
+
+@InterfaceAudience.Private
+public abstract class RegionFileSystem {
+  private static Log LOG = LogFactory.getLog(RegionFileSystem.class);
+
+  private final Configuration conf;
+  private final HRegionInfo hri;
+  private final FileSystem fs;
+  private final Path rootDir;
+
+  protected RegionFileSystem(Configuration conf, FileSystem fs, Path rootDir, 
HRegionInfo hri) {
+    this.conf = conf;
+    this.rootDir = rootDir;
+    this.hri = hri;
+    this.fs = fs;
+  }
+
+  public Configuration getConfiguration() { return conf; }
+  public FileSystem getFileSystem() { return fs; }
+  public Path getRootDir() { return rootDir; }
+
+  public HRegionInfo getRegionInfo() { return hri; }
+  public TableName getTable() { return getRegionInfo().getTable(); }
+
+  // ==========================================================================
+  //  PUBLIC Interfaces - Visitors
+  // ==========================================================================
+  public interface StoreFileVisitor {
+    void storeFile(HRegionInfo region, String family, StoreFileInfo storeFile)
+       throws IOException;
+  }
+
+  public void visitStoreFiles(final StoreFileVisitor visitor) throws 
IOException {
+    for (String familyName: getFamilies()) {
+      for (StoreFileInfo storeFile: getStoreFiles(familyName)) {
+        visitor.storeFile(getRegionInfo(), familyName, storeFile);
+      }
+    }
+  }
+
+  // ==========================================================================
+  //  PUBLIC Methods - Families Related
+  // ==========================================================================
+
+  /**
+   * @return the set of families present on disk
+   * @throws IOException
+   */
+  public abstract Collection<String> getFamilies() throws IOException;
+
+  public void deleteFamily(byte[] familyName, boolean hasMob) throws 
IOException {
+    deleteFamily(Bytes.toString(familyName), hasMob);
+  }
+
+  public abstract void deleteFamily(String familyName, boolean hasMob) throws 
IOException;
+
+  // ==========================================================================
+  //  PUBLIC Methods - Store Files related
+  // ==========================================================================
+
+  /**
+   * Returns the store files available for the family.
+   * This methods performs the filtering based on the valid store files.
+   * @param familyName Column Family Name
+   * @return a set of {@link StoreFileInfo} for the specified family.
+   */
+  public Collection<StoreFileInfo> getStoreFiles(final byte[] familyName) 
throws IOException {
+    return getStoreFiles(Bytes.toString(familyName));
+  }
+
+  public Collection<StoreFileInfo> getStoreFiles(final String familyName) 
throws IOException {
+    return getStoreFiles(familyName, true);
+  }
+
+  /**
+   * Returns the store files available for the family.
+   * This methods performs the filtering based on the valid store files.
+   * @param familyName Column Family Name
+   * @return a set of {@link StoreFileInfo} for the specified family.
+   */
+  public abstract Collection<StoreFileInfo> getStoreFiles(String familyName, 
boolean validate)
+      throws IOException;
+
+  // ==========================================================================
+  //  PUBLIC bootstrap
+  // ==========================================================================
+  protected abstract void bootstrap() throws IOException;
+  protected abstract void destroy() throws IOException;
+
+  // ==========================================================================
+  //  PUBLIC
+  // ==========================================================================
+  public static RegionFileSystem open(Configuration conf, HRegionInfo 
regionInfo, boolean bootstrap)
+      throws IOException {
+    return open(conf, FSUtils.getCurrentFileSystem(conf), 
FSUtils.getRootDir(conf),
+        regionInfo, bootstrap);
+  }
+
+  public static RegionFileSystem open(Configuration conf, FileSystem fs, Path 
rootDir,
+      HRegionInfo regionInfo, boolean bootstrap) throws IOException {
+    // Cover both bases, the old way of setting default fs and the new.
+    // We're supposed to run on 0.20 and 0.21 anyways.
+    fs = rootDir.getFileSystem(conf);
+    FSUtils.setFsDefault(conf, new Path(fs.getUri()));
+    // make sure the fs has the same conf
+    fs.setConf(conf);
+
+    RegionFileSystem rfs = getInstance(conf, fs, rootDir, regionInfo);
+    if (bootstrap) {
+      // TODO: are bootstrap and create two different things?
+      // should switch to bootstrap & read-only 
+      // legacy region wants to recover the .regioninfo :(
+    }
+    return rfs;
+  }
+
+  public static void destroy(Configuration conf, HRegionInfo regionInfo) 
throws IOException {
+    destroy(conf, FSUtils.getCurrentFileSystem(conf), 
FSUtils.getRootDir(conf), regionInfo);
+  }
+
+  public static void destroy(Configuration conf, FileSystem fs,
+      Path rootDir, HRegionInfo regionInfo) throws IOException {
+    getInstance(conf, fs, rootDir, regionInfo).destroy();
+  }
+
+  private static RegionFileSystem getInstance(Configuration conf, FileSystem 
fs,
+      Path rootDir, HRegionInfo regionInfo) throws IOException {
+    String fsType = conf.get("hbase.fs.layout.type", "legacy").toLowerCase();
+    switch (fsType) {
+      case "legacy":
+        return new LegacyRegionFileSystem(conf, fs, rootDir, regionInfo);
+      default:
+        throw new IOException("Invalid filesystem type " + fsType);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fe2918ba/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/HFileArchiveUtil.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/HFileArchiveUtil.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/HFileArchiveUtil.java
new file mode 100644
index 0000000..2fbaa92
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/HFileArchiveUtil.java
@@ -0,0 +1,156 @@
+/**
+ * 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);
+  }
+}

Reply via email to