You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2015/12/27 19:02:17 UTC

[1/6] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov)

Repository: hbase
Updated Branches:
  refs/heads/master 5eefe1317 -> de69f0df3


http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
new file mode 100644
index 0000000..ad23f9b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
@@ -0,0 +1,85 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestRestoreBoundaryTests extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRestoreBoundaryTests.class);
+
+  /**
+   * Verify that a single empty table is restored to a new table
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreSingleEmpty() throws Exception {
+
+    LOG.info("test full restore on a single table empty table");
+    String backupId =
+ BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+    String[] tableset = new String[] { table1.getNameAsString() };
+    String[] tablemap = new String[] { table1_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
+      false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(TableName.valueOf(table1_restore)));
+    TEST_UTIL.deleteTable(TableName.valueOf(table1_restore));
+  }
+
+  /**
+   * Verify that multiple tables are restored to new tables.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreMultipleEmpty() throws Exception {
+    LOG.info("create full backup image on multiple tables");
+    String tableset =
+        table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table3.getNameAsString();
+    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
+    assertTrue(checkSucceeded(backupId));
+    String[] restore_tableset = new String[] { table2.getNameAsString(), table3.getNameAsString() };
+    String[] tablemap = new String[] { table2_restore, table3_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, restore_tableset,
+      tablemap,
+      false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(TableName.valueOf(table2_restore)));
+    assertTrue(hba.tableExists(TableName.valueOf(table3_restore)));
+    TEST_UTIL.deleteTable(TableName.valueOf(table2_restore));
+    TEST_UTIL.deleteTable(TableName.valueOf(table3_restore));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
index 7620bbb..cd2efad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
@@ -49,7 +49,7 @@ public class SimpleRSProcedureManager extends RegionServerProcedureManager {
   private ProcedureMember member;
 
   @Override
-  public void initialize(RegionServerServices rss) throws KeeperException {
+  public void initialize(RegionServerServices rss) throws IOException {
     this.rss = rss;
     ZooKeeperWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw, getProcedureSignature());


[3/6] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
new file mode 100644
index 0000000..ae21b33
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
@@ -0,0 +1,496 @@
+/**
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ * The main class which interprets the given arguments and trigger restore operation.
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class RestoreClient {
+
+  private static final Log LOG = LogFactory.getLog(RestoreClient.class);
+
+  private static Options opt;
+  private static Configuration conf;
+  private static Set<BackupImage> lastRestoreImagesSet;
+
+  // delimiter in tablename list in restore command
+  private static final String DELIMITER_IN_COMMAND = ",";
+
+  private static final String OPTION_OVERWRITE = "overwrite";
+  private static final String OPTION_CHECK = "check";
+  private static final String OPTION_AUTOMATIC = "automatic";
+
+  private static final String USAGE =
+      "Usage: hbase restore <backup_root_path> <backup_id> <tables> [tableMapping] \n"
+          + "       [-overwrite] [-check] [-automatic]\n"
+          + " backup_root_path  The parent location where the backup images are stored\n"
+          + " backup_id         The id identifying the backup image\n"
+          + " table(s)          Table(s) from the backup image to be restored.\n"
+          + "                   Tables are separated by comma.\n"
+          + " Options:\n"
+          + "   tableMapping    A comma separated list of target tables.\n"
+          + "                   If specified, each table in <tables> must have a mapping.\n"
+          + "   -overwrite      With this option, restore overwrites to the existing table "
+          + "if there's any in\n"
+          + "                   restore target. The existing table must be online before restore.\n"
+          + "   -check          With this option, restore sequence and dependencies are checked\n"
+          + "                   and verified without executing the restore\n"
+          + "   -automatic      With this option, all the dependencies are automatically restored\n"
+          + "                   together with this backup image following the correct order.\n"
+          + "                   The restore dependencies can be checked by using \"-check\" "
+          + "option,\n"
+          + "                   or using \"hbase backup describe\" command. Without this option, "
+          + "only\n" + "                   this backup image is restored\n";
+
+  private RestoreClient(){
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  protected static void init() throws IOException {
+    // define supported options
+    opt = new Options();
+    opt.addOption(OPTION_OVERWRITE, false,
+        "Overwrite the data if any of the restore target tables exists");
+    opt.addOption(OPTION_CHECK, false, "Check restore sequence and dependencies");
+    opt.addOption(OPTION_AUTOMATIC, false, "Restore all dependencies");
+    opt.addOption("debug", false, "Enable debug logging");
+
+    conf = getConf();
+
+    // disable irrelevant loggers to avoid it mess up command output
+    disableUselessLoggers();
+  }
+
+  public static void main(String[] args) throws IOException {
+    init();
+    parseAndRun(args);
+  }
+
+  private static void parseAndRun(String[] args) {
+    CommandLine cmd = null;
+    try {
+      cmd = new PosixParser().parse(opt, args);
+    } catch (ParseException e) {
+      LOG.error("Could not parse command", e);
+      System.exit(-1);
+    }
+
+    // enable debug logging
+    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+    if (cmd.hasOption("debug")) {
+      backupClientLogger.setLevel(Level.DEBUG);
+    }
+
+    // whether to overwrite to existing table if any, false by default
+    boolean isOverwrite = cmd.hasOption(OPTION_OVERWRITE);
+    if (isOverwrite) {
+      LOG.debug("Found -overwrite option in restore command, "
+          + "will overwrite to existing table if any in the restore target");
+    }
+
+    // whether to only check the dependencies, false by default
+    boolean check = cmd.hasOption(OPTION_CHECK);
+    if (check) {
+      LOG.debug("Found -check option in restore command, "
+          + "will check and verify the dependencies");
+    }
+
+    // whether to restore all dependencies, false by default
+    boolean autoRestore = cmd.hasOption(OPTION_AUTOMATIC);
+    if (autoRestore) {
+      LOG.debug("Found -automatic option in restore command, "
+          + "will automatically retore all the dependencies");
+    }
+
+    // parse main restore command options
+    String[] remainArgs = cmd.getArgs();
+    if (remainArgs.length < 3) {
+      System.out.println("ERROR: missing arguments");
+      System.out.println(USAGE);
+      System.exit(-1);
+    }
+
+    String backupRootDir = remainArgs[0];
+    String backupId = remainArgs[1];
+    String tables = remainArgs[2];
+
+    String tableMapping = (remainArgs.length > 3) ? remainArgs[3] : null;
+
+    String[] sTableArray = (tables != null) ? tables.split(DELIMITER_IN_COMMAND) : null;
+    String[] tTableArray = (tableMapping != null) ? tableMapping.split(DELIMITER_IN_COMMAND) : null;
+
+    if (tableMapping != null && tTableArray != null && (sTableArray.length != tTableArray.length)) {
+      System.err.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping);
+      System.out.println(USAGE);
+      System.exit(-1);
+    }
+
+    try {
+      HBackupFileSystem hBackupFS = new HBackupFileSystem(conf, new Path(backupRootDir), backupId);
+      restore_stage1(hBackupFS, backupRootDir, backupId, check, autoRestore, sTableArray,
+        tTableArray, isOverwrite);
+    } catch (IOException e) {
+      System.err.println("ERROR: " + e.getMessage());
+      System.exit(-1);
+    }
+  }
+
+  /**
+   * Restore operation. Stage 1: validate backupManifest, and check target tables
+   * @param hBackupFS to access the backup image
+   * @param backupRootDir The root dir for backup image
+   * @param backupId The backup id for image to be restored
+   * @param check True if only do dependency check
+   * @param autoRestore True if automatically restore following the dependency
+   * @param sTableArray The array of tables to be restored
+   * @param tTableArray The array of mapping tables to restore to
+   * @param isOverwrite True then do restore overwrite if target table exists, otherwise fail the
+   *          request if target table exists
+   * @return True if only do dependency check
+   * @throws IOException if any failure during restore
+   */
+  public static boolean restore_stage1(HBackupFileSystem hBackupFS, String backupRootDir,
+      String backupId, boolean check, boolean autoRestore, String[] sTableArray,
+      String[] tTableArray, boolean isOverwrite) throws IOException {
+
+    HashMap<String, BackupManifest> backupManifestMap = new HashMap<String, BackupManifest>();
+    // check and load backup image manifest for the tables
+    hBackupFS.checkImageManifestExist(backupManifestMap, sTableArray);
+
+    try {
+      // Check and validate the backup image and its dependencies
+      if (check || autoRestore) {
+        if (validate(backupManifestMap)) {
+          LOG.info("Checking backup images: ok");
+        } else {
+          String errMsg = "Some dependencies are missing for restore";
+          LOG.error(errMsg);
+          throw new IOException(errMsg);
+        }
+      }
+
+      // return true if only for check
+      if (check) {
+        return true;
+      }
+
+      if (tTableArray == null) {
+        tTableArray = sTableArray;
+      }
+
+      // check the target tables
+      checkTargetTables(tTableArray, isOverwrite);
+
+      // start restore process
+      Set<BackupImage> restoreImageSet =
+          restore_stage2(hBackupFS, backupManifestMap, sTableArray, tTableArray, autoRestore);
+
+      LOG.info("Restore for " + Arrays.asList(sTableArray) + " are successful!");
+      lastRestoreImagesSet = restoreImageSet;
+
+    } catch (IOException e) {
+      LOG.error("ERROR: restore failed with error: " + e.getMessage());
+      throw e;
+    }
+
+    // not only for check, return false
+    return false;
+  }
+
+  /**
+   * Get last restore image set. The value is globally set for the latest finished restore.
+   * @return the last restore image set
+   */
+  public static Set<BackupImage> getLastRestoreImagesSet() {
+    return lastRestoreImagesSet;
+  }
+
+  private static boolean validate(HashMap<String, BackupManifest> backupManifestMap)
+      throws IOException {
+    boolean isValid = true;
+
+    for (Entry<String, BackupManifest> manifestEntry : backupManifestMap.entrySet()) {
+
+      String table = manifestEntry.getKey();
+      TreeSet<BackupImage> imageSet = new TreeSet<BackupImage>();
+
+      ArrayList<BackupImage> depList = manifestEntry.getValue().getDependentListByTable(table);
+      if (depList != null && !depList.isEmpty()) {
+        imageSet.addAll(depList);
+      }
+
+      // todo merge
+      LOG.debug("merge will be implemented in future jira");
+      // BackupUtil.clearMergedImages(table, imageSet, conf);
+
+      LOG.info("Dependent image(s) from old to new:");
+      for (BackupImage image : imageSet) {
+        String imageDir =
+            HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table);
+        if (!HBackupFileSystem.checkPathExist(imageDir, getConf())) {
+          LOG.error("ERROR: backup image does not exist: " + imageDir);
+          isValid = false;
+          break;
+        }
+        // TODO More validation?
+        LOG.info("Backup image: " + image.getBackupId() + " for '" + table + "' is available");
+      }
+    }
+    return isValid;
+  }
+
+  /**
+   * Validate target Tables
+   * @param tTableArray: target tables
+   * @param isOverwrite overwrite existing table
+   * @throws IOException exception
+   */
+  private static void checkTargetTables(String[] tTableArray, boolean isOverwrite)
+      throws IOException {
+    ArrayList<String> existTableList = new ArrayList<String>();
+    ArrayList<String> disabledTableList = new ArrayList<String>();
+
+    // check if the tables already exist
+    HBaseAdmin admin = null;
+    Connection conn = null;
+    try {
+      conn = ConnectionFactory.createConnection(conf);
+      admin = (HBaseAdmin) conn.getAdmin();
+      for (String tableName : tTableArray) {
+        if (admin.tableExists(TableName.valueOf(tableName))) {
+          existTableList.add(tableName);
+          if (admin.isTableDisabled(TableName.valueOf(tableName))) {
+            disabledTableList.add(tableName);
+          }
+        } else {
+          LOG.info("HBase table " + tableName
+              + " does not exist. It will be create during backup process");
+        }
+      }
+    } finally {
+      if (admin != null) {
+        admin.close();
+      }
+      if (conn != null) {
+        conn.close();
+      }
+    }
+
+    if (existTableList.size() > 0) {
+      if (!isOverwrite) {
+        LOG.error("Existing table found in the restore target, please add \"-overwrite\" "
+            + "option in the command if you mean to restore to these existing tables");
+        LOG.info("Existing table list in restore target: " + existTableList);
+        throw new IOException("Existing table found in target while no \"-overwrite\" "
+            + "option found");
+      } else {
+        if (disabledTableList.size() > 0) {
+          LOG.error("Found offline table in the restore target, "
+              + "please enable them before restore with \"-overwrite\" option");
+          LOG.info("Offline table list in restore target: " + disabledTableList);
+          throw new IOException(
+              "Found offline table in the target when restore with \"-overwrite\" option");
+        }
+      }
+    }
+
+  }
+
+  /**
+   * Restore operation. Stage 2: resolved Backup Image dependency
+   * @param hBackupFS to access the backup image
+   * @param backupManifestMap : tableName,  Manifest
+   * @param sTableArray The array of tables to be restored
+   * @param tTableArray The array of mapping tables to restore to
+   * @param autoRestore : yes, restore all the backup images on the dependency list
+   * @return set of BackupImages restored
+   * @throws IOException exception
+   */
+  private static Set<BackupImage> restore_stage2(HBackupFileSystem hBackupFS,
+    HashMap<String, BackupManifest> backupManifestMap, String[] sTableArray,
+    String[] tTableArray, boolean autoRestore) throws IOException {
+    TreeSet<BackupImage> restoreImageSet = new TreeSet<BackupImage>();
+
+    for (int i = 0; i < sTableArray.length; i++) {
+      restoreImageSet.clear();
+      String table = sTableArray[i];
+      BackupManifest manifest = backupManifestMap.get(table);
+      if (autoRestore) {
+        // Get the image list of this backup for restore in time order from old
+        // to new.
+        TreeSet<BackupImage> restoreList =
+            new TreeSet<BackupImage>(manifest.getDependentListByTable(table));
+        LOG.debug("need to clear merged Image. to be implemented in future jira");
+
+        for (BackupImage image : restoreList) {
+          restoreImage(image, table, tTableArray[i]);
+        }
+        restoreImageSet.addAll(restoreList);
+      } else {
+        BackupImage image = manifest.getBackupImage();
+        List<BackupImage> depList = manifest.getDependentListByTable(table);
+        // The dependency list always contains self.
+        if (depList != null && depList.size() > 1) {
+          LOG.warn("Backup image " + image.getBackupId() + " depends on other images.\n"
+              + "this operation will only restore the delta contained within backupImage "
+              + image.getBackupId());
+        }
+        restoreImage(image, table, tTableArray[i]);
+        restoreImageSet.add(image);
+      }
+
+      if (autoRestore) {
+        if (restoreImageSet != null && !restoreImageSet.isEmpty()) {
+          LOG.info("Restore includes the following image(s):");
+          for (BackupImage image : restoreImageSet) {
+            LOG.info("  Backup: "
+                + image.getBackupId()
+                + " "
+                + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(),
+                  table));
+          }
+        }
+      }
+
+    }
+    return restoreImageSet;
+  }
+
+  /**
+   * Restore operation handle each backupImage
+   * @param image: backupImage
+   * @param sTable: table to be restored
+   * @param tTable: table to be restored to
+   * @throws IOException exception
+   */
+  private static void restoreImage(BackupImage image, String sTable, String tTable)
+      throws IOException {
+
+    Configuration conf = getConf();
+
+    String rootDir = image.getRootDir();
+    LOG.debug("Image root dir " + rootDir);
+    String backupId = image.getBackupId();
+
+    HBackupFileSystem hFS = new HBackupFileSystem(conf, new Path(rootDir), backupId);
+    RestoreUtil restoreTool = new RestoreUtil(conf, hFS);
+    BackupManifest manifest = hFS.getManifest(sTable);
+
+    Path tableBackupPath = hFS.getTableBackupPath(sTable);
+
+    // todo: convert feature will be provided in a future jira
+    boolean converted = false;
+
+    if (manifest.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL) || converted) {
+      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from "
+          + (converted ? "converted" : "full") + " backup image " + tableBackupPath.toString());
+      restoreTool.fullRestoreTable(tableBackupPath, sTable, tTable, converted);
+    } else { // incremental Backup
+      String logBackupDir =
+          HBackupFileSystem.getLogBackupDir(image.getRootDir(), image.getBackupId());
+      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from incremental backup image "
+          + logBackupDir);
+      restoreTool.incrementalRestoreTable(logBackupDir, new String[] { sTable },
+        new String[] { tTable });
+    }
+
+    LOG.info(sTable + " has been successfully restored to " + tTable);
+  }
+
+  /**
+   * Set the configuration from a given one.
+   * @param newConf A new given configuration
+   */
+  public synchronized static void setConf(Configuration newConf) {
+    conf = newConf;
+  }
+
+  /**
+   * Get and merge Hadoop and HBase configuration.
+   * @throws IOException exception
+   */
+  protected static Configuration getConf() {
+    if (conf == null) {
+      synchronized (RestoreClient.class) {
+        conf = new Configuration();
+        HBaseConfiguration.merge(conf, HBaseConfiguration.create());
+      }
+    }
+    return conf;
+  }
+
+  private static void disableUselessLoggers() {
+    // disable zookeeper log to avoid it mess up command output
+    Logger zkLogger = Logger.getLogger("org.apache.zookeeper");
+    LOG.debug("Zookeeper log level before set: " + zkLogger.getLevel());
+    zkLogger.setLevel(Level.OFF);
+    LOG.debug("Zookeeper log level after set: " + zkLogger.getLevel());
+
+    // disable hbase zookeeper tool log to avoid it mess up command output
+    Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper");
+    LOG.debug("HBase zookeeper log level before set: " + hbaseZkLogger.getLevel());
+    hbaseZkLogger.setLevel(Level.OFF);
+    LOG.debug("HBase Zookeeper log level after set: " + hbaseZkLogger.getLevel());
+
+    // disable hbase client log to avoid it mess up command output
+    Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client");
+    LOG.debug("HBase client log level before set: " + hbaseClientLogger.getLevel());
+    hbaseClientLogger.setLevel(Level.OFF);
+    LOG.debug("HBase client log level after set: " + hbaseClientLogger.getLevel());
+
+    // disable other related log to avoid mess up command output
+    Logger otherLogger = Logger.getLogger("org.apache.hadoop.hbase.io.hfile");
+    otherLogger.setLevel(Level.OFF);
+    otherLogger = Logger.getLogger("org.apache.hadoop.hbase.util");
+    otherLogger.setLevel(Level.OFF);
+    otherLogger = Logger.getLogger("org.apache.hadoop.hbase.mapreduce");
+    otherLogger.setLevel(Level.OFF);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java
new file mode 100644
index 0000000..bdb7988
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java
@@ -0,0 +1,503 @@
+/**
+ * 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.EOFException;
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.NavigableSet;
+
+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;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALSplitter;
+
+/**
+ * A collection for methods used by multiple classes to restore HBase tables.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RestoreUtil {
+
+  public static final Log LOG = LogFactory.getLog(RestoreUtil.class);
+
+  protected Configuration conf = null;
+
+  protected HBackupFileSystem hBackupFS = null;
+
+  // store table name and snapshot dir mapping
+  private final HashMap<String, Path> snapshotMap = new HashMap<String, Path>();
+
+  public RestoreUtil(Configuration conf, HBackupFileSystem hBackupFS) throws IOException {
+    this.conf = conf;
+    this.hBackupFS = hBackupFS;
+  }
+
+  /**
+   * During incremental backup operation. Call WalPlayer to replay WAL in backup image Currently
+   * tableNames and newTablesNames only contain single table, will be expanded to multiple tables in
+   * the future
+   * @param logDir : incremental backup folders, which contains WAL
+   * @param tableNames : source tableNames(table names were backuped)
+   * @param newTableNames : target tableNames(table names to be restored to)
+   * @throws IOException exception
+   */
+  public void incrementalRestoreTable(String logDir, String[] tableNames, String[] newTableNames)
+      throws IOException {
+
+    if (tableNames.length != newTableNames.length) {
+      throw new IOException("Number of source tables adn taget Tables does not match!");
+    }
+
+    // for incremental backup image, expect the table already created either by user or previous
+    // full backup. Here, check that all new tables exists
+    HBaseAdmin admin = null;
+    Connection conn = null;
+    try {
+      conn = ConnectionFactory.createConnection(conf);
+      admin = (HBaseAdmin) conn.getAdmin();
+      for (String tableName : newTableNames) {
+        if (!admin.tableExists(TableName.valueOf(tableName))) {
+          admin.close();
+          throw new IOException("HBase table " + tableName
+            + " does not exist. Create the table first, e.g. by restoring a full backup.");
+        }
+      }
+      IncrementalRestoreService restoreService =
+          BackupRestoreServiceFactory.getIncrementalRestoreService(conf);
+
+      restoreService.run(logDir, tableNames, newTableNames);
+    } finally {
+      if (admin != null) {
+        admin.close();
+      }
+      if(conn != null){
+        conn.close();
+      }
+    }
+  }
+
+  public void fullRestoreTable(Path tableBackupPath, String tableName, String newTableName,
+      boolean converted) throws IOException {
+
+    restoreTableAndCreate(tableName, newTableName, tableBackupPath, converted);
+  }
+
+  private void restoreTableAndCreate(String tableName, String newTableName, Path tableBackupPath,
+       boolean converted) throws IOException {
+    if (newTableName == null || newTableName.equals("")) {
+      newTableName = tableName;
+    }
+
+    FileSystem fileSys = tableBackupPath.getFileSystem(this.conf);
+
+    // get table descriptor first
+    HTableDescriptor tableDescriptor = null;
+
+    Path tableSnapshotPath = hBackupFS.getTableSnapshotPath(tableName);
+
+    if (fileSys.exists(tableSnapshotPath)) {
+      // snapshot path exist means the backup path is in HDFS
+      // check whether snapshot dir already recorded for target table
+      if (snapshotMap.get(tableName) != null) {
+        SnapshotDescription desc =
+            SnapshotDescriptionUtils.readSnapshotInfo(fileSys, tableSnapshotPath);
+        SnapshotManifest manifest = SnapshotManifest.open(conf, fileSys, tableSnapshotPath, desc);
+        tableDescriptor = manifest.getTableDescriptor();
+        LOG.debug("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString()
+          + " while tableName = " + tableName);
+        // HBase 96.0 and 98.0
+        // tableDescriptor =
+        // FSTableDescriptors.getTableDescriptorFromFs(fileSys, snapshotMap.get(tableName));
+      } else {
+        tableDescriptor = hBackupFS.getTableDesc(tableName);
+        LOG.debug("tableSnapshotPath=" + tableSnapshotPath.toString());
+        snapshotMap.put(tableName, hBackupFS.getTableInfoPath(tableName));
+      }
+      if (tableDescriptor == null) {
+        LOG.debug("Found no table descriptor in the snapshot dir, previous schema would be lost");
+      }
+    } else if (converted) {
+      // first check if this is a converted backup image
+      LOG.error("convert will be supported in a future jira");
+    }
+
+    Path tableArchivePath = hBackupFS.getTableArchivePath(tableName);
+    if (tableArchivePath == null) {
+      if (tableDescriptor != null) {
+        // find table descriptor but no archive dir means the table is empty, create table and exit
+        LOG.debug("find table descriptor but no archive dir for table " + tableName
+          + ", will only create table");
+        tableDescriptor.setName(Bytes.toBytes(newTableName));
+        checkAndCreateTable(tableBackupPath, tableName, newTableName, null, tableDescriptor);
+        return;
+      } else {
+        throw new IllegalStateException("Cannot restore hbase table because directory '"
+            + " tableArchivePath is null.");
+      }
+    }
+
+    if (tableDescriptor == null) {
+      tableDescriptor = new HTableDescriptor(newTableName);
+    } else {
+      tableDescriptor.setName(Bytes.toBytes(newTableName));
+    }
+
+    if (!converted) {
+      // record all region dirs:
+      // load all files in dir
+      try {
+        ArrayList<Path> regionPathList = hBackupFS.getRegionList(tableName);
+
+        // should only try to create the table with all region informations, so we could pre-split
+        // the regions in fine grain
+        checkAndCreateTable(tableBackupPath, tableName, newTableName, regionPathList,
+          tableDescriptor);
+        if (tableArchivePath != null) {
+          // start real restore through bulkload
+          // if the backup target is on local cluster, special action needed
+          Path tempTableArchivePath = hBackupFS.checkLocalAndBackup(tableArchivePath);
+          if (tempTableArchivePath.equals(tableArchivePath)) {
+            LOG.debug("TableArchivePath for bulkload using existPath: " + tableArchivePath);
+          } else {
+            regionPathList = hBackupFS.getRegionList(tempTableArchivePath); // point to the tempDir
+            LOG.debug("TableArchivePath for bulkload using tempPath: " + tempTableArchivePath);
+          }
+
+          LoadIncrementalHFiles loader = createLoader(tempTableArchivePath, false);
+          for (Path regionPath : regionPathList) {
+            String regionName = regionPath.toString();
+            LOG.debug("Restoring HFiles from directory " + regionName);
+            String[] args = { regionName, newTableName };
+            loader.run(args);
+          }
+        }
+        // restore the recovered.edits if exists
+        replayRecoveredEditsIfAny(tableBackupPath, tableName, tableDescriptor);
+      } catch (Exception e) {
+        throw new IllegalStateException("Cannot restore hbase table", e);
+      }
+    } else {
+      LOG.debug("convert will be supported in a future jira");
+    }
+  }
+
+  /**
+   * Replay recovered edits from backup.
+   */
+  private void replayRecoveredEditsIfAny(Path tableBackupPath, String tableName,
+      HTableDescriptor newTableHtd) throws IOException {
+
+    LOG.debug("Trying to replay the recovered.edits if exist to the target table "
+        + newTableHtd.getNameAsString() + " from the backup of table " + tableName + ".");
+
+    FileSystem fs = tableBackupPath.getFileSystem(this.conf);
+    ArrayList<Path> regionDirs = hBackupFS.getRegionList(tableName);
+
+    if (regionDirs == null || regionDirs.size() == 0) {
+      LOG.warn("No recovered.edits to be replayed for empty backup of table " + tableName + ".");
+      return;
+    }
+
+    Connection conn = null;
+    try {
+
+      conn = ConnectionFactory.createConnection(conf);
+
+      for (Path regionDir : regionDirs) {
+        // OLD: NavigableSet<Path> files = HLogUtil.getSplitEditFilesSorted(fs, regionDir);
+        NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regionDir);
+
+        if (files == null || files.isEmpty()) {
+          LOG.warn("No recovered.edits found for the region " + regionDir.getName() + ".");
+          return;
+        }
+
+        for (Path edits : files) {
+          if (edits == null || !fs.exists(edits)) {
+            LOG.warn("Null or non-existent edits file: " + edits);
+            continue;
+          }
+
+          HTable table = null;
+          try {
+            table = (HTable) conn.getTable(newTableHtd.getTableName());
+            replayRecoveredEdits(table, fs, edits);
+            table.flushCommits();
+            table.close();
+          } catch (IOException e) {
+            boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
+            if (skipErrors) {
+              Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
+              LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
+                + "=true so continuing. Renamed " + edits + " as " + p, e);
+            } else {
+              throw e;
+            }
+          } finally {
+            if (table != null) {
+              table.close();
+            }
+          }
+        } // for each edit file under a region
+      } // for each region
+
+    } finally {
+      if (conn != null) {
+        conn.close();
+      }
+    }
+  }
+
+  /**
+   * Restore process for an edit entry.
+   * @param htable The target table of restore
+   * @param key HLog key
+   * @param val KVs
+   * @throws IOException exception
+   */
+  private void restoreEdit(HTable htable, WALKey key, WALEdit val) throws IOException {
+    Put put = null;
+    Delete del = null;
+    Cell lastKV = null;
+    for (Cell kv : val.getCells()) {
+      // filtering HLog meta entries, see HLog.completeCacheFlushLogEdit
+      if (WALEdit.isMetaEditFamily(CellUtil.cloneFamily(kv))) {
+        continue;
+      }
+
+      // A WALEdit may contain multiple operations (HBASE-3584) and/or
+      // multiple rows (HBASE-5229).
+      // Aggregate as much as possible into a single Put/Delete
+      // operation before apply the action to the table.
+      if (lastKV == null || lastKV.getTypeByte() != kv.getTypeByte()
+          || !CellUtil.matchingRow(lastKV, kv)) {
+        // row or type changed, write out aggregate KVs.
+        if (put != null) {
+          applyAction(htable, put);
+        }
+        if (del != null) {
+          applyAction(htable, del);
+        }
+
+        if (CellUtil.isDelete(kv)) {
+          del = new Delete(CellUtil.cloneRow(kv));
+        } else {
+          put = new Put(CellUtil.cloneRow(kv));
+        }
+      }
+      if (CellUtil.isDelete(kv)) {
+        del.addDeleteMarker(kv);
+      } else {
+        put.add(kv);
+      }
+      lastKV = kv;
+    }
+    // write residual KVs
+    if (put != null) {
+      applyAction(htable, put);
+    }
+    if (del != null) {
+      applyAction(htable, del);
+    }
+  }
+
+  /**
+   * Apply an action (Put/Delete) to table.
+   * @param table table
+   * @param action action
+   * @throws IOException exception
+   */
+  private void applyAction(HTable table, Mutation action) throws IOException {
+    // The actions are not immutable, so we defensively copy them
+    if (action instanceof Put) {
+      Put put = new Put((Put) action);
+      // put.setWriteToWAL(false);
+      // why do not we do WAL?
+      put.setDurability(Durability.SKIP_WAL);
+      table.put(put);
+    } else if (action instanceof Delete) {
+      Delete delete = new Delete((Delete) action);
+      table.delete(delete);
+    } else {
+      throw new IllegalArgumentException("action must be either Delete or Put");
+    }
+  }
+
+  /**
+   * Replay the given edits.
+   * @param htable The target table of restore
+   * @param fs File system
+   * @param edits Recovered.edits to be replayed
+   * @throws IOException exception
+   */
+  private void replayRecoveredEdits(HTable htable, FileSystem fs, Path edits) throws IOException {
+    LOG.debug("Replaying edits from " + edits + "; path=" + edits);
+
+    WAL.Reader reader = null;
+    try {
+      reader = WALFactory.createReader(fs, edits, this.conf);
+      long editsCount = 0;
+      WAL.Entry entry;
+
+      try {
+        while ((entry = reader.next()) != null) {
+          restoreEdit(htable, entry.getKey(), entry.getEdit());
+          editsCount++;
+        }
+        LOG.debug(editsCount + " edits from " + edits + " have been replayed.");
+
+      } catch (EOFException eof) {
+        Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
+        String msg =
+            "Encountered EOF. Most likely due to Master failure during "
+                + "log spliting, so we have this data in another edit.  "
+                + "Continuing, but renaming " + edits + " as " + p;
+        LOG.warn(msg, eof);
+      } catch (IOException ioe) {
+        // If the IOE resulted from bad file format,
+        // then this problem is idempotent and retrying won't help
+        if (ioe.getCause() instanceof ParseException) {
+          Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
+          String msg =
+              "File corruption encountered!  " + "Continuing, but renaming " + edits + " as " + p;
+          LOG.warn(msg, ioe);
+        } else {
+          // other IO errors may be transient (bad network connection,
+          // checksum exception on one datanode, etc). throw & retry
+          throw ioe;
+        }
+      }
+    } finally {
+      if (reader != null) {
+        reader.close();
+      }
+    }
+  }
+
+  /**
+   * Create a {@link LoadIncrementalHFiles} instance to be used to restore the HFiles of a full
+   * backup.
+   * @return the {@link LoadIncrementalHFiles} instance
+   * @throws IOException exception
+   */
+  private LoadIncrementalHFiles createLoader(Path tableArchivePath, boolean multipleTables)
+      throws IOException {
+    // set configuration for restore:
+    // LoadIncrementalHFile needs more time
+    // <name>hbase.rpc.timeout</name> <value>600000</value>
+    // calculates
+    Integer milliSecInMin = 60000;
+    Integer previousMillis = this.conf.getInt("hbase.rpc.timeout", 0);
+    Integer numberOfFilesInDir =
+        multipleTables ? hBackupFS.getMaxNumberOfFilesInSubDir(tableArchivePath) : hBackupFS
+            .getNumberOfFilesInDir(tableArchivePath);
+    Integer calculatedMillis = numberOfFilesInDir * milliSecInMin; // 1 minute per file
+    Integer resultMillis = Math.max(calculatedMillis, previousMillis);
+    if (resultMillis > previousMillis) {
+      LOG.info("Setting configuration for restore with LoadIncrementalHFile: "
+          + "hbase.rpc.timeout to " + calculatedMillis / milliSecInMin
+          + " minutes, to handle the number of files in backup " + tableArchivePath);
+      this.conf.setInt("hbase.rpc.timeout", resultMillis);
+    }
+
+    LoadIncrementalHFiles loader = null;
+    try {
+      loader = new LoadIncrementalHFiles(this.conf);
+    } catch (Exception e1) {
+      throw new IOException(e1);
+    }
+    return loader;
+  }
+
+  /**
+   * Prepare the table for bulkload, most codes copied from
+   * {@link LoadIncrementalHFiles#createTable(String, String)}
+   * @param tableBackupPath path
+   * @param tableName table name
+   * @param targetTableName target table name
+   * @param regionDirList region directory list
+   * @param htd table descriptor
+   * @throws IOException exception
+   */
+  private void checkAndCreateTable(Path tableBackupPath, String tableName, String targetTableName,
+      ArrayList<Path> regionDirList, HTableDescriptor htd) throws IOException {
+    HBaseAdmin hbadmin = null;
+    Connection conn = null;
+    try {
+      conn = ConnectionFactory.createConnection(conf);
+      hbadmin = (HBaseAdmin) conn.getAdmin();
+      if (hbadmin.tableExists(TableName.valueOf(targetTableName))) {
+        LOG.info("Using exising target table '" + targetTableName + "'");
+      } else {
+        LOG.info("Creating target table '" + targetTableName + "'");
+
+        // if no region dir given, create the table and return
+        if (regionDirList == null || regionDirList.size() == 0) {
+
+          hbadmin.createTable(htd);
+          return;
+        }
+
+        byte[][] keys = hBackupFS.generateBoundaryKeys(regionDirList);
+
+        // create table using table decriptor and region boundaries
+        hbadmin.createTable(htd, keys);
+      }
+    } catch (Exception e) {
+      throw new IOException(e);
+    } finally {
+      if (hbadmin != null) {
+        hbadmin.close();
+      }
+      if(conn != null){
+        conn.close();
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java
new file mode 100644
index 0000000..a3b5db5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java
@@ -0,0 +1,292 @@
+/**
+ * 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.mapreduce;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.math.BigDecimal;
+import java.util.Arrays;
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.backup.BackupCopyService;
+import org.apache.hadoop.hbase.backup.BackupHandler;
+import org.apache.hadoop.hbase.backup.BackupUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.snapshot.ExportSnapshot;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.tools.DistCp;
+import org.apache.hadoop.tools.DistCpConstants;
+import org.apache.hadoop.tools.DistCpOptions;
+/**
+ * Copier for backup operation. Basically, there are 2 types of copy. One is copying from snapshot,
+ * which bases on extending ExportSnapshot's function with copy progress reporting to ZooKeeper
+ * implementation. The other is copying for incremental log files, which bases on extending 
+ * DistCp's function with copy progress reporting to ZooKeeper implementation.
+ *
+ * For now this is only a wrapper. The other features such as progress and increment backup will be
+ * implemented in future jira
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MapReduceBackupCopyService implements BackupCopyService {
+  private static final Log LOG = LogFactory.getLog(MapReduceBackupCopyService.class);
+
+  private Configuration conf;
+  // private static final long BYTES_PER_MAP = 2 * 256 * 1024 * 1024;
+
+  // Accumulated progress within the whole backup process for the copy operation
+  private float progressDone = 0.1f;
+  private long bytesCopied = 0;
+  private static float INIT_PROGRESS = 0.1f;
+
+  // The percentage of the current copy task within the whole task if multiple time copies are
+  // needed. The default value is 100%, which means only 1 copy task for the whole.
+  private float subTaskPercntgInWholeTask = 1f;
+
+  public MapReduceBackupCopyService() {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Get the current copy task percentage within the whole task if multiple copies are needed.
+   * @return the current copy task percentage
+   */
+  public float getSubTaskPercntgInWholeTask() {
+    return subTaskPercntgInWholeTask;
+  }
+
+  /**
+   * Set the current copy task percentage within the whole task if multiple copies are needed. Must
+   * be called before calling
+   * {@link #copy(BackupHandler, Configuration, Type, String[])}
+   * @param subTaskPercntgInWholeTask The percentage of the copy subtask
+   */
+  public void setSubTaskPercntgInWholeTask(float subTaskPercntgInWholeTask) {
+    this.subTaskPercntgInWholeTask = subTaskPercntgInWholeTask;
+  }
+
+  class SnapshotCopy extends ExportSnapshot {
+    private BackupHandler backupHandler;
+    private String table;
+
+    public SnapshotCopy(BackupHandler backupHandler, String table) {
+      super();
+      this.backupHandler = backupHandler;
+      this.table = table;
+    }
+
+    public BackupHandler getBackupHandler() {
+      return this.backupHandler;
+    }
+
+    public String getTable() {
+      return this.table;
+    }
+  }  
+  
+  // Extends DistCp for progress updating to hbase:backup
+  // during backup. Using DistCpV2 (MAPREDUCE-2765).
+  // Simply extend it and override execute() method to get the 
+  // Job reference for progress updating.
+  // Only the argument "src1, [src2, [...]] dst" is supported, 
+  // no more DistCp options.
+  class BackupDistCp extends DistCp {
+
+    private BackupHandler backupHandler;
+
+    public BackupDistCp(Configuration conf, DistCpOptions options, BackupHandler backupHandler)
+        throws Exception {
+      super(conf, options);
+      this.backupHandler = backupHandler;
+    }
+
+    @Override
+    public Job execute() throws Exception {
+
+      // reflection preparation for private methods and fields
+      Class<?> classDistCp = org.apache.hadoop.tools.DistCp.class;
+      Method methodCreateMetaFolderPath = classDistCp.getDeclaredMethod("createMetaFolderPath");
+      Method methodCreateJob = classDistCp.getDeclaredMethod("createJob");
+      Method methodCreateInputFileListing =
+          classDistCp.getDeclaredMethod("createInputFileListing", Job.class);
+      Method methodCleanup = classDistCp.getDeclaredMethod("cleanup");
+
+      Field fieldInputOptions = classDistCp.getDeclaredField("inputOptions");
+      Field fieldMetaFolder = classDistCp.getDeclaredField("metaFolder");
+      Field fieldJobFS = classDistCp.getDeclaredField("jobFS");
+      Field fieldSubmitted = classDistCp.getDeclaredField("submitted");
+
+      methodCreateMetaFolderPath.setAccessible(true);
+      methodCreateJob.setAccessible(true);
+      methodCreateInputFileListing.setAccessible(true);
+      methodCleanup.setAccessible(true);
+
+      fieldInputOptions.setAccessible(true);
+      fieldMetaFolder.setAccessible(true);
+      fieldJobFS.setAccessible(true);
+      fieldSubmitted.setAccessible(true);
+
+      // execute() logic starts here
+      assert fieldInputOptions.get(this) != null;
+      assert getConf() != null;
+
+      Job job = null;
+      try {
+        synchronized (this) {
+          // Don't cleanup while we are setting up.
+          fieldMetaFolder.set(this, methodCreateMetaFolderPath.invoke(this));
+          fieldJobFS.set(this, ((Path) fieldMetaFolder.get(this)).getFileSystem(getConf()));
+
+          job = (Job) methodCreateJob.invoke(this);
+        }
+        methodCreateInputFileListing.invoke(this, job);
+
+        // Get the total length of the source files
+        List<Path> srcs = ((DistCpOptions) fieldInputOptions.get(this)).getSourcePaths();
+        long totalSrcLgth = 0;
+        for (Path aSrc : srcs) {
+          totalSrcLgth += BackupUtil.getFilesLength(aSrc.getFileSystem(getConf()), aSrc);
+        }
+
+        // submit the copy job
+        job.submit();
+        fieldSubmitted.set(this, true);
+
+        // after submit the MR job, set its handler in backup handler for cancel process
+        // this.backupHandler.copyJob = job;
+
+        // Update the copy progress to ZK every 0.5s if progress value changed
+        int progressReportFreq =
+            this.getConf().getInt("hbase.backup.progressreport.frequency", 500);
+        float lastProgress = progressDone;
+        while (!job.isComplete()) {
+          float newProgress =
+              progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS);
+
+          if (newProgress > lastProgress) {
+
+            BigDecimal progressData =
+                new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP);
+            String newProgressStr = progressData + "%";
+            LOG.info("Progress: " + newProgressStr);
+            this.backupHandler.updateProgress(newProgressStr, bytesCopied);
+            LOG.debug("Backup progress data updated to hbase:backup: \"Progress: " + newProgressStr
+              + ".\"");
+            lastProgress = newProgress;
+          }
+          Thread.sleep(progressReportFreq);
+        }
+
+        // update the progress data after copy job complete
+        float newProgress =
+            progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS);
+        BigDecimal progressData =
+            new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP);
+
+        String newProgressStr = progressData + "%";
+        LOG.info("Progress: " + newProgressStr);
+
+        // accumulate the overall backup progress
+        progressDone = newProgress;
+        bytesCopied += totalSrcLgth;
+
+        this.backupHandler.updateProgress(newProgressStr, bytesCopied);
+        LOG.debug("Backup progress data updated to hbase:backup: \"Progress: " + newProgressStr
+          + " - " + bytesCopied + " bytes copied.\"");
+
+      } finally {
+        if (!fieldSubmitted.getBoolean(this)) {
+          methodCleanup.invoke(this);
+        }
+      }
+
+      String jobID = job.getJobID().toString();
+      job.getConfiguration().set(DistCpConstants.CONF_LABEL_DISTCP_JOB_ID, jobID);
+
+      LOG.debug("DistCp job-id: " + jobID);
+      return job;
+    }
+
+  }
+
+  /**
+   * Do backup copy based on different types.
+   * @param handler The backup handler reference
+   * @param conf The hadoop configuration
+   * @param copyType The backup copy type
+   * @param options Options for customized ExportSnapshot or DistCp
+   * @throws Exception exception
+   */
+  public int copy(BackupHandler handler, Configuration conf, BackupCopyService.Type copyType,
+      String[] options) throws IOException {
+
+    int res = 0;
+
+    try {
+      if (copyType == Type.FULL) {
+        SnapshotCopy snapshotCp =
+            new SnapshotCopy(handler, handler.getBackupContext().getTableBySnapshot(options[1]));
+        LOG.debug("Doing SNAPSHOT_COPY");
+        // Make a new instance of conf to be used by the snapshot copy class.
+        snapshotCp.setConf(new Configuration(conf));
+        res = snapshotCp.run(options);
+      } else if (copyType == Type.INCREMENTAL) {
+        LOG.debug("Doing COPY_TYPE_DISTCP");
+        setSubTaskPercntgInWholeTask(1f);
+
+        BackupDistCp distcp = new BackupDistCp(new Configuration(conf), null, handler);
+        // Handle a special case where the source file is a single file.
+        // In this case, distcp will not create the target dir. It just take the
+        // target as a file name and copy source file to the target (as a file name).
+        // We need to create the target dir before run distcp.
+        LOG.debug("DistCp options: " + Arrays.toString(options));
+        if (options.length == 2) {
+          Path dest = new Path(options[1]);
+          FileSystem destfs = dest.getFileSystem(conf);
+          if (!destfs.exists(dest)) {
+            destfs.mkdirs(dest);
+          }
+        }
+
+        res = distcp.run(options);
+      }
+      return res;
+
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java
new file mode 100644
index 0000000..deefbf7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java
@@ -0,0 +1,72 @@
+/**
+ * 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.mapreduce;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.IncrementalRestoreService;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.mapreduce.WALPlayer;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MapReduceRestoreService implements IncrementalRestoreService {
+  public static final Log LOG = LogFactory.getLog(MapReduceRestoreService.class);
+
+  private WALPlayer player;
+
+  public MapReduceRestoreService() {
+    this.player = new WALPlayer();
+  }
+
+  @Override
+  public void run(String logDir, String[] tableNames, String[] newTableNames) throws IOException {
+    String tableStr = HBackupFileSystem.join(tableNames);
+    String newTableStr = HBackupFileSystem.join(newTableNames);
+
+    // WALPlayer reads all files in arbitrary directory structure and creates a Map task for each
+    // log file
+
+    String[] playerArgs = { logDir, tableStr, newTableStr };
+    LOG.info("Restore incremental backup from directory " + logDir + " from hbase tables "
+        + HBackupFileSystem.join(tableNames) + " to tables "
+        + HBackupFileSystem.join(newTableNames));
+    try {
+      player.run(playerArgs);
+    } catch (Exception e) {
+      throw new IOException("cannot restore from backup directory " + logDir
+        + " (check Hadoop and HBase logs) " + e);
+    }
+  }
+
+  @Override
+  public Configuration getConf() {
+    return player.getConf();
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.player.setConf(conf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
new file mode 100644
index 0000000..4712548
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
@@ -0,0 +1,121 @@
+/**
+ *
+ * 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.master;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+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.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.backup.BackupSystemTable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
+
+
+
+
+/**
+ * Implementation of a log cleaner that checks if a log is still scheduled for
+ * incremental backup before deleting it when its TTL is over.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class BackupLogCleaner extends BaseLogCleanerDelegate {
+  private static final Log LOG = LogFactory.getLog(BackupLogCleaner.class);
+
+  private boolean stopped = false;
+
+  public BackupLogCleaner() {
+  }
+
+  @Override
+  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
+    // all members of this class are null if backup is disabled,
+    // so we cannot filter the files
+    if (this.getConf() == null) {
+      return files;
+    }
+
+    try {
+      final BackupSystemTable table = BackupSystemTable.getTable(getConf());
+      // If we do not have recorded backup sessions
+      if (table.hasBackupSessions() == false) {
+        return files;
+      }
+      return Iterables.filter(files, new Predicate<FileStatus>() {
+        @Override
+        public boolean apply(FileStatus file) {
+          try {
+            String wal = file.getPath().toString();
+            boolean logInSystemTable = table.checkWALFile(wal);
+            if (LOG.isDebugEnabled()) {
+              if (logInSystemTable) {
+                LOG.debug("Found log file in hbase:backup, deleting: " + wal);
+              } else {
+                LOG.debug("Didn't find this log in hbase:backup, keeping: " + wal);
+              }
+            }
+            return logInSystemTable;
+          } catch (IOException e) {
+            LOG.error(e);
+            return false;// keep file for a while, HBase failed
+          }
+        }
+      });
+    } catch (IOException e) {
+      LOG.error("Failed to get hbase:backup table, therefore will keep all files", e);
+      // nothing to delete
+      return new ArrayList<FileStatus>();
+    }
+
+  }
+
+  @Override
+  public void setConf(Configuration config) {
+    // If backup is disabled, keep all members null
+    if (!config.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) {
+      LOG.warn("Backup is disabled - allowing all wals to be deleted");
+      return;
+    }
+    super.setConf(config);
+  }
+
+  @Override
+  public void stop(String why) {
+    if (this.stopped) {
+      return;
+    }
+    this.stopped = true;
+    LOG.info("Stopping BackupLogCleaner");
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
new file mode 100644
index 0000000..f96682f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
@@ -0,0 +1,129 @@
+/**
+ * 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.master;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ThreadPoolExecutor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
+import org.apache.hadoop.hbase.errorhandling.ForeignException;
+import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.MetricsMaster;
+import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
+import org.apache.hadoop.hbase.procedure.Procedure;
+import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
+import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
+import org.apache.zookeeper.KeeperException;
+
+public class LogRollMasterProcedureManager extends MasterProcedureManager {
+
+  public static final String ROLLLOG_PROCEDURE_SIGNATURE = "rolllog-proc";
+  public static final String ROLLLOG_PROCEDURE_NAME = "rolllog";
+  private static final Log LOG = LogFactory.getLog(LogRollMasterProcedureManager.class);
+
+  private MasterServices master;
+  private ProcedureCoordinator coordinator;
+  private boolean done;
+
+  @Override
+  public void stop(String why) {
+    LOG.info("stop: " + why);
+  }
+
+  @Override
+  public boolean isStopped() {
+    return false;
+  }
+
+  @Override
+  public void initialize(MasterServices master, MetricsMaster metricsMaster)
+      throws KeeperException, IOException, UnsupportedOperationException {
+    this.master = master;
+    this.done = false;
+
+    // setup the default procedure coordinator
+    String name = master.getServerName().toString();
+    ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, 1);
+    BaseCoordinatedStateManager coordManager =
+        (BaseCoordinatedStateManager) CoordinatedStateManagerFactory
+        .getCoordinatedStateManager(master.getConfiguration());
+    coordManager.initialize(master);
+
+    ProcedureCoordinatorRpcs comms =
+        coordManager.getProcedureCoordinatorRpcs(getProcedureSignature(), name);
+
+    this.coordinator = new ProcedureCoordinator(comms, tpool);
+  }
+
+  @Override
+  public String getProcedureSignature() {
+    return ROLLLOG_PROCEDURE_SIGNATURE;
+  }
+
+  @Override
+  public void execProcedure(ProcedureDescription desc) throws IOException {
+    this.done = false;
+    // start the process on the RS
+    ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(desc.getInstance());
+    List<ServerName> serverNames = master.getServerManager().getOnlineServersList();
+    List<String> servers = new ArrayList<String>();
+    for (ServerName sn : serverNames) {
+      servers.add(sn.toString());
+    }
+    Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), new byte[0], servers);
+    if (proc == null) {
+      String msg = "Failed to submit distributed procedure for '" + desc.getInstance() + "'";
+      LOG.error(msg);
+      throw new IOException(msg);
+    }
+
+    try {
+      // wait for the procedure to complete. A timer thread is kicked off that should cancel this
+      // if it takes too long.
+      proc.waitForCompleted();
+      LOG.info("Done waiting - exec procedure for " + desc.getInstance());
+      LOG.info("Distributed roll log procedure is successful!");
+      this.done = true;
+    } catch (InterruptedException e) {
+      ForeignException ee =
+          new ForeignException("Interrupted while waiting for roll log procdure to finish", e);
+      monitor.receive(ee);
+      Thread.currentThread().interrupt();
+    } catch (ForeignException e) {
+      ForeignException ee =
+          new ForeignException("Exception while waiting for roll log procdure to finish", e);
+      monitor.receive(ee);
+    }
+    monitor.rethrowException();
+  }
+
+  @Override
+  public boolean isProcedureDone(ProcedureDescription desc) throws IOException {
+    return done;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
new file mode 100644
index 0000000..618748e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
@@ -0,0 +1,138 @@
+/**
+ * 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.regionserver;
+
+import java.util.HashMap;
+import java.util.concurrent.Callable;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.backup.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.errorhandling.ForeignException;
+import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.procedure.ProcedureMember;
+import org.apache.hadoop.hbase.procedure.Subprocedure;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
+
+
+/**
+ * This backup subprocedure implementation forces a log roll on the RS.
+ */
+public class LogRollBackupSubprocedure extends Subprocedure {
+  private static final Log LOG = LogFactory.getLog(LogRollBackupSubprocedure.class);
+
+  private final RegionServerServices rss;
+  private final LogRollBackupSubprocedurePool taskManager;
+  private FSHLog hlog;
+
+  public LogRollBackupSubprocedure(RegionServerServices rss, ProcedureMember member,
+      ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
+      LogRollBackupSubprocedurePool taskManager) {
+
+    super(member, LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, errorListener,
+      wakeFrequency, timeout);
+    LOG.info("Constructing a LogRollBackupSubprocedure.");
+    this.rss = rss;
+    this.taskManager = taskManager;
+  }
+
+  /**
+   * Callable task. TODO. We don't need a thread pool to execute roll log. This can be simplified
+   * with no use of subprocedurepool.
+   */
+  class RSRollLogTask implements Callable<Void> {
+    RSRollLogTask() {
+    }
+
+    @Override
+    public Void call() throws Exception {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("++ DRPC started: " + rss.getServerName());
+      }
+      hlog = (FSHLog) rss.getWAL(null);
+      long filenum = hlog.getFilenum();
+
+      LOG.info("Trying to roll log in backup subprocedure, current log number: " + filenum);
+      hlog.rollWriter(true);
+      LOG.info("After roll log in backup subprocedure, current log number: " + hlog.getFilenum());
+      // write the log number to hbase:backup.
+      BackupSystemTable table = BackupSystemTable.getTable(rss.getConfiguration());
+      // sanity check, good for testing
+      HashMap<String, String> serverTimestampMap = table.readRegionServerLastLogRollResult();
+      String host = rss.getServerName().getHostname();
+      String sts = serverTimestampMap.get(host);
+      if (sts != null && Long.parseLong(sts) > filenum) {
+        LOG.warn("Won't update server's last roll log result: current=" + sts + " new=" + filenum);
+        return null;
+      }
+      table.writeRegionServerLastLogRollResult(host, Long.toString(filenum));
+      // TODO: potential leak of HBase connection
+      // BackupSystemTable.close();
+      return null;
+    }
+
+  }
+
+  private void rolllog() throws ForeignException {
+
+    monitor.rethrowException();
+
+    taskManager.submitTask(new RSRollLogTask());
+    monitor.rethrowException();
+
+    // wait for everything to complete.
+    taskManager.waitForOutstandingTasks();
+    monitor.rethrowException();
+
+  }
+
+  @Override
+  public void acquireBarrier() throws ForeignException {
+    // do nothing, executing in inside barrier step.
+  }
+
+  /**
+   * do a log roll.
+   * @return some bytes
+   */
+  @Override
+  public byte[] insideBarrier() throws ForeignException {
+    rolllog();
+    // FIXME
+    return null;
+  }
+
+  /**
+   * Cancel threads if they haven't finished.
+   */
+  @Override
+  public void cleanup(Exception e) {
+    taskManager.abort("Aborting log roll subprocedure tasks for backup due to error", e);
+  }
+
+  /**
+   * Hooray!
+   */
+  public void releaseBarrier() {
+    // NO OP
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
new file mode 100644
index 0000000..1ca638c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
@@ -0,0 +1,137 @@
+/**
+ * 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.regionserver;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.DaemonThreadFactory;
+import org.apache.hadoop.hbase.errorhandling.ForeignException;
+
+/**
+ * Handle running each of the individual tasks for completing a backup procedure
+ * on a regionserver.
+ */
+public class LogRollBackupSubprocedurePool implements Closeable, Abortable {
+  private static final Log LOG = LogFactory.getLog(LogRollBackupSubprocedurePool.class);
+
+  /** Maximum number of concurrent snapshot region tasks that can run concurrently */
+  private static final String CONCURENT_BACKUP_TASKS_KEY = "hbase.backup.region.concurrentTasks";
+  private static final int DEFAULT_CONCURRENT_BACKUP_TASKS = 3;
+
+  private final ExecutorCompletionService<Void> taskPool;
+  private final ThreadPoolExecutor executor;
+  private volatile boolean aborted;
+  private final List<Future<Void>> futures = new ArrayList<Future<Void>>();
+  private final String name;
+
+  public LogRollBackupSubprocedurePool(String name, Configuration conf) {
+    // configure the executor service
+    long keepAlive =
+        conf.getLong(LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_KEY,
+          LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_DEFAULT);
+    int threads = conf.getInt(CONCURENT_BACKUP_TASKS_KEY, DEFAULT_CONCURRENT_BACKUP_TASKS);
+    this.name = name;
+    executor =
+        new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
+          new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("rs(" + name
+            + ")-backup-pool"));
+    taskPool = new ExecutorCompletionService<Void>(executor);
+  }
+
+  /**
+   * Submit a task to the pool.
+   */
+  public void submitTask(final Callable<Void> task) {
+    Future<Void> f = this.taskPool.submit(task);
+    futures.add(f);
+  }
+
+  /**
+   * Wait for all of the currently outstanding tasks submitted via {@link #submitTask(Callable)}
+   * @return <tt>true</tt> on success, <tt>false</tt> otherwise
+   * @throws ForeignException exception
+   */
+  public boolean waitForOutstandingTasks() throws ForeignException {
+    LOG.debug("Waiting for backup procedure to finish.");
+
+    try {
+      for (Future<Void> f : futures) {
+        f.get();
+      }
+      return true;
+    } catch (InterruptedException e) {
+      if (aborted) {
+        throw new ForeignException("Interrupted and found to be aborted while waiting for tasks!",
+            e);
+      }
+      Thread.currentThread().interrupt();
+    } catch (ExecutionException e) {
+      if (e.getCause() instanceof ForeignException) {
+        throw (ForeignException) e.getCause();
+      }
+      throw new ForeignException(name, e.getCause());
+    } finally {
+      // close off remaining tasks
+      for (Future<Void> f : futures) {
+        if (!f.isDone()) {
+          f.cancel(true);
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Attempt to cleanly shutdown any running tasks - allows currently running tasks to cleanly
+   * finish
+   */
+  @Override
+  public void close() {
+    executor.shutdown();
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    if (this.aborted) {
+      return;
+    }
+
+    this.aborted = true;
+    LOG.warn("Aborting because: " + why, e);
+    this.executor.shutdownNow();
+  }
+
+  @Override
+  public boolean isAborted() {
+    return this.aborted;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
new file mode 100644
index 0000000..aca190c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
@@ -0,0 +1,168 @@
+/**
+ * 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.regionserver;
+
+
+import java.io.IOException;
+import java.util.concurrent.ThreadPoolExecutor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
+import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.procedure.ProcedureMember;
+import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
+import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
+import org.apache.hadoop.hbase.procedure.Subprocedure;
+import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+
+/**
+ * This manager class handles the work dealing with backup for a {@link HRegionServer}.
+ * <p>
+ * This provides the mechanism necessary to kick off a backup specific {@link Subprocedure} that is
+ * responsible by this region server. If any failures occur with the subprocedure, the manager's
+ * procedure member notifies the procedure coordinator to abort all others.
+ * <p>
+ * On startup, requires {@link #start()} to be called.
+ * <p>
+ * On shutdown, requires org.apache.hadoop.hbase.procedure.ProcedureMember.close() to be
+ * called
+ */
+public class LogRollRegionServerProcedureManager extends RegionServerProcedureManager {
+
+  private static final Log LOG = LogFactory.getLog(LogRollRegionServerProcedureManager.class);
+
+  /** Conf key for number of request threads to start backup on regionservers */
+  public static final String BACKUP_REQUEST_THREADS_KEY = "hbase.backup.region.pool.threads";
+  /** # of threads for backup work on the rs. */
+  public static final int BACKUP_REQUEST_THREADS_DEFAULT = 10;
+
+  public static final String BACKUP_TIMEOUT_MILLIS_KEY = "hbase.backup.timeout";
+  public static final long BACKUP_TIMEOUT_MILLIS_DEFAULT = 60000;
+
+  /** Conf key for millis between checks to see if backup work completed or if there are errors */
+  public static final String BACKUP_REQUEST_WAKE_MILLIS_KEY = "hbase.backup.region.wakefrequency";
+  /** Default amount of time to check for errors while regions finish backup work */
+  private static final long BACKUP_REQUEST_WAKE_MILLIS_DEFAULT = 500;
+
+  private RegionServerServices rss;
+  private ProcedureMemberRpcs memberRpcs;
+  private ProcedureMember member;
+
+  /**
+   * Create a default backup procedure manager
+   */
+  public LogRollRegionServerProcedureManager() {
+  }
+
+  /**
+   * Start accepting backup procedure requests.
+   */
+  @Override
+  public void start() {
+    this.memberRpcs.start(rss.getServerName().toString(), member);
+    LOG.info("Started region server backup manager.");
+  }
+
+  /**
+   * Close <tt>this</tt> and all running backup procedure tasks
+   * @param force forcefully stop all running tasks
+   * @throws IOException exception
+   */
+  @Override
+  public void stop(boolean force) throws IOException {
+    String mode = force ? "abruptly" : "gracefully";
+    LOG.info("Stopping RegionServerBackupManager " + mode + ".");
+
+    try {
+      this.member.close();
+    } finally {
+      this.memberRpcs.close();
+    }
+  }
+
+  /**
+   * If in a running state, creates the specified subprocedure for handling a backup procedure.
+   * @return Subprocedure to submit to the ProcedureMemeber.
+   */
+  public Subprocedure buildSubprocedure() {
+
+    // don't run a backup if the parent is stop(ping)
+    if (rss.isStopping() || rss.isStopped()) {
+      throw new IllegalStateException("Can't start backup procedure on RS: " + rss.getServerName()
+        + ", because stopping/stopped!");
+    }
+
+    LOG.info("Attempting to run a roll log procedure for backup.");
+    ForeignExceptionDispatcher errorDispatcher = new ForeignExceptionDispatcher();
+    Configuration conf = rss.getConfiguration();
+    long timeoutMillis = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT);
+    long wakeMillis =
+        conf.getLong(BACKUP_REQUEST_WAKE_MILLIS_KEY, BACKUP_REQUEST_WAKE_MILLIS_DEFAULT);
+
+    LogRollBackupSubprocedurePool taskManager =
+        new LogRollBackupSubprocedurePool(rss.getServerName().toString(), conf);
+    return new LogRollBackupSubprocedure(rss, member, errorDispatcher, wakeMillis, timeoutMillis,
+      taskManager);
+
+  }
+
+  /**
+   * Build the actual backup procedure runner that will do all the 'hard' work
+   */
+  public class BackupSubprocedureBuilder implements SubprocedureFactory {
+
+    @Override
+    public Subprocedure buildSubprocedure(String name, byte[] data) {
+      return LogRollRegionServerProcedureManager.this.buildSubprocedure();
+    }
+  }
+
+  @Override
+  public void initialize(RegionServerServices rss) throws IOException {
+    this.rss = rss;
+    BaseCoordinatedStateManager coordManager =
+        (BaseCoordinatedStateManager) CoordinatedStateManagerFactory.getCoordinatedStateManager(rss
+          .getConfiguration());
+    coordManager.initialize(rss);
+    this.memberRpcs =
+        coordManager
+        .getProcedureMemberRpcs(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE);
+
+    // read in the backup handler configuration properties
+    Configuration conf = rss.getConfiguration();
+    long keepAlive = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT);
+    int opThreads = conf.getInt(BACKUP_REQUEST_THREADS_KEY, BACKUP_REQUEST_THREADS_DEFAULT);
+    // create the actual cohort member
+    ThreadPoolExecutor pool =
+        ProcedureMember.defaultPool(rss.getServerName().toString(), opThreads, keepAlive);
+    this.member = new ProcedureMember(memberRpcs, pool, new BackupSubprocedureBuilder());
+  }
+
+  @Override
+  public String getProcedureSignature() {
+    return "backup-proc";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
index ae36f08..3342743 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
@@ -17,7 +17,11 @@
  */
 package org.apache.hadoop.hbase.coordination;
 
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.Server;
 
@@ -51,8 +55,21 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan
    * Method to retrieve coordination for split log worker
    */
   public abstract  SplitLogWorkerCoordination getSplitLogWorkerCoordination();
+  
   /**
    * Method to retrieve coordination for split log manager
    */
   public abstract SplitLogManagerCoordination getSplitLogManagerCoordination();
+  /**
+   * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs}
+   */
+  public abstract ProcedureCoordinatorRpcs
+    getProcedureCoordinatorRpcs(String procType, String coordNode) throws IOException;
+  
+  /**
+   * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureMemberRpc}
+   */
+  public abstract ProcedureMemberRpcs
+    getProcedureMemberRpcs(String procType) throws IOException;
+    
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
index 3e89be7..7cf4aab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
@@ -17,9 +17,15 @@
  */
 package org.apache.hadoop.hbase.coordination;
 
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
+import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 /**
@@ -49,9 +55,21 @@ public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
   @Override
   public SplitLogWorkerCoordination getSplitLogWorkerCoordination() {
     return splitLogWorkerCoordination;
-    }
+  }
+
   @Override
   public SplitLogManagerCoordination getSplitLogManagerCoordination() {
     return splitLogManagerCoordination;
   }
+
+  @Override
+  public ProcedureCoordinatorRpcs getProcedureCoordinatorRpcs(String procType, String coordNode)
+      throws IOException {
+    return new ZKProcedureCoordinatorRpcs(watcher, procType, coordNode);
+  }
+
+  @Override
+  public ProcedureMemberRpcs getProcedureMemberRpcs(String procType) throws IOException {
+    return new ZKProcedureMemberRpcs(watcher, procType);
+  }
 }


[2/6] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index 9d9cee0..2ceeda5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -85,6 +85,9 @@ public class WALPlayer extends Configured implements Tool {
 
   private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
 
+  public WALPlayer(){
+  }
+
   protected WALPlayer(final Configuration c) {
     super(c);
   }
@@ -94,7 +97,7 @@ public class WALPlayer extends Configured implements Tool {
    * This one can be used together with {@link KeyValueSortReducer}
    */
   static class WALKeyValueMapper
-  extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
+    extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
     private byte[] table;
 
     @Override
@@ -106,7 +109,9 @@ public class WALPlayer extends Configured implements Tool {
         if (Bytes.equals(table, key.getTablename().getName())) {
           for (Cell cell : value.getCells()) {
             KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-            if (WALEdit.isMetaEditFamily(kv)) continue;
+            if (WALEdit.isMetaEditFamily(kv)) {
+              continue;
+            }
             context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)), kv);
           }
         }
@@ -132,7 +137,7 @@ public class WALPlayer extends Configured implements Tool {
    * a running HBase instance.
    */
   protected static class WALMapper
-  extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
+    extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
     private Map<TableName, TableName> tables = new TreeMap<TableName, TableName>();
 
     @Override
@@ -149,7 +154,9 @@ public class WALPlayer extends Configured implements Tool {
           Cell lastCell = null;
           for (Cell cell : value.getCells()) {
             // filtering WAL meta entries
-            if (WALEdit.isMetaEditFamily(cell)) continue;
+            if (WALEdit.isMetaEditFamily(cell)) {
+              continue;
+            }
 
             // Allow a subclass filter out this cell.
             if (filter(context, cell)) {
@@ -160,8 +167,12 @@ public class WALPlayer extends Configured implements Tool {
               if (lastCell == null || lastCell.getTypeByte() != cell.getTypeByte()
                   || !CellUtil.matchingRow(lastCell, cell)) {
                 // row or type changed, write out aggregate KVs.
-                if (put != null) context.write(tableOut, put);
-                if (del != null) context.write(tableOut, del);
+                if (put != null) {
+                  context.write(tableOut, put);
+                }
+                if (del != null) {
+                  context.write(tableOut, del);
+                }
                 if (CellUtil.isDelete(cell)) {
                   del = new Delete(CellUtil.cloneRow(cell));
                 } else {
@@ -177,8 +188,12 @@ public class WALPlayer extends Configured implements Tool {
             lastCell = cell;
           }
           // write residual KVs
-          if (put != null) context.write(tableOut, put);
-          if (del != null) context.write(tableOut, del);
+          if (put != null) {
+            context.write(tableOut, put);
+          }
+          if (del != null) {
+            context.write(tableOut, del);
+          }
         }
       } catch (InterruptedException e) {
         e.printStackTrace();
@@ -186,7 +201,8 @@ public class WALPlayer extends Configured implements Tool {
     }
 
     /**
-     * @param cell
+     * Filter cell
+     * @param cell cell
      * @return Return true if we are to emit this cell.
      */
     protected boolean filter(Context context, final Cell cell) {
@@ -197,9 +213,7 @@ public class WALPlayer extends Configured implements Tool {
     public void setup(Context context) throws IOException {
       String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY);
       String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY);
-      if (tablesToUse == null && tableMap == null) {
-        // Then user wants all tables.
-      } else if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
+      if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
         // this can only happen when WALMapper is used directly by a class other than WALPlayer
         throw new IOException("No tables or incorrect table mapping specified.");
       }
@@ -215,7 +229,9 @@ public class WALPlayer extends Configured implements Tool {
 
   void setupTime(Configuration conf, String option) throws IOException {
     String val = conf.get(option);
-    if (null == val) return;
+    if (null == val) {
+      return;
+    }
     long ms;
     try {
       // first try to parse in user friendly form
@@ -295,7 +311,8 @@ public class WALPlayer extends Configured implements Tool {
     return job;
   }
 
-  /*
+  /**
+   * Print usage
    * @param errorMsg Error message.  Can be null.
    */
   private void usage(final String errorMsg) {
@@ -305,7 +322,8 @@ public class WALPlayer extends Configured implements Tool {
     System.err.println("Usage: " + NAME + " [options] <wal inputdir> <tables> [<tableMappings>]");
     System.err.println("Read all WAL entries for <tables>.");
     System.err.println("If no tables (\"\") are specific, all tables are imported.");
-    System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported in that case.)");
+    System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported"+
+      " in that case.)");
     System.err.println("Otherwise <tables> is a comma separated list of tables.\n");
     System.err.println("The WAL entries can be mapped to new set of tables via <tableMapping>.");
     System.err.println("<tableMapping> is a command separated list of targettables.");
@@ -318,10 +336,10 @@ public class WALPlayer extends Configured implements Tool {
     System.err.println("  -D" + WALInputFormat.START_TIME_KEY + "=[date|ms]");
     System.err.println("  -D" + WALInputFormat.END_TIME_KEY + "=[date|ms]");
     System.err.println("   -D " + JOB_NAME_CONF_KEY
-        + "=jobName - use the specified mapreduce job name for the wal player");
+      + "=jobName - use the specified mapreduce job name for the wal player");
     System.err.println("For performance also consider the following options:\n"
-        + "  -Dmapreduce.map.speculative=false\n"
-        + "  -Dmapreduce.reduce.speculative=false");
+      + "  -Dmapreduce.map.speculative=false\n"
+      + "  -Dmapreduce.reduce.speculative=false");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index bdb19f4..5cd38b5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -75,6 +75,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
+import org.apache.hadoop.hbase.backup.BackupManager;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
@@ -384,6 +385,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
 
     Replication.decorateMasterConfiguration(this.conf);
+    BackupManager.decorateMasterConfiguration(this.conf);
 
     // Hack! Maps DFSClient => Master for logs.  HDFS made this
     // config param for task trackers, but we can piggyback off of it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
index 95c3ffe..b6e11ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
@@ -37,7 +37,7 @@ public abstract class RegionServerProcedureManager extends ProcedureManager {
    * @param rss Region Server service interface
    * @throws KeeperException
    */
-  public abstract void initialize(RegionServerServices rss) throws KeeperException;
+  public abstract void initialize(RegionServerServices rss) throws IOException;
 
   /**
    * Start accepting procedure requests.

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
index 0f4ea64..adb3604 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.procedure.flush.RegionServerFlushTableProcedureManager;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
-import org.apache.zookeeper.KeeperException;
 
 /**
  * Provides the globally barriered procedure framework and environment
@@ -39,7 +38,7 @@ public class RegionServerProcedureManagerHost extends
   private static final Log LOG = LogFactory
       .getLog(RegionServerProcedureManagerHost.class);
 
-  public void initialize(RegionServerServices rss) throws KeeperException {
+  public void initialize(RegionServerServices rss) throws IOException {
     for (RegionServerProcedureManager proc : procedures) {
       LOG.debug("Procedure " + proc.getProcedureSignature() + " is initializing");
       proc.initialize(rss);

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
index 085d642..3865ba9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
@@ -54,7 +54,7 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs {
    * @throws KeeperException if an unexpected zk error occurs
    */
   public ZKProcedureCoordinatorRpcs(ZooKeeperWatcher watcher,
-      String procedureClass, String coordName) throws KeeperException {
+      String procedureClass, String coordName) throws IOException {
     this.watcher = watcher;
     this.procedureType = procedureClass;
     this.coordName = coordName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
index 2e03a60..fff75a7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
@@ -68,49 +68,54 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
    * @throws KeeperException if we can't reach zookeeper
    */
   public ZKProcedureMemberRpcs(final ZooKeeperWatcher watcher, final String procType)
-      throws KeeperException {
-    this.zkController = new ZKProcedureUtil(watcher, procType) {
-      @Override
-      public void nodeCreated(String path) {
-        if (!isInProcedurePath(path)) {
-          return;
-        }
+      throws IOException {
+    try {
+      this.zkController = new ZKProcedureUtil(watcher, procType) {
+        @Override
+        public void nodeCreated(String path) {
+          if (!isInProcedurePath(path)) {
+            return;
+          }
 
-        LOG.info("Received created event:" + path);
-        // if it is a simple start/end/abort then we just rewatch the node
-        if (isAcquiredNode(path)) {
-          waitForNewProcedures();
-          return;
-        } else if (isAbortNode(path)) {
-          watchForAbortedProcedures();
-          return;
+          LOG.info("Received created event:" + path);
+          // if it is a simple start/end/abort then we just rewatch the node
+          if (isAcquiredNode(path)) {
+            waitForNewProcedures();
+            return;
+          } else if (isAbortNode(path)) {
+            watchForAbortedProcedures();
+            return;
+          }
+          String parent = ZKUtil.getParent(path);
+          // if its the end barrier, the procedure can be completed
+          if (isReachedNode(parent)) {
+            receivedReachedGlobalBarrier(path);
+            return;
+          } else if (isAbortNode(parent)) {
+            abort(path);
+            return;
+          } else if (isAcquiredNode(parent)) {
+            startNewSubprocedure(path);
+          } else {
+            LOG.debug("Ignoring created notification for node:" + path);
+          }
         }
-        String parent = ZKUtil.getParent(path);
-        // if its the end barrier, the procedure can be completed
-        if (isReachedNode(parent)) {
-          receivedReachedGlobalBarrier(path);
-          return;
-        } else if (isAbortNode(parent)) {
-          abort(path);
-          return;
-        } else if (isAcquiredNode(parent)) {
-          startNewSubprocedure(path);
-        } else {
-          LOG.debug("Ignoring created notification for node:" + path);
-        }
-      }
 
-      @Override
-      public void nodeChildrenChanged(String path) {
-        if (path.equals(this.acquiredZnode)) {
-          LOG.info("Received procedure start children changed event: " + path);
-          waitForNewProcedures();
-        } else if (path.equals(this.abortZnode)) {
-          LOG.info("Received procedure abort children changed event: " + path);
-          watchForAbortedProcedures();
+        @Override
+        public void nodeChildrenChanged(String path) {
+          if (path.equals(this.acquiredZnode)) {
+            LOG.info("Received procedure start children changed event: " + path);
+            waitForNewProcedures();
+          } else if (path.equals(this.abortZnode)) {
+            LOG.info("Received procedure abort children changed event: " + path);
+            watchForAbortedProcedures();
+          }
         }
-      }
-    };
+      };
+    } catch (KeeperException e) {
+      // TODO Auto-generated catch block
+      throw new IOException(e);
+    }
   }
 
   public ZKProcedureUtil getZkController() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
index 1aa959c..bd65cc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
@@ -317,7 +317,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur
    * @throws KeeperException if the zookeeper cannot be reached
    */
   @Override
-  public void initialize(RegionServerServices rss) throws KeeperException {
+  public void initialize(RegionServerServices rss) throws IOException {
     this.rss = rss;
     ZooKeeperWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw,

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 211fed5..1cd54fa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -807,8 +807,8 @@ public class HRegionServer extends HasThread implements
       rspmHost = new RegionServerProcedureManagerHost();
       rspmHost.loadProcedures(conf);
       rspmHost.initialize(this);
-    } catch (KeeperException e) {
-      this.abort("Failed to reach zk cluster when creating procedure handler.", e);
+    } catch (IOException e) {
+      this.abort("Failed to reach coordination cluster when creating procedure handler.", e);
     }
     // register watcher for recovering regions
     this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
index 537329a..e56dd28 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
@@ -390,7 +390,7 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
    * @throws KeeperException if the zookeeper cluster cannot be reached
    */
   @Override
-  public void initialize(RegionServerServices rss) throws KeeperException {
+  public void initialize(RegionServerServices rss) throws IOException {
     this.rss = rss;
     ZooKeeperWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw,

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 9ae72e6..acde21e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -97,6 +97,8 @@ import com.lmax.disruptor.TimeoutException;
 import com.lmax.disruptor.dsl.Disruptor;
 import com.lmax.disruptor.dsl.ProducerType;
 
+
+
 /**
  * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS.
  * Only one WAL is ever being written at a time.  When a WAL hits a configured maximum size,
@@ -359,7 +361,9 @@ public class FSHLog implements WAL {
     public int compare(Path o1, Path o2) {
       long t1 = getFileNumFromFileName(o1);
       long t2 = getFileNumFromFileName(o2);
-      if (t1 == t2) return 0;
+      if (t1 == t2) {
+        return 0;
+      }
       return (t1 > t2) ? 1 : -1;
     }
   };
@@ -402,7 +406,7 @@ public class FSHLog implements WAL {
    * @param root path for stored and archived wals
    * @param logDir dir where wals are stored
    * @param conf configuration to use
-   * @throws IOException
+   * @throws IOException exception
    */
   public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf)
       throws IOException {
@@ -410,7 +414,7 @@ public class FSHLog implements WAL {
   }
 
   /**
-   * Create an edit log at the given <code>dir</code> location.
+   * Create an edit log at the given directory location.
    *
    * You should never have to load an existing log. If there is a log at
    * startup, it should have already been processed and deleted by the time the
@@ -425,13 +429,13 @@ public class FSHLog implements WAL {
    * be registered before we do anything else; e.g. the
    * Constructor {@link #rollWriter()}.
    * @param failIfWALExists If true IOException will be thrown if files related to this wal
-   *        already exist.
+   *     already exist.
    * @param prefix should always be hostname and port in distributed env and
-   *        it will be URL encoded before being used.
-   *        If prefix is null, "wal" will be used
+   *     it will be URL encoded before being used.
+   *     If prefix is null, "wal" will be used
    * @param suffix will be url encoded. null is treated as empty. non-empty must start with
-   *        {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
-   * @throws IOException
+   *     {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
+   * @throws IOException exception
    */
   public FSHLog(final FileSystem fs, final Path rootDir, final String logDir,
       final String archiveDir, final Configuration conf,
@@ -593,7 +597,9 @@ public class FSHLog implements WAL {
   @VisibleForTesting
   OutputStream getOutputStream() {
     FSDataOutputStream fsdos = this.hdfs_out;
-    if (fsdos == null) return null;
+    if (fsdos == null) {
+      return null;
+    }
     return fsdos.getWrappedStream();
   }
 
@@ -628,7 +634,7 @@ public class FSHLog implements WAL {
 
   /**
    * Tell listeners about pre log roll.
-   * @throws IOException
+   * @throws IOException exception
    */
   private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath)
   throws IOException {
@@ -641,7 +647,7 @@ public class FSHLog implements WAL {
 
   /**
    * Tell listeners about post log roll.
-   * @throws IOException
+   * @throws IOException exception
    */
   private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
   throws IOException {
@@ -654,8 +660,7 @@ public class FSHLog implements WAL {
 
   /**
    * Run a sync after opening to set up the pipeline.
-   * @param nextWriter
-   * @param startTimeNanos
+   * @param nextWriter next writer
    */
   private void preemptiveSync(final ProtobufLogWriter nextWriter) {
     long startTimeNanos = System.nanoTime();
@@ -673,7 +678,9 @@ public class FSHLog implements WAL {
     rollWriterLock.lock();
     try {
       // Return if nothing to flush.
-      if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null;
+      if (!force && (this.writer != null && this.numEntries.get() <= 0)) {
+        return null;
+      }
       byte [][] regionsToFlush = null;
       if (this.closed) {
         LOG.debug("WAL closed. Skipping rolling of writer");
@@ -728,7 +735,7 @@ public class FSHLog implements WAL {
 
   /**
    * Archive old logs. A WAL is eligible for archiving if all its WALEdits have been flushed.
-   * @throws IOException
+   * @throws IOException exception
    */
   private void cleanOldLogs() throws IOException {
     List<Path> logsToArchive = null;
@@ -738,9 +745,13 @@ public class FSHLog implements WAL {
       Path log = e.getKey();
       Map<byte[], Long> sequenceNums = e.getValue();
       if (this.sequenceIdAccounting.areAllLower(sequenceNums)) {
-        if (logsToArchive == null) logsToArchive = new ArrayList<Path>();
+        if (logsToArchive == null) {
+          logsToArchive = new ArrayList<Path>();
+        }
         logsToArchive.add(log);
-        if (LOG.isTraceEnabled()) LOG.trace("WAL file ready for archiving " + log);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("WAL file ready for archiving " + log);
+        }
       }
     }
     if (logsToArchive != null) {
@@ -770,7 +781,9 @@ public class FSHLog implements WAL {
     if (regions != null) {
       StringBuilder sb = new StringBuilder();
       for (int i = 0; i < regions.length; i++) {
-        if (i > 0) sb.append(", ");
+        if (i > 0) {
+          sb.append(", ");
+        }
         sb.append(Bytes.toStringBinary(regions[i]));
       }
       LOG.info("Too many WALs; count=" + logCount + ", max=" + this.maxLogs +
@@ -836,7 +849,9 @@ public class FSHLog implements WAL {
         }
       } catch (FailedSyncBeforeLogCloseException e) {
         // If unflushed/unsynced entries on close, it is reason to abort.
-        if (isUnflushedEntries()) throw e;
+        if (isUnflushedEntries()) {
+          throw e;
+        }
         LOG.warn("Failed sync-before-close but no outstanding appends; closing WAL: " +
           e.getMessage());
       }
@@ -897,7 +912,9 @@ public class FSHLog implements WAL {
             try {
               blockOnSync(syncFuture);
             } catch (IOException ioe) {
-              if (LOG.isTraceEnabled()) LOG.trace("Stale sync exception", ioe);
+              if (LOG.isTraceEnabled()) {
+                LOG.trace("Stale sync exception", ioe);
+              }
             }
           }
         }
@@ -968,7 +985,15 @@ public class FSHLog implements WAL {
   public Path getCurrentFileName() {
     return computeFilename(this.filenum.get());
   }
-
+  
+  /**
+   * To support old API compatibility
+   * @return current file number (timestamp)
+   */
+  public long getFilenum() {
+    return filenum.get();
+  }
+  
   @Override
   public String toString() {
     return "FSHLog " + logFilePrefix + ":" + logFileSuffix + "(num " + filenum + ")";

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java
new file mode 100644
index 0000000..0360000
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java
@@ -0,0 +1,42 @@
+/**
+ * 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.snapshot;
+
+import org.apache.hadoop.hbase.backup.BackupHandler;
+
+/* this class will be extended in future jira to support progress report */
+public class SnapshotCopy extends ExportSnapshot {
+  private BackupHandler backupHandler;
+  private String table;
+
+  public SnapshotCopy(BackupHandler backupHandler, String table) {
+    super();
+    this.backupHandler = backupHandler;
+    this.table = table;
+  }
+
+  public BackupHandler getBackupHandler() {
+    return this.backupHandler;
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
index 027e7a2..dd4d337 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
@@ -209,13 +209,18 @@ public class DefaultWALProvider implements WALProvider {
   @VisibleForTesting
   public static long extractFileNumFromWAL(final WAL wal) {
     final Path walName = ((FSHLog)wal).getCurrentFileName();
+    return extractFileNumFromWAL(walName);
+  }
+
+  @VisibleForTesting
+  public static long extractFileNumFromWAL(final Path walName) {
     if (walName == null) {
       throw new IllegalArgumentException("The WAL path couldn't be null");
     }
     final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
     return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2:1)]);
   }
-
+  
   /**
    * Pattern used to validate a WAL file name
    * see {@link #validateWALFilename(String)} for description.

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
new file mode 100644
index 0000000..bc0c848
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
@@ -0,0 +1,194 @@
+/*
+ *
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+/**
+ * This class is only a base for other integration-level backup tests.
+ * Do not add tests here.
+ * TestBackupSmallTests is where tests that don't require bring machines up/down should go
+ * All other tests should have their own classes and extend this one
+ */
+public class TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupBase.class);
+
+  protected static Configuration conf1;
+  protected static Configuration conf2;
+
+  protected static HBaseTestingUtility TEST_UTIL;
+  protected static HBaseTestingUtility TEST_UTIL2;
+
+  protected static TableName table1;
+  protected static TableName table2;
+  protected static TableName table3;
+  protected static TableName table4;
+
+  protected static String table1_restore = "table1_restore";
+  protected static String table2_restore = "table2_restore";
+  protected static String table3_restore = "table3_restore";
+  protected static String table4_restore = "table4_restore";
+
+  protected static final int NB_ROWS_IN_BATCH = 100;
+  protected static final byte[] qualName = Bytes.toBytes("q1");
+  protected static final byte[] famName = Bytes.toBytes("f");
+
+  protected static String BACKUP_ROOT_DIR = "/backupUT";
+  protected static String BACKUP_REMOTE_ROOT_DIR = "/backupUT";
+
+  protected static final String BACKUP_ZNODE = "/backup/hbase";
+  protected static final String BACKUP_SUCCEED_NODE = "complete";
+  protected static final String BACKUP_FAILED_NODE = "failed";
+
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+
+    TEST_UTIL = new HBaseTestingUtility();
+    TEST_UTIL.getConfiguration().set("hbase.procedure.regionserver.classes",
+      LogRollRegionServerProcedureManager.class.getName());
+    TEST_UTIL.getConfiguration().set("hbase.procedure.master.classes",
+      LogRollMasterProcedureManager.class.getName());
+    TEST_UTIL.getConfiguration().set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
+    TEST_UTIL.startMiniZKCluster();
+    MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
+
+    conf1 = TEST_UTIL.getConfiguration();
+    conf2 = HBaseConfiguration.create(conf1);
+    conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
+    TEST_UTIL2 = new HBaseTestingUtility(conf2);
+    TEST_UTIL2.setZkCluster(miniZK);
+    TEST_UTIL.startMiniCluster();
+    TEST_UTIL2.startMiniCluster();
+    conf1 = TEST_UTIL.getConfiguration();
+
+    TEST_UTIL.startMiniMapReduceCluster();
+    BACKUP_ROOT_DIR = TEST_UTIL.getConfiguration().get("fs.defaultFS") + "/backupUT";
+    LOG.info("ROOTDIR " + BACKUP_ROOT_DIR);
+    BACKUP_REMOTE_ROOT_DIR = TEST_UTIL2.getConfiguration().get("fs.defaultFS") + "/backupUT";
+    LOG.info("REMOTE ROOTDIR " + BACKUP_REMOTE_ROOT_DIR);
+
+    BackupClient.setConf(conf1);
+    RestoreClient.setConf(conf1);
+    createTables();
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
+    //zkw1.close();
+    TEST_UTIL2.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniMapReduceCluster();
+  }
+
+  protected static void loadTable(HTable table) throws Exception {
+
+    Put p; // 100 + 1 row to t1_syncup
+    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+      p = new Put(Bytes.toBytes("row" + i));
+      p.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      table.put(p);
+    }
+  }
+
+  protected static void createTables() throws Exception {
+
+    long tid = System.currentTimeMillis();
+    table1 = TableName.valueOf("test-" + tid);
+    HBaseAdmin ha = TEST_UTIL.getHBaseAdmin();
+    HTableDescriptor desc = new HTableDescriptor(table1);
+    HColumnDescriptor fam = new HColumnDescriptor(famName);
+    desc.addFamily(fam);
+    ha.createTable(desc);
+    Connection conn = ConnectionFactory.createConnection(conf1);    
+    HTable table = (HTable) conn.getTable(table1);
+    loadTable(table);
+    table.close();
+    table2 = TableName.valueOf("test-" + tid + 1);
+    desc = new HTableDescriptor(table2);
+    desc.addFamily(fam);
+    ha.createTable(desc);
+    table = (HTable) conn.getTable(table2);
+    loadTable(table);
+    table.close();
+    table3 = TableName.valueOf("test-" + tid + 2);
+    table = TEST_UTIL.createTable(table3, famName);
+    table.close();
+    table4 = TableName.valueOf("test-" + tid + 3);
+    table = TEST_UTIL.createTable(table4, famName);
+    table.close();
+    ha.close();
+    conn.close();
+  }
+
+  protected boolean checkSucceeded(String backupId) throws IOException
+  {
+    BackupContext status = getBackupContext(backupId);
+    if(status == null) return false;
+    return status.getFlag() == BACKUPSTATUS.COMPLETE;
+  }
+
+  protected boolean checkFailed(String backupId) throws IOException
+  {
+    BackupContext status = getBackupContext(backupId);
+    if(status == null) return false;
+    return status.getFlag() == BACKUPSTATUS.FAILED;
+  }
+
+  private BackupContext getBackupContext(String backupId) throws IOException
+  {
+    Configuration conf = BackupClient.getConf();
+    BackupSystemTable table = BackupSystemTable.getTable(conf);
+    BackupContext status =  table.readBackupStatus(backupId);
+    return status;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
new file mode 100644
index 0000000..8be07bc
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
@@ -0,0 +1,99 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestBackupBoundaryTests extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupBoundaryTests.class);
+
+  /**
+   * Verify that full backup is created on a single empty table correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupSingleEmpty() throws Exception {
+
+    LOG.info("create full backup image on single table");
+
+    String backupId =
+        BackupClient.create("full", BACKUP_ROOT_DIR, table3.getNameAsString(), null);
+    LOG.info("Finished Backup");
+    assertTrue(checkSucceeded(backupId));
+  }
+
+  /**
+   * Verify that full backup is created on multiple empty tables correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupMultipleEmpty() throws Exception {
+    LOG.info("create full backup image on mulitple empty tables");
+    String tableset =
+        table3.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table4.getNameAsString();
+    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
+    assertTrue(checkSucceeded(backupId));
+
+  }
+
+  /**
+   * Verify that full backup fails on a single table that does not exist.
+   * @throws Exception
+   */
+  @Test(expected = RuntimeException.class)
+  public void testFullBackupSingleDNE() throws Exception {
+
+    LOG.info("test full backup fails on a single table that does not exist");
+    BackupClient.create("full", BACKUP_ROOT_DIR, "tabledne", null);
+  }
+
+  /**
+   * Verify that full backup fails on multiple tables that do not exist.
+   * @throws Exception
+   */
+  @Test(expected = RuntimeException.class)
+  public void testFullBackupMultipleDNE() throws Exception {
+
+    LOG.info("test full backup fails on multiple tables that do not exist");
+    BackupClient.create("full", BACKUP_ROOT_DIR, "table1dne,table2dne", null);
+  }
+
+  /**
+   * Verify that full backup fails on tableset containing real and fake tables.
+   * @throws Exception
+   */
+  @Test(expected = RuntimeException.class)
+  public void testFullBackupMixExistAndDNE() throws Exception {
+    LOG.info("create full backup fails on tableset containing real and fake table");
+    String tableset =
+        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + "tabledne";
+    BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
new file mode 100644
index 0000000..158479b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
@@ -0,0 +1,161 @@
+/**
+ * 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 static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Iterables;
+
+@Category(LargeTests.class)
+public class TestBackupLogCleaner extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestBackupLogCleaner.class);
+
+  // implements all test cases in 1 test since incremental full backup/
+  // incremental backup has dependencies
+  @Test
+  public void testBackupLogCleaner() throws Exception {
+
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+    String tablesetFull =
+        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table3.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table4.getNameAsString();
+
+    BackupSystemTable systemTable = BackupSystemTable.getTable(TEST_UTIL.getConfiguration());
+    // Verify that we have no backup sessions yet
+    assertFalse(systemTable.hasBackupSessions());
+
+    List<FileStatus> walFiles = getListOfWALFiles(TEST_UTIL.getConfiguration());
+    List<String> swalFiles = convert(walFiles);
+    BackupLogCleaner cleaner = new BackupLogCleaner();
+    cleaner.setConf(TEST_UTIL.getConfiguration());
+
+    Iterable<FileStatus> deletable = cleaner.getDeletableFiles(walFiles);
+    // We can delete all files because we do not have yet recorded backup sessions
+    assertTrue(Iterables.size(deletable) == walFiles.size());
+
+    systemTable.addWALFiles(swalFiles, "backup");
+    String backupIdFull = BackupClient.create("full", BACKUP_ROOT_DIR, tablesetFull, null);
+    assertTrue(checkSucceeded(backupIdFull));
+    // Check one more time
+    deletable = cleaner.getDeletableFiles(walFiles);
+    // We can delete wal files because they were saved into hbase:backup table
+    int size = Iterables.size(deletable);
+    assertTrue(size == walFiles.size());
+
+    List<FileStatus> newWalFiles = getListOfWALFiles(TEST_UTIL.getConfiguration());
+    LOG.debug("WAL list after full backup");
+    convert(newWalFiles);
+
+    // New list of wal files is greater than the previous one,
+    // because new wal per RS have been opened after full backup
+    assertTrue(walFiles.size() < newWalFiles.size());
+    // TODO : verify that result files are not walFiles collection
+    Connection conn = ConnectionFactory.createConnection(conf1);
+    // #2 - insert some data to table
+    HTable t1 = (HTable) conn.getTable(table1);
+    Put p1;
+    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+      p1 = new Put(Bytes.toBytes("row-t1" + i));
+      p1.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t1.put(p1);
+    }
+
+    t1.close();
+
+    HTable t2 = (HTable) conn.getTable(table2);
+    Put p2;
+    for (int i = 0; i < 5; i++) {
+      p2 = new Put(Bytes.toBytes("row-t2" + i));
+      p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t2.put(p2);
+    }
+
+    t2.close();
+
+    // #3 - incremental backup for multiple tables
+    String tablesetIncMultiple =
+        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table3.getNameAsString();
+
+    String backupIdIncMultiple =
+        BackupClient.create("incremental", BACKUP_ROOT_DIR, tablesetIncMultiple, null);
+    assertTrue(checkSucceeded(backupIdIncMultiple));
+    deletable = cleaner.getDeletableFiles(newWalFiles);
+
+    assertTrue(Iterables.size(deletable) == newWalFiles.size());
+
+    conn.close();
+
+  }
+
+  private List<String> convert(List<FileStatus> walFiles) {
+    List<String> result = new ArrayList<String>();
+    for (FileStatus fs : walFiles) {
+      LOG.debug("+++WAL: " + fs.getPath().toString());
+      result.add(fs.getPath().toString());
+    }
+    return result;
+  }
+
+  private List<FileStatus> getListOfWALFiles(Configuration c) throws IOException {
+    Path logRoot = new Path(FSUtils.getRootDir(c), HConstants.HREGION_LOGDIR_NAME);
+    FileSystem fs = FileSystem.get(c);
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(logRoot, true);
+    List<FileStatus> logFiles = new ArrayList<FileStatus>();
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (lfs.isFile() && !DefaultWALProvider.isMetaFile(lfs.getPath())) {
+        logFiles.add(lfs);
+        LOG.info(lfs);
+      }
+    }
+    return logFiles;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
new file mode 100644
index 0000000..7421707
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
@@ -0,0 +1,341 @@
+/**
+ *
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS;
+import org.apache.hadoop.hbase.backup.BackupUtil.BackupCompleteData;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test cases for hbase:backup API 
+ *
+ */
+@Category(MediumTests.class)
+public class TestBackupSystemTable {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  protected static Configuration conf = UTIL.getConfiguration();
+  protected static MiniHBaseCluster cluster;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    cluster = UTIL.startMiniCluster();
+
+  }
+
+  @Test
+  public void testUpdateReadDeleteBackupStatus() throws IOException {
+    BackupSystemTable table = BackupSystemTable.getTable(conf);
+    BackupContext ctx = createBackupContext();
+    table.updateBackupStatus(ctx);
+    BackupContext readCtx = table.readBackupStatus(ctx.getBackupId());
+    assertTrue(compare(ctx, readCtx));
+
+    // try fake backup id
+    readCtx = table.readBackupStatus("fake");
+
+    assertNull(readCtx);
+    // delete backup context
+    table.deleteBackupStatus(ctx.getBackupId());
+    readCtx = table.readBackupStatus(ctx.getBackupId());
+    assertNull(readCtx);
+    cleanBackupTable();
+  }
+
+  @Test
+  public void testWriteReadBackupStartCode() throws IOException {
+    BackupSystemTable table = BackupSystemTable.getTable(conf);
+    String code = "100";
+    table.writeBackupStartCode(code);
+    String readCode = table.readBackupStartCode();
+    assertEquals(code, readCode);
+    cleanBackupTable();
+  }
+
+  private void cleanBackupTable() throws IOException {
+    Admin admin = UTIL.getHBaseAdmin();
+    admin.disableTable(BackupSystemTable.getTableName());
+    admin.truncateTable(BackupSystemTable.getTableName(), true);
+    if (admin.isTableDisabled(BackupSystemTable.getTableName())) {
+      admin.enableTable(BackupSystemTable.getTableName());
+    }
+  }
+
+  @Test
+  public void testBackupHistory() throws IOException {
+    BackupSystemTable table = BackupSystemTable.getTable(conf);
+    int n = 10;
+    List<BackupContext> list = createBackupContextList(n);
+
+    // Load data
+    for (BackupContext bc : list) {
+      // Make sure we set right status
+      bc.setFlag(BACKUPSTATUS.COMPLETE);
+      table.updateBackupStatus(bc);
+    }
+
+    // Reverse list for comparison
+    Collections.reverse(list);
+    ArrayList<BackupCompleteData> history = table.getBackupHistory();
+    assertTrue(history.size() == n);
+
+    for (int i = 0; i < n; i++) {
+      BackupContext ctx = list.get(i);
+      BackupCompleteData data = history.get(i);
+      assertTrue(compare(ctx, data));
+    }
+
+    cleanBackupTable();
+
+  }
+
+  @Test
+  public void testRegionServerLastLogRollResults() throws IOException {
+    BackupSystemTable table = BackupSystemTable.getTable(conf);
+
+    String[] servers = new String[] { "server1", "server2", "server3" };
+    String[] timestamps = new String[] { "100", "102", "107" };
+
+    for (int i = 0; i < servers.length; i++) {
+      table.writeRegionServerLastLogRollResult(servers[i], timestamps[i]);
+    }
+
+    HashMap<String, String> result = table.readRegionServerLastLogRollResult();
+    assertTrue(servers.length == result.size());
+    Set<String> keys = result.keySet();
+    String[] keysAsArray = new String[keys.size()];
+    keys.toArray(keysAsArray);
+    Arrays.sort(keysAsArray);
+
+    for (int i = 0; i < keysAsArray.length; i++) {
+      assertEquals(keysAsArray[i], servers[i]);
+      String ts1 = timestamps[i];
+      String ts2 = result.get(keysAsArray[i]);
+      assertEquals(ts1, ts2);
+    }
+
+    cleanBackupTable();
+
+  }
+
+  @Test
+  public void testIncrementalBackupTableSet() throws IOException {
+    BackupSystemTable table = BackupSystemTable.getTable(conf);
+
+    TreeSet<String> tables1 = new TreeSet<String>();
+
+    tables1.add("t1");
+    tables1.add("t2");
+    tables1.add("t3");
+
+    TreeSet<String> tables2 = new TreeSet<String>();
+
+    tables2.add("t3");
+    tables2.add("t4");
+    tables2.add("t5");
+
+    table.addIncrementalBackupTableSet(tables1);
+    TreeSet<String> res1 = (TreeSet<String>) table.getIncrementalBackupTableSet();
+    assertTrue(tables1.size() == res1.size());
+    Iterator<String> desc1 = tables1.descendingIterator();
+    Iterator<String> desc2 = res1.descendingIterator();
+    while (desc1.hasNext()) {
+      assertEquals(desc1.next(), desc2.next());
+    }
+
+    table.addIncrementalBackupTableSet(tables2);
+    TreeSet<String> res2 = (TreeSet<String>) table.getIncrementalBackupTableSet();
+    assertTrue((tables2.size() + tables1.size() - 1) == res2.size());
+
+    tables1.addAll(tables2);
+
+    desc1 = tables1.descendingIterator();
+    desc2 = res2.descendingIterator();
+
+    while (desc1.hasNext()) {
+      assertEquals(desc1.next(), desc2.next());
+    }
+    cleanBackupTable();
+
+  }
+
+  @Test
+  public void testRegionServerLogTimestampMap() throws IOException {
+    BackupSystemTable table = BackupSystemTable.getTable(conf);
+
+    TreeSet<String> tables = new TreeSet<String>();
+
+    tables.add("t1");
+    tables.add("t2");
+    tables.add("t3");
+
+    HashMap<String, String> rsTimestampMap = new HashMap<String, String>();
+
+    rsTimestampMap.put("rs1", "100");
+    rsTimestampMap.put("rs2", "101");
+    rsTimestampMap.put("rs3", "103");
+
+    table.writeRegionServerLogTimestamp(tables, rsTimestampMap);
+
+    HashMap<String, HashMap<String, String>> result = table.readLogTimestampMap();
+
+    assertTrue(tables.size() == result.size());
+
+    for (String t : tables) {
+      HashMap<String, String> rstm = result.get(t);
+      assertNotNull(rstm);
+      assertEquals(rstm.get("rs1"), "100");
+      assertEquals(rstm.get("rs2"), "101");
+      assertEquals(rstm.get("rs3"), "103");
+    }
+
+    Set<String> tables1 = new TreeSet<String>();
+
+    tables1.add("t3");
+    tables1.add("t4");
+    tables1.add("t5");
+
+    HashMap<String, String> rsTimestampMap1 = new HashMap<String, String>();
+
+    rsTimestampMap1.put("rs1", "200");
+    rsTimestampMap1.put("rs2", "201");
+    rsTimestampMap1.put("rs3", "203");
+
+    table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1);
+
+    result = table.readLogTimestampMap();
+
+    assertTrue(5 == result.size());
+
+    for (String t : tables) {
+      HashMap<String, String> rstm = result.get(t);
+      assertNotNull(rstm);
+      if (t.equals("t3") == false) {
+        assertEquals(rstm.get("rs1"), "100");
+        assertEquals(rstm.get("rs2"), "101");
+        assertEquals(rstm.get("rs3"), "103");
+      } else {
+        assertEquals(rstm.get("rs1"), "200");
+        assertEquals(rstm.get("rs2"), "201");
+        assertEquals(rstm.get("rs3"), "203");
+      }
+    }
+
+    for (String t : tables1) {
+      HashMap<String, String> rstm = result.get(t);
+      assertNotNull(rstm);
+      assertEquals(rstm.get("rs1"), "200");
+      assertEquals(rstm.get("rs2"), "201");
+      assertEquals(rstm.get("rs3"), "203");
+    }
+
+    cleanBackupTable();
+
+  }
+
+  @Test
+  public void testAddWALFiles() throws IOException {
+    BackupSystemTable table = BackupSystemTable.getTable(conf);
+    FileSystem fs = FileSystem.get(conf);
+    List<String> files =
+        Arrays.asList("hdfs://server/WALs/srv1,101,15555/srv1,101,15555.default.1",
+          "hdfs://server/WALs/srv2,102,16666/srv2,102,16666.default.2",
+            "hdfs://server/WALs/srv3,103,17777/srv3,103,17777.default.3");
+    String newFile = "hdfs://server/WALs/srv1,101,15555/srv1,101,15555.default.5";
+
+    table.addWALFiles(files, "backup");
+
+    assertTrue(table.checkWALFile(files.get(0)));
+    assertTrue(table.checkWALFile(files.get(1)));
+    assertTrue(table.checkWALFile(files.get(2)));
+    assertFalse(table.checkWALFile(newFile));
+
+    cleanBackupTable();
+  }
+
+  private boolean compare(BackupContext ctx, BackupCompleteData data) {
+
+    return ctx.getBackupId().equals(data.getBackupToken())
+        && ctx.getTargetRootDir().equals(data.getBackupRootPath())
+        && ctx.getType().equals(data.getType())
+        && ctx.getStartTs() == Long.parseLong(data.getStartTime())
+        && ctx.getEndTs() == Long.parseLong(data.getEndTime());
+
+  }
+
+  private boolean compare(BackupContext one, BackupContext two) {
+    return one.getBackupId().equals(two.getBackupId()) && one.getType().equals(two.getType())
+        && one.getTargetRootDir().equals(two.getTargetRootDir())
+        && one.getStartTs() == two.getStartTs() && one.getEndTs() == two.getEndTs();
+  }
+
+  private BackupContext createBackupContext() {
+
+    BackupContext ctxt =
+        new BackupContext("backup_" + System.nanoTime(), "full", new String[] { "t1", "t2", "t3" },
+          "/hbase/backup", null);
+    ctxt.setStartTs(System.currentTimeMillis());
+    ctxt.setEndTs(System.currentTimeMillis() + 1);
+    return ctxt;
+  }
+
+  private List<BackupContext> createBackupContextList(int size) {
+    List<BackupContext> list = new ArrayList<BackupContext>();
+    for (int i = 0; i < size; i++) {
+      list.add(createBackupContext());
+      try {
+        Thread.sleep(10);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+    }
+    return list;
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    if (cluster != null) cluster.shutdown();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
new file mode 100644
index 0000000..19caf37
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
@@ -0,0 +1,83 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestFullBackup extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestFullBackup.class);
+
+  /**
+   * Verify that full backup is created on a single table with data correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupSingle() throws Exception {
+
+    LOG.info("test full backup on a single table with data");
+    String backupId =
+        BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+  }
+
+  /**
+   * Verify that full backup is created on multiple tables correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupMultiple() throws Exception {
+    LOG.info("create full backup image on multiple tables with data");
+    String tableset =
+        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table2.getNameAsString();
+    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
+    assertTrue(checkSucceeded(backupId));
+
+  }
+
+  /**
+   * Verify that full backup is created on all tables correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupAll() throws Exception {
+    LOG.info("create full backup image on all tables");
+    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, null, null);
+    assertTrue(checkSucceeded(backupId));
+
+  }
+
+  /**
+   * Verify that full backup is created on a table correctly using a snapshot.
+   * @throws Exception
+   */
+  //@Test
+  //public void testFullBackupUsingSnapshot() throws Exception {
+   // HBaseAdmin hba = new HBaseAdmin(conf1);
+    //String snapshot = "snapshot";
+    //hba.snapshot(snapshot, table1);
+    //LOG.info("create full backup image on a table using snapshot");
+    //String backupId =
+    //    BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(),
+    //      snapshot);
+  // }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
new file mode 100644
index 0000000..1262c54
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
@@ -0,0 +1,166 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestFullRestore extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestFullRestore.class);
+
+  /**
+   * Verify that a single table is restored to a new table
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreSingle() throws Exception {
+
+    LOG.info("test full restore on a single table empty table");
+    String backupId =
+        BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+
+    String[] tableset = new String[] { table1.getNameAsString() };
+    String[] tablemap = new String[] { table1_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
+      false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(TableName.valueOf(table1_restore)));
+    TEST_UTIL.deleteTable(TableName.valueOf(table1_restore));
+    hba.close();
+  }
+
+  /**
+   * Verify that multiple tables are restored to new tables.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreMultiple() throws Exception {
+    LOG.info("create full backup image on multiple tables");
+    String tableset =
+        table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table3.getNameAsString();
+    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
+    assertTrue(checkSucceeded(backupId));
+
+    String[] restore_tableset = new String[] { table2.getNameAsString(), table3.getNameAsString() };
+    String[] tablemap = new String[] { table2_restore, table3_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, 
+      restore_tableset, tablemap, false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(TableName.valueOf(table2_restore)));
+    assertTrue(hba.tableExists(TableName.valueOf(table3_restore)));
+    TEST_UTIL.deleteTable(TableName.valueOf(table2_restore));
+    TEST_UTIL.deleteTable(TableName.valueOf(table3_restore));
+    hba.close();
+  }
+
+  /**
+   * Verify that a single table is restored using overwrite
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreSingleOverwrite() throws Exception {
+
+    LOG.info("test full restore on a single table empty table");
+    String backupId =
+        BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+
+    String[] tableset = new String[] { table1.getNameAsString() };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, null,
+      true);
+  }
+
+  /**
+   * Verify that multiple tables are restored to new tables using overwrite.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreMultipleOverwrite() throws Exception {
+    LOG.info("create full backup image on multiple tables");
+    String tableset =
+        table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table3.getNameAsString();
+    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
+    assertTrue(checkSucceeded(backupId));
+
+    String[] restore_tableset = new String[] { table2.getNameAsString(), table3.getNameAsString() };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, 
+      false, restore_tableset, null, true);
+  }
+
+  /**
+   * Verify that restore fails on a single table that does not exist.
+   * @throws Exception
+   */
+  @Test(expected = IOException.class)
+  public void testFullRestoreSingleDNE() throws Exception {
+
+    LOG.info("test restore fails on a single table that does not exist");
+    String backupId =
+        BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+
+    String[] tableset = new String[] { "faketable" };
+    String[] tablemap = new String[] { table1_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
+      false);
+  }
+
+  /**
+   * Verify that restore fails on multiple tables that do not exist.
+   * @throws Exception
+   */
+  @Test(expected = IOException.class)
+  public void testFullRestoreMultipleDNE() throws Exception {
+
+    LOG.info("test restore fails on multiple tables that do not exist");
+    String tableset =
+        table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table3.getNameAsString();
+    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
+    assertTrue(checkSucceeded(backupId));
+
+    String[] restore_tableset = new String[] { "faketable1", "faketable2" };
+    String[] tablemap = new String[] { table2_restore, table3_restore };
+    Path path = new Path(BACKUP_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, 
+      false, restore_tableset, tablemap, false);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
new file mode 100644
index 0000000..5437b84
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
@@ -0,0 +1,179 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestIncrementalBackup extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackup.class);
+  //implement all testcases in 1 test since incremental backup/restore has dependencies
+  @Test
+  public void TestIncBackupRestore() throws Exception {
+    HBackupFileSystem hbfs;
+
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+    String tablesetFull =
+        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table3.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table4.getNameAsString();
+
+    String backupIdFull =
+        BackupClient.create("full", BACKUP_ROOT_DIR, tablesetFull, null);
+    assertTrue(checkSucceeded(backupIdFull));
+
+    Connection conn = ConnectionFactory.createConnection(conf1); 
+    // #2 - insert some data to table
+    HTable t1 = (HTable) conn.getTable(table1);
+    Put p1;
+    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+      p1 = new Put(Bytes.toBytes("row-t1" + i));
+      p1.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t1.put(p1);
+    }
+
+    Assert.assertThat(TEST_UTIL.countRows(t1), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
+    t1.close();
+
+    HTable t2 =  (HTable) conn.getTable(table2);
+    Put p2;
+    for (int i = 0; i < 5; i++) {
+      p2 = new Put(Bytes.toBytes("row-t2" + i));
+      p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t2.put(p2);
+    }
+
+    Assert.assertThat(TEST_UTIL.countRows(t2), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
+    t2.close();
+
+    // #3 - incremental backup for multiple tables
+    String tablesetIncMultiple =
+        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
+            + table3.getNameAsString();
+
+    String backupIdIncMultiple = BackupClient.create("incremental", BACKUP_ROOT_DIR, 
+      tablesetIncMultiple, null);
+    assertTrue(checkSucceeded(backupIdIncMultiple));
+
+
+    // #4 - restore full backup for all tables, without overwrite
+    String[] tablesRestoreFull =
+        new String[] { table1.getNameAsString(), table2.getNameAsString(),
+            table3.getNameAsString(), table4.getNameAsString() };
+
+    String[] tablesMapFull =
+        new String[] { table1_restore, table2_restore, table3_restore, table4_restore };
+
+    hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdFull);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupIdFull, false, false,
+      tablesRestoreFull,
+      tablesMapFull, false);
+
+    // #5.1 - check tables for full restore
+    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hAdmin.tableExists(TableName.valueOf(table1_restore)));
+    assertTrue(hAdmin.tableExists(TableName.valueOf(table2_restore)));
+    assertTrue(hAdmin.tableExists(TableName.valueOf(table3_restore)));
+    assertTrue(hAdmin.tableExists(TableName.valueOf(table4_restore)));
+
+    hAdmin.close();
+
+    // #5.2 - checking row count of tables for full restore
+    HTable hTable = (HTable) conn.getTable(TableName.valueOf(table1_restore));
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(TableName.valueOf(table2_restore));
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(TableName.valueOf(table3_restore));
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(TableName.valueOf(table4_restore));
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
+    hTable.close();
+
+    // #6 - restore incremental backup for multiple tables, with overwrite
+    String[] tablesRestoreIncMultiple =
+        new String[]
+        { table1.getNameAsString(), table2.getNameAsString(), table3.getNameAsString() };
+    String[] tablesMapIncMultiple =
+        new String[] { table1_restore, table2_restore, table3_restore };
+    hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdIncMultiple);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupIdIncMultiple, false, false,
+      tablesRestoreIncMultiple, tablesMapIncMultiple, true);
+
+    hTable = (HTable) conn.getTable(TableName.valueOf(table1_restore));
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(TableName.valueOf(table2_restore));
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(TableName.valueOf(table3_restore));
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
+    hTable.close();
+
+    // #7 - incremental backup for single, empty table
+
+    String tablesetIncEmpty = table4.getNameAsString();
+    String backupIdIncEmpty =
+        BackupClient.create("incremental", BACKUP_ROOT_DIR, tablesetIncEmpty, null);
+    assertTrue(checkSucceeded(backupIdIncEmpty));
+
+
+    // #8 - restore incremental backup for single empty table, with overwrite
+    String[] tablesRestoreIncEmpty = new String[] { table4.getNameAsString() };
+    String[] tablesMapIncEmpty = new String[] { table4_restore };
+    hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdIncEmpty);
+    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupIdIncEmpty, false, false,
+      tablesRestoreIncEmpty,
+      tablesMapIncEmpty, true);
+
+    hTable = (HTable) conn.getTable(TableName.valueOf(table4_restore));
+    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
+    hTable.close();
+    conn.close();
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
new file mode 100644
index 0000000..03822a2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
@@ -0,0 +1,44 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestRemoteBackup extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRemoteBackup.class);
+
+  /**
+   * Verify that a remote full backup is created on a single table with data correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupRemote() throws Exception {
+
+    LOG.info("test remote full backup on a single table");
+
+    // String rootdir = TEST_UTIL2.getDefaultRootDirPath() + BACKUP_ROOT_DIR;
+    // LOG.info("ROOTDIR " + rootdir);
+    String backupId =
+        BackupClient.create("full", BACKUP_REMOTE_ROOT_DIR, table1.getNameAsString(), null);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
new file mode 100644
index 0000000..e1315c5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
@@ -0,0 +1,54 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestRemoteRestore extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRemoteRestore.class);
+
+  /**
+   * Verify that a remote restore on a single table is successful.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreRemote() throws Exception {
+
+    LOG.info("test remote full backup on a single table");
+    String backupId =
+        BackupClient.create("full", BACKUP_REMOTE_ROOT_DIR, table1.getNameAsString(), null);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+    String[] tableset = new String[] { table1.getNameAsString() };
+    String[] tablemap = new String[] { table1_restore };
+    Path path = new Path(BACKUP_REMOTE_ROOT_DIR);
+    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
+    RestoreClient.restore_stage1(hbfs, BACKUP_REMOTE_ROOT_DIR, backupId, false, false, tableset,
+      tablemap, false);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(TableName.valueOf(table1_restore)));
+    TEST_UTIL.deleteTable(TableName.valueOf(table1_restore));
+    hba.close();
+  }
+
+}
\ No newline at end of file


[5/6] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java
new file mode 100644
index 0000000..f764f18
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java
@@ -0,0 +1,744 @@
+/**
+ * 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.HashMap;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+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.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+
+/**
+ * A Handler to carry the operations of backup progress
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupHandler implements Callable<Object> {
+  private static final Log LOG = LogFactory.getLog(BackupHandler.class);
+
+  // backup phase
+  // for overall backup (for table list, some table may go online, while some may go offline)
+  protected static enum BACKUPPHASE {
+    REQUEST, SNAPSHOT, PREPARE_INCREMENTAL, SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
+  }
+
+  // backup status flag
+  protected static enum BACKUPSTATUS {
+    WAITING, ONGOING, COMPLETE, FAILED, CANCELLED;
+  }
+
+  protected BackupContext backupContext;
+  private BackupManager backupManager;
+  private Configuration conf;
+
+  public BackupHandler(BackupContext backupContext, 
+      BackupManager backupManager, Configuration conf) {
+    this.backupContext = backupContext;
+    this.backupManager = backupManager;
+    this.conf = conf;
+  }
+
+  public BackupContext getBackupContext() {
+    return backupContext;
+  }
+
+  @Override
+  public Object call() throws Exception {
+    try {
+      // overall backup begin
+      this.beginBackup(backupContext);
+      HashMap<String, String> newTimestamps = null;
+      boolean fromExistingSnapshot = false; // supported by future jira
+      // handle full or incremental backup for table or table list
+      if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
+        String savedStartCode = null;
+        HBaseAdmin hbadmin = null;
+        Connection conn = null;
+        boolean firstBackup = false;
+        // do snapshot for full table backup, if backing up from existing snapshot, then skip the
+        // step of taking snapshot
+        if (fromExistingSnapshot) {
+          LOG.error("Backup from existing snapshot, so skip the snapshot step. ");
+          LOG.error("This feature will be supported by a future jira");
+          throw new RuntimeException("Backup from existing snapshot is not supported");
+        } else {
+          try {
+            savedStartCode = backupManager.readBackupStartCode();
+            firstBackup = savedStartCode == null;
+            if (firstBackup) {
+              // This is our first backup. Let's put some marker on ZK so that we can hold the logs
+              // while we do the backup.
+              backupManager.writeBackupStartCode("0");
+            }
+            // We roll log here before we do the snapshot. It is possible there is duplicate data
+            // in the log that is already in the snapshot. But if we do it after the snapshot, we
+            // could have data loss.
+            // A better approach is to do the roll log on each RS in the same global procedure as
+            // the snapshot.
+            LOG.info("Execute roll log procedure for full backup ...");
+            conn = ConnectionFactory.createConnection(conf);
+            hbadmin = (HBaseAdmin) conn.getAdmin();
+            hbadmin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
+              LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, new HashMap<String, String>());
+            newTimestamps = backupManager.readRegionServerLastLogRollResult();
+            if (firstBackup) {
+              // Updates registered log files
+              // We record ALL old WAL files as registered, because
+              // this is a first full backup in the system and these
+              // files are not needed for next incremental backup
+              List<String> logFiles = BackupUtil.getWALFilesOlderThan(conf, newTimestamps);
+              backupManager.recordWALFiles(logFiles);
+            }
+            this.snapshotForFullBackup(backupContext);
+          } catch (BackupException e) {
+            // fail the overall backup and return
+            this.failBackup(backupContext, e, "Unexpected BackupException : ");
+            return null;
+          } finally {
+            if (hbadmin != null) {
+              hbadmin.close();
+            }
+            if (conn != null) {
+              conn.close();
+            }
+          }
+        }
+
+        // update the faked progress currently for snapshot done
+        this.updateProgress("10.0%", 0);
+        // do snapshot copy
+        try {
+          this.snapshotCopy(backupContext);
+        } catch (Exception e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected BackupException : ");
+          return null;
+        }
+        // Updates incremental backup table set
+        backupManager.addIncrementalBackupTableSet(backupContext.getTables());
+
+      } else if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+        LOG.info("For incremental backup, current table set is "
+            + backupManager.getIncrementalBackupTableSet());
+        // do incremental table backup preparation
+        backupContext.setPhase(BACKUPPHASE.PREPARE_INCREMENTAL);
+        // avoid action if has been cancelled
+        if (backupContext.isCancelled()) {
+          return null;
+        }
+        try {
+          IncrementalBackupManager incrBackupManager = new IncrementalBackupManager(backupManager);
+
+          newTimestamps = incrBackupManager.getIncrBackupLogFileList(backupContext);
+        } catch (Exception e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected Exception : ");
+          return null;
+        }
+        // update the faked progress currently for incremental preparation done
+        this.updateProgress("10.0%", 0);
+
+        // do incremental copy
+        try {
+          // copy out the table and region info files for each table
+          BackupUtil.copyTableRegionInfo(backupContext, conf);
+          this.incrementalCopy(backupContext);
+          // Save list of WAL files copied
+          backupManager.recordWALFiles(backupContext.getIncrBackupFileList());
+        } catch (Exception e) {
+          // fail the overall backup and return
+          this.failBackup(backupContext, e, "Unexpected exception doing incremental copy : ");
+          return null;
+        }
+
+      } else {
+        LOG.error("Unsupport backup type: " + backupContext.getType());
+      }
+
+      // set overall backup status: complete. Here we make sure to complete the backup. After this
+      // checkpoint, even if entering cancel process, will let the backup finished
+      backupContext.setFlag(BACKUPSTATUS.COMPLETE);
+
+      if (!fromExistingSnapshot) {
+        if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+          // Set the previousTimestampMap which is before this current log roll to the manifest.
+          HashMap<String, HashMap<String, String>> previousTimestampMap =
+              backupManager.readLogTimestampMap();
+          backupContext.setIncrTimestampMap(previousTimestampMap);
+        }
+        // The table list in backupContext is good for both full backup and incremental backup.
+        // For incremental backup, it contains the incremental backup table set.
+
+        backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps);
+
+        HashMap<String, HashMap<String, String>> newTableSetTimestampMap =
+            backupManager.readLogTimestampMap();
+
+        String newStartCode =
+            BackupUtil.getMinValue(BackupUtil.getRSLogTimestampMins(newTableSetTimestampMap));
+        backupManager.writeBackupStartCode(newStartCode);
+      }
+
+      // backup complete
+      this.completeBackup(backupContext);
+    } catch (Exception e) {
+      // even during completing backup (#completeBackup(backupContext)), exception may occur, or
+      // exception occur during other process, fail the backup finally
+      this.failBackup(backupContext, e, "Error caught during backup progress: ");
+    }
+    return null;
+  }
+
+  /**
+   * Begin the overall backup.
+   * @param backupContext backup context
+   * @throws IOException exception
+   */
+  private void beginBackup(BackupContext backupContext) throws IOException {
+
+    // set the start timestamp of the overall backup
+    long startTs = EnvironmentEdgeManager.currentTime();
+    backupContext.setStartTs(startTs);
+    // set overall backup status: ongoing
+    backupContext.setFlag(BACKUPSTATUS.ONGOING);
+    LOG.info("Backup " + backupContext.getBackupId() + " starts at " + startTs + ".");
+
+    backupManager.updateBackupStatus(backupContext);
+    LOG.debug("Backup session " + backupContext.getBackupId() + " has been started.");
+
+  }
+
+  /**
+   * Snapshot for full table backup.
+   * @param backupContext backup context
+   * @throws IOException exception
+   */
+  private void snapshotForFullBackup(BackupContext backupContext) throws IOException {
+
+    LOG.info("HBase snapshot full backup for " + backupContext.getBackupId());
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    HBaseAdmin hbadmin = null;
+    Connection conn = null;
+
+    // we do HBase snapshot for tables in the table list one by one currently
+    for (String table : backupContext.getTables()) {
+
+      // avoid action if it has been cancelled
+      if (backupContext.isCancelled()) {
+        return;
+      }
+
+      HBaseProtos.SnapshotDescription backupSnapshot;
+      try {
+        // wrap a SnapshotDescription for offline/online snapshot
+        backupSnapshot = this.wrapSnapshotDescription(table);
+
+        // set the snapshot name in BackupStatus of this table
+        backupContext.setSnapshotName(table, backupSnapshot.getName());
+
+        // Kick off snapshot for backup
+        conn = ConnectionFactory.createConnection(conf);
+        hbadmin = (HBaseAdmin) conn.getAdmin();
+        hbadmin.snapshot(backupSnapshot);
+
+        if (LOG.isDebugEnabled() == false) {
+          // In DEBUG mode we log message already.
+          // This is not to duplicate that message.
+          LOG.info("Snapshot has been launched, waiting to finish ...");
+        }
+
+      } catch (Exception e) {
+        LOG.error("Snapshot failed to create " + getMessage(e));
+
+        // currently, we fail the overall backup if any table in the list failed, so throw the
+        // exception out for overall backup failing
+        throw new BackupException("Backup snapshot failed on table " + table, e);
+      } finally {
+        if (hbadmin != null) {
+          hbadmin.close();
+        }
+        if (conn != null) {
+          conn.close();
+        }
+      }
+
+      // set the snapshot name in BackupStatus of this table, only after snapshot success.
+      backupContext.setSnapshotName(table, backupSnapshot.getName());
+
+    } // for each table in the backup table list
+
+  }
+
+  /**
+   * Fail the overall backup.
+   * @param backupContext backup context
+   * @param e exception
+   * @throws Exception exception
+   */
+  private void failBackup(BackupContext backupContext, Exception e, String msg) throws Exception {
+
+    LOG.error(msg + getMessage(e));
+
+    // If this is a cancel exception, then we've already cleaned.
+
+    if (this.backupContext.getFlag().equals(BACKUPSTATUS.CANCELLED)) {
+      return;
+    }
+
+    // set the failure timestamp of the overall backup
+    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
+
+    // set failure message
+    backupContext.setFailedMsg(e.getMessage());
+
+    // set overall backup status: failed
+    backupContext.setFlag(BACKUPSTATUS.FAILED);
+
+    // compose the backup failed data
+    String backupFailedData =
+        "BackupId=" + backupContext.getBackupId() + ",startts=" + backupContext.getStartTs()
+        + ",failedts=" + backupContext.getEndTs() + ",failedphase=" + backupContext.getPhase()
+        + ",failedmessage=" + backupContext.getFailedMsg();
+    LOG.error(backupFailedData);
+
+    backupManager.updateBackupStatus(backupContext);
+
+    // if full backup, then delete HBase snapshots if there already have snapshots taken
+    // and also clean up export snapshot log files if exist
+    if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
+      if (!backupContext.fromExistingSnapshot()) {
+        this.deleteSnapshot(backupContext);
+      }
+      this.cleanupExportSnapshotLog();
+    } /*
+     * else { // support incremental backup code in future jira // TODO. See HBASE-14124 }
+     */
+
+    // clean up the uncompleted data at target directory if the ongoing backup has already entered
+    // the copy phase
+    // For incremental backup, DistCp logs will be cleaned with the targetDir.
+    this.cleanupTargetDir();
+
+    LOG.info("Backup " + backupContext.getBackupId() + " failed.");
+  }
+
+  /**
+   * Update the ongoing back token znode with new progress.
+   * @param newProgress progress
+   * @param bytesCopied bytes copied
+   * @throws NoNodeException exception
+   */
+  public void updateProgress(String newProgress, long bytesCopied) throws IOException {
+
+    // compose the new backup progress data, using fake number for now
+    String backupProgressData = newProgress;
+
+    backupContext.setProgress(newProgress);
+    backupManager.updateBackupStatus(backupContext);
+    LOG.debug("Backup progress data \"" + backupProgressData
+      + "\" has been updated to hbase:backup for " + backupContext.getBackupId());
+  }
+
+  /**
+   * Complete the overall backup.
+   * @param backupContext backup context
+   * @throws Exception exception
+   */
+  private void completeBackup(BackupContext backupContext) throws Exception {
+
+    // set the complete timestamp of the overall backup
+    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
+    // set overall backup status: complete
+    backupContext.setFlag(BACKUPSTATUS.COMPLETE);
+    // add and store the manifest for the backup
+    this.addManifest(backupContext);
+
+    // after major steps done and manifest persisted, do convert if needed for incremental backup
+    /* in-fly convert code here, provided by future jira */
+    LOG.debug("in-fly convert code here, provided by future jira");
+
+    // compose the backup complete data
+    String backupCompleteData =
+        this.obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs()
+        + ",completets=" + backupContext.getEndTs() + ",bytescopied="
+        + backupContext.getTotalBytesCopied();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData);
+    }
+    backupManager.updateBackupStatus(backupContext);
+
+    // when full backup is done:
+    // - delete HBase snapshot
+    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
+    // snapshots
+    if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
+      if (!backupContext.fromExistingSnapshot()) {
+        this.deleteSnapshot(backupContext);
+      }
+      this.cleanupExportSnapshotLog();
+    } else if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+      this.cleanupDistCpLog();
+    } else {
+      LOG.error(" other backup types have not been implemented yet");
+    }
+
+    LOG.info("Backup " + backupContext.getBackupId() + " completed.");
+  }
+
+  /**
+   * Get backup request meta data dir as string.
+   * @param backupContext backup context
+   * @return meta data dir
+   */
+  private String obtainBackupMetaDataStr(BackupContext backupContext) {
+    StringBuffer sb = new StringBuffer();
+    sb.append("type=" + backupContext.getType() + ",tablelist=");
+    for (String table : backupContext.getTables()) {
+      sb.append(table + ";");
+    }
+    if (sb.lastIndexOf(";") > 0) {
+      sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1);
+    }
+    sb.append(",targetRootDir=" + backupContext.getTargetRootDir());
+    if (backupContext.fromExistingSnapshot()) {
+      sb.append(",snapshot=" + backupContext.getExistingSnapshot());
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Do snapshot copy.
+   * @param backupContext backup context
+   * @throws Exception exception
+   */
+  private void snapshotCopy(BackupContext backupContext) throws Exception {
+
+    LOG.info("Snapshot copy is starting.");
+
+    // set overall backup phase: snapshot_copy
+    backupContext.setPhase(BACKUPPHASE.SNAPSHOTCOPY);
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    // call ExportSnapshot to copy files based on hbase snapshot for backup
+    // ExportSnapshot only support single snapshot export, need loop for multiple tables case
+    BackupCopyService copyService = BackupRestoreServiceFactory.getBackupCopyService(conf);
+
+    // number of snapshots matches number of tables
+    float numOfSnapshots = backupContext.getSnapshotNames().size();
+
+    LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied.");
+
+    for (String table : backupContext.getTables()) {
+      // Currently we simply set the sub copy tasks by counting the table snapshot number, we can
+      // calculate the real files' size for the percentage in the future.
+      // TODO this below
+      // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots);
+      int res = 0;
+      String[] args = new String[4];
+      args[0] = "-snapshot";
+      args[1] = backupContext.getSnapshotName(table);
+      args[2] = "-copy-to";
+      args[3] = backupContext.getBackupStatus(table).getTargetDir();
+
+      LOG.debug("Copy snapshot " + args[1] + " to " + args[3]);
+      res = copyService.copy(this, conf, BackupCopyService.Type.FULL, args);
+      // if one snapshot export failed, do not continue for remained snapshots
+      if (res != 0) {
+
+        LOG.error("Exporting Snapshot " + args[1] + " failed with return code: " + res + ".");
+
+        throw new IOException("Failed of exporting snapshot " + args[1] + " to " + args[3]
+            + " with reason code " + res);
+      }
+
+      LOG.info("Snapshot copy " + args[1] + " finished.");
+    }
+  }
+
+  /**
+   * Wrap a SnapshotDescription for a target table.
+   * @param table table
+   * @return a SnapshotDescription especially for backup.
+   */
+  private SnapshotDescription wrapSnapshotDescription(String table) {
+    // Mock a SnapshotDescription from backupContext to call SnapshotManager function,
+    // Name it in the format "snapshot_<timestamp>_<table>"
+    HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
+    builder.setTable(table);
+    TableName tableName = TableName.valueOf(table);
+    builder.setName("snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_"
+        + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString());
+    HBaseProtos.SnapshotDescription backupSnapshot = builder.build();
+
+    LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName()
+      + " from backupContext to request snapshot for backup.");
+
+    return backupSnapshot;
+  }
+
+  /**
+   * Delete HBase snapshot for backup.
+   * @param backupCtx backup context
+   * @throws Exception exception
+   */
+  private void deleteSnapshot(BackupContext backupCtx) throws IOException {
+
+    LOG.debug("Trying to delete snapshot for full backup.");
+    Connection conn = null;
+    Admin admin = null;
+    try {
+      conn = ConnectionFactory.createConnection(conf);
+      admin = conn.getAdmin();
+      for (String snapshotName : backupCtx.getSnapshotNames()) {
+        if (snapshotName == null) {
+          continue;
+        }
+        LOG.debug("Trying to delete snapshot: " + snapshotName);
+        admin.deleteSnapshot(snapshotName);
+        LOG.debug("Deleting the snapshot " + snapshotName + " for backup "
+            + backupCtx.getBackupId() + " succeeded.");
+      }
+    } finally {
+      if (admin != null) {
+        admin.close();
+      }
+      if (conn != null) {
+        conn.close();
+      }
+    }
+  }
+
+  /**
+   * Clean up directories with prefix "exportSnapshot-", which are generated when exporting
+   * snapshots.
+   * @throws IOException exception
+   */
+  private void cleanupExportSnapshotLog() throws IOException {
+    FileSystem fs = FSUtils.getCurrentFileSystem(conf);
+    Path stagingDir =
+        new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory()
+          .toString()));
+    FileStatus[] files = FSUtils.listStatus(fs, stagingDir);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      if (file.getPath().getName().startsWith("exportSnapshot-")) {
+        LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName());
+        if (FSUtils.delete(fs, file.getPath(), true) == false) {
+          LOG.warn("Can not delete " + file.getPath());
+        }
+      }
+    }
+  }
+
+  /**
+   * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying
+   * hlogs.
+   * @throws IOException exception
+   */
+  private void cleanupDistCpLog() throws IOException {
+    Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent();
+    FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
+    FileStatus[] files = FSUtils.listStatus(fs, rootPath);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      if (file.getPath().getName().startsWith("_distcp_logs")) {
+        LOG.debug("Delete log files of DistCp: " + file.getPath().getName());
+        FSUtils.delete(fs, file.getPath(), true);
+      }
+    }
+  }
+
+  /**
+   * Clean up the uncompleted data at target directory if the ongoing backup has already entered the
+   * copy phase.
+   */
+  private void cleanupTargetDir() {
+    try {
+      // clean up the uncompleted data at target directory if the ongoing backup has already entered
+      // the copy phase
+      LOG.debug("Trying to cleanup up target dir. Current backup phase: "
+          + backupContext.getPhase());
+      if (backupContext.getPhase().equals(BACKUPPHASE.SNAPSHOTCOPY)
+          || backupContext.getPhase().equals(BACKUPPHASE.INCREMENTAL_COPY)
+          || backupContext.getPhase().equals(BACKUPPHASE.STORE_MANIFEST)) {
+        FileSystem outputFs =
+            FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf);
+
+        // now treat one backup as a transaction, clean up data that has been partially copied at
+        // table level
+        for (String table : backupContext.getTables()) {
+          Path targetDirPath =
+              new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(),
+                backupContext.getBackupId(), table));
+          if (outputFs.delete(targetDirPath, true)) {
+            LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString()
+              + " done.");
+          } else {
+            LOG.info("No data has been copied to " + targetDirPath.toString() + ".");
+          }
+
+          Path tableDir = targetDirPath.getParent();
+          FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir);
+          if (backups == null || backups.length == 0) {
+            outputFs.delete(tableDir, true);
+            LOG.debug(tableDir.toString() + " is empty, remove it.");
+          }
+        }
+      }
+
+    } catch (IOException e1) {
+      LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at "
+          + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
+    }
+  }
+
+  /**
+   * Add manifest for the current backup. The manifest is stored 
+   * within the table backup directory.  
+   * @param backupContext The current backup context
+   * @throws IOException exception
+   * @throws BackupException exception
+   */
+  private void addManifest(BackupContext backupContext) throws IOException, BackupException {
+    // set the overall backup phase : store manifest
+    backupContext.setPhase(BACKUPPHASE.STORE_MANIFEST);
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    BackupManifest manifest;
+    boolean fromExistingSnapshot = false; // to be implemented in future jira
+
+    // Since we have each table's backup in its own directory structure,
+    // we'll store its manifest with the table directory.
+    for (String table : backupContext.getTables()) {
+      manifest = new BackupManifest(backupContext, table);
+      if (fromExistingSnapshot) {
+        // mark backing up from existing snapshot in manifest, so that later, dependency analysis
+        // can skip this backup image
+        LOG.debug("backup using existing snapshot will be supported in future jira");
+      } else {
+        ArrayList<BackupImage> ancestors = this.backupManager.getAncestors(backupContext, table);
+        for (BackupImage image : ancestors) {
+          manifest.addDependentImage(image);
+        }
+      }
+      if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+        // We'll store the log timestamps for this table only in its manifest.
+        HashMap<String, HashMap<String, String>> tableTimestampMap =
+            new HashMap<String, HashMap<String, String>>();
+        tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table));
+        manifest.setIncrTimestampMap(tableTimestampMap);
+      }
+      manifest.store(conf);
+    }
+
+    // For incremental backup, we store a overall manifest in
+    // <backup-root-dir>/WALs/<backup-id>
+    // This is used when created the next incremental backup
+    if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+      manifest = new BackupManifest(backupContext);
+      // set the table region server start and end timestamps for incremental backup
+      manifest.setIncrTimestampMap(backupContext.getIncrTimestampMap());
+      ArrayList<BackupImage> ancestors = this.backupManager.getAncestors(backupContext);
+      for (BackupImage image : ancestors) {
+        manifest.addDependentImage(image);
+      }
+      manifest.store(conf);
+    }
+  }
+
+  /**
+   * Do incremental copy.
+   * @param backupContext backup context
+   */
+  private void incrementalCopy(BackupContext backupContext) throws Exception {
+
+    LOG.info("Incremental copy is starting.");
+
+    // set overall backup phase: incremental_copy
+    backupContext.setPhase(BACKUPPHASE.INCREMENTAL_COPY);
+
+    // avoid action if has been cancelled
+    if (backupContext.isCancelled()) {
+      return;
+    }
+
+    // get incremental backup file list and prepare parms for DistCp
+    List<String> incrBackupFileList = backupContext.getIncrBackupFileList();
+    String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
+    strArr[strArr.length - 1] = backupContext.getHLogTargetDir();
+
+    BackupCopyService copyService = BackupRestoreServiceFactory.getBackupCopyService(conf);
+    int res = copyService.copy(this, conf, BackupCopyService.Type.INCREMENTAL, strArr);
+
+    if (res != 0) {
+      LOG.error("Copy incremental log files failed with return code: " + res + ".");
+      throw new IOException("Failed of Hadoop Distributed Copy from " + incrBackupFileList + " to "
+          + backupContext.getHLogTargetDir());
+    }
+    LOG.info("Incremental copy from " + incrBackupFileList + " to "
+        + backupContext.getHLogTargetDir() + " finished.");
+
+  }
+
+  private String getMessage(Exception e) {
+    String msg = e.getMessage();
+    if (msg == null || msg.equals("")) {
+      msg = e.getClass().getName();
+    }
+    return msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java
new file mode 100644
index 0000000..fdb3c46
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java
@@ -0,0 +1,488 @@
+/**
+ * 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 com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS;
+import org.apache.hadoop.hbase.backup.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.BackupUtil.BackupCompleteData;
+import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+
+
+
+/**
+ * Handles backup requests on server-side, creates backup context records in hbase:backup 
+ * to keep track backup. The timestamps kept in hbase:backup table will be used for future 
+ * incremental backup. Creates BackupContext and DispatchRequest.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupManager {
+  private static final Log LOG = LogFactory.getLog(BackupManager.class);
+
+  private Configuration conf = null;
+  private BackupContext backupContext = null;
+  private ExecutorService pool = null;
+
+  private boolean backupComplete = false;
+
+  private BackupSystemTable systemTable;
+
+  /**
+   * Backup manager constructor.
+   * @param conf configuration
+   * @throws IOException exception
+   */
+  public BackupManager(Configuration conf) throws IOException {
+    if (!conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) {
+      throw new BackupException("HBase backup is not enabled. Check your " + 
+          HConstants.BACKUP_ENABLE_KEY + " setting.");
+    }
+    this.conf = conf;
+    this.systemTable = BackupSystemTable.getTable(conf);
+    Runtime.getRuntime().addShutdownHook(new ExitHandler());
+  }
+
+  /**
+   * This method modifies the master's configuration in order to inject backup-related features
+   * @param conf configuration
+   */
+  public static void decorateMasterConfiguration(Configuration conf) {
+    if (!isBackupEnabled(conf)) {
+      return;
+    }
+    String plugins = conf.get(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS);
+    String cleanerClass = BackupLogCleaner.class.getCanonicalName();
+    if (!plugins.contains(cleanerClass)) {
+      conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Added log cleaner: " + cleanerClass);
+    }
+  }
+
+  private static boolean isBackupEnabled(Configuration conf) {
+    return conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT);
+  }
+
+  private class ExitHandler extends Thread {
+    public ExitHandler() {
+      super("Backup Manager Exit Handler");
+    }
+
+    public void run() {
+
+      if (backupContext != null && !backupComplete) {
+
+        // program exit and backup is not complete, then mark as cancelled to avoid submitted backup
+        // handler's taking further action
+        backupContext.markCancel();
+
+        LOG.debug("Backup is cancelled due to force program exiting.");
+        try {
+          cancelBackup(backupContext.getBackupId());
+        } catch (Exception e) {
+          String msg = e.getMessage();
+          if (msg == null || msg.equals("")) {
+            msg = e.getClass().getName();
+          }
+          LOG.error("Failed to cancel backup " + backupContext.getBackupId() + " due to " + msg);
+        }
+      }
+
+      exit();
+    }
+  }
+
+  /**
+   * Cancel the ongoing backup via backup id.
+   * @param backupId The id of the ongoing backup to be cancelled
+   * @throws Exception exception
+   */
+  private void cancelBackup(String backupId) throws Exception {
+    // TODO: will be implemented in Phase 2: HBASE-14125
+    LOG.debug("Try to cancel the backup " + backupId + ". the feature is NOT implemented yet");
+
+  }
+
+  /**
+   * Stop all the work of backup.
+   */
+  public void exit() {
+
+    // currently, we shutdown now for all ongoing back handlers, we may need to do something like
+    // record the failed list somewhere later
+    if (this.pool != null) {
+      this.pool.shutdownNow();
+    }
+
+  }
+
+  /**
+   * Create a BackupContext based on input backup request.
+   * @param backupId backup id
+   * @param type    type
+   * @param tablelist table list
+   * @param targetRootDir root dir
+   * @param snapshot snapshot name
+   * @return BackupContext context
+   * @throws BackupException exception
+   */
+  protected BackupContext createBackupContext(String backupId, String type, String tablelist,
+      String targetRootDir, String snapshot) throws BackupException {
+
+    if (targetRootDir == null) {
+      throw new BackupException("Wrong backup request parameter: target backup root directory");
+    }
+
+    if (type.equals(BackupRestoreConstants.BACKUP_TYPE_FULL) && tablelist == null) {
+      // If table list is null for full backup, which means backup all tables. Then fill the table
+      // list with all user tables from meta. It no table available, throw the request exception.
+
+      HTableDescriptor[] htds = null;
+      try (Connection conn = ConnectionFactory.createConnection(conf);
+          HBaseAdmin hbadmin = (HBaseAdmin) conn.getAdmin()) {
+
+        htds = hbadmin.listTables();
+      } catch (Exception e) {
+        throw new BackupException(e);
+      }
+
+      if (htds == null) {
+        throw new BackupException("No table exists for full backup of all tables.");
+      } else {
+        StringBuilder sb = new StringBuilder();
+        for (HTableDescriptor hTableDescriptor : htds) {
+          sb.append(hTableDescriptor.getNameAsString()
+              + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+        }
+        sb.deleteCharAt(sb.lastIndexOf(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND));
+        tablelist = sb.toString();
+
+        LOG.info("Full backup all the tables available in the cluster: " + tablelist);
+      }
+    }
+
+    // there are one or more tables in the table list
+    return new BackupContext(backupId, type,
+        tablelist.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND), targetRootDir,
+        snapshot);
+
+  }
+
+  /**
+   * Check if any ongoing backup. Currently, we only reply on checking status in hbase:backup. We
+   * need to consider to handle the case of orphan records in the future. Otherwise, all the coming
+   * request will fail.
+   * @return the ongoing backup id if on going backup exists, otherwise null
+   * @throws IOException exception
+   */
+  private String getOngoingBackupId() throws IOException {
+
+    ArrayList<BackupContext> sessions = systemTable.getBackupContexts(BACKUPSTATUS.ONGOING);
+    if (sessions.size() == 0) {
+      return null;
+    }
+    return sessions.get(0).getBackupId();
+  }
+
+  /**
+   * Start the backup manager service.
+   * @throws IOException exception
+   */
+  public void initialize() throws IOException {
+    String ongoingBackupId = this.getOngoingBackupId();
+    if (ongoingBackupId != null) {
+      LOG.info("There is a ongoing backup " + ongoingBackupId
+          + ". Can not launch new backup until no ongoing backup remains.");
+      throw new BackupException("There is ongoing backup.");
+    }
+
+    // Initialize thread pools
+    int nrThreads = this.conf.getInt("hbase.backup.threads.max", 1);
+    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
+    builder.setNameFormat("BackupHandler-%1$d");
+    this.pool =
+        new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,
+            new LinkedBlockingQueue<Runnable>(), builder.build());
+    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
+  }
+
+  /**
+   * Dispatch and handle a backup request.
+   * @param backupContext backup context
+   * @throws BackupException exception
+   */
+  public void dispatchRequest(BackupContext backupContext) throws BackupException {
+
+    this.backupContext = backupContext;
+
+    LOG.info("Got a backup request: " + "Type: " + backupContext.getType() + "; Tables: "
+        + backupContext.getTableListAsString() + "; TargetRootDir: "
+        + backupContext.getTargetRootDir());
+
+    // dispatch the request to a backup handler and put it handler map
+
+    BackupHandler handler = new BackupHandler(this.backupContext, this, conf);
+    Future<Object> future = this.pool.submit(handler);
+    // wait for the execution to complete
+    try {
+      future.get();
+    } catch (InterruptedException e) {
+      throw new BackupException(e);
+    } catch (CancellationException e) {
+      throw new BackupException(e);
+    } catch (ExecutionException e) {
+      throw new BackupException(e);
+    }
+
+    // mark the backup complete for exit handler's processing
+    backupComplete = true;
+
+    LOG.info("Backup request " + backupContext.getBackupId() + " has been executed.");
+  }
+
+  /**
+   * Get direct ancestors of the current backup.
+   * @param backupCtx The backup context for the current backup
+   * @return The ancestors for the current backup
+   * @throws IOException exception
+   * @throws BackupException exception
+   */
+  protected ArrayList<BackupImage> getAncestors(BackupContext backupCtx) throws IOException,
+      BackupException {
+    LOG.debug("Getting the direct ancestors of the current backup ...");
+
+    ArrayList<BackupImage> ancestors = new ArrayList<BackupImage>();
+
+    // full backup does not have ancestor
+    if (backupCtx.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
+      LOG.debug("Current backup is a full backup, no direct ancestor for it.");
+      return ancestors;
+    }
+
+    // get all backup history list in descending order
+
+    ArrayList<BackupCompleteData> allHistoryList = getBackupHistory();
+    for (BackupCompleteData backup : allHistoryList) {
+      BackupImage image =
+          new BackupImage(backup.getBackupToken(), backup.getType(), backup.getBackupRootPath(),
+              backup.getTableList(), Long.parseLong(backup.getStartTime()), Long.parseLong(backup
+                  .getEndTime()));
+
+      // add the full backup image as an ancestor until the last incremental backup
+      if (backup.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
+
+        // backup image from existing snapshot does not involve in dependency
+        if (backup.fromExistingSnapshot()) {
+          continue;
+        }
+        // check the backup image coverage, if previous image could be covered by the newer ones,
+        // then no need to add
+        if (!BackupManifest.canCoverImage(ancestors, image)) {
+          ancestors.add(image);
+        }
+      } else {
+        // found last incremental backup, if previously added full backup ancestor images can cover
+        // it, then this incremental ancestor is not the dependent of the current incremental
+        // backup, that is to say, this is the backup scope boundary of current table set.
+        // Otherwise, this incremental backup ancestor is the dependent ancestor of the ongoing
+        // incremental backup
+        if (BackupManifest.canCoverImage(ancestors, image)) {
+          LOG.debug("Met the backup boundary of the current table set. "
+              + "The root full backup images for the current backup scope:");
+          for (BackupImage image1 : ancestors) {
+            LOG.debug("  BackupId: " + image1.getBackupId() + ", Backup directory: "
+                + image1.getRootDir());
+          }
+        } else {
+          Path logBackupPath =
+              HBackupFileSystem.getLogBackupPath(backup.getBackupRootPath(),
+                backup.getBackupToken());
+          LOG.debug("Current backup has an incremental backup ancestor, "
+              + "touching its image manifest in " + logBackupPath.toString()
+              + " to construct the dependency.");
+
+          BackupManifest lastIncrImgManifest = new BackupManifest(conf, logBackupPath);
+          BackupImage lastIncrImage = lastIncrImgManifest.getBackupImage();
+          ancestors.add(lastIncrImage);
+
+          LOG.debug("Last dependent incremental backup image information:");
+          LOG.debug("  Token: " + lastIncrImage.getBackupId());
+          LOG.debug("  Backup directory: " + lastIncrImage.getRootDir());
+        }
+      }
+    }
+    LOG.debug("Got " + ancestors.size() + " ancestors for the current backup.");
+    return ancestors;
+  }
+
+  /**
+   * Get the direct ancestors of this backup for one table involved.
+   * @param backupContext backup context
+   * @param table table
+   * @return backupImages on the dependency list
+   * @throws BackupException exception
+   * @throws IOException exception
+   */
+  protected ArrayList<BackupImage> getAncestors(BackupContext backupContext, String table)
+      throws BackupException, IOException {
+    ArrayList<BackupImage> ancestors = getAncestors(backupContext);
+    ArrayList<BackupImage> tableAncestors = new ArrayList<BackupImage>();
+    for (BackupImage image : ancestors) {
+      if (image.hasTable(table)) {
+        tableAncestors.add(image);
+        if (image.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
+          break;
+        }
+      }
+    }
+    return tableAncestors;
+  }
+
+  /**
+   * hbase:backup operations
+   */
+
+  /**
+   * Updates status (state) of a backup session in a persistent store
+   * @param context context
+   * @throws IOException exception
+   */
+  public void updateBackupStatus(BackupContext context) throws IOException {
+    systemTable.updateBackupStatus(context);
+  }
+
+  /**
+   * Read the last backup start code (timestamp) of last successful backup. Will return null 
+   * if there is no startcode stored in hbase:backup or the value is of length 0. These two 
+   * cases indicate there is no successful backup completed so far.
+   * @return the timestamp of a last successful backup
+   * @throws IOException exception
+   */
+  public String readBackupStartCode() throws IOException {
+    return systemTable.readBackupStartCode();
+  }
+
+  /**
+   * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
+   * @param startCode start code
+   * @throws IOException exception
+   */
+  public void writeBackupStartCode(String startCode) throws IOException {
+    systemTable.writeBackupStartCode(startCode);
+  }
+
+  /**
+   * Get the RS log information after the last log roll from hbase:backup.
+   * @return RS log info
+   * @throws IOException exception
+   */
+  public HashMap<String, String> readRegionServerLastLogRollResult() throws IOException {
+    return systemTable.readRegionServerLastLogRollResult();
+  }
+
+  /**
+   * Get all completed backup information (in desc order by time)
+   * @return history info of BackupCompleteData
+   * @throws IOException exception
+   */
+  public ArrayList<BackupCompleteData> getBackupHistory() throws IOException {
+    return systemTable.getBackupHistory();
+  }
+
+  /**
+   * Write the current timestamps for each regionserver to hbase:backup after a successful full or
+   * incremental backup. Each table may have a different set of log timestamps. The saved timestamp
+   * is of the last log file that was backed up already.
+   * @param tables tables
+   * @throws IOException exception
+   */
+  public void writeRegionServerLogTimestamp(Set<String> tables,
+      HashMap<String, String> newTimestamps) throws IOException {
+    systemTable.writeRegionServerLogTimestamp(tables, newTimestamps);
+  }
+
+  /**
+   * Read the timestamp for each region server log after the last successful backup. Each table has
+   * its own set of the timestamps. The info is stored for each table as a concatinated string on ZK
+   * under //hbase//backup//incr//tablelogtimestamp//table_name
+   * @return the timestamp for each region server. key: tableName value:
+   *         RegionServer,PreviousTimeStamp
+   * @throws IOException exception
+   */
+  public HashMap<String, HashMap<String, String>> readLogTimestampMap() throws IOException {
+    return systemTable.readLogTimestampMap();
+  }
+
+  /**
+   * Return the current tables covered by incremental backup.
+   * @return set of tableNames
+   * @throws IOException exception
+   */
+  public Set<String> getIncrementalBackupTableSet() throws IOException {
+    return systemTable.getIncrementalBackupTableSet();
+  }
+
+  /**
+   * Adds set of tables to overall incremental backup table set
+   * @param tables tables
+   * @throws IOException exception
+   */
+  public void addIncrementalBackupTableSet(Set<String> tables) throws IOException {
+    systemTable.addIncrementalBackupTableSet(tables);
+  }
+  
+  /**
+   * Saves list of WAL files after incremental backup operation. These files will be stored until
+   * TTL expiration and are used by Backup Log Cleaner plugin to determine which WAL files can be
+   * safely purged.
+   */
+
+  public void recordWALFiles(List<String> files) throws IOException {
+    systemTable.addWALFiles(files, backupContext.getBackupId());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java
new file mode 100644
index 0000000..f41540b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java
@@ -0,0 +1,814 @@
+/**
+ * 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.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.codehaus.jackson.map.ObjectMapper;
+
+/**
+ * Backup manifest Contains all the meta data of a backup image. The manifest info will be bundled
+ * as manifest file together with data. So that each backup image will contain all the info needed
+ * for restore.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupManifest {
+
+  private static final Log LOG = LogFactory.getLog(BackupManifest.class);
+
+  // manifest file name
+  public static final String FILE_NAME = ".backup.manifest";
+
+  // manifest file version, current is 1.0
+  public static final String MANIFEST_VERSION = "1.0";
+
+  // tags of fields for manifest file
+  public static final String TAG_VERSION = "Manifest-Version";
+  public static final String TAG_BACKUPID = "Backup-Id";
+  public static final String TAG_BACKUPTYPE = "Backup-Type";
+  public static final String TAG_TABLESET = "Table-Set";
+  public static final String TAG_STARTTS = "Start-Timestamp";
+  public static final String TAG_COMPLETETS = "Complete-Timestamp";
+  public static final String TAG_TABLEBYTES = "Total-Table-Bytes";
+  public static final String TAG_LOGBYTES = "Total-Log-Bytes";
+  public static final String TAG_INCRTIMERANGE = "Incremental-Time-Range";
+  public static final String TAG_DEPENDENCY = "Dependency";
+  public static final String TAG_IMAGESTATE = "Image-State";
+  public static final String TAG_COMPACTION = "Compaction";
+
+  public static final String ERROR_DEPENDENCY = "DEPENDENCY_ERROR";
+
+  public static final int DELETE_SUCCESS = 0;
+  public static final int DELETE_FAILED = -1;
+
+  // currently only one state, will have CONVERTED, and MERGED in future JIRA
+  public static final String IMAGE_STATE_ORIG = "ORIGINAL";
+  public static final String IMAGE_STATE_CONVERT = "CONVERTED";
+  public static final String IMAGE_STATE_MERGE = "MERGED";
+  public static final String IMAGE_STATE_CONVERT_MERGE = "CONVERTED,MERGED";
+
+  // backup image, the dependency graph is made up by series of backup images
+
+  public static class BackupImage implements Comparable<BackupImage> {
+
+    private String backupId;
+    private String type;
+    private String rootDir;
+    private String tableSet;
+    private long startTs;
+    private long completeTs;
+    private ArrayList<BackupImage> ancestors;
+
+    public BackupImage() {
+      super();
+    }
+
+    public BackupImage(String backupId, String type, String rootDir, String tableSet, long startTs,
+        long completeTs) {
+      this.backupId = backupId;
+      this.type = type;
+      this.rootDir = rootDir;
+      this.tableSet = tableSet;
+      this.startTs = startTs;
+      this.completeTs = completeTs;
+    }
+
+    public String getBackupId() {
+      return backupId;
+    }
+
+    public void setBackupId(String backupId) {
+      this.backupId = backupId;
+    }
+
+    public String getType() {
+      return type;
+    }
+
+    public void setType(String type) {
+      this.type = type;
+    }
+
+    public String getRootDir() {
+      return rootDir;
+    }
+
+    public void setRootDir(String rootDir) {
+      this.rootDir = rootDir;
+    }
+
+    public String getTableSet() {
+      return tableSet;
+    }
+
+    public void setTableSet(String tableSet) {
+      this.tableSet = tableSet;
+    }
+
+    public long getStartTs() {
+      return startTs;
+    }
+
+    public void setStartTs(long startTs) {
+      this.startTs = startTs;
+    }
+
+    public long getCompleteTs() {
+      return completeTs;
+    }
+
+    public void setCompleteTs(long completeTs) {
+      this.completeTs = completeTs;
+    }
+
+    public ArrayList<BackupImage> getAncestors() {
+      if (this.ancestors == null) {
+        this.ancestors = new ArrayList<BackupImage>();
+      }
+      return this.ancestors;
+    }
+
+    public void addAncestor(BackupImage backupImage) {
+      this.getAncestors().add(backupImage);
+    }
+
+    public boolean hasAncestor(String token) {
+      for (BackupImage image : this.getAncestors()) {
+        if (image.getBackupId().equals(token)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public boolean hasTable(String table) {
+      String[] tables = this.getTableSet().split(";");
+      for (String t : tables) {
+        if (t.equals(table)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public int compareTo(BackupImage other) {
+      String thisBackupId = this.getBackupId();
+      String otherBackupId = other.getBackupId();
+      Long thisTS = new Long(thisBackupId.substring(thisBackupId.lastIndexOf("_") + 1));
+      Long otherTS = new Long(otherBackupId.substring(otherBackupId.lastIndexOf("_") + 1));
+      return thisTS.compareTo(otherTS);
+    }
+  }
+
+  // manifest version
+  private String version = MANIFEST_VERSION;
+
+  // hadoop hbase configuration
+  protected Configuration config = null;
+
+  // backup root directory
+  private String rootDir = null;
+
+  // backup image directory
+  private String tableBackupDir = null;
+
+  // backup log directory if this is an incremental backup
+  private String logBackupDir = null;
+
+  // backup token
+  private String token;
+
+  // backup type, full or incremental
+  private String type;
+
+  // the table set for the backup
+  private ArrayList<String> tableSet;
+
+  // actual start timestamp of the backup process
+  private long startTs;
+
+  // actual complete timestamp of the backup process
+  private long completeTs;
+
+  // total bytes for table backup image
+  private long tableBytes;
+
+  // total bytes for the backed-up logs for incremental backup
+  private long logBytes;
+
+  // the region server timestamp for tables:
+  // <table, <rs, timestamp>>
+  private Map<String, HashMap<String, String>> incrTimeRanges;
+
+  // dependency of this backup, including all the dependent images to do PIT recovery
+  private Map<String, BackupImage> dependency;
+
+  // the state of backup image
+  private String imageState;
+
+  // the indicator of the image compaction
+  private boolean isCompacted = false;
+
+  // the merge chain of the original backups, null if not a merged backup
+  private LinkedList<String> mergeChain;
+
+  /**
+   * Construct manifest for a ongoing backup.
+   * @param backupCtx The ongoing backup context
+   */
+  public BackupManifest(BackupContext backupCtx) {
+    this.token = backupCtx.getBackupId();
+    this.type = backupCtx.getType();
+    this.rootDir = backupCtx.getTargetRootDir();
+    if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+      this.logBackupDir = backupCtx.getHLogTargetDir();
+      this.logBytes = backupCtx.getTotalBytesCopied();
+    }
+    this.startTs = backupCtx.getStartTs();
+    this.completeTs = backupCtx.getEndTs();
+    this.loadTableSet(backupCtx.getTableListAsString());
+    this.setImageOriginal();
+  }
+
+  /**
+   * Construct a table level manifest for a backup of the named table.
+   * @param backupCtx The ongoing backup context
+   */
+  public BackupManifest(BackupContext backupCtx, String table) {
+    this.token = backupCtx.getBackupId();
+    this.type = backupCtx.getType();
+    this.rootDir = backupCtx.getTargetRootDir();
+    this.tableBackupDir = backupCtx.getBackupStatus(table).getTargetDir();
+    if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+      this.logBackupDir = backupCtx.getHLogTargetDir();
+      this.logBytes = backupCtx.getTotalBytesCopied();
+    }
+    this.startTs = backupCtx.getStartTs();
+    this.completeTs = backupCtx.getEndTs();
+    this.loadTableSet(table);
+    this.setImageOriginal();
+  }
+
+  /**
+   * Construct manifest from a backup directory.
+   * @param conf configuration
+   * @param backupPath backup path
+   * @throws BackupException exception
+   */
+  public BackupManifest(Configuration conf, Path backupPath) throws BackupException {
+
+    LOG.debug("Loading manifest from: " + backupPath.toString());
+    // The input backupDir may not exactly be the backup table dir.
+    // It could be the backup log dir where there is also a manifest file stored.
+    // This variable's purpose is to keep the correct and original location so
+    // that we can store/persist it.
+    this.tableBackupDir = backupPath.toString();
+    this.config = conf;
+    try {
+
+      FileSystem fs = backupPath.getFileSystem(conf);
+      FileStatus[] subFiles = FSUtils.listStatus(fs, backupPath);
+      if (subFiles == null) {
+        String errorMsg = backupPath.toString() + " does not exist";
+        LOG.error(errorMsg);
+        throw new IOException(errorMsg);
+      }
+      for (FileStatus subFile : subFiles) {
+        if (subFile.getPath().getName().equals(FILE_NAME)) {
+
+          // load and set manifest field from file content
+          FSDataInputStream in = fs.open(subFile.getPath());
+          Properties props = new Properties();
+          try {
+            props.load(in);
+          } catch (IOException e) {
+            LOG.error("Error when loading from manifest file!");
+            throw e;
+          } finally {
+            in.close();
+          }
+
+          this.version = props.getProperty(TAG_VERSION);
+          this.token = props.getProperty(TAG_BACKUPID);
+          this.type = props.getProperty(TAG_BACKUPTYPE);
+          // Here the parameter backupDir is where the manifest file is.
+          // There should always be a manifest file under:
+          // backupRootDir/namespace/table/backupId/.backup.manifest
+          this.rootDir = backupPath.getParent().getParent().getParent().toString();
+
+          Path p = backupPath.getParent();
+          if (p.getName().equals(HConstants.HREGION_LOGDIR_NAME)) {
+            this.rootDir = p.getParent().toString();
+          } else {
+            this.rootDir = p.getParent().getParent().toString();
+          }
+
+          this.loadTableSet(props.getProperty(TAG_TABLESET));
+
+          this.startTs = Long.parseLong(props.getProperty(TAG_STARTTS));
+          this.completeTs = Long.parseLong(props.getProperty(TAG_COMPLETETS));
+          this.tableBytes = Long.parseLong(props.getProperty(TAG_TABLEBYTES));
+          if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+            this.logBytes = (Long.parseLong(props.getProperty(TAG_LOGBYTES)));
+            LOG.debug("convert will be implemented by future jira");
+          }
+          this.loadIncrementalTimeRanges(props.getProperty(TAG_INCRTIMERANGE));
+          this.loadDependency(props.getProperty(TAG_DEPENDENCY));
+          this.imageState = props.getProperty(TAG_IMAGESTATE);
+          this.isCompacted =
+              props.getProperty(TAG_COMPACTION).equalsIgnoreCase("TRUE") ? true : false;
+          LOG.debug("merge and from existing snapshot will be implemented by future jira");
+          LOG.debug("Loaded manifest instance from manifest file: "
+              + FSUtils.getPath(subFile.getPath()));
+          return;
+        }
+      }
+      String errorMsg = "No manifest file found in: " + backupPath.toString();
+      LOG.error(errorMsg);
+      throw new IOException(errorMsg);
+
+    } catch (IOException e) {
+      throw new BackupException(e.getMessage());
+    }
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  /**
+   * Load table set from a table set list string (t1;t2;t3;...).
+   * @param tableSetStr Table set list string
+   */
+  private void loadTableSet(String tableSetStr) {
+
+    LOG.debug("Loading table set: " + tableSetStr);
+
+    String[] tableSet = tableSetStr.split(";");
+    this.tableSet = this.getTableSet();
+    if (this.tableSet.size() > 0) {
+      this.tableSet.clear();
+    }
+    for (int i = 0; i < tableSet.length; i++) {
+      this.tableSet.add(tableSet[i]);
+    }
+
+    LOG.debug(tableSet.length + " tables exist in table set.");
+  }
+
+  public void setImageOriginal() {
+    this.imageState = IMAGE_STATE_ORIG;
+  }
+
+  /**
+   * Get the table set of this image.
+   * @return The table set list
+   */
+  public ArrayList<String> getTableSet() {
+    if (this.tableSet == null) {
+      this.tableSet = new ArrayList<String>();
+    }
+    return this.tableSet;
+  }
+
+  /**
+   * Persist the manifest file.
+   * @throws IOException IOException when storing the manifest file.
+   */
+  public void store(Configuration conf) throws BackupException {
+    Properties props = new Properties();
+    props.setProperty(TAG_VERSION, this.version);
+    props.setProperty(TAG_BACKUPID, this.token);
+    props.setProperty(TAG_BACKUPTYPE, this.type);
+    props.setProperty(TAG_TABLESET, this.getTableSetStr());
+    LOG.debug("convert will be supported in future jira");
+    // String convertedTables = this.getConvertedTableSetStr();
+    // if (convertedTables != null )
+    // props.setProperty(TAG_CONVERTEDTABLESET, convertedTables);
+    props.setProperty(TAG_STARTTS, Long.toString(this.startTs));
+    props.setProperty(TAG_COMPLETETS, Long.toString(this.completeTs));
+    props.setProperty(TAG_TABLEBYTES, Long.toString(this.tableBytes));
+    if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+      props.setProperty(TAG_LOGBYTES, Long.toString(this.logBytes));
+    }
+    props.setProperty(TAG_INCRTIMERANGE, this.getIncrTimestampStr());
+    props.setProperty(TAG_DEPENDENCY, this.getDependencyStr());
+    props.setProperty(TAG_IMAGESTATE, this.getImageState());
+    props.setProperty(TAG_COMPACTION, this.isCompacted ? "TRUE" : "FALSE");
+    LOG.debug("merge will be supported in future jira");
+    // props.setProperty(TAG_MERGECHAIN, this.getMergeChainStr());
+    LOG.debug("backup from existing snapshot will be supported in future jira");
+    // props.setProperty(TAG_FROMSNAPSHOT, this.isFromSnapshot() ? "TRUE" : "FALSE");
+
+    // write the file, overwrite if already exist
+    Path manifestFilePath =
+        new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir)
+            + File.separator + FILE_NAME);
+    try {
+      FSDataOutputStream out = manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);
+      props.store(out, "HBase backup manifest.");
+      out.close();
+    } catch (IOException e) {
+      throw new BackupException(e.getMessage());
+    }
+
+    LOG.debug("Manifest file stored to " + this.tableBackupDir != null ? this.tableBackupDir
+        : this.logBackupDir + File.separator + FILE_NAME);
+  }
+
+  /**
+   * Get the table set string in the format of t1;t2;t3...
+   */
+  private String getTableSetStr() {
+    return BackupUtil.concat(getTableSet(), ";");
+  }
+
+  public String getImageState() {
+    return imageState;
+  }
+
+  public String getVersion() {
+    return version;
+  }
+
+  /**
+   * Get this backup image.
+   * @return the backup image.
+   */
+  public BackupImage getBackupImage() {
+    return this.getDependency().get(this.token);
+  }
+
+  /**
+   * Add dependent backup image for this backup.
+   * @param image The direct dependent backup image
+   */
+  public void addDependentImage(BackupImage image) {
+    this.getDependency().get(this.token).addAncestor(image);
+    this.setDependencyMap(this.getDependency(), image);
+  }
+
+  /**
+   * Get the dependency' string in the json format.
+   */
+  private String getDependencyStr() {
+    BackupImage thisImage = this.getDependency().get(this.token);
+    if (thisImage == null) {
+      LOG.warn("There is no dependency set yet.");
+      return null;
+    }
+
+    ObjectMapper mapper = new ObjectMapper();
+    try {
+      return mapper.writeValueAsString(thisImage);
+    } catch (JsonGenerationException e) {
+      LOG.error("Error when generating dependency string from backup image.", e);
+      return ERROR_DEPENDENCY;
+    } catch (JsonMappingException e) {
+      LOG.error("Error when generating dependency string from backup image.", e);
+      return ERROR_DEPENDENCY;
+    } catch (IOException e) {
+      LOG.error("Error when generating dependency string from backup image.", e);
+      return ERROR_DEPENDENCY;
+    }
+  }
+
+  /**
+   * Get all dependent backup images. The image of this backup is also contained.
+   * @return The dependent backup images map
+   */
+  public Map<String, BackupImage> getDependency() {
+    if (this.dependency == null) {
+      this.dependency = new HashMap<String, BackupImage>();
+      LOG.debug(this.rootDir + " " + this.token + " " + this.type);
+      this.dependency.put(this.token,
+        new BackupImage(this.token, this.type, this.rootDir, this.getTableSetStr(), this.startTs,
+            this.completeTs));
+    }
+    return this.dependency;
+  }
+
+  /**
+   * Set the incremental timestamp map directly.
+   * @param incrTimestampMap timestamp map
+   */
+  public void setIncrTimestampMap(HashMap<String, HashMap<String, String>> incrTimestampMap) {
+    this.incrTimeRanges = incrTimestampMap;
+  }
+
+  /**
+   * Get the incremental time range string in the format of:
+   * t1,rs1:ts,rs2:ts,...;t2,rs1:ts,rs2:ts,...;t3,rs1:ts,rs2:ts,...
+   */
+  private String getIncrTimestampStr() {
+    StringBuilder sb = new StringBuilder();
+    for (Entry<String, HashMap<String, String>> tableEntry : this.getIncrTimestamps().entrySet()) {
+      sb.append(tableEntry.getKey() + ","); // table
+      for (Entry<String, String> rsEntry : tableEntry.getValue().entrySet()) {
+        sb.append(rsEntry.getKey() + ":"); // region server
+        sb.append(rsEntry.getValue() + ","); // timestamp
+      }
+      if (sb.length() > 1 && sb.charAt(sb.length() - 1) == ',') {
+        sb.deleteCharAt(sb.length() - 1);
+      }
+      sb.append(";");
+    }
+    if (sb.length() > 1 && sb.charAt(sb.length() - 1) == ';') {
+      sb.deleteCharAt(sb.length() - 1);
+    }
+    return sb.toString();
+  }
+
+  public Map<String, HashMap<String, String>> getIncrTimestamps() {
+    if (this.incrTimeRanges == null) {
+      this.incrTimeRanges = new HashMap<String, HashMap<String, String>>();
+    }
+    return this.incrTimeRanges;
+  }
+
+  /**
+   * Load incremental timestamps from a given string, and store them in the collection. The
+   * timestamps in string is in the format of
+   * t1,rs1:ts,rs2:ts,...;t2,rs1:ts,rs2:ts,...;t3,rs1:ts,rs2:ts,...
+   * @param timeRangesInStr Incremental time ranges in string
+   */
+  private void loadIncrementalTimeRanges(String timeRangesStr) throws IOException {
+
+    LOG.debug("Loading table's incremental time ranges of region servers from string in manifest: "
+        + timeRangesStr);
+
+    Map<String, HashMap<String, String>> timeRangeMap = this.getIncrTimestamps();
+
+    String[] entriesOfTables = timeRangesStr.split(";");
+    for (int i = 0; i < entriesOfTables.length; i++) {
+      String[] itemsForTable = entriesOfTables[i].split(",");
+
+      // validate the incremental timestamps string format for a table:
+      // t1,rs1:ts,rs2:ts,...
+      if (itemsForTable.length < 1) {
+        String errorMsg = "Wrong incremental time range format: " + timeRangesStr;
+        LOG.error(errorMsg);
+        throw new IOException(errorMsg);
+      }
+
+      HashMap<String, String> rsTimestampMap = new HashMap<String, String>();
+      for (int j = 1; j < itemsForTable.length; j++) {
+        String[] rsTsEntry = itemsForTable[j].split(":");
+
+        // validate the incremental timestamps string format for a region server:
+        // rs1:ts
+        if (rsTsEntry.length != 2) {
+          String errorMsg = "Wrong incremental timestamp format: " + itemsForTable[j];
+          LOG.error(errorMsg);
+          throw new IOException(errorMsg);
+        }
+
+        // an entry for timestamp of a region server
+        rsTimestampMap.put(rsTsEntry[0], rsTsEntry[1]);
+      }
+
+      timeRangeMap.put(itemsForTable[0], rsTimestampMap);
+    }
+
+    // all entries have been loaded
+    LOG.debug(entriesOfTables.length + " tables' incremental time ranges have been loaded.");
+  }
+
+  /**
+   * Get the image list of this backup for restore in time order.
+   * @param reverse If true, then output in reverse order, otherwise in time order from old to new
+   * @return the backup image list for restore in time order
+   */
+  public ArrayList<BackupImage> getRestoreDependentList(boolean reverse) {
+    TreeMap<Long, BackupImage> restoreImages = new TreeMap<Long, BackupImage>();
+    for (BackupImage image : this.getDependency().values()) {
+      restoreImages.put(Long.valueOf(image.startTs), image);
+    }
+    return new ArrayList<BackupImage>(reverse ? (restoreImages.descendingMap().values())
+        : (restoreImages.values()));
+  }
+
+  /**
+   * Get the dependent image list for a specific table of this backup in time order from old to new
+   * if want to restore to this backup image level.
+   * @param table table
+   * @return the backup image list for a table in time order
+   */
+  public ArrayList<BackupImage> getDependentListByTable(String table) {
+    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
+    ArrayList<BackupImage> imageList = getRestoreDependentList(true);
+    for (BackupImage image : imageList) {
+      if (image.hasTable(table)) {
+        tableImageList.add(image);
+        if (image.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
+          break;
+        }
+      }
+    }
+    Collections.reverse(tableImageList);
+    return tableImageList;
+  }
+
+  /**
+   * Get the full dependent image list in the whole dependency scope for a specific table of this
+   * backup in time order from old to new.
+   * @param table table
+   * @return the full backup image list for a table in time order in the whole scope of the
+   *         dependency of this image
+   */
+  public ArrayList<BackupImage> getAllDependentListByTable(String table) {
+    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
+    ArrayList<BackupImage> imageList = getRestoreDependentList(false);
+    for (BackupImage image : imageList) {
+      if (image.hasTable(table)) {
+        tableImageList.add(image);
+      }
+    }
+    return tableImageList;
+  }
+
+  /**
+   * Load dependency from a dependency json string.
+   * @param dependencyStr The dependency string
+   * @throws IOException exception
+   */
+  private void loadDependency(String dependencyStr) throws IOException {
+
+    LOG.debug("Loading dependency: " + dependencyStr);
+
+    String msg = "Dependency is broken in the manifest.";
+    if (dependencyStr.equals(ERROR_DEPENDENCY)) {
+      throw new IOException(msg);
+    }
+
+    ObjectMapper mapper = new ObjectMapper();
+    BackupImage image = null;
+    try {
+      image = mapper.readValue(dependencyStr, BackupImage.class);
+    } catch (JsonParseException e) {
+      LOG.error(msg);
+      throw new IOException(e.getMessage());
+    } catch (JsonMappingException e) {
+      LOG.error(msg);
+      throw new IOException(e.getMessage());
+    } catch (IOException e) {
+      LOG.error(msg);
+      throw new IOException(e.getMessage());
+    }
+    LOG.debug("Manifest's current backup image information:");
+    LOG.debug("  Token: " + image.getBackupId());
+    LOG.debug("  Backup directory: " + image.getRootDir());
+    this.setDependencyMap(this.getDependency(), image);
+
+    LOG.debug("Dependent images map:");
+    for (Entry<String, BackupImage> entry : this.getDependency().entrySet()) {
+      LOG.debug("  " + entry.getKey() + " : " + entry.getValue().getBackupId() + " -- "
+          + entry.getValue().getRootDir());
+    }
+
+    LOG.debug("Dependency has been loaded.");
+  }
+
+  /**
+   * Recursively set the dependency map of the backup images.
+   * @param map The dependency map
+   * @param image The backup image
+   */
+  private void setDependencyMap(Map<String, BackupImage> map, BackupImage image) {
+    if (image == null) {
+      return;
+    } else {
+      map.put(image.getBackupId(), image);
+      for (BackupImage img : image.getAncestors()) {
+        setDependencyMap(map, img);
+      }
+    }
+  }
+
+  /**
+   * Check whether backup image1 could cover backup image2 or not.
+   * @param image1 backup image 1
+   * @param image2 backup image 2
+   * @return true if image1 can cover image2, otherwise false
+   */
+  public static boolean canCoverImage(BackupImage image1, BackupImage image2) {
+    // image1 can cover image2 only when the following conditions are satisfied:
+    // - image1 must not be an incremental image;
+    // - image1 must be taken after image2 has been taken;
+    // - table set of image1 must cover the table set of image2.
+    if (image1.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+      return false;
+    }
+    if (image1.getStartTs() < image2.getStartTs()) {
+      return false;
+    }
+    String[] image1TableSet = image1.getTableSet().split(";");
+    String[] image2TableSet = image2.getTableSet().split(";");
+    boolean found = false;
+    for (int i = 0; i < image2TableSet.length; i++) {
+      found = false;
+      for (int j = 0; j < image1TableSet.length; j++) {
+        if (image2TableSet[i].equals(image1TableSet[j])) {
+          found = true;
+          break;
+        }
+      }
+      if (!found) {
+        return false;
+      }
+    }
+
+    LOG.debug("Backup image " + image1.getBackupId() + " can cover " + image2.getBackupId());
+    return true;
+  }
+
+  /**
+   * Check whether backup image set could cover a backup image or not.
+   * @param fullImages The backup image set
+   * @param image The target backup image
+   * @return true if fullImages can cover image, otherwise false
+   */
+  public static boolean canCoverImage(ArrayList<BackupImage> fullImages, BackupImage image) {
+    // fullImages can cover image only when the following conditions are satisfied:
+    // - each image of fullImages must not be an incremental image;
+    // - each image of fullImages must be taken after image has been taken;
+    // - sum table set of fullImages must cover the table set of image.
+    for (BackupImage image1 : fullImages) {
+      if (image1.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+        return false;
+      }
+      if (image1.getStartTs() < image.getStartTs()) {
+        return false;
+      }
+    }
+
+    ArrayList<String> image1TableSet = new ArrayList<String>();
+    for (BackupImage image1 : fullImages) {
+      String[] tableSet = image1.getTableSet().split(";");
+      for (String table : tableSet) {
+        image1TableSet.add(table);
+      }
+    }
+    ArrayList<String> image2TableSet = new ArrayList<String>();
+    String[] tableSet = image.getTableSet().split(";");
+    for (String table : tableSet) {
+      image2TableSet.add(table);
+    }
+    
+    for (int i = 0; i < image2TableSet.size(); i++) {
+      if (image1TableSet.contains(image2TableSet.get(i)) == false) {
+        return false;
+      }
+    }
+
+    LOG.debug("Full image set can cover image " + image.getBackupId());
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
new file mode 100644
index 0000000..9610b27
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
@@ -0,0 +1,66 @@
+/**
+ * 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 org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * HConstants holds a bunch of HBase Backup and Restore constants
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public final class BackupRestoreConstants {
+
+  // constants for znode data keys in backup znode
+  public static final String BACKUP_PROGRESS = "progress";
+  public static final String BACKUP_START_TIME = "startTs";
+  public static final String BACKUP_INPROGRESS_PHASE = "phase";
+  public static final String BACKUP_COMPLETE_TIME = "completeTs";
+  public static final String BACKUP_FAIL_TIME = "failedTs";
+  public static final String BACKUP_FAIL_PHASE = "failedphase";
+  public static final String BACKUP_FAIL_MSG = "failedmessage";
+  public static final String BACKUP_ROOT_PATH = "targetRootDir";
+  public static final String BACKUP_REQUEST_TABLE_LIST = "tablelist";
+  public static final String BACKUP_REQUEST_TYPE = "type";
+  public static final String BACKUP_BYTES_COPIED = "bytescopied";
+  public static final String BACKUP_ANCESTORS = "ancestors";
+  public static final String BACKUP_EXISTINGSNAPSHOT = "snapshot";
+
+  public static final String BACKUP_TYPE_FULL = "full";
+  public static final String BACKUP_TYPE_INCR = "incremental";
+
+  // delimiter in tablename list in restore command
+  public static final String TABLENAME_DELIMITER_IN_COMMAND = ",";
+
+  // delimiter in znode data
+  public static final String ZNODE_DATA_DELIMITER = ",";
+
+  public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root";
+
+  public static final String BACKUPID_PREFIX = "backup_";
+
+  public static enum BACKUP_COMMAND {
+    CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP,
+  }
+
+  private BackupRestoreConstants() {
+    // Can't be instantiated with this ctor.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServiceFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServiceFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServiceFactory.java
new file mode 100644
index 0000000..1c38cf4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreServiceFactory.java
@@ -0,0 +1,61 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyService;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreService;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.util.ReflectionUtils;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupRestoreServiceFactory {
+
+  public final static String HBASE_INCR_RESTORE_IMPL_CLASS = "hbase.incremental.restore.class";
+  public final static String HBASE_BACKUP_COPY_IMPL_CLASS = "hbase.backup.copy.class";
+
+  private BackupRestoreServiceFactory(){
+    throw new AssertionError("Instantiating utility class...");
+  }
+  
+  /**
+   * Gets incremental restore service
+   * @param conf - configuration
+   * @return incremental backup service instance
+   */
+  public static IncrementalRestoreService getIncrementalRestoreService(Configuration conf) {
+    Class<? extends IncrementalRestoreService> cls =
+        conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreService.class,
+          IncrementalRestoreService.class);
+    return ReflectionUtils.newInstance(cls, conf);
+  }
+  
+  /**
+   * Gets backup copy service
+   * @param conf - configuration
+   * @return backup copy service
+   */
+  public static BackupCopyService getBackupCopyService(Configuration conf) {
+    Class<? extends BackupCopyService> cls =
+        conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, MapReduceBackupCopyService.class,
+          BackupCopyService.class);
+    return ReflectionUtils.newInstance(cls, conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
new file mode 100644
index 0000000..c26e1d4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
@@ -0,0 +1,67 @@
+/**
+ * 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.Serializable;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Backup status and related information encapsulated for a table.
+ * At this moment only TargetDir and SnapshotName is encapsulated here.
+ * future Jira will be implemented for progress, bytesCopies, phase, etc.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupStatus implements Serializable {
+
+  private static final long serialVersionUID = -5968397963548535982L;
+
+  // table name for backup
+  private String table;
+
+  // target directory of the backup image for this table
+  private String targetDir;
+
+  // snapshot name for offline/online snapshot
+  private String snapshotName = null;
+
+  public BackupStatus(String table, String targetRootDir, String backupId) {
+    this.table = table;
+    this.targetDir = HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
+  }
+
+  public String getSnapshotName() {
+    return snapshotName;
+  }
+
+  public void setSnapshotName(String snapshotName) {
+    this.snapshotName = snapshotName;
+  }
+
+  public String getTargetDir() {
+    return targetDir;
+  }
+
+  public String getTable() {
+    return table;
+  }
+}


[4/6] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java
new file mode 100644
index 0000000..14769f9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java
@@ -0,0 +1,642 @@
+/**
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS;
+import org.apache.hadoop.hbase.backup.BackupUtil.BackupCompleteData;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+
+/**
+ * This class provides 'hbase:backup' table API
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupSystemTable {
+
+  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
+  private final static String TABLE_NAMESPACE = "hbase";
+  private final static String TABLE_NAME = "backup";
+  private final static TableName tableName = TableName.valueOf(TABLE_NAMESPACE, TABLE_NAME);
+  public final static byte[] familyName = "f".getBytes();
+
+  // Connection to HBase cluster
+  private static Connection connection;
+  // Cluster configuration
+  private static Configuration config;
+  // singleton
+  private static BackupSystemTable table;
+
+  /**
+   * Get instance by a given configuration
+   * @param conf - HBase configuration
+   * @return instance of BackupSystemTable
+   * @throws IOException exception
+   */
+  public synchronized static BackupSystemTable getTable(Configuration conf) throws IOException {
+    if (connection == null) {
+      connection = ConnectionFactory.createConnection(conf);
+      config = conf;
+      // Verify hbase:system exists
+      createSystemTableIfNotExists();
+      table = new BackupSystemTable();
+    }
+    return table;
+  }
+
+  /**
+   * TODO: refactor
+   * @throws IOException exception
+   */
+  public static void close() throws IOException {
+    connection.close();
+    table = null;
+  }
+
+  /**
+   * Gets table name
+   * @return table name
+   */
+  public static TableName getTableName() {
+    return tableName;
+  }
+
+  private static void createSystemTableIfNotExists() throws IOException {
+    Admin admin = null;
+    try {
+      admin = connection.getAdmin();
+      if (admin.tableExists(tableName) == false) {
+        HTableDescriptor tableDesc = new HTableDescriptor(tableName);
+        HColumnDescriptor colDesc = new HColumnDescriptor(familyName);
+        colDesc.setMaxVersions(1);
+        int ttl =
+            config.getInt(HConstants.BACKUP_SYSTEM_TTL_KEY, HConstants.BACKUP_SYSTEM_TTL_DEFAULT);
+        colDesc.setTimeToLive(ttl);
+        tableDesc.addFamily(colDesc);
+        admin.createTable(tableDesc);
+      }
+    } catch (IOException e) {
+      LOG.error(e);
+      throw e;
+    } finally {
+      if (admin != null) {
+        admin.close();
+      }
+    }
+  }
+
+  private BackupSystemTable() {
+  }
+
+  /**
+   * Updates status (state) of a backup session in hbase:backup table
+   * @param context context
+   * @throws IOException exception
+   */
+  public void updateBackupStatus(BackupContext context) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("update backup status in hbase:backup for: " + context.getBackupId()
+        + " set status=" + context.getFlag());
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Put put = BackupSystemTableHelper.createPutForBackupContext(context);
+      table.put(put);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Deletes backup status from hbase:backup table
+   * @param backupId backup id
+   * @throws IOException exception
+   */
+
+  public void deleteBackupStatus(String backupId) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("delete backup status in hbase:backup for " + backupId);
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Delete del = BackupSystemTableHelper.createDeletForBackupContext(backupId);
+      table.delete(del);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Reads backup status object (instance of BackupContext) from hbase:backup table
+   * @param backupId - backupId
+   * @return Current status of backup session or null
+   */
+
+  public BackupContext readBackupStatus(String backupId) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read backup status from hbase:backup for: " + backupId);
+    }
+
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Get get = BackupSystemTableHelper.createGetForBackupContext(backupId);
+      Result res = table.get(get);
+      if(res.isEmpty()){
+        return null;
+      }
+      return BackupSystemTableHelper.resultToBackupContext(res);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Read the last backup start code (timestamp) of last successful backup. Will return null if
+   * there is no start code stored on hbase or the value is of length 0. These two cases indicate
+   * there is no successful backup completed so far.
+   * @return the timestamp of last successful backup
+   * @throws IOException exception
+   */
+  public String readBackupStartCode() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read backup start code from hbase:backup");
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Get get = BackupSystemTableHelper.createGetForStartCode();
+      Result res = table.get(get);
+      if (res.isEmpty()){
+        return null;
+      }
+      Cell cell = res.listCells().get(0);
+      byte[] val = CellUtil.cloneValue(cell);
+      if (val.length == 0){
+        return null;
+      }
+      return new String(val);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
+   * @param startCode start code
+   * @throws IOException exception
+   */
+  public void writeBackupStartCode(String startCode) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write backup start code to hbase:backup " + startCode);
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Put put = BackupSystemTableHelper.createPutForStartCode(startCode);
+      table.put(put);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Get the Region Servers log information after the last log roll from hbase:backup.
+   * @return RS log info
+   * @throws IOException exception
+   */
+  public HashMap<String, String> readRegionServerLastLogRollResult() 
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read region server last roll log result to hbase:backup");
+    }
+    Table table = null;
+    ResultScanner scanner = null;
+
+    try {
+      table = connection.getTable(tableName);
+      Scan scan = BackupSystemTableHelper.createScanForReadRegionServerLastLogRollResult();
+      scan.setMaxVersions(1);
+      scanner = table.getScanner(scan);
+      Result res = null;
+      HashMap<String, String> rsTimestampMap = new HashMap<String, String>();
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        Cell cell = res.current();
+        byte[] row = CellUtil.cloneRow(cell);
+        String server =
+            BackupSystemTableHelper.getServerNameForReadRegionServerLastLogRollResult(row);
+
+        byte[] data = CellUtil.cloneValue(cell);
+        rsTimestampMap.put(server, new String(data));
+      }
+      return rsTimestampMap;
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Writes Region Server last roll log result (timestamp) to hbase:backup table
+   * @param server - Region Server name
+   * @param fileName - last log timestamp
+   * @throws IOException exception
+   */
+  public void writeRegionServerLastLogRollResult(String server, String fileName) 
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write region server last roll log result to hbase:backup");
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Put put = 
+          BackupSystemTableHelper.createPutForRegionServerLastLogRollResult(server, fileName);
+      table.put(put);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Get all completed backup information (in desc order by time)
+   * @return history info of BackupCompleteData
+   * @throws IOException exception
+   */
+  public ArrayList<BackupCompleteData> getBackupHistory() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get backup history from hbase:backup");
+    }
+    Table table = null;
+    ResultScanner scanner = null;
+    ArrayList<BackupCompleteData> list = new ArrayList<BackupCompleteData>();
+    try {
+      table = connection.getTable(tableName);
+      Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
+      scan.setMaxVersions(1);
+      scanner = table.getScanner(scan);
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        BackupContext context = BackupSystemTableHelper.cellToBackupContext(res.current());
+        if (context.getFlag() != BACKUPSTATUS.COMPLETE) {
+          continue;
+        }
+
+        BackupCompleteData history = new BackupCompleteData();
+        history.setBackupToken(context.getBackupId());
+        history.setStartTime(Long.toString(context.getStartTs()));
+        history.setEndTime(Long.toString(context.getEndTs()));
+        history.setBackupRootPath(context.getTargetRootDir());
+        history.setTableList(context.getTableListAsString());
+        history.setType(context.getType());
+        history.setBytesCopied(Long.toString(context.getTotalBytesCopied()));
+
+        if (context.fromExistingSnapshot()) {
+          history.markFromExistingSnapshot();
+        }
+        list.add(history);
+      }
+      return BackupUtil.sortHistoryListDesc(list);
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Get all backup session with a given status (in desc order by time)
+   * @param status status
+   * @return history info of backup contexts
+   * @throws IOException exception
+   */
+  public ArrayList<BackupContext> getBackupContexts(BACKUPSTATUS status) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get backup contexts from hbase:backup");
+    }
+    Table table = null;
+    ResultScanner scanner = null;
+    ArrayList<BackupContext> list = new ArrayList<BackupContext>();
+    try {
+      table = connection.getTable(tableName);
+      Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
+      scan.setMaxVersions(1);
+      scanner = table.getScanner(scan);
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        BackupContext context = BackupSystemTableHelper.cellToBackupContext(res.current());
+        if (context.getFlag() != status){
+          continue;
+        }
+        list.add(context);
+      }
+      return list;
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Write the current timestamps for each regionserver to hbase:backup after a successful full or
+   * incremental backup. The saved timestamp is of the last log file that was backed up already.
+   * @param tables tables
+   * @param newTimestamps timestamps
+   * @throws IOException exception
+   */
+  public void writeRegionServerLogTimestamp(Set<String> tables,
+      HashMap<String, String> newTimestamps) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write RS log ts to HBASE_BACKUP");
+    }
+    StringBuilder sb = new StringBuilder();
+    for (Map.Entry<String, String> entry : newTimestamps.entrySet()) {
+      String host = entry.getKey();
+      String timestamp = entry.getValue();
+      sb.append(host).append(BackupUtil.FIELD_SEPARATOR).append(timestamp)
+      .append(BackupUtil.RECORD_SEPARATOR);
+    }
+    String smap = sb.toString();
+    List<Put> puts = new ArrayList<Put>();
+    for (String table : tables) {
+      Put put = BackupSystemTableHelper.createPutForWriteRegionServerLogTimestamp(table, smap);
+      puts.add(put);
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      table.put(puts);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Read the timestamp for each region server log after the last successful backup. Each table has
+   * its own set of the timestamps. The info is stored for each table as a concatenated string of
+   * rs->timestapmp
+   * @return the timestamp for each region server. key: tableName value:
+   *         RegionServer,PreviousTimeStamp
+   * @throws IOException exception
+   */
+  public HashMap<String, HashMap<String, String>> readLogTimestampMap() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read RS log ts from HBASE_BACKUP");
+    }
+
+    Table table = null;
+    ResultScanner scanner = null;
+    HashMap<String, HashMap<String, String>> tableTimestampMap =
+        new HashMap<String, HashMap<String, String>>();
+
+    try {
+      table = connection.getTable(tableName);
+      Scan scan = BackupSystemTableHelper.createScanForReadLogTimestampMap();
+      scanner = table.getScanner(scan);
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        Cell cell = res.current();
+        byte[] row = CellUtil.cloneRow(cell);
+        String tabName = BackupSystemTableHelper.getTableNameForReadLogTimestampMap(row);
+        HashMap<String, String> lastBackup = new HashMap<String, String>();
+        byte[] data = CellUtil.cloneValue(cell);
+        if (data == null) {
+          // TODO
+          throw new IOException("Data of last backup data from HBASE_BACKUP "
+              + "is empty. Create a backup first.");
+        }
+        if (data != null && data.length > 0) {
+          String s = new String(data);
+          String[] records = s.split(BackupUtil.RECORD_SEPARATOR);
+          for (String record : records) {
+            String[] flds = record.split(BackupUtil.FIELD_SEPARATOR);
+            if (flds.length != 2) {
+              throw new IOException("data from HBASE_BACKUP is corrupted: "
+                  + Arrays.toString(flds));
+            }
+            lastBackup.put(flds[0], flds[1]);
+          }
+          tableTimestampMap.put(tabName, lastBackup);
+        }
+      }
+      return tableTimestampMap;
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Return the current tables covered by incremental backup.
+   * @return set of tableNames
+   * @throws IOException exception
+   */
+  public Set<String> getIncrementalBackupTableSet() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get incr backup table set from hbase:backup");
+    }
+    Table table = null;
+    TreeSet<String> set = new TreeSet<String>();
+
+    try {
+      table = connection.getTable(tableName);
+      Get get = BackupSystemTableHelper.createGetForIncrBackupTableSet();
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return set;
+      }
+      List<Cell> cells = res.listCells();
+      for (Cell cell : cells) {
+        // qualifier = table name - we use table names as qualifiers
+        // TODO ns:table as qualifier?
+        set.add(new String(CellUtil.cloneQualifier(cell)));
+      }
+      return set;
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Add tables to global incremental backup set
+   * @param tables - set of tables
+   * @throws IOException exception
+   */
+  public void addIncrementalBackupTableSet(Set<String> tables) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("add incr backup table set to hbase:backup");
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Put put = BackupSystemTableHelper.createPutForIncrBackupTableSet(tables);
+      table.put(put);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Register WAL files as eligible for deletion
+   * @param files files
+   * @throws IOException exception
+   */
+  public void addWALFiles(List<String> files, String backupId) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("add WAL files to hbase:backup");
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      List<Put> puts = BackupSystemTableHelper.createPutsForAddWALFiles(files, backupId);
+      table.put(puts);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Check if WAL file is eligible for deletion
+   * @param file file
+   * @return true, if - yes.
+   * @throws IOException exception
+   */
+  public boolean checkWALFile(String file) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Check if WAL file has been already backuped in hbase:backup");
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Get get = BackupSystemTableHelper.createGetForCheckWALFile(file);
+      Result res = table.get(get);
+      if (res.isEmpty()){
+        return false;
+      }
+      return true;
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Checks if we have at least one backup session in hbase:backup This API is used by
+   * BackupLogCleaner
+   * @return true, if - at least one session exists in hbase:backup table
+   * @throws IOException exception
+   */
+  public boolean hasBackupSessions() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("has backup sessions from hbase:backup");
+    }
+    Table table = null;
+    ResultScanner scanner = null;
+    boolean result = false;
+    try {
+      table = connection.getTable(tableName);
+      Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
+      scan.setMaxVersions(1);
+      scan.setCaching(1);
+      scanner = table.getScanner(scan);
+      if (scanner.next() != null) {
+        result = true;
+      }
+      return result;
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java
new file mode 100644
index 0000000..bf62a84
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java
@@ -0,0 +1,314 @@
+/**
+ * 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.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+
+
+/**
+ * A collection for methods used by BackupSystemTable.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupSystemTableHelper {
+
+  /**
+   * hbase:backup schema: 
+   * 1. Backup sessions rowkey= "session." + backupId; value = serialized
+   * BackupContext 
+   * 2. Backup start code rowkey = "startcode"; value = startcode 
+   * 3. Incremental backup set rowkey="incrbackupset"; value=[list of tables] 
+   * 4. Table-RS-timestamp map rowkey="trslm"+ table_name; value = map[RS-> last WAL timestamp] 
+   * 5. RS - WAL ts map rowkey="rslogts."+server; value = last WAL timestamp 
+   * 6. WALs recorded rowkey="wals."+WAL unique file name; value = NULL (value is not used)
+   */
+  private static final Log LOG = LogFactory.getLog(BackupSystemTableHelper.class);
+
+  private final static String BACKUP_CONTEXT_PREFIX = "session.";
+  private final static String START_CODE_ROW = "startcode";
+  private final static String INCR_BACKUP_SET = "incrbackupset";
+  private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm.";
+  private final static String RS_LOG_TS_PREFIX = "rslogts.";
+  private final static String WALS_PREFIX = "wals.";
+
+  private final static byte[] q0 = "0".getBytes();
+  private final static byte[] EMPTY_VALUE = new byte[] {};
+
+  private BackupSystemTableHelper() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Creates Put operation for a given backup context object
+   * @param context backup context
+   * @return put operation
+   * @throws IOException exception
+   */
+  static Put createPutForBackupContext(BackupContext context) throws IOException {
+
+    Put put = new Put((BACKUP_CONTEXT_PREFIX + context.getBackupId()).getBytes());
+    put.addColumn(BackupSystemTable.familyName, q0, context.toByteArray());
+    return put;
+  }
+
+  /**
+   * Creates Get operation for a given backup id
+   * @param backupId - backup's ID
+   * @return get operation
+   * @throws IOException exception
+   */
+  static Get createGetForBackupContext(String backupId) throws IOException {
+    Get get = new Get((BACKUP_CONTEXT_PREFIX + backupId).getBytes());
+    get.addFamily(BackupSystemTable.familyName);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Delete operation for a given backup id
+   * @param backupId - backup's ID
+   * @return delete operation
+   * @throws IOException exception
+   */
+  public static Delete createDeletForBackupContext(String backupId) {
+    Delete del = new Delete((BACKUP_CONTEXT_PREFIX + backupId).getBytes());
+    del.addFamily(BackupSystemTable.familyName);
+    return del;
+  }
+
+  /**
+   * Converts Result to BackupContext
+   * @param res - HBase result
+   * @return backup context instance
+   * @throws IOException exception
+   */
+  static BackupContext resultToBackupContext(Result res) throws IOException {
+    res.advance();
+    Cell cell = res.current();
+    return cellToBackupContext(cell);
+  }
+
+  /**
+   * Creates Get operation to retrieve start code from hbase:backup
+   * @return get operation
+   * @throws IOException exception
+   */
+  static Get createGetForStartCode() throws IOException {
+    Get get = new Get(START_CODE_ROW.getBytes());
+    get.addFamily(BackupSystemTable.familyName);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Put operation to store start code to hbase:backup
+   * @return put operation
+   * @throws IOException exception
+   */
+  static Put createPutForStartCode(String startCode) {
+    Put put = new Put(START_CODE_ROW.getBytes());
+    put.addColumn(BackupSystemTable.familyName, q0, startCode.getBytes());
+    return put;
+  }
+
+  /**
+   * Creates Get to retrieve incremental backup table set from hbase:backup
+   * @return get operation
+   * @throws IOException exception
+   */
+  static Get createGetForIncrBackupTableSet() throws IOException {
+    Get get = new Get(INCR_BACKUP_SET.getBytes());
+    get.addFamily(BackupSystemTable.familyName);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Put to store incremental backup table set
+   * @param tables tables
+   * @return put operation
+   */
+  static Put createPutForIncrBackupTableSet(Set<String> tables) {
+    Put put = new Put(INCR_BACKUP_SET.getBytes());
+    for (String table : tables) {
+      put.addColumn(BackupSystemTable.familyName, table.getBytes(), EMPTY_VALUE);
+    }
+    return put;
+  }
+
+  /**
+   * Creates Scan operation to load backup history
+   * @return scan operation
+   */
+  static Scan createScanForBackupHistory() {
+    Scan scan = new Scan();
+    byte[] startRow = BACKUP_CONTEXT_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.familyName);
+
+    return scan;
+  }
+
+  /**
+   * Converts cell to backup context instance.
+   * @param current - cell
+   * @return backup context instance
+   * @throws IOException exception
+   */
+  static BackupContext cellToBackupContext(Cell current) throws IOException {
+    byte[] data = CellUtil.cloneValue(current);
+    try {
+      BackupContext ctxt = BackupContext.fromByteArray(data);
+      return ctxt;
+    } catch (ClassNotFoundException e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Creates Put to write RS last roll log timestamp map
+   * @param table - table
+   * @param smap - map, containing RS:ts
+   * @return put operation
+   */
+  static Put createPutForWriteRegionServerLogTimestamp(String table, String smap) {
+    Put put = new Put((TABLE_RS_LOG_MAP_PREFIX + table).getBytes());
+    put.addColumn(BackupSystemTable.familyName, q0, smap.getBytes());
+    return put;
+  }
+
+  /**
+   * Creates Scan to load table-> { RS -> ts} map of maps
+   * @return scan operation
+   */
+  static Scan createScanForReadLogTimestampMap() {
+    Scan scan = new Scan();
+    byte[] startRow = TABLE_RS_LOG_MAP_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.familyName);
+
+    return scan;
+  }
+
+  /**
+   * Get table name from rowkey
+   * @param cloneRow rowkey
+   * @return table name
+   */
+  static String getTableNameForReadLogTimestampMap(byte[] cloneRow) {
+    int prefixSize = TABLE_RS_LOG_MAP_PREFIX.length();
+    return new String(cloneRow, prefixSize, cloneRow.length - prefixSize);
+  }
+
+  /**
+   * Creates Put to store RS last log result
+   * @param server - server name
+   * @param fileName - log roll result (timestamp)
+   * @return put operation
+   */
+  static Put createPutForRegionServerLastLogRollResult(String server, String fileName) {
+    Put put = new Put((RS_LOG_TS_PREFIX + server).getBytes());
+    put.addColumn(BackupSystemTable.familyName, q0, fileName.getBytes());
+    return put;
+  }
+
+  /**
+   * Creates Scan operation to load last RS log roll results
+   * @return scan operation
+   */
+  static Scan createScanForReadRegionServerLastLogRollResult() {
+    Scan scan = new Scan();
+    byte[] startRow = RS_LOG_TS_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.familyName);
+
+    return scan;
+  }
+
+  /**
+   * Get server's name from rowkey
+   * @param row - rowkey
+   * @return server's name
+   */
+  static String getServerNameForReadRegionServerLastLogRollResult(byte[] row) {
+    int prefixSize = RS_LOG_TS_PREFIX.length();
+    return new String(row, prefixSize, row.length - prefixSize);
+  }
+
+  /**
+   * Creates put list for list of WAL files
+   * @param files list of WAL file paths
+   * @param backupId backup id
+   * @return put list
+   * @throws IOException exception
+   */
+  public static List<Put> createPutsForAddWALFiles(List<String> files, String backupId)
+      throws IOException {
+
+    List<Put> puts = new ArrayList<Put>();
+    for (String file : files) {
+      LOG.debug("+++ put: " + BackupUtil.getUniqueWALFileNamePart(file));
+      byte[] row = (WALS_PREFIX + BackupUtil.getUniqueWALFileNamePart(file)).getBytes();
+      Put put = new Put(row);
+      put.addColumn(BackupSystemTable.familyName, q0, backupId.getBytes());
+      puts.add(put);
+    }
+    return puts;
+  }
+
+  /**
+   * Creates Get operation for a given wal file name
+   * @param file file
+   * @return get operation
+   * @throws IOException exception
+   */
+  public static Get createGetForCheckWALFile(String file) throws IOException {
+    byte[] row = (WALS_PREFIX + BackupUtil.getUniqueWALFileNamePart(file)).getBytes();
+    Get get = new Get(row);
+    get.addFamily(BackupSystemTable.familyName);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java
new file mode 100644
index 0000000..ff8bd2e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java
@@ -0,0 +1,564 @@
+/**
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+
+/**
+ * A collection for methods used by multiple classes to backup HBase tables.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupUtil {
+  protected static final Log LOG = LogFactory.getLog(BackupUtil.class);
+
+  public static final String FIELD_SEPARATOR = "\001";
+  public static final String RECORD_SEPARATOR = "\002";
+  public static final String LOGNAME_SEPARATOR = ".";
+  protected static final String HDFS = "hdfs://";
+  protected static Configuration conf = null;
+
+  private BackupUtil(){
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Set the configuration from a given one.
+   * @param newConf A new given configuration
+   */
+  public synchronized static void setConf(Configuration newConf) {
+    conf = newConf;
+  }
+
+  /**
+   * Get and merge Hadoop and HBase configuration.
+   * @throws IOException exception
+   */
+  protected static Configuration getConf() {
+    if (conf == null) {
+      conf = new Configuration();
+      HBaseConfiguration.merge(conf, HBaseConfiguration.create());
+    }
+    return conf;
+  }
+
+  /**
+   * Loop through the RS log timestamp map for the tables, for each RS, find the min timestamp value
+   * for the RS among the tables.
+   * @param rsLogTimestampMap timestamp map
+   * @return the min timestamp of each RS
+   */
+  protected static HashMap<String, String> getRSLogTimestampMins(
+    HashMap<String, HashMap<String, String>> rsLogTimestampMap) {
+
+    if (rsLogTimestampMap == null || rsLogTimestampMap.isEmpty()) {
+      return null;
+    }
+
+    HashMap<String, String> rsLogTimestamptMins = new HashMap<String, String>();
+    HashMap<String, HashMap<String, String>> rsLogTimestampMapByRS =
+        new HashMap<String, HashMap<String, String>>();
+
+    for (Entry<String, HashMap<String, String>> tableEntry : rsLogTimestampMap.entrySet()) {
+      String table = tableEntry.getKey();
+      HashMap<String, String> rsLogTimestamp = tableEntry.getValue();
+      for (Entry<String, String> rsEntry : rsLogTimestamp.entrySet()) {
+        String rs = rsEntry.getKey();
+        String ts = rsEntry.getValue();
+        if (!rsLogTimestampMapByRS.containsKey(rs)) {
+          rsLogTimestampMapByRS.put(rs, new HashMap<String, String>());
+          rsLogTimestampMapByRS.get(rs).put(table, ts);
+        } else {
+          rsLogTimestampMapByRS.get(rs).put(table, ts);
+        }
+      }
+    }
+
+    for (String rs : rsLogTimestampMapByRS.keySet()) {
+      rsLogTimestamptMins.put(rs, getMinValue(rsLogTimestampMapByRS.get(rs)));
+    }
+
+    return rsLogTimestamptMins;
+  }
+
+  /**
+   * Get the min value for all the Values a map.
+   * @param map map
+   * @return the min value
+   */
+  protected static String getMinValue(HashMap<String, String> map) {
+    String minTimestamp = null;
+    if (map != null) {
+      ArrayList<String> timestampList = new ArrayList<String>(map.values());
+      Collections.sort(timestampList, new Comparator<String>() {
+        @Override
+        public int compare(String s1, String s2) {
+          long l1 = Long.valueOf(s1);
+          long l2 = Long.valueOf(s2);
+          if (l1 > l2) {
+            return 1;
+          } else if (l1 < l2) {
+            return -1;
+          } else {
+            return 0;
+          }
+        }
+      });
+      // The min among all the RS log timestamps will be kept in ZK.
+      minTimestamp = timestampList.get(0);
+    }
+    return minTimestamp;
+  }
+
+  /**
+   * copy out Table RegionInfo into incremental backup image need to consider move this logic into
+   * HBackupFileSystem
+   * @param backupContext backup context
+   * @param conf configuration
+   * @throws IOException exception
+   * @throws InterruptedException exception
+   */
+  protected static void copyTableRegionInfo(BackupContext backupContext, Configuration conf)
+      throws IOException, InterruptedException {
+
+    Path rootDir = FSUtils.getRootDir(conf);
+    FileSystem fs = rootDir.getFileSystem(conf);
+
+    // for each table in the table set, copy out the table info and region info files in the correct
+    // directory structure
+    for (String table : backupContext.getTables()) {
+
+      LOG.debug("Attempting to copy table info for:" + table);
+      TableDescriptor orig =
+          FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, TableName.valueOf(table));
+
+      // write a copy of descriptor to the target directory
+      Path target = new Path(backupContext.getBackupStatus(table).getTargetDir());
+      FileSystem targetFs = target.getFileSystem(conf);
+      FSTableDescriptors descriptors =
+          new FSTableDescriptors(conf, targetFs, FSUtils.getRootDir(conf));
+      descriptors.createTableDescriptorForTableDirectory(target, orig, false);
+      LOG.debug("Finished copying tableinfo.");
+
+      HBaseAdmin hbadmin = null;
+      // TODO: optimize
+      Connection conn = null;
+      List<HRegionInfo> regions = null;
+      try {
+        conn = ConnectionFactory.createConnection(conf);
+        hbadmin = (HBaseAdmin) conn.getAdmin();
+        regions = hbadmin.getTableRegions(TableName.valueOf(table));
+      } catch (Exception e) {
+        throw new BackupException(e);
+      } finally {
+        if (hbadmin != null) {
+          hbadmin.close();
+        }
+        if(conn != null){
+          conn.close();
+        }
+      }
+
+      // For each region, write the region info to disk
+      LOG.debug("Starting to write region info for table " + table);
+      for (HRegionInfo regionInfo : regions) {
+        Path regionDir =
+            HRegion.getRegionDir(new Path(backupContext.getBackupStatus(table).getTargetDir()),
+              regionInfo);
+        regionDir =
+            new Path(backupContext.getBackupStatus(table).getTargetDir(), regionDir.getName());
+        writeRegioninfoOnFilesystem(conf, targetFs, regionDir, regionInfo);
+      }
+      LOG.debug("Finished writing region info for table " + table);
+    }
+  }
+
+  /**
+   * Write the .regioninfo file on-disk.
+   */
+  public static void writeRegioninfoOnFilesystem(final Configuration conf, final FileSystem fs,
+      final Path regionInfoDir, HRegionInfo regionInfo) throws IOException {
+    final byte[] content = regionInfo.toDelimitedByteArray();
+    Path regionInfoFile = new Path(regionInfoDir, ".regioninfo");
+    // First check to get the permissions
+    FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
+    // Write the RegionInfo file content
+    FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null);
+    try {
+      out.write(content);
+    } finally {
+      out.close();
+    }
+  }
+
+  /**
+   * TODO: verify the code
+   * @param p path
+   * @return host name
+   * @throws IOException exception
+   */
+  protected static String parseHostFromOldLog(Path p) throws IOException {
+    String n = p.getName();
+    int idx = n.lastIndexOf(LOGNAME_SEPARATOR);
+    String s = URLDecoder.decode(n.substring(0, idx), "UTF8");
+    return ServerName.parseHostname(s);
+  }
+
+  public static String parseHostNameFromLogFile(Path p) throws IOException {
+    if (isArchivedLogFile(p)) {
+      return parseHostFromOldLog(p);
+    } else {
+      return DefaultWALProvider.getServerNameFromWALDirectoryName(p).getHostname();
+    }
+  }
+
+  private static boolean isArchivedLogFile(Path p) {
+    String oldLog = Path.SEPARATOR + HConstants.HREGION_OLDLOGDIR_NAME + Path.SEPARATOR;
+    return p.toString().contains(oldLog);
+  }
+
+  /**
+   * Return WAL file name
+   * @param walFileName WAL file name
+   * @return WAL file name
+   * @throws IOException exception
+   * @throws IllegalArgumentException exception
+   */
+  public static String getUniqueWALFileNamePart(String walFileName) throws IOException {
+    return new Path(walFileName).getName();
+  }
+
+  /**
+   * Return WAL file name
+   * @param p - WAL file path
+   * @return WAL file name
+   * @throws IOException exception
+   */
+  public static String getUniqueWALFileNamePart(Path p) throws IOException {
+    return p.getName();
+  }
+
+  /**
+   * Given the log file, parse the timestamp from the file name. The timestamp is the last number.
+   * @param p a path to the log file
+   * @return the timestamp
+   * @throws IOException exception
+   */
+  protected static String getCreationTime(Path p, Configuration conf) throws IOException {
+    int idx = p.getName().lastIndexOf(LOGNAME_SEPARATOR);
+    if (idx < 0) {
+      throw new IOException("Cannot parse timestamp from path " + p);
+    }
+    String ts = p.getName().substring(idx + 1);
+    return ts;
+  }
+
+  /**
+   * Get the total length of files under the given directory recursively.
+   * @param fs The hadoop file system
+   * @param dir The target directory
+   * @return the total length of files
+   * @throws IOException exception
+   */
+  public static long getFilesLength(FileSystem fs, Path dir) throws IOException {
+    long totalLength = 0;
+    FileStatus[] files = FSUtils.listStatus(fs, dir);
+    if (files != null) {
+      for (FileStatus fileStatus : files) {
+        if (fileStatus.isDir()) {
+          totalLength += getFilesLength(fs, fileStatus.getPath());
+        } else {
+          totalLength += fileStatus.getLen();
+        }
+      }
+    }
+    return totalLength;
+  }
+
+  /**
+   * Keep the record for dependency for incremental backup and history info p.s, we may be able to
+   * merge this class into backupImage class later
+   */
+  public static class BackupCompleteData implements Comparable<BackupCompleteData> {
+    private String startTime;
+    private String endTime;
+    private String type;
+    private String backupRootPath;
+    private String tableList;
+    private String backupToken;
+    private String bytesCopied;
+    private List<String> ancestors;
+    private boolean fromExistingSnapshot = false;
+
+    public List<String> getAncestors() {
+      if (fromExistingSnapshot) {
+        return null;
+      }
+      if (this.ancestors == null) {
+        this.ancestors = new ArrayList<String>();
+      }
+      return this.ancestors;
+    }
+
+    public void addAncestor(String backupToken) {
+      this.getAncestors().add(backupToken);
+    }
+
+    public String getBytesCopied() {
+      return bytesCopied;
+    }
+
+    public void setBytesCopied(String bytesCopied) {
+      this.bytesCopied = bytesCopied;
+    }
+
+    public String getBackupToken() {
+      return backupToken;
+    }
+
+    public void setBackupToken(String backupToken) {
+      this.backupToken = backupToken;
+    }
+
+    public String getStartTime() {
+      return startTime;
+    }
+
+    public void setStartTime(String startTime) {
+      this.startTime = startTime;
+    }
+
+    public String getEndTime() {
+      return endTime;
+    }
+
+    public void setEndTime(String endTime) {
+      this.endTime = endTime;
+    }
+
+    public String getType() {
+      return type;
+    }
+
+    public void setType(String type) {
+      this.type = type;
+    }
+
+    public String getBackupRootPath() {
+      return backupRootPath;
+    }
+
+    public void setBackupRootPath(String backupRootPath) {
+      this.backupRootPath = backupRootPath;
+    }
+
+    public String getTableList() {
+      return tableList;
+    }
+
+    public void setTableList(String tableList) {
+      this.tableList = tableList;
+    }
+
+    public boolean fromExistingSnapshot() {
+      return this.fromExistingSnapshot;
+    }
+
+    public void markFromExistingSnapshot() {
+      this.fromExistingSnapshot = true;
+    }
+
+    @Override
+    public int compareTo(BackupCompleteData o) {
+      Long thisTS =
+          new Long(this.getBackupToken().substring(this.getBackupToken().lastIndexOf("_") + 1));
+      Long otherTS =
+          new Long(o.getBackupToken().substring(o.getBackupToken().lastIndexOf("_") + 1));
+      return thisTS.compareTo(otherTS);
+    }
+
+  }
+
+  /**
+   * Sort history list by start time in descending order.
+   * @param historyList history list
+   * @return sorted list of BackupCompleteData
+   */
+  public static ArrayList<BackupCompleteData> sortHistoryListDesc(
+    ArrayList<BackupCompleteData> historyList) {
+    ArrayList<BackupCompleteData> list = new ArrayList<BackupCompleteData>();
+    TreeMap<String, BackupCompleteData> map = new TreeMap<String, BackupCompleteData>();
+    for (BackupCompleteData h : historyList) {
+      map.put(h.getStartTime(), h);
+    }
+    Iterator<String> i = map.descendingKeySet().iterator();
+    while (i.hasNext()) {
+      list.add(map.get(i.next()));
+    }
+    return list;
+  }
+
+  /**
+   * Get list of all WAL files (WALs and archive)
+   * @param c - configuration
+   * @return list of WAL files
+   * @throws IOException exception
+   */
+  public static List<String> getListOfWALFiles(Configuration c) throws IOException {
+    Path rootDir = FSUtils.getRootDir(c);
+    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
+    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    List<String> logFiles = new ArrayList<String>();
+
+    FileSystem fs = FileSystem.get(c);
+    logFiles = getFiles(fs, logDir, logFiles, null);
+    logFiles = getFiles(fs, oldLogDir, logFiles, null);
+    return logFiles;
+  }
+
+  /**
+   * Get list of all WAL files (WALs and archive)
+   * @param c - configuration
+   * @return list of WAL files
+   * @throws IOException exception
+   */
+  public static List<String> getListOfWALFiles(Configuration c, PathFilter filter)
+      throws IOException {
+    Path rootDir = FSUtils.getRootDir(c);
+    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
+    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    List<String> logFiles = new ArrayList<String>();
+
+    FileSystem fs = FileSystem.get(c);
+    logFiles = getFiles(fs, logDir, logFiles, filter);
+    logFiles = getFiles(fs, oldLogDir, logFiles, filter);
+    return logFiles;
+  }
+
+  /**
+   * Get list of all old WAL files (WALs and archive)
+   * @param c - configuration
+   * @return list of WAL files
+   * @throws IOException exception
+   */
+  public static List<String> getWALFilesOlderThan(final Configuration c,
+    final HashMap<String, String> hostTimestampMap) throws IOException {
+    Path rootDir = FSUtils.getRootDir(c);
+    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
+    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
+    List<String> logFiles = new ArrayList<String>();
+
+    PathFilter filter = new PathFilter() {
+
+      @Override
+      public boolean accept(Path p) {
+        try {
+          if (DefaultWALProvider.isMetaFile(p)) {
+            return false;
+          }
+          String host = BackupUtil.parseHostNameFromLogFile(p);
+          String oldTimestamp = hostTimestampMap.get(host);
+          String currentLogTS = getCreationTime(p, c);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("path=" + p);
+            LOG.debug("oldTimestamp=" + oldTimestamp);
+            LOG.debug("currentLogTS=" + currentLogTS);
+          }
+          return Long.parseLong(currentLogTS) <= Long.parseLong(oldTimestamp);
+        } catch (IOException e) {
+          LOG.error(e);
+          return false;
+        }
+      }
+    };
+    FileSystem fs = FileSystem.get(c);
+    logFiles = getFiles(fs, logDir, logFiles, filter);
+    logFiles = getFiles(fs, oldLogDir, logFiles, filter);
+    return logFiles;
+  }
+
+  private static List<String> getFiles(FileSystem fs, Path rootDir, List<String> files,
+    PathFilter filter) throws FileNotFoundException, IOException {
+
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(rootDir, true);
+
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (lfs.isDirectory()) {
+        continue;
+      }
+      // apply filter
+      if (filter.accept(lfs.getPath())) {
+        files.add(lfs.getPath().toString());
+        LOG.info(lfs.getPath());
+      }
+    }
+    return files;
+  }
+
+  public static String concat(Collection<String> col, String separator) {
+    if (col.size() == 0) {
+      return "";
+    }
+    StringBuilder sb = new StringBuilder();
+    for (String s : col) {
+      sb.append(s + separator);
+    }
+    sb.deleteCharAt(sb.lastIndexOf(";"));
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
new file mode 100644
index 0000000..74411da
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
@@ -0,0 +1,511 @@
+/**
+ *
+ * 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.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.TreeMap;
+
+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.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * View to an on-disk Backup Image FileSytem
+ * Provides the set of methods necessary to interact with the on-disk Backup Image data.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HBackupFileSystem {
+  public static final Log LOG = LogFactory.getLog(HBackupFileSystem.class);
+
+  private final String RESTORE_TMP_PATH = "/tmp/restoreTemp";
+  private final String[] ignoreDirs = { "recovered.edits" };
+
+  private final Configuration conf;
+  private final FileSystem fs;
+  private final Path backupRootPath;
+  private final String backupId;
+
+  /**
+   * Create a view to the on-disk Backup Image. 
+   * @param conf  to use
+   * @param backupPath  to where the backup Image stored
+   * @param backupId represent backup Image
+   */
+  HBackupFileSystem(final Configuration conf, final Path backupRootPath, final String backupId)
+      throws IOException {
+    this.conf = conf;
+    this.fs = backupRootPath.getFileSystem(conf);
+    this.backupRootPath = backupRootPath;
+    this.backupId = backupId; // the backup ID for the lead backup Image
+  }
+
+  /**
+   * @param tableName is the table backuped
+   * @return {@link HTableDescriptor} saved in backup image of the table
+   */
+  protected HTableDescriptor getTableDesc(String tableName) throws FileNotFoundException,
+  IOException {
+
+    Path tableInfoPath = this.getTableInfoPath(tableName);
+    LOG.debug("tableInfoPath = " + tableInfoPath.toString());
+    SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath);
+    LOG.debug("desc = " + desc.getName());
+    SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc);
+    HTableDescriptor tableDescriptor = manifest.getTableDescriptor();
+    /*
+     * for HBase 0.96 or 0.98 HTableDescriptor tableDescriptor =
+     * FSTableDescriptors.getTableDescriptorFromFs(fs, tableInfoPath);
+     */
+    if (!tableDescriptor.getNameAsString().equals(tableName)) {
+      LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: "
+          + tableInfoPath.toString());
+      LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString());
+    }
+    return tableDescriptor;
+  }
+
+  /**
+   * Given the backup root dir, backup id and the table name, return the backup image location,
+   * which is also where the backup manifest file is. return value look like:
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup1/default/t1_dn/backup_1396650096738"
+   * @param backupRootDir backup root directory
+   * @param backupId  backup id
+   * @param table table name
+   * @return backupPath String for the particular table
+   */
+  protected static String getTableBackupDir(String backupRootDir, String backupId, String table) {
+    TableName tableName = TableName.valueOf(table);
+    return backupRootDir + File.separator + tableName.getNamespaceAsString() + File.separator
+        + tableName.getQualifierAsString() + File.separator + backupId;
+  }
+
+  /**
+   * Given the backup root dir, backup id and the table name, return the backup image location,
+   * which is also where the backup manifest file is. return value look like:
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup1/default/t1_dn/backup_1396650096738"
+   * @param tableN table name
+   * @return backupPath for the particular table
+   */
+  protected Path getTableBackupPath(String tableN) {
+    TableName tableName = TableName.valueOf(tableN);
+    return new Path(this.backupRootPath, tableName.getNamespaceAsString() + File.separator
+      + tableName.getQualifierAsString() + File.separator + backupId);
+  }
+
+  /**
+   * return value represent path for:
+   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/.hbase-snapshot"
+   * @param tableName table name
+   * @return path for snapshot
+   */
+  protected Path getTableSnapshotPath(String tableName) {
+    return new Path(this.getTableBackupPath(tableName), HConstants.SNAPSHOT_DIR_NAME);
+  }
+
+  /**
+   * return value represent path for:
+   * "..../default/t1_dn/backup_1396650096738/.hbase-snapshot/snapshot_1396650097621_default_t1_dn"
+   * this path contains .snapshotinfo, .tabledesc (0.96 and 0.98) this path contains .snapshotinfo,
+   * .data.manifest (trunk)
+   * @param tableName table name
+   * @return path to table info
+   * @throws FileNotFoundException exception
+   * @throws IOException exception
+   */
+  protected Path getTableInfoPath(String tableName) throws FileNotFoundException, IOException {
+
+    Path tableSnapShotPath = this.getTableSnapshotPath(tableName);
+    Path tableInfoPath = null;
+
+    // can't build the path directly as the timestamp values are different
+    FileStatus[] snapshots = fs.listStatus(tableSnapShotPath);
+    for (FileStatus snapshot : snapshots) {
+      tableInfoPath = snapshot.getPath();
+      // SnapshotManifest.DATA_MANIFEST_NAME = "data.manifest";
+      if (tableInfoPath.getName().endsWith("data.manifest")) {
+        LOG.debug("find Snapshot Manifest");
+        break;
+      }
+    }
+    return tableInfoPath;
+  }
+
+  /**
+   * return value represent path for:
+   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn"
+   * @param tabelName table name
+   * @return path to table archive
+   * @throws IOException exception
+   */
+  protected Path getTableArchivePath(String tableName) throws IOException {
+    Path baseDir = new Path(getTableBackupPath(tableName), HConstants.HFILE_ARCHIVE_DIRECTORY);
+    Path dataDir = new Path(baseDir, HConstants.BASE_NAMESPACE_DIR);
+    Path archivePath = new Path(dataDir, TableName.valueOf(tableName).getNamespaceAsString());
+    Path tableArchivePath =
+        new Path(archivePath, TableName.valueOf(tableName).getQualifierAsString());
+    if (!fs.exists(tableArchivePath) || !fs.getFileStatus(tableArchivePath).isDirectory()) {
+      LOG.debug("Folder tableArchivePath: " + tableArchivePath.toString() + " does not exists");
+      tableArchivePath = null; // empty table has no archive
+    }
+    return tableArchivePath;
+  }
+
+  /**
+   * Given the backup root dir and the backup id, return the log file location for an incremental
+   * backup.
+   * @param backupRootDir backup root directory
+   * @param backupId backup id
+   * @return logBackupDir: ".../user/biadmin/backup1/WALs/backup_1396650096738"
+   */
+  protected static String getLogBackupDir(String backupRootDir, String backupId) {
+    return backupRootDir + File.separator + HConstants.HREGION_LOGDIR_NAME + File.separator
+        + backupId;
+  }
+
+  protected static Path getLogBackupPath(String backupRootDir, String backupId) {
+    return new Path(getLogBackupDir(backupRootDir, backupId));
+  }
+
+  private Path getManifestPath(String tableName) throws IOException {
+    Path manifestPath = new Path(getTableBackupPath(tableName), BackupManifest.FILE_NAME);
+
+    LOG.debug("Looking for " + manifestPath.toString());
+    if (!fs.exists(manifestPath)) {
+      // check log dir for incremental backup case
+      manifestPath =
+          new Path(getLogBackupDir(this.backupRootPath.toString(), this.backupId) + File.separator
+            + BackupManifest.FILE_NAME);
+      LOG.debug("Looking for " + manifestPath.toString());
+      if (!fs.exists(manifestPath)) {
+        String errorMsg =
+            "Could not find backup manifest for " + backupId + " in " + backupRootPath.toString();
+        throw new IOException(errorMsg);
+      }
+    }
+    return manifestPath;
+  }
+
+  protected BackupManifest getManifest(String tableName) throws IOException {
+    BackupManifest manifest = new BackupManifest(conf, this.getManifestPath(tableName));
+    return manifest;
+  }
+
+  /**
+   * Gets region list
+   * @param tableName table name
+   * @return RegionList region list
+   * @throws FileNotFoundException exception
+   * @throws IOException exception
+   */
+
+  protected ArrayList<Path> getRegionList(String tableName) throws FileNotFoundException,
+  IOException {
+    Path tableArchivePath = this.getTableArchivePath(tableName);
+    ArrayList<Path> regionDirList = new ArrayList<Path>();
+    FileStatus[] children = fs.listStatus(tableArchivePath);
+    for (FileStatus childStatus : children) {
+      // here child refer to each region(Name)
+      Path child = childStatus.getPath();
+      regionDirList.add(child);
+    }
+    return regionDirList;
+  }
+
+  /**
+   * Gets region list
+   * @param tableArchivePath table archive path
+   * @return RegionList region list
+   * @throws FileNotFoundException exception
+   * @throws IOException exception
+   */
+  protected ArrayList<Path> getRegionList(Path tableArchivePath) throws FileNotFoundException,
+  IOException {
+    ArrayList<Path> regionDirList = new ArrayList<Path>();
+    FileStatus[] children = fs.listStatus(tableArchivePath);
+    for (FileStatus childStatus : children) {
+      // here child refer to each region(Name)
+      Path child = childStatus.getPath();
+      regionDirList.add(child);
+    }
+    return regionDirList;
+  }
+
+  /**
+   * Counts the number of files in all subdirectories of an HBase tables, i.e. HFiles. And finds the
+   * maximum number of files in one HBase table.
+   * @param tableArchivePath archive path
+   * @return the maximum number of files found in 1 HBase table
+   * @throws IOException exception
+   */
+  protected int getMaxNumberOfFilesInSubDir(Path tableArchivePath) throws IOException {
+    int result = 1;
+    ArrayList<Path> regionPathList = this.getRegionList(tableArchivePath);
+    // tableArchivePath = this.getTableArchivePath(tableName);
+
+    if (regionPathList == null || regionPathList.size() == 0) {
+      throw new IllegalStateException("Cannot restore hbase table because directory '"
+          + tableArchivePath + "' is not a directory.");
+    }
+
+    for (Path regionPath : regionPathList) {
+      result = Math.max(result, getNumberOfFilesInDir(regionPath));
+    }
+    return result;
+  }
+
+  /**
+   * Counts the number of files in all subdirectories of an HBase table, i.e. HFiles.
+   * @param regionPath Path to an HBase table directory
+   * @return the number of files all directories
+   * @throws IOException exception
+   */
+  protected int getNumberOfFilesInDir(Path regionPath) throws IOException {
+    int result = 0;
+
+    if (!fs.exists(regionPath) || !fs.getFileStatus(regionPath).isDirectory()) {
+      throw new IllegalStateException("Cannot restore hbase table because directory '"
+          + regionPath.toString() + "' is not a directory.");
+    }
+
+    FileStatus[] tableDirContent = fs.listStatus(regionPath);
+    for (FileStatus subDirStatus : tableDirContent) {
+      FileStatus[] colFamilies = fs.listStatus(subDirStatus.getPath());
+      for (FileStatus colFamilyStatus : colFamilies) {
+        FileStatus[] colFamilyContent = fs.listStatus(colFamilyStatus.getPath());
+        result += colFamilyContent.length;
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Duplicate the backup image if it's on local cluster
+   * @see HStore#bulkLoadHFile(String, long)
+   * @see HRegionFileSystem#bulkLoadStoreFile(String familyName, Path srcPath, long seqNum)
+   * @param tableArchivePath archive path
+   * @return the new tableArchivePath 
+   * @throws IOException exception
+   */
+  protected Path checkLocalAndBackup(Path tableArchivePath) throws IOException {
+    // Move the file if it's on local cluster
+    boolean isCopyNeeded = false;
+
+    FileSystem srcFs = tableArchivePath.getFileSystem(conf);
+    FileSystem desFs = FileSystem.get(conf);
+    if (tableArchivePath.getName().startsWith("/")) {
+      isCopyNeeded = true;
+    } else {
+      // This should match what is done in @see HRegionFileSystem#bulkLoadStoreFile(String, Path,
+      // long)
+      if (srcFs.getUri().equals(desFs.getUri())) {
+        LOG.debug("cluster hold the backup image: " + srcFs.getUri() + "; local cluster node: "
+            + desFs.getUri());
+        isCopyNeeded = true;
+      }
+    }
+    if (isCopyNeeded) {
+      LOG.debug("File " + tableArchivePath + " on local cluster, back it up before restore");
+      Path tmpPath = new Path(RESTORE_TMP_PATH);
+      if (desFs.exists(tmpPath)) {
+        try {
+          desFs.delete(tmpPath, true);
+        } catch (IOException e) {
+          LOG.debug("Failed to delete path: " + tmpPath
+            + ", need to check whether restore target DFS cluster is healthy");
+        }
+      }
+      FileUtil.copy(srcFs, tableArchivePath, desFs, tmpPath, false, conf);
+      LOG.debug("Copied to temporary path on local cluster: " + tmpPath);
+      tableArchivePath = tmpPath;
+    }
+    return tableArchivePath;
+  }
+
+  /**
+   * Calculate region boundaries and add all the column families to the table descriptor
+   * @param regionDirList region dir list
+   * @return a set of keys to store the boundaries
+   */
+  protected byte[][] generateBoundaryKeys(ArrayList<Path> regionDirList)
+      throws FileNotFoundException, IOException {
+    TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+    // Build a set of keys to store the boundaries
+    byte[][] keys = null;
+    // calculate region boundaries and add all the column families to the table descriptor
+    for (Path regionDir : regionDirList) {
+      LOG.debug("Parsing region dir: " + regionDir);
+      Path hfofDir = regionDir;
+
+      if (!fs.exists(hfofDir)) {
+        LOG.warn("HFileOutputFormat dir " + hfofDir + " not found");
+      }
+
+      FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
+      if (familyDirStatuses == null) {
+        throw new IOException("No families found in " + hfofDir);
+      }
+
+      for (FileStatus stat : familyDirStatuses) {
+        if (!stat.isDirectory()) {
+          LOG.warn("Skipping non-directory " + stat.getPath());
+          continue;
+        }
+        boolean isIgnore = false;
+        String pathName = stat.getPath().getName();
+        for (String ignore : ignoreDirs) {
+          if (pathName.contains(ignore)) {
+            LOG.warn("Skipping non-family directory" + pathName);
+            isIgnore = true;
+            break;
+          }
+        }
+        if (isIgnore) {
+          continue;
+        }
+        Path familyDir = stat.getPath();
+        LOG.debug("Parsing family dir [" + familyDir.toString() + " in region [" + regionDir + "]");
+        // Skip _logs, etc
+        if (familyDir.getName().startsWith("_") || familyDir.getName().startsWith(".")) {
+          continue;
+        }
+
+        // start to parse hfile inside one family dir
+        Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
+        for (Path hfile : hfiles) {
+          if (hfile.getName().startsWith("_") || hfile.getName().startsWith(".")
+              || StoreFileInfo.isReference(hfile.getName())
+              || HFileLink.isHFileLink(hfile.getName())) {
+            continue;
+          }
+          HFile.Reader reader = HFile.createReader(fs, hfile, new CacheConfig(conf), conf);
+          final byte[] first, last;
+          try {
+            reader.loadFileInfo();
+            first = reader.getFirstRowKey();
+            last = reader.getLastRowKey();
+            LOG.debug("Trying to figure out region boundaries hfile=" + hfile + " first="
+                + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));
+
+            // To eventually infer start key-end key boundaries
+            Integer value = map.containsKey(first) ? (Integer) map.get(first) : 0;
+            map.put(first, value + 1);
+            value = map.containsKey(last) ? (Integer) map.get(last) : 0;
+            map.put(last, value - 1);
+          } finally {
+            reader.close();
+          }
+        }
+      }
+    }
+    keys = LoadIncrementalHFiles.inferBoundaries(map);
+    return keys;
+  }
+
+  /**
+   * Check whether the backup path exist
+   * @param backupStr backup
+   * @param conf configuration
+   * @return Yes if path exists
+   * @throws IOException exception
+   */
+  protected static boolean checkPathExist(String backupStr, Configuration conf) 
+    throws IOException {
+    boolean isExist = false;
+    Path backupPath = new Path(backupStr);
+    FileSystem fileSys = backupPath.getFileSystem(conf);
+    String targetFsScheme = fileSys.getUri().getScheme();
+    LOG.debug("Schema of given url: " + backupStr + " is: " + targetFsScheme);
+    if (fileSys.exists(backupPath)) {
+      isExist = true;
+    }
+    return isExist;
+  }
+
+  /**
+   * Check whether the backup image path and there is manifest file in the path.
+   * @param backupManifestMap If all the manifests are found, then they are put into this map
+   * @param tableArray the tables involved
+   * @throws IOException exception
+   */
+  protected void checkImageManifestExist(HashMap<String, BackupManifest> backupManifestMap,
+      String[] tableArray) throws IOException {
+
+    try {
+      for (String tableName : tableArray) {
+        BackupManifest manifest = this.getManifest(tableName);
+        backupManifestMap.put(tableName, manifest);
+      }
+    } catch (IOException e) {
+      String expMsg = e.getMessage();
+      if (expMsg.contains("No FileSystem for scheme")) {
+        if (expMsg.contains("gpfs")) {
+          LOG.error("Please change to use webhdfs url when "
+              + "the backup image to restore locates on gpfs cluster");
+        } else {
+          LOG.error("Unsupported filesystem scheme found in the backup target url, "
+              + "please check the url to make sure no typo in it");
+        }
+        throw e;
+      } else if (expMsg.contains("no authority supported")) {
+        LOG.error("Please change to use webhdfs url when "
+            + "the backup image to restore locates on gpfs cluster");
+        throw e;
+      } else {
+        LOG.error(expMsg);
+        throw e;
+      }
+    }
+  }
+
+  public static String join(String[] names) {
+    StringBuilder sb = new StringBuilder();
+    String sep = BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND;
+    for (String s : names) {
+      sb.append(sep).append(s);
+    }
+    return sb.toString();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java
new file mode 100644
index 0000000..e91857f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java
@@ -0,0 +1,269 @@
+/**
+ * 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.HashMap;
+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.HConstants;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+
+/**
+ * After a full backup was created, the incremental backup will only store the changes made
+ * after the last full or incremental backup.
+ *
+ * Creating the backup copies the logfiles in .logs and .oldlogs since the last backup timestamp.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class IncrementalBackupManager {
+  // parent manager
+  private BackupManager backupManager;
+
+  public static final Log LOG = LogFactory.getLog(IncrementalBackupManager.class);
+
+  public IncrementalBackupManager(BackupManager bm) {
+    this.backupManager = bm;
+  }
+
+  /**
+   * Obtain the list of logs that need to be copied out for this incremental backup. The list is set
+   * in BackupContext.
+   * @param backupContext backup context
+   * @return The new HashMap of RS log timestamps after the log roll for this incremental backup.
+   * @throws IOException exception
+   */
+  public HashMap<String, String> getIncrBackupLogFileList(BackupContext backupContext)
+      throws IOException {
+    List<String> logList;
+    HashMap<String, String> newTimestamps;
+    HashMap<String, String> previousTimestampMins;
+
+    Configuration conf = BackupUtil.getConf();
+    String savedStartCode = backupManager.readBackupStartCode();
+
+    // key: tableName
+    // value: <RegionServer,PreviousTimeStamp>
+    HashMap<String, HashMap<String, String>> previousTimestampMap =
+        backupManager.readLogTimestampMap();
+
+    previousTimestampMins = BackupUtil.getRSLogTimestampMins(previousTimestampMap);
+
+    LOG.debug("StartCode " + savedStartCode + "for backupID " + backupContext.getBackupId());
+    LOG.debug("Timestamps " + previousTimestampMap);
+    // get all new log files from .logs and .oldlogs after last TS and before new timestamp
+    if (savedStartCode == null || 
+        previousTimestampMins == null || 
+          previousTimestampMins.isEmpty()) {
+      throw new IOException("Cannot read any previous back up timestamps from hbase:backup. "
+          + "In order to create an incremental backup, at least one full backup is needed.");
+    }
+
+    HBaseAdmin hbadmin = null;
+    Connection conn = null;
+    try {
+      LOG.info("Execute roll log procedure for incremental backup ...");
+      conn = ConnectionFactory.createConnection(conf);
+      hbadmin = (HBaseAdmin) conn.getAdmin();
+      hbadmin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
+        LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, new HashMap<String, String>());
+    } finally {
+      if (hbadmin != null) {
+        hbadmin.close();
+      }
+      if(conn != null){
+        conn.close();
+      }
+    }
+
+    newTimestamps = backupManager.readRegionServerLastLogRollResult();
+
+    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
+
+    backupContext.setIncrBackupFileList(logList);
+
+    return newTimestamps;
+  }
+
+  /**
+   * For each region server: get all log files newer than the last timestamps but not newer than the
+   * newest timestamps.
+   * @param olderTimestamps the timestamp for each region server of the last backup.
+   * @param newestTimestamps the timestamp for each region server that the backup should lead to.
+   * @param conf the Hadoop and Hbase configuration
+   * @param savedStartCode the startcode (timestamp) of last successful backup.
+   * @return a list of log files to be backed up
+   * @throws IOException exception
+   */
+  private List<String> getLogFilesForNewBackup(HashMap<String, String> olderTimestamps,
+    HashMap<String, String> newestTimestamps, Configuration conf, String savedStartCode)
+        throws IOException {
+    LOG.debug("In getLogFilesForNewBackup()\n" + "olderTimestamps: " + olderTimestamps
+      + "\n newestTimestamps: " + newestTimestamps);
+    Path rootdir = FSUtils.getRootDir(conf);
+    Path logDir = new Path(rootdir, HConstants.HREGION_LOGDIR_NAME);
+    Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
+    FileSystem fs = rootdir.getFileSystem(conf);
+    NewestLogFilter pathFilter = new NewestLogFilter(conf);
+
+    List<String> resultLogFiles = new ArrayList<String>();
+    List<String> newestLogs = new ArrayList<String>();
+
+    /*
+     * The old region servers and timestamps info we kept in hbase:backup may be out of sync if new
+     * region server is added or existing one lost. We'll deal with it here when processing the
+     * logs. If data in hbase:backup has more hosts, just ignore it. If the .logs directory includes
+     * more hosts, the additional hosts will not have old timestamps to compare with. We'll just use
+     * all the logs in that directory. We always write up-to-date region server and timestamp info
+     * to hbase:backup at the end of successful backup.
+     */
+
+    FileStatus[] rss;
+    Path p;
+    String host;
+    String oldTimeStamp;
+    String currentLogFile;
+    String currentLogTS;
+
+    // Get the files in .logs.
+    rss = fs.listStatus(logDir);
+    for (FileStatus rs : rss) {
+      p = rs.getPath();
+      host = DefaultWALProvider.getServerNameFromWALDirectoryName(p).getHostname();
+      FileStatus[] logs;
+      oldTimeStamp = olderTimestamps.get(host);
+      // It is possible that there is no old timestamp in hbase:backup for this host if
+      // this region server is newly added after our last backup.
+      if (oldTimeStamp == null) {
+        logs = fs.listStatus(p);
+      } else {
+        pathFilter.setLastBackupTS(oldTimeStamp);
+        logs = fs.listStatus(p, pathFilter);
+      }
+      for (FileStatus log : logs) {
+        LOG.debug("currentLogFile: " + log.getPath().toString());
+        if (DefaultWALProvider.isMetaFile(log.getPath())) {
+          LOG.debug("Skip hbase:meta log file: " + log.getPath().getName());
+          continue;
+        }
+        currentLogFile = log.getPath().toString();
+        resultLogFiles.add(currentLogFile);
+        currentLogTS = BackupUtil.getCreationTime(log.getPath(), conf);
+        // newestTimestamps is up-to-date with the current list of hosts
+        // so newestTimestamps.get(host) will not be null.
+        if (Long.valueOf(currentLogTS) > Long.valueOf(newestTimestamps.get(host))) {
+          newestLogs.add(currentLogFile);
+        }
+      }
+    }
+
+    // Include the .oldlogs files too.
+    FileStatus[] oldlogs = fs.listStatus(oldLogDir);
+    for (FileStatus oldlog : oldlogs) {
+      p = oldlog.getPath();
+      currentLogFile = p.toString();
+      if (DefaultWALProvider.isMetaFile(p)) {
+        LOG.debug("Skip .meta log file: " + currentLogFile);
+        continue;
+      }
+      host = BackupUtil.parseHostFromOldLog(p);
+      currentLogTS = BackupUtil.getCreationTime(p, conf);
+      oldTimeStamp = olderTimestamps.get(host);
+      /*
+       * It is possible that there is no old timestamp in hbase:backup for this host. At the time of
+       * our last backup operation, this rs did not exist. The reason can be one of the two: 1. The
+       * rs already left/crashed. Its logs were moved to .oldlogs. 2. The rs was added after our
+       * last backup.
+       */
+      if (oldTimeStamp == null) {
+        if (Long.valueOf(currentLogTS) < Long.valueOf(savedStartCode)) {
+          // This log file is really old, its region server was before our last backup.
+          continue;
+        } else {
+          resultLogFiles.add(currentLogFile);
+        }
+      } else if (Long.valueOf(currentLogTS) > Long.valueOf(oldTimeStamp)) {
+        resultLogFiles.add(currentLogFile);
+      }
+
+      LOG.debug("resultLogFiles before removal of newestLogs: " + resultLogFiles);
+      // It is possible that a host in .oldlogs is an obsolete region server
+      // so newestTimestamps.get(host) here can be null.
+      // Even if these logs belong to a obsolete region server, we still need
+      // to include they to avoid loss of edits for backup.
+      String newTimestamp = newestTimestamps.get(host);
+      if (newTimestamp != null && Long.valueOf(currentLogTS) > Long.valueOf(newTimestamp)) {
+        newestLogs.add(currentLogFile);
+      }
+    }
+    LOG.debug("newestLogs: " + newestLogs);
+    // remove newest log per host because they are still in use
+    resultLogFiles.removeAll(newestLogs);
+    LOG.debug("resultLogFiles after removal of newestLogs: " + resultLogFiles);
+    return resultLogFiles;
+  }
+
+  class NewestLogFilter implements PathFilter {
+    private String lastBackupTS = "0";
+    final private Configuration conf;
+
+    public NewestLogFilter(Configuration conf) {
+      this.conf = conf;
+    }
+
+    protected void setLastBackupTS(String ts) {
+      this.lastBackupTS = ts;
+    }
+
+    @Override
+    public boolean accept(Path path) {
+      // skip meta table log -- ts.meta file
+      if (DefaultWALProvider.isMetaFile(path)) {
+        LOG.debug("Skip .meta log file: " + path.getName());
+        return false;
+      }
+      String timestamp;
+      try {
+        timestamp = BackupUtil.getCreationTime(path, conf);
+        return Long.valueOf(timestamp) > Long.valueOf(lastBackupTS);
+      } catch (IOException e) {
+        LOG.warn("Cannot read timestamp of log file " + path);
+        return false;
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java
new file mode 100644
index 0000000..72e4879
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java
@@ -0,0 +1,33 @@
+/**
+ * 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 org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface IncrementalRestoreService extends Configurable{
+
+  public void run(String logDirectory, String[] fromTables, String[] toTables)
+    throws IOException;
+}


[6/6] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov)

Posted by te...@apache.org.
HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov)


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

Branch: refs/heads/master
Commit: de69f0df34a6cb00a9fd2e370fdf898274871294
Parents: 5eefe13
Author: tedyu <yu...@gmail.com>
Authored: Sun Dec 27 10:02:09 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Sun Dec 27 10:02:09 2015 -0800

----------------------------------------------------------------------
 bin/hbase                                       |   6 +
 .../org/apache/hadoop/hbase/HConstants.java     | 445 +++++-----
 hbase-server/pom.xml                            |  10 +
 .../hadoop/hbase/backup/BackupClient.java       | 338 ++++++++
 .../hadoop/hbase/backup/BackupCommands.java     | 147 ++++
 .../hadoop/hbase/backup/BackupContext.java      | 318 ++++++++
 .../hadoop/hbase/backup/BackupCopyService.java  |  37 +
 .../hadoop/hbase/backup/BackupException.java    |  85 ++
 .../hadoop/hbase/backup/BackupHandler.java      | 744 +++++++++++++++++
 .../hadoop/hbase/backup/BackupManager.java      | 488 +++++++++++
 .../hadoop/hbase/backup/BackupManifest.java     | 814 +++++++++++++++++++
 .../hbase/backup/BackupRestoreConstants.java    |  66 ++
 .../backup/BackupRestoreServiceFactory.java     |  61 ++
 .../hadoop/hbase/backup/BackupStatus.java       |  67 ++
 .../hadoop/hbase/backup/BackupSystemTable.java  | 642 +++++++++++++++
 .../hbase/backup/BackupSystemTableHelper.java   | 314 +++++++
 .../apache/hadoop/hbase/backup/BackupUtil.java  | 564 +++++++++++++
 .../hadoop/hbase/backup/HBackupFileSystem.java  | 511 ++++++++++++
 .../hbase/backup/IncrementalBackupManager.java  | 269 ++++++
 .../hbase/backup/IncrementalRestoreService.java |  33 +
 .../hadoop/hbase/backup/RestoreClient.java      | 496 +++++++++++
 .../apache/hadoop/hbase/backup/RestoreUtil.java | 503 ++++++++++++
 .../mapreduce/MapReduceBackupCopyService.java   | 292 +++++++
 .../mapreduce/MapReduceRestoreService.java      |  72 ++
 .../hbase/backup/master/BackupLogCleaner.java   | 121 +++
 .../master/LogRollMasterProcedureManager.java   | 129 +++
 .../regionserver/LogRollBackupSubprocedure.java | 138 ++++
 .../LogRollBackupSubprocedurePool.java          | 137 ++++
 .../LogRollRegionServerProcedureManager.java    | 168 ++++
 .../BaseCoordinatedStateManager.java            |  17 +
 .../coordination/ZkCoordinatedStateManager.java |  20 +-
 .../hadoop/hbase/mapreduce/WALPlayer.java       |  54 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   2 +
 .../procedure/RegionServerProcedureManager.java |   2 +-
 .../RegionServerProcedureManagerHost.java       |   3 +-
 .../procedure/ZKProcedureCoordinatorRpcs.java   |   2 +-
 .../hbase/procedure/ZKProcedureMemberRpcs.java  |  83 +-
 .../RegionServerFlushTableProcedureManager.java |   2 +-
 .../hbase/regionserver/HRegionServer.java       |   4 +-
 .../snapshot/RegionServerSnapshotManager.java   |   2 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  67 +-
 .../hadoop/hbase/snapshot/SnapshotCopy.java     |  42 +
 .../hadoop/hbase/wal/DefaultWALProvider.java    |   7 +-
 .../hadoop/hbase/backup/TestBackupBase.java     | 194 +++++
 .../hbase/backup/TestBackupBoundaryTests.java   |  99 +++
 .../hbase/backup/TestBackupLogCleaner.java      | 161 ++++
 .../hbase/backup/TestBackupSystemTable.java     | 341 ++++++++
 .../hadoop/hbase/backup/TestFullBackup.java     |  83 ++
 .../hadoop/hbase/backup/TestFullRestore.java    | 166 ++++
 .../hbase/backup/TestIncrementalBackup.java     | 179 ++++
 .../hadoop/hbase/backup/TestRemoteBackup.java   |  44 +
 .../hadoop/hbase/backup/TestRemoteRestore.java  |  54 ++
 .../hbase/backup/TestRestoreBoundaryTests.java  |  85 ++
 .../procedure/SimpleRSProcedureManager.java     |   2 +-
 54 files changed, 9389 insertions(+), 341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index 5064451..9fd903a 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -99,6 +99,8 @@ if [ $# = 0 ]; then
   echo "  pe              Run PerformanceEvaluation"
   echo "  ltt             Run LoadTestTool"
   echo "  version         Print the version"
+  echo "  backup          backup tables for recovery"
+  echo "  restore         restore tables from existing backup image"
   echo "  CLASSNAME       Run the class named CLASSNAME"
   exit 1
 fi
@@ -303,6 +305,10 @@ elif [ "$COMMAND" = "hfile" ] ; then
   CLASS='org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter'
 elif [ "$COMMAND" = "zkcli" ] ; then
   CLASS="org.apache.hadoop.hbase.zookeeper.ZooKeeperMainServer"
+elif [ "$COMMAND" = "backup" ] ; then
+  CLASS='org.apache.hadoop.hbase.backup.BackupClient'
+elif [ "$COMMAND" = "restore" ] ; then
+  CLASS='org.apache.hadoop.hbase.backup.RestoreClient'
 elif [ "$COMMAND" = "upgrade" ] ; then
   echo "This command was used to upgrade to HBase 0.96, it was removed in HBase 2.0.0."
   echo "Please follow the documentation at http://hbase.apache.org/book.html#upgrading."

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 6fafad3..0f572e6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -40,11 +40,11 @@ import org.apache.hadoop.hbase.util.Bytes;
 public final class HConstants {
   // NOTICE!!!! Please do not add a constants here, unless they are referenced by a lot of classes.
 
-  //Bytes.UTF8_ENCODING should be updated if this changed
+  // Bytes.UTF8_ENCODING should be updated if this changed
   /** When we encode strings, we always specify UTF8 encoding */
   public static final String UTF8_ENCODING = "UTF-8";
 
-  //Bytes.UTF8_CHARSET should be updated if this changed
+  // Bytes.UTF8_CHARSET should be updated if this changed
   /** When we encode strings, we always specify UTF8 encoding */
   public static final Charset UTF8_CHARSET = Charset.forName(UTF8_ENCODING);
   /**
@@ -55,9 +55,9 @@ public final class HConstants {
   /** Used as a magic return value while optimized index key feature enabled(HBASE-7845) */
   public final static int INDEX_KEY_MAGIC = -2;
   /*
-     * Name of directory that holds recovered edits written by the wal log
-     * splitting code, one per region
-     */
+   * Name of directory that holds recovered edits written by the wal log splitting code, one per
+   * region
+   */
   public static final String RECOVERED_EDITS_DIR = "recovered.edits";
   /**
    * The first four bytes of Hadoop RPC connections
@@ -70,27 +70,24 @@ public final class HConstants {
   /** The size data structures with minor version is 0 */
   public static final int HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM = MAGIC_LENGTH + 2 * Bytes.SIZEOF_INT
       + Bytes.SIZEOF_LONG;
-  /** The size of a version 2 HFile block header, minor version 1.
-   * There is a 1 byte checksum type, followed by a 4 byte bytesPerChecksum
-   * followed by another 4 byte value to store sizeofDataOnDisk.
+  /**
+   * The size of a version 2 HFile block header, minor version 1. There is a 1 byte checksum type,
+   * followed by a 4 byte bytesPerChecksum followed by another 4 byte value to store
+   * sizeofDataOnDisk.
    */
-  public static final int HFILEBLOCK_HEADER_SIZE = HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM +
-    Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT;
+  public static final int HFILEBLOCK_HEADER_SIZE = HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM
+      + Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT;
   /** Just an array of bytes of the right size. */
   public static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HFILEBLOCK_HEADER_SIZE];
 
-  //End HFileBlockConstants.
+  // End HFileBlockConstants.
 
   /**
    * Status codes used for return values of bulk operations.
    */
   @InterfaceAudience.Private
   public enum OperationStatusCode {
-    NOT_RUN,
-    SUCCESS,
-    BAD_FAMILY,
-    SANITY_CHECK_FAILURE,
-    FAILURE;
+    NOT_RUN, SUCCESS, BAD_FAMILY, SANITY_CHECK_FAILURE, FAILURE;
   }
 
   /** long constant for zero */
@@ -104,19 +101,16 @@ public final class HConstants {
   public static final String VERSION_FILE_NAME = "hbase.version";
 
   /**
-   * Current version of file system.
-   * Version 4 supports only one kind of bloom filter.
-   * Version 5 changes versions in catalog table regions.
-   * Version 6 enables blockcaching on catalog tables.
-   * Version 7 introduces hfile -- hbase 0.19 to 0.20..
-   * Version 8 introduces namespace
+   * Current version of file system. Version 4 supports only one kind of bloom filter. Version 5
+   * changes versions in catalog table regions. Version 6 enables blockcaching on catalog tables.
+   * Version 7 introduces hfile -- hbase 0.19 to 0.20.. Version 8 introduces namespace
    */
   // public static final String FILE_SYSTEM_VERSION = "6";
   public static final String FILE_SYSTEM_VERSION = "8";
 
   // Configuration parameters
 
-  //TODO: Is having HBase homed on port 60k OK?
+  // TODO: Is having HBase homed on port 60k OK?
 
   /** Cluster is in distributed mode or not */
   public static final String CLUSTER_DISTRIBUTED = "hbase.cluster.distributed";
@@ -131,12 +125,10 @@ public final class HConstants {
   public static final String ENSEMBLE_TABLE_NAME = "hbase:ensemble";
 
   /** Config for pluggable region normalizer */
-  public static final String HBASE_MASTER_NORMALIZER_CLASS =
-    "hbase.master.normalizer.class";
+  public static final String HBASE_MASTER_NORMALIZER_CLASS = "hbase.master.normalizer.class";
 
   /** Config for enabling/disabling pluggable region normalizer */
-  public static final String HBASE_NORMALIZER_ENABLED =
-    "hbase.normalizer.enabled";
+  public static final String HBASE_NORMALIZER_ENABLED = "hbase.normalizer.enabled";
 
   /** Cluster is standalone or pseudo-distributed */
   public static final boolean CLUSTER_IS_LOCAL = false;
@@ -174,21 +166,18 @@ public final class HConstants {
   public static final String ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
 
   /** Common prefix of ZooKeeper configuration properties */
-  public static final String ZK_CFG_PROPERTY_PREFIX =
-      "hbase.zookeeper.property.";
+  public static final String ZK_CFG_PROPERTY_PREFIX = "hbase.zookeeper.property.";
 
-  public static final int ZK_CFG_PROPERTY_PREFIX_LEN =
-      ZK_CFG_PROPERTY_PREFIX.length();
+  public static final int ZK_CFG_PROPERTY_PREFIX_LEN = ZK_CFG_PROPERTY_PREFIX.length();
 
   /**
-   * The ZK client port key in the ZK properties map. The name reflects the
-   * fact that this is not an HBase configuration key.
+   * The ZK client port key in the ZK properties map. The name reflects the fact that this is not an
+   * HBase configuration key.
    */
   public static final String CLIENT_PORT_STR = "clientPort";
 
   /** Parameter name for the client port that the zookeeper listens on */
-  public static final String ZOOKEEPER_CLIENT_PORT =
-      ZK_CFG_PROPERTY_PREFIX + CLIENT_PORT_STR;
+  public static final String ZOOKEEPER_CLIENT_PORT = ZK_CFG_PROPERTY_PREFIX + CLIENT_PORT_STR;
 
   /** Default client port that the zookeeper listens on */
   public static final int DEFAULT_ZOOKEPER_CLIENT_PORT = 2181;
@@ -208,19 +197,15 @@ public final class HConstants {
   public static final String DEFAULT_ZOOKEEPER_ZNODE_PARENT = "/hbase";
 
   /**
-   * Parameter name for the limit on concurrent client-side zookeeper
-   * connections
+   * Parameter name for the limit on concurrent client-side zookeeper connections
    */
-  public static final String ZOOKEEPER_MAX_CLIENT_CNXNS =
-      ZK_CFG_PROPERTY_PREFIX + "maxClientCnxns";
+  public static final String ZOOKEEPER_MAX_CLIENT_CNXNS = ZK_CFG_PROPERTY_PREFIX + "maxClientCnxns";
 
   /** Parameter name for the ZK data directory */
-  public static final String ZOOKEEPER_DATA_DIR =
-      ZK_CFG_PROPERTY_PREFIX + "dataDir";
+  public static final String ZOOKEEPER_DATA_DIR = ZK_CFG_PROPERTY_PREFIX + "dataDir";
 
   /** Parameter name for the ZK tick time */
-  public static final String ZOOKEEPER_TICK_TIME =
-      ZK_CFG_PROPERTY_PREFIX + "tickTime";
+  public static final String ZOOKEEPER_TICK_TIME = ZK_CFG_PROPERTY_PREFIX + "tickTime";
 
   /** Default limit on concurrent client-side zookeeper connections */
   public static final int DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS = 300;
@@ -244,21 +229,19 @@ public final class HConstants {
   public static final int DEFAULT_REGIONSERVER_INFOPORT = 16030;
 
   /** A configuration key for regionserver info port */
-  public static final String REGIONSERVER_INFO_PORT =
-    "hbase.regionserver.info.port";
+  public static final String REGIONSERVER_INFO_PORT = "hbase.regionserver.info.port";
 
   /** A flag that enables automatic selection of regionserver info port */
-  public static final String REGIONSERVER_INFO_PORT_AUTO =
-      REGIONSERVER_INFO_PORT + ".auto";
+  public static final String REGIONSERVER_INFO_PORT_AUTO = REGIONSERVER_INFO_PORT + ".auto";
 
   /** Parameter name for what region server implementation to use. */
-  public static final String REGION_SERVER_IMPL= "hbase.regionserver.impl";
+  public static final String REGION_SERVER_IMPL = "hbase.regionserver.impl";
 
   /** Parameter name for what master implementation to use. */
-  public static final String MASTER_IMPL= "hbase.master.impl";
+  public static final String MASTER_IMPL = "hbase.master.impl";
 
   /** Parameter name for what hbase client implementation to use. */
-  public static final String HBASECLIENT_IMPL= "hbase.hbaseclient.impl";
+  public static final String HBASECLIENT_IMPL = "hbase.hbaseclient.impl";
 
   /** Parameter name for how often threads should wake up */
   public static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency";
@@ -293,7 +276,7 @@ public final class HConstants {
 
   /** Parameter name for HBase client operation timeout, which overrides RPC timeout */
   public static final String HBASE_CLIENT_META_OPERATION_TIMEOUT =
-    "hbase.client.meta.operation.timeout";
+      "hbase.client.meta.operation.timeout";
 
   /** Default HBase client operation timeout, which is tantamount to a blocking call */
   public static final int DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT = 1200000;
@@ -316,9 +299,8 @@ public final class HConstants {
   public static final String MIGRATION_NAME = ".migration";
 
   /**
-   * The directory from which co-processor/custom filter jars can be loaded
-   * dynamically by the region servers. This value can be overridden by the
-   * hbase.dynamic.jars.dir config.
+   * The directory from which co-processor/custom filter jars can be loaded dynamically by the
+   * region servers. This value can be overridden by the hbase.dynamic.jars.dir config.
    */
   public static final String LIB_DIR = "lib";
 
@@ -326,8 +308,7 @@ public final class HConstants {
   public static final String HREGION_COMPACTIONDIR_NAME = "compaction.dir";
 
   /** Conf key for the max file size after which we split the region */
-  public static final String HREGION_MAX_FILESIZE =
-      "hbase.hregion.max.filesize";
+  public static final String HREGION_MAX_FILESIZE = "hbase.hregion.max.filesize";
 
   /** Default maximum file size */
   public static final long DEFAULT_MAX_FILE_SIZE = 10 * 1024 * 1024 * 1024L;
@@ -343,25 +324,24 @@ public final class HConstants {
   public static final long TABLE_MAX_ROWSIZE_DEFAULT = 1024 * 1024 * 1024L;
 
   /**
-   * The max number of threads used for opening and closing stores or store
-   * files in parallel
+   * The max number of threads used for opening and closing stores or store files in parallel
    */
   public static final String HSTORE_OPEN_AND_CLOSE_THREADS_MAX =
-    "hbase.hstore.open.and.close.threads.max";
+      "hbase.hstore.open.and.close.threads.max";
 
   /**
-   * The default number for the max number of threads used for opening and
-   * closing stores or store files in parallel
+   * The default number for the max number of threads used for opening and closing stores or store
+   * files in parallel
    */
   public static final int DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX = 1;
 
   /**
-   * Block updates if memstore has hbase.hregion.memstore.block.multiplier
-   * times hbase.hregion.memstore.flush.size bytes.  Useful preventing
-   * runaway memstore during spikes in update traffic.
+   * Block updates if memstore has hbase.hregion.memstore.block.multiplier times
+   * hbase.hregion.memstore.flush.size bytes. Useful preventing runaway memstore during spikes in
+   * update traffic.
    */
   public static final String HREGION_MEMSTORE_BLOCK_MULTIPLIER =
-          "hbase.hregion.memstore.block.multiplier";
+      "hbase.hregion.memstore.block.multiplier";
 
   /**
    * Default value for hbase.hregion.memstore.block.multiplier
@@ -369,14 +349,12 @@ public final class HConstants {
   public static final int DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER = 4;
 
   /** Conf key for the memstore size at which we flush the memstore */
-  public static final String HREGION_MEMSTORE_FLUSH_SIZE =
-      "hbase.hregion.memstore.flush.size";
+  public static final String HREGION_MEMSTORE_FLUSH_SIZE = "hbase.hregion.memstore.flush.size";
 
   public static final String HREGION_EDITS_REPLAY_SKIP_ERRORS =
       "hbase.hregion.edits.replay.skip.errors";
 
-  public static final boolean DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS =
-      false;
+  public static final boolean DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS = false;
 
   /** Maximum value length, enforced on KeyValue construction */
   public static final int MAXIMUM_VALUE_LENGTH = Integer.MAX_VALUE - 1;
@@ -412,12 +390,12 @@ public final class HConstants {
   // be the first to be reassigned if the server(s) they are being served by
   // should go down.
 
-
   /**
    * The hbase:meta table's name.
    * @deprecated For upgrades of 0.94 to 0.96
    */
-  @Deprecated  // for compat from 0.94 -> 0.96.
+  @Deprecated
+  // for compat from 0.94 -> 0.96.
   public static final byte[] META_TABLE_NAME = TableName.META_TABLE_NAME.getName();
 
   public static final String BASE_NAMESPACE_DIR = "data";
@@ -425,52 +403,52 @@ public final class HConstants {
   /** delimiter used between portions of a region name */
   public static final int META_ROW_DELIMITER = ',';
 
-  /** The catalog family as a string*/
+  /** The catalog family as a string */
   public static final String CATALOG_FAMILY_STR = "info";
 
   /** The catalog family */
-  public static final byte [] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR);
+  public static final byte[] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR);
 
   /** The RegionInfo qualifier as a string */
   public static final String REGIONINFO_QUALIFIER_STR = "regioninfo";
 
   /** The regioninfo column qualifier */
-  public static final byte [] REGIONINFO_QUALIFIER = Bytes.toBytes(REGIONINFO_QUALIFIER_STR);
+  public static final byte[] REGIONINFO_QUALIFIER = Bytes.toBytes(REGIONINFO_QUALIFIER_STR);
 
   /** The server column qualifier */
   public static final String SERVER_QUALIFIER_STR = "server";
   /** The server column qualifier */
-  public static final byte [] SERVER_QUALIFIER = Bytes.toBytes(SERVER_QUALIFIER_STR);
+  public static final byte[] SERVER_QUALIFIER = Bytes.toBytes(SERVER_QUALIFIER_STR);
 
   /** The startcode column qualifier */
   public static final String STARTCODE_QUALIFIER_STR = "serverstartcode";
   /** The startcode column qualifier */
-  public static final byte [] STARTCODE_QUALIFIER = Bytes.toBytes(STARTCODE_QUALIFIER_STR);
+  public static final byte[] STARTCODE_QUALIFIER = Bytes.toBytes(STARTCODE_QUALIFIER_STR);
 
   /** The open seqnum column qualifier */
   public static final String SEQNUM_QUALIFIER_STR = "seqnumDuringOpen";
   /** The open seqnum column qualifier */
-  public static final byte [] SEQNUM_QUALIFIER = Bytes.toBytes(SEQNUM_QUALIFIER_STR);
+  public static final byte[] SEQNUM_QUALIFIER = Bytes.toBytes(SEQNUM_QUALIFIER_STR);
 
   /** The state column qualifier */
   public static final String STATE_QUALIFIER_STR = "state";
 
-  public static final byte [] STATE_QUALIFIER = Bytes.toBytes(STATE_QUALIFIER_STR);
+  public static final byte[] STATE_QUALIFIER = Bytes.toBytes(STATE_QUALIFIER_STR);
 
   /**
-   * The serverName column qualifier. Its the server where the region is
-   * transitioning on, while column server is the server where the region is
-   * opened on. They are the same when the region is in state OPEN.
+   * The serverName column qualifier. Its the server where the region is transitioning on, while
+   * column server is the server where the region is opened on. They are the same when the region is
+   * in state OPEN.
    */
   public static final String SERVERNAME_QUALIFIER_STR = "sn";
 
-  public static final byte [] SERVERNAME_QUALIFIER = Bytes.toBytes(SERVERNAME_QUALIFIER_STR);
+  public static final byte[] SERVERNAME_QUALIFIER = Bytes.toBytes(SERVERNAME_QUALIFIER_STR);
 
   /** The lower-half split region column qualifier */
-  public static final byte [] SPLITA_QUALIFIER = Bytes.toBytes("splitA");
+  public static final byte[] SPLITA_QUALIFIER = Bytes.toBytes("splitA");
 
   /** The upper-half split region column qualifier */
-  public static final byte [] SPLITB_QUALIFIER = Bytes.toBytes("splitB");
+  public static final byte[] SPLITB_QUALIFIER = Bytes.toBytes("splitB");
 
   /** The lower-half merge region column qualifier */
   public static final byte[] MERGEA_QUALIFIER = Bytes.toBytes("mergeA");
@@ -478,32 +456,28 @@ public final class HConstants {
   /** The upper-half merge region column qualifier */
   public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes("mergeB");
 
-  /** The catalog family as a string*/
+  /** The catalog family as a string */
   public static final String TABLE_FAMILY_STR = "table";
 
   /** The catalog family */
-  public static final byte [] TABLE_FAMILY = Bytes.toBytes(TABLE_FAMILY_STR);
+  public static final byte[] TABLE_FAMILY = Bytes.toBytes(TABLE_FAMILY_STR);
 
   /** The serialized table state qualifier */
   public static final byte[] TABLE_STATE_QUALIFIER = Bytes.toBytes("state");
 
-
   /**
-   * The meta table version column qualifier.
-   * We keep current version of the meta table in this column in <code>-ROOT-</code>
-   * table: i.e. in the 'info:v' column.
+   * The meta table version column qualifier. We keep current version of the meta table in this
+   * column in <code>-ROOT-</code> table: i.e. in the 'info:v' column.
    */
-  public static final byte [] META_VERSION_QUALIFIER = Bytes.toBytes("v");
+  public static final byte[] META_VERSION_QUALIFIER = Bytes.toBytes("v");
 
   /**
-   * The current version of the meta table.
-   * - pre-hbase 0.92.  There is no META_VERSION column in the root table
-   * in this case. The meta has HTableDescriptor serialized into the HRegionInfo;
-   * - version 0 is 0.92 and 0.94. Meta data has serialized HRegionInfo's using
-   * Writable serialization, and HRegionInfo's does not contain HTableDescriptors.
-   * - version 1 for 0.96+ keeps HRegionInfo data structures, but changes the
-   * byte[] serialization from Writables to Protobuf.
-   * See HRegionInfo.VERSION
+   * The current version of the meta table. - pre-hbase 0.92. There is no META_VERSION column in the
+   * root table in this case. The meta has HTableDescriptor serialized into the HRegionInfo; -
+   * version 0 is 0.92 and 0.94. Meta data has serialized HRegionInfo's using Writable
+   * serialization, and HRegionInfo's does not contain HTableDescriptors. - version 1 for 0.96+
+   * keeps HRegionInfo data structures, but changes the byte[] serialization from Writables to
+   * Protobuf. See HRegionInfo.VERSION
    */
   public static final short META_VERSION = 1;
 
@@ -512,25 +486,24 @@ public final class HConstants {
   /**
    * An empty instance.
    */
-  public static final byte [] EMPTY_BYTE_ARRAY = new byte [0];
+  public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
 
   public static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.wrap(EMPTY_BYTE_ARRAY);
 
   /**
    * Used by scanners, etc when they want to start at the beginning of a region
    */
-  public static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY;
+  public static final byte[] EMPTY_START_ROW = EMPTY_BYTE_ARRAY;
 
   /**
    * Last row in a table.
    */
-  public static final byte [] EMPTY_END_ROW = EMPTY_START_ROW;
+  public static final byte[] EMPTY_END_ROW = EMPTY_START_ROW;
 
   /**
-    * Used by scanners and others when they're trying to detect the end of a
-    * table
-    */
-  public static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY;
+   * Used by scanners and others when they're trying to detect the end of a table
+   */
+  public static final byte[] LAST_ROW = EMPTY_BYTE_ARRAY;
 
   /**
    * Max length a row can have because of the limitation in TFile.
@@ -538,9 +511,8 @@ public final class HConstants {
   public static final int MAX_ROW_LENGTH = Short.MAX_VALUE;
 
   /**
-   * Timestamp to use when we want to refer to the latest cell.
-   * This is the timestamp sent by clients when no timestamp is specified on
-   * commit.
+   * Timestamp to use when we want to refer to the latest cell. This is the timestamp sent by
+   * clients when no timestamp is specified on commit.
    */
   public static final long LATEST_TIMESTAMP = Long.MAX_VALUE;
 
@@ -552,17 +524,12 @@ public final class HConstants {
   /**
    * LATEST_TIMESTAMP in bytes form
    */
-  public static final byte [] LATEST_TIMESTAMP_BYTES = {
+  public static final byte[] LATEST_TIMESTAMP_BYTES = {
     // big-endian
-    (byte) (LATEST_TIMESTAMP >>> 56),
-    (byte) (LATEST_TIMESTAMP >>> 48),
-    (byte) (LATEST_TIMESTAMP >>> 40),
-    (byte) (LATEST_TIMESTAMP >>> 32),
-    (byte) (LATEST_TIMESTAMP >>> 24),
-    (byte) (LATEST_TIMESTAMP >>> 16),
-    (byte) (LATEST_TIMESTAMP >>> 8),
-    (byte) LATEST_TIMESTAMP,
-  };
+    (byte) (LATEST_TIMESTAMP >>> 56), (byte) (LATEST_TIMESTAMP >>> 48),
+    (byte) (LATEST_TIMESTAMP >>> 40), (byte) (LATEST_TIMESTAMP >>> 32),
+    (byte) (LATEST_TIMESTAMP >>> 24), (byte) (LATEST_TIMESTAMP >>> 16),
+    (byte) (LATEST_TIMESTAMP >>> 8), (byte) LATEST_TIMESTAMP };
 
   /**
    * Define for 'return-all-versions'.
@@ -572,7 +539,7 @@ public final class HConstants {
   /**
    * Unlimited time-to-live.
    */
-//  public static final int FOREVER = -1;
+  // public static final int FOREVER = -1;
   public static final int FOREVER = Integer.MAX_VALUE;
 
   /**
@@ -587,10 +554,10 @@ public final class HConstants {
   public static final int HOUR_IN_SECONDS = 60 * 60;
   public static final int MINUTE_IN_SECONDS = 60;
 
-  //TODO: although the following are referenced widely to format strings for
-  //      the shell. They really aren't a part of the public API. It would be
-  //      nice if we could put them somewhere where they did not need to be
-  //      public. They could have package visibility
+  // TODO: although the following are referenced widely to format strings for
+  // the shell. They really aren't a part of the public API. It would be
+  // nice if we could put them somewhere where they did not need to be
+  // public. They could have package visibility
   public static final String NAME = "NAME";
   public static final String VERSIONS = "VERSIONS";
   public static final String IN_MEMORY = "IN_MEMORY";
@@ -598,45 +565,38 @@ public final class HConstants {
   public static final String CONFIGURATION = "CONFIGURATION";
 
   /**
-   * Retrying we multiply hbase.client.pause setting by what we have in this array until we
-   * run out of array items.  Retries beyond this use the last number in the array.  So, for
-   * example, if hbase.client.pause is 1 second, and maximum retries count
-   * hbase.client.retries.number is 10, we will retry at the following intervals:
-   * 1, 2, 3, 5, 10, 20, 40, 100, 100, 100.
-   * With 100ms, a back-off of 200 means 20s
+   * Retrying we multiply hbase.client.pause setting by what we have in this array until we run out
+   * of array items. Retries beyond this use the last number in the array. So, for example, if
+   * hbase.client.pause is 1 second, and maximum retries count hbase.client.retries.number is 10, we
+   * will retry at the following intervals: 1, 2, 3, 5, 10, 20, 40, 100, 100, 100. With 100ms, a
+   * back-off of 200 means 20s
    */
-  public static final int [] RETRY_BACKOFF = {1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200};
+  public static final int[] RETRY_BACKOFF =
+    { 1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200 };
 
   public static final String REGION_IMPL = "hbase.hregion.impl";
 
   /** modifyTable op for replacing the table descriptor */
   @InterfaceAudience.Private
   public static enum Modify {
-    CLOSE_REGION,
-    TABLE_COMPACT,
-    TABLE_FLUSH,
-    TABLE_MAJOR_COMPACT,
-    TABLE_SET_HTD,
-    TABLE_SPLIT
+    CLOSE_REGION, TABLE_COMPACT, TABLE_FLUSH, TABLE_MAJOR_COMPACT, TABLE_SET_HTD, TABLE_SPLIT
   }
 
   /**
-   * Scope tag for locally scoped data.
-   * This data will not be replicated.
+   * Scope tag for locally scoped data. This data will not be replicated.
    */
   public static final int REPLICATION_SCOPE_LOCAL = 0;
 
   /**
-   * Scope tag for globally scoped data.
-   * This data will be replicated to all peers.
+   * Scope tag for globally scoped data. This data will be replicated to all peers.
    */
   public static final int REPLICATION_SCOPE_GLOBAL = 1;
 
   /**
-   * Default cluster ID, cannot be used to identify a cluster so a key with
-   * this value means it wasn't meant for replication.
+   * Default cluster ID, cannot be used to identify a cluster so a key with this value means it
+   * wasn't meant for replication.
    */
-  public static final UUID DEFAULT_CLUSTER_ID = new UUID(0L,0L);
+  public static final UUID DEFAULT_CLUSTER_ID = new UUID(0L, 0L);
 
   /**
    * Parameter name for maximum number of bytes returned when calling a scanner's next method.
@@ -653,27 +613,22 @@ public final class HConstants {
       "hbase.server.scanner.max.result.size";
 
   /**
-   * Maximum number of bytes returned when calling a scanner's next method.
-   * Note that when a single row is larger than this limit the row is still
-   * returned completely.
-   *
-   * The default value is 2MB.
+   * Maximum number of bytes returned when calling a scanner's next method. Note that when a single
+   * row is larger than this limit the row is still returned completely. The default value is 2MB.
    */
   public static final long DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE = 2 * 1024 * 1024;
 
   /**
-   * Maximum number of bytes returned when calling a scanner's next method.
-   * Note that when a single row is larger than this limit the row is still
-   * returned completely.
-   * Safety setting to protect the region server.
-   *
-   * The default value is 100MB. (a client would rarely request larger chunks on purpose)
+   * Maximum number of bytes returned when calling a scanner's next method. Note that when a single
+   * row is larger than this limit the row is still returned completely. Safety setting to protect
+   * the region server. The default value is 100MB. (a client would rarely request larger chunks on
+   * purpose)
    */
   public static final long DEFAULT_HBASE_SERVER_SCANNER_MAX_RESULT_SIZE = 100 * 1024 * 1024;
 
   /**
-   * Parameter name for client pause value, used mostly as value to wait
-   * before running a retry of a failed get, region lookup, etc.
+   * Parameter name for client pause value, used mostly as value to wait before running a retry of a
+   * failed get, region lookup, etc.
    */
   public static final String HBASE_CLIENT_PAUSE = "hbase.client.pause";
 
@@ -693,8 +648,7 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS = 100;
 
   /**
-   * The maximum number of concurrent connections the client will maintain to a single
-   * RegionServer.
+   * The maximum number of concurrent connections the client will maintain to a single RegionServer.
    */
   public static final String HBASE_CLIENT_MAX_PERSERVER_TASKS = "hbase.client.max.perserver.tasks";
 
@@ -704,8 +658,7 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS = 2;
 
   /**
-   * The maximum number of concurrent connections the client will maintain to a single
-   * Region.
+   * The maximum number of concurrent connections the client will maintain to a single Region.
    */
   public static final String HBASE_CLIENT_MAX_PERREGION_TASKS = "hbase.client.max.perregion.tasks";
 
@@ -715,8 +668,8 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS = 1;
 
   /**
-   * Parameter name for server pause value, used mostly as value to wait before
-   * running a retry of a failed operation.
+   * Parameter name for server pause value, used mostly as value to wait before running a retry of a
+   * failed operation.
    */
   public static final String HBASE_SERVER_PAUSE = "hbase.server.pause";
 
@@ -726,9 +679,9 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_SERVER_PAUSE = 1000;
 
   /**
-   * Parameter name for maximum retries, used as maximum for all retryable
-   * operations such as fetching of the root region from root region server,
-   * getting a cell's value, starting a row update, etc.
+   * Parameter name for maximum retries, used as maximum for all retryable operations such as
+   * fetching of the root region from root region server, getting a cell's value, starting a row
+   * update, etc.
    */
   public static final String HBASE_CLIENT_RETRIES_NUMBER = "hbase.client.retries.number";
 
@@ -748,10 +701,9 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_CLIENT_SCANNER_CACHING = Integer.MAX_VALUE;
 
   /**
-   * Parameter name for number of rows that will be fetched when calling next on
-   * a scanner if it is not served from memory. Higher caching values will
-   * enable faster scanners but will eat up more memory and some calls of next
-   * may take longer and longer times when the cache is empty.
+   * Parameter name for number of rows that will be fetched when calling next on a scanner if it is
+   * not served from memory. Higher caching values will enable faster scanners but will eat up more
+   * memory and some calls of next may take longer and longer times when the cache is empty.
    */
   public static final String HBASE_META_SCANNER_CACHING = "hbase.meta.scanner.caching";
 
@@ -918,18 +870,13 @@ public final class HConstants {
   public static final String LOCALHOST = "localhost";
 
   /**
-   * If this parameter is set to true, then hbase will read
-   * data and then verify checksums. Checksum verification
-   * inside hdfs will be switched off.  However, if the hbase-checksum
-   * verification fails, then it will switch back to using
-   * hdfs checksums for verifiying data that is being read from storage.
-   *
-   * If this parameter is set to false, then hbase will not
-   * verify any checksums, instead it will depend on checksum verification
-   * being done in the hdfs client.
+   * If this parameter is set to true, then hbase will read data and then verify checksums. Checksum
+   * verification inside hdfs will be switched off. However, if the hbase-checksum verification
+   * fails, then it will switch back to using hdfs checksums for verifiying data that is being read
+   * from storage. If this parameter is set to false, then hbase will not verify any checksums,
+   * instead it will depend on checksum verification being done in the hdfs client.
    */
-  public static final String HBASE_CHECKSUM_VERIFICATION =
-      "hbase.regionserver.checksum.verify";
+  public static final String HBASE_CHECKSUM_VERIFICATION = "hbase.regionserver.checksum.verify";
 
   public static final String LOCALHOST_IP = "127.0.0.1";
 
@@ -944,17 +891,15 @@ public final class HConstants {
   public static final int DEFAULT_REGION_SERVER_HANDLER_COUNT = 30;
 
   /*
-   * REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT:
-   * -1  => Disable aborting
-   * 0   => Abort if even a single handler has died
-   * 0.x => Abort only when this percent of handlers have died
-   * 1   => Abort only all of the handers have died
+   * REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT: -1 => Disable aborting 0 => Abort if even a
+   * single handler has died 0.x => Abort only when this percent of handlers have died 1 => Abort
+   * only all of the handers have died
    */
   public static final String REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT =
       "hbase.regionserver.handler.abort.on.error.percent";
   public static final double DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT = 0.5;
 
-  //High priority handlers to deal with admin requests and system table operation requests
+  // High priority handlers to deal with admin requests and system table operation requests
   public static final String REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT =
       "hbase.regionserver.metahandler.count";
   public static final int DEFAULT_REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT = 20;
@@ -977,19 +922,16 @@ public final class HConstants {
   public static final int DEFAULT_META_REPLICA_NUM = 1;
 
   /**
-   * The name of the configuration parameter that specifies
-   * the number of bytes in a newly created checksum chunk.
+   * The name of the configuration parameter that specifies the number of bytes in a newly created
+   * checksum chunk.
    */
-  public static final String BYTES_PER_CHECKSUM =
-      "hbase.hstore.bytes.per.checksum";
+  public static final String BYTES_PER_CHECKSUM = "hbase.hstore.bytes.per.checksum";
 
   /**
-   * The name of the configuration parameter that specifies
-   * the name of an algorithm that is used to compute checksums
-   * for newly created blocks.
+   * The name of the configuration parameter that specifies the name of an algorithm that is used to
+   * compute checksums for newly created blocks.
    */
-  public static final String CHECKSUM_TYPE_NAME =
-      "hbase.hstore.checksum.algorithm";
+  public static final String CHECKSUM_TYPE_NAME = "hbase.hstore.checksum.algorithm";
 
   /** Enable file permission modification from standard hbase */
   public static final String ENABLE_DATA_FILE_UMASK = "hbase.data.umask.enable";
@@ -997,16 +939,14 @@ public final class HConstants {
   public static final String DATA_FILE_UMASK_KEY = "hbase.data.umask";
 
   /** Configuration name of WAL Compression */
-  public static final String ENABLE_WAL_COMPRESSION =
-    "hbase.regionserver.wal.enablecompression";
-
-  /** Configuration name of WAL storage policy
-   * Valid values are:
-   *  NONE: no preference in destination of block replicas
-   *  ONE_SSD: place only one block replica in SSD and the remaining in default storage
-   *  and ALL_SSD: place all block replicas on SSD
-   *
-   * See http://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html*/
+  public static final String ENABLE_WAL_COMPRESSION = "hbase.regionserver.wal.enablecompression";
+
+  /**
+   * Configuration name of WAL storage policy Valid values are: NONE: no preference in destination
+   * of block replicas ONE_SSD: place only one block replica in SSD and the remaining in default
+   * storage and ALL_SSD: place all block replicas on SSD See
+   * http://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html
+   */
   public static final String WAL_STORAGE_POLICY = "hbase.wal.storage.policy";
   public static final String DEFAULT_WAL_STORAGE_POLICY = "NONE";
 
@@ -1017,17 +957,16 @@ public final class HConstants {
   public static final String LOAD_BALANCER_SLOP_KEY = "hbase.regions.slop";
 
   /**
-   * The byte array represents for NO_NEXT_INDEXED_KEY;
-   * The actual value is irrelevant because this is always compared by reference.
+   * The byte array represents for NO_NEXT_INDEXED_KEY; The actual value is irrelevant because this
+   * is always compared by reference.
    */
   public static final Cell NO_NEXT_INDEXED_KEY = new KeyValue();
   /** delimiter used between portions of a region name */
   public static final int DELIMITER = ',';
 
   /**
-   * QOS attributes: these attributes are used to demarcate RPC call processing
-   * by different set of handlers. For example, HIGH_QOS tagged methods are
-   * handled by high priority handlers.
+   * QOS attributes: these attributes are used to demarcate RPC call processing by different set of
+   * handlers. For example, HIGH_QOS tagged methods are handled by high priority handlers.
    */
   // normal_QOS < QOS_threshold < replication_QOS < replay_QOS < admin_QOS < high_QOS
   public static final int NORMAL_QOS = 0;
@@ -1042,8 +981,8 @@ public final class HConstants {
   public static final String HFILE_ARCHIVE_DIRECTORY = "archive";
 
   /**
-   * Name of the directory to store all snapshots. See SnapshotDescriptionUtils for
-   * remaining snapshot constants; this is here to keep HConstants dependencies at a minimum and
+   * Name of the directory to store all snapshots. See SnapshotDescriptionUtils for remaining
+   * snapshot constants; this is here to keep HConstants dependencies at a minimum and
    * uni-directional.
    */
   public static final String SNAPSHOT_DIR_NAME = ".hbase-snapshot";
@@ -1059,49 +998,42 @@ public final class HConstants {
   public static final String REGIONSERVER_METRICS_PERIOD = "hbase.regionserver.metrics.period";
   public static final long DEFAULT_REGIONSERVER_METRICS_PERIOD = 5000;
   /** Directories that are not HBase table directories */
-  public static final List<String> HBASE_NON_TABLE_DIRS =
-    Collections.unmodifiableList(Arrays.asList(new String[] {
-      HBCK_SIDELINEDIR_NAME, HBASE_TEMP_DIRECTORY, MIGRATION_NAME
-    }));
+  public static final List<String> HBASE_NON_TABLE_DIRS = Collections.unmodifiableList(Arrays
+      .asList(new String[] { HBCK_SIDELINEDIR_NAME, HBASE_TEMP_DIRECTORY, MIGRATION_NAME }));
 
   /** Directories that are not HBase user table directories */
-  public static final List<String> HBASE_NON_USER_TABLE_DIRS =
-    Collections.unmodifiableList(Arrays.asList((String[])ArrayUtils.addAll(
-      new String[] { TableName.META_TABLE_NAME.getNameAsString() },
-      HBASE_NON_TABLE_DIRS.toArray())));
+  public static final List<String> HBASE_NON_USER_TABLE_DIRS = Collections.unmodifiableList(Arrays
+      .asList((String[]) ArrayUtils.addAll(
+        new String[] { TableName.META_TABLE_NAME.getNameAsString() },
+        HBASE_NON_TABLE_DIRS.toArray())));
 
   /** Health script related settings. */
   public static final String HEALTH_SCRIPT_LOC = "hbase.node.health.script.location";
   public static final String HEALTH_SCRIPT_TIMEOUT = "hbase.node.health.script.timeout";
-  public static final String HEALTH_CHORE_WAKE_FREQ =
-      "hbase.node.health.script.frequency";
+  public static final String HEALTH_CHORE_WAKE_FREQ = "hbase.node.health.script.frequency";
   public static final long DEFAULT_HEALTH_SCRIPT_TIMEOUT = 60000;
   /**
    * The maximum number of health check failures a server can encounter consecutively.
    */
-  public static final String HEALTH_FAILURE_THRESHOLD =
-      "hbase.node.health.failure.threshold";
+  public static final String HEALTH_FAILURE_THRESHOLD = "hbase.node.health.failure.threshold";
   public static final int DEFAULT_HEALTH_FAILURE_THRESHOLD = 3;
 
-
   /**
-   * Setting to activate, or not, the publication of the status by the master. Default
-   *  notification is by a multicast message.
+   * Setting to activate, or not, the publication of the status by the master. Default notification
+   * is by a multicast message.
    */
   public static final String STATUS_PUBLISHED = "hbase.status.published";
   public static final boolean STATUS_PUBLISHED_DEFAULT = false;
 
   /**
-   * IP to use for the multicast status messages between the master and the clients.
-   * The default address is chosen as one among others within the ones suitable for multicast
-   * messages.
+   * IP to use for the multicast status messages between the master and the clients. The default
+   * address is chosen as one among others within the ones suitable for multicast messages.
    */
   public static final String STATUS_MULTICAST_ADDRESS = "hbase.status.multicast.address.ip";
   public static final String DEFAULT_STATUS_MULTICAST_ADDRESS = "226.1.1.3";
 
   /**
-   * The address to use for binding the local socket for receiving multicast. Defaults to
-   * 0.0.0.0.
+   * The address to use for binding the local socket for receiving multicast. Defaults to 0.0.0.0.
    * @see <a href="https://issues.apache.org/jira/browse/HBASE-9961">HBASE-9961</a>
    */
   public static final String STATUS_MULTICAST_BIND_ADDRESS =
@@ -1134,7 +1066,7 @@ public final class HConstants {
 
   /** Configuration key for the name of the alternate master key for the cluster, a string */
   public static final String CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY =
-    "hbase.crypto.master.alternate.key.name";
+      "hbase.crypto.master.alternate.key.name";
 
   /** Configuration key for the algorithm to use when encrypting the WAL, a string */
   public static final String CRYPTO_WAL_ALGORITHM_CONF_KEY = "hbase.crypto.wal.algorithm";
@@ -1166,7 +1098,7 @@ public final class HConstants {
 
   /** Config for pluggable consensus provider */
   public static final String HBASE_COORDINATED_STATE_MANAGER_CLASS =
-    "hbase.coordinated.state.manager.class";
+      "hbase.coordinated.state.manager.class";
 
   /** Configuration key for SplitLog manager timeout */
   public static final String HBASE_SPLITLOG_MANAGER_TIMEOUT = "hbase.splitlog.manager.timeout";
@@ -1180,17 +1112,18 @@ public final class HConstants {
   // hbase-common?
 
   /**
-   * Current ioengine options in include: heap, offheap and file:PATH (where PATH is the path
-   * to the file that will host the file-based cache.  See BucketCache#getIOEngineFromName() for
-   * list of supported ioengine options.
-   * <p>Set this option and a non-zero {@link #BUCKET_CACHE_SIZE_KEY} to enable bucket cache.
+   * Current ioengine options in include: heap, offheap and file:PATH (where PATH is the path to the
+   * file that will host the file-based cache. See BucketCache#getIOEngineFromName() for list of
+   * supported ioengine options.
+   * <p>
+   * Set this option and a non-zero {@link #BUCKET_CACHE_SIZE_KEY} to enable bucket cache.
    */
   public static final String BUCKET_CACHE_IOENGINE_KEY = "hbase.bucketcache.ioengine";
 
   /**
    * When using bucket cache, this is a float that EITHER represents a percentage of total heap
-   * memory size to give to the cache (if &lt; 1.0) OR, it is the capacity in
-   * megabytes of the cache.
+   * memory size to give to the cache (if &lt; 1.0) OR, it is the capacity in megabytes of the
+   * cache.
    */
   public static final String BUCKET_CACHE_SIZE_KEY = "hbase.bucketcache.size";
 
@@ -1203,26 +1136,25 @@ public final class HConstants {
   public static final String HBASE_CLIENT_FAST_FAIL_MODE_ENABLED =
       "hbase.client.fast.fail.mode.enabled";
 
-  public static final boolean HBASE_CLIENT_ENABLE_FAST_FAIL_MODE_DEFAULT =
-      false;
+  public static final boolean HBASE_CLIENT_ENABLE_FAST_FAIL_MODE_DEFAULT = false;
 
   public static final String HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS =
       "hbase.client.fastfail.threshold";
 
-  public static final long HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS_DEFAULT =
-      60000;
+  public static final long HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS_DEFAULT = 60000;
 
   public static final String HBASE_CLIENT_FAST_FAIL_CLEANUP_MS_DURATION_MS =
       "hbase.client.fast.fail.cleanup.duration";
 
-  public static final long HBASE_CLIENT_FAST_FAIL_CLEANUP_DURATION_MS_DEFAULT =
-      600000;
+  public static final long HBASE_CLIENT_FAST_FAIL_CLEANUP_DURATION_MS_DEFAULT = 600000;
 
   public static final String HBASE_CLIENT_FAST_FAIL_INTERCEPTOR_IMPL =
       "hbase.client.fast.fail.interceptor.impl";
 
-  /** Config key for if the server should send backpressure and if the client should listen to
-   * that backpressure from the server */
+  /**
+   * Config key for if the server should send backpressure and if the client should listen to that
+   * backpressure from the server
+   */
   public static final String ENABLE_CLIENT_BACKPRESSURE = "hbase.client.backpressure.enabled";
   public static final boolean DEFAULT_ENABLE_CLIENT_BACKPRESSURE = false;
 
@@ -1234,11 +1166,11 @@ public final class HConstants {
   public static final float DEFAULT_HEAP_OCCUPANCY_HIGH_WATERMARK = 0.98f;
 
   /**
-   * The max number of threads used for splitting storefiles in parallel during
-   * the region split process.
+   * The max number of threads used for splitting storefiles in parallel during the region split
+   * process.
    */
   public static final String REGION_SPLIT_THREADS_MAX =
-    "hbase.regionserver.region.split.threads.max";
+      "hbase.regionserver.region.split.threads.max";
 
   /** Canary config keys */
   public static final String HBASE_CANARY_WRITE_DATA_TTL_KEY = "hbase.canary.write.data.ttl";
@@ -1263,6 +1195,15 @@ public final class HConstants {
   public static final String ZK_SERVER_KEYTAB_FILE = "hbase.zookeeper.server.keytab.file";
   public static final String ZK_SERVER_KERBEROS_PRINCIPAL =
       "hbase.zookeeper.server.kerberos.principal";
+  /**
+   * Backup/Restore constants
+   */
+
+  public final static String BACKUP_ENABLE_KEY = "hbase.backup.enable";
+  public final static boolean BACKUP_ENABLE_DEFAULT = true;
+  public final static String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl";
+  // Default TTL = 1 year
+  public final static int BACKUP_SYSTEM_TTL_DEFAULT = 365 * 24 * 3600;
 
   private HConstants() {
     // Can't be instantiated with this ctor.

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 26aad71..807b021 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -394,6 +394,11 @@
        <version>${project.version}</version>
        <optional>true</optional>
     </dependency>
+     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <version>${hadoop-two.version}</version>
+    </dependency>
     <dependency>
       <groupId>commons-httpclient</groupId>
       <artifactId>commons-httpclient</artifactId>
@@ -407,6 +412,11 @@
       <artifactId>commons-collections</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <version>${hadoop-two.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
new file mode 100644
index 0000000..28cf627
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
@@ -0,0 +1,338 @@
+/**
+ * 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.Set;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+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;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_COMMAND;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Backup HBase tables locally or on a remote cluster Serve as client entry point for the following
+ * features: - Full Backup provide local and remote back/restore for a list of tables - Incremental
+ * backup to build on top of full backup as daily/weekly backup - Convert incremental backup WAL
+ * files into hfiles - Merge several backup images into one(like merge weekly into monthly) - Add
+ * and remove table to and from Backup image - Cancel a backup process - Full backup based on
+ * existing snapshot - Describe information of a backup image
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class BackupClient {
+  private static final Log LOG = LogFactory.getLog(BackupClient.class);
+  private static Options opt;
+  private static Configuration conf = null;
+
+  private BackupClient() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  protected static void init() throws IOException {
+    // define supported options
+    opt = new Options();
+
+    opt.addOption("all", false, "All tables");
+    opt.addOption("debug", false, "Enable debug loggings");
+    opt.addOption("t", true, "Table name");
+
+    // create configuration instance
+    conf = getConf();
+
+    // disable irrelevant loggers to avoid it mess up command output
+    disableUselessLoggers();
+
+  }
+
+
+  public static void main(String[] args) throws IOException {
+    init();
+    parseAndRun(args);
+    System.exit(0);
+  }
+
+  /**
+   * Set the configuration from a given one.
+   * @param newConf A new given configuration
+   */
+  public synchronized static void setConf(Configuration newConf) {
+    conf = newConf;
+    BackupUtil.setConf(newConf);
+  }
+
+  public static Configuration getConf() {
+    if (conf == null) {
+      conf = BackupUtil.getConf();
+    }
+    return conf;
+  }
+
+  private static void disableUselessLoggers() {
+    // disable zookeeper log to avoid it mess up command output
+    Logger zkLogger = Logger.getLogger("org.apache.zookeeper");
+    LOG.debug("Zookeeper log level before set: " + zkLogger.getLevel());
+    zkLogger.setLevel(Level.OFF);
+    LOG.debug("Zookeeper log level after set: " + zkLogger.getLevel());
+
+    // disable hbase zookeeper tool log to avoid it mess up command output
+    Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper");
+    LOG.debug("HBase zookeeper log level before set: " + hbaseZkLogger.getLevel());
+    hbaseZkLogger.setLevel(Level.OFF);
+    LOG.debug("HBase Zookeeper log level after set: " + hbaseZkLogger.getLevel());
+
+    // disable hbase client log to avoid it mess up command output
+    Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client");
+    LOG.debug("HBase client log level before set: " + hbaseClientLogger.getLevel());
+    hbaseClientLogger.setLevel(Level.OFF);
+    LOG.debug("HBase client log level after set: " + hbaseClientLogger.getLevel());
+  }
+
+  private static void parseAndRun(String[] args) throws IOException {
+
+    String cmd = null;
+    String[] remainArgs = null;
+    if (args == null || args.length == 0) {
+      BackupCommands.createCommand(BackupRestoreConstants.BACKUP_COMMAND.HELP, null).execute();
+    } else {
+      cmd = args[0];
+      remainArgs = new String[args.length - 1];
+      if (args.length > 1) {
+        System.arraycopy(args, 1, remainArgs, 0, args.length - 1);
+      }
+    }
+    CommandLine cmdline = null;
+    try {
+      cmdline = new PosixParser().parse(opt, remainArgs);
+    } catch (ParseException e) {
+      LOG.error("Could not parse command", e);
+      System.exit(-1);
+    }
+
+    BACKUP_COMMAND type = BACKUP_COMMAND.HELP;
+    if (BACKUP_COMMAND.CREATE.name().equalsIgnoreCase(cmd)) {
+      type = BACKUP_COMMAND.CREATE;
+    } else if (BACKUP_COMMAND.HELP.name().equalsIgnoreCase(cmd)) {
+      type = BACKUP_COMMAND.HELP;
+    } else {
+      System.out.println("Unsupported command for backup: " + cmd);
+    }
+
+    // enable debug logging
+    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+    if (cmdline.hasOption("debug")) {
+      backupClientLogger.setLevel(Level.DEBUG);
+    } else {
+      backupClientLogger.setLevel(Level.INFO);
+    }
+
+    BackupCommands.createCommand(type, cmdline).execute();
+  }
+
+  /**
+   * Send backup request to server, and monitor the progress if necessary
+   * @param backupType : full or incremental
+   * @param backupRootPath : the rooPath specified by user
+   * @param tableListStr : the table list specified by user
+   * @param snapshot : using existing snapshot if specified by user (in future jira)
+   * @return backupId backup id
+   * @throws IOException exception
+   * @throws KeeperException excpetion
+   */
+  public static String create(String backupType, String backupRootPath, String tableListStr,
+      String snapshot) throws IOException {
+
+    String backupId = BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime();
+
+    // check target path first, confirm it doesn't exist before backup
+    boolean isTargetExist = false;
+    try {
+      isTargetExist = HBackupFileSystem.checkPathExist(backupRootPath, conf);
+    } catch (IOException e) {
+      String expMsg = e.getMessage();
+      String newMsg = null;
+      if (expMsg.contains("No FileSystem for scheme")) {
+        newMsg =
+            "Unsupported filesystem scheme found in the backup target url. Error Message: "
+                + newMsg;
+        LOG.error(newMsg);
+        throw new IOException(newMsg);
+      } else {
+        throw e;
+      }
+    } catch (RuntimeException e) {
+      LOG.error(e.getMessage());
+      throw e;
+    }
+
+    if (isTargetExist) {
+      LOG.info("Using existing backup root dir: " + backupRootPath);
+    } else {
+      LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created.");
+    }
+
+    // table list specified for backup, trigger backup on specified tables
+    String tableList = tableListStr;
+    // (tableListStr == null) ? null : tableListStr.replaceAll(
+    // BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND,
+    // BackupRestoreConstants.TABLENAME_DELIMITER_IN_ZNODE);
+    try {
+      requestBackup(backupId, backupType, tableList, backupRootPath, snapshot);
+    } catch (RuntimeException e) {
+      String errMsg = e.getMessage();
+      if (errMsg != null
+          && (errMsg.startsWith("Non-existing tables found") || errMsg
+              .startsWith("Snapshot is not found"))) {
+        LOG.error(errMsg + ", please check your command");
+        throw e;
+      } else {
+        throw e;
+      }
+    }
+    return backupId;
+  }
+
+  /**
+   * Prepare and submit Backup request
+   * @param backupId : backup_timestame (something like backup_1398729212626)
+   * @param backupType : full or incremental
+   * @param tableList : tables to be backuped
+   * @param targetRootDir : specified by user
+   * @param snapshot : use existing snapshot if specified by user (for future jira)
+   * @throws IOException exception
+   */
+  protected static void requestBackup(String backupId, String backupType, String tableList,
+      String targetRootDir, String snapshot) throws IOException {
+
+    Configuration conf = getConf();
+    BackupManager backupManager = null;
+    BackupContext backupContext = null;
+    if (snapshot != null) {
+      LOG.warn("Snapshot option specified, backup type and table option will be ignored,\n"
+          + "full backup will be taken based on the given snapshot.");
+      throw new IOException("backup using existing Snapshot will be implemented in future jira");
+    }
+
+    HBaseAdmin hbadmin = null;
+    Connection conn = null;
+    try {
+
+      backupManager = new BackupManager(conf);
+      String tables = tableList;
+      if (backupType.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+        Set<String> incrTableSet = backupManager.getIncrementalBackupTableSet();
+        if (incrTableSet.isEmpty()) {
+          LOG.warn("Incremental backup table set contains no table.\n"
+              + "Use 'backup create full' or 'backup stop' to \n "
+              + "change the tables covered by incremental backup.");
+          throw new RuntimeException("No table covered by incremental backup.");
+        }
+        StringBuilder sb = new StringBuilder();
+        for (String tableName : incrTableSet) {
+          sb.append(tableName + " ");
+        }
+        LOG.info("Incremental backup for the following table set: " + sb.toString());
+        tables =
+            sb.toString().trim()
+            .replaceAll(" ", BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+      }
+
+      // check whether table exists first before starting real request
+      if (tables != null) {
+        String[] tableNames = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+        ArrayList<String> noneExistingTableList = null;
+        conn = ConnectionFactory.createConnection(conf);
+        hbadmin = (HBaseAdmin) conn.getAdmin();
+        for (String tableName : tableNames) {
+          if (!hbadmin.tableExists(TableName.valueOf(tableName))) {
+            if (noneExistingTableList == null) {
+              noneExistingTableList = new ArrayList<String>();
+            }
+            noneExistingTableList.add(tableName);
+          }
+        }
+        if (noneExistingTableList != null) {
+          if (backupType.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+            LOG.warn("Incremental backup table set contains no-exising table: "
+                + noneExistingTableList);
+          } else {
+            // Throw exception only in full mode - we try to backup non-existing table
+            throw new RuntimeException("Non-existing tables found in the table list: "
+                + noneExistingTableList);
+          }
+        }
+      }
+
+      // if any target table backup dir already exist, then no backup action taken
+      String[] tableNames = null;
+      if (tables != null && !tables.equals("")) {
+        tableNames = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+      }
+      if (tableNames != null && tableNames.length > 0) {
+        for (String table : tableNames) {
+          String targetTableBackupDir =
+              HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
+          Path targetTableBackupDirPath = new Path(targetTableBackupDir);
+          FileSystem outputFs = FileSystem.get(targetTableBackupDirPath.toUri(), conf);
+          if (outputFs.exists(targetTableBackupDirPath)) {
+            throw new IOException("Target backup directory " + targetTableBackupDir
+              + " exists already.");
+          }
+        }
+      }
+      backupContext =
+          backupManager.createBackupContext(backupId, backupType, tables, targetRootDir, snapshot);
+      backupManager.initialize();
+      backupManager.dispatchRequest(backupContext);
+    } catch (BackupException e) {
+      // suppress the backup exception wrapped within #initialize or #dispatchRequest, backup
+      // exception has already been handled normally
+      StackTraceElement[] stes = e.getStackTrace();
+      for (StackTraceElement ste : stes) {
+        LOG.info(ste);
+      }
+      LOG.error("Backup Exception " + e.getMessage());
+    } finally {
+      if (hbadmin != null) {
+        hbadmin.close();
+      }
+      if (conn != null) {
+        conn.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java
new file mode 100644
index 0000000..c7aff15
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java
@@ -0,0 +1,147 @@
+/**
+ * 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 org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_COMMAND;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * General backup commands, options and usage messages
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+final class BackupCommands {
+
+  private static final String USAGE = "Usage: hbase backup COMMAND\n"
+      + "where COMMAND is one of:\n" + "  create     create a new backup image\n"
+      + "Enter \'help COMMAND\' to see help message for each command\n";
+
+  private static final String CREATE_CMD_USAGE =
+      "Usage: hbase backup create <type> <backup_root_path> [tables] [-s name] [-convert] "
+          + "[-silent]\n" + " type          \"full\" to create a full backup image;\n"
+          + "               \"incremental\" to create an incremental backup image\n"
+          + " backup_root_path   The full root path to store the backup image,\n"
+          + "                    the prefix can be gpfs, hdfs or webhdfs\n" + " Options:\n"
+          + "   tables      If no tables (\"\") are specified, all tables are backed up. "
+          + "Otherwise it is a\n" + "               comma separated list of tables.\n"
+          + "   -s name     Use the specified snapshot for full backup\n"
+          + "   -convert    For an incremental backup, convert WAL files to HFiles\n";
+
+  interface Command {
+    void execute() throws IOException;
+  }
+
+  private BackupCommands() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  static Command createCommand(BACKUP_COMMAND type, CommandLine cmdline) {
+    Command cmd = null;
+    switch (type) {
+      case CREATE:
+        cmd = new CreateCommand(cmdline);
+        break;
+      case HELP:
+      default:
+        cmd = new HelpCommand(cmdline);
+        break;
+    }
+    return cmd;
+  }
+
+  private static class CreateCommand implements Command {
+    CommandLine cmdline;
+
+    CreateCommand(CommandLine cmdline) {
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null || cmdline.getArgs() == null) {
+        System.out.println("ERROR: missing arguments");
+        System.out.println(CREATE_CMD_USAGE);
+        System.exit(-1);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length < 2 || args.length > 3) {
+        System.out.println("ERROR: wrong number of arguments");
+        System.out.println(CREATE_CMD_USAGE);
+        System.exit(-1);
+      }
+
+      if (!BackupRestoreConstants.BACKUP_TYPE_FULL.equalsIgnoreCase(args[0])
+          && !BackupRestoreConstants.BACKUP_TYPE_INCR.equalsIgnoreCase(args[0])) {
+        System.out.println("ERROR: invalid backup type");
+        System.out.println(CREATE_CMD_USAGE);
+        System.exit(-1);
+      }
+
+      String snapshot = cmdline.hasOption('s') ? cmdline.getOptionValue('s') : null;
+      String tables = (args.length == 3) ? args[2] : null;
+
+      try {
+        BackupClient.create(args[0], args[1], tables, snapshot);
+      } catch (RuntimeException e) {
+        System.out.println("ERROR: " + e.getMessage());
+        System.exit(-1);
+      }
+    }
+  }
+
+  private static class HelpCommand implements Command {
+    CommandLine cmdline;
+
+    HelpCommand(CommandLine cmdline) {
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null) {
+        System.out.println(USAGE);
+        System.exit(0);
+      }
+
+      String[] args = cmdline.getArgs();
+      if (args == null || args.length == 0) {
+        System.out.println(USAGE);
+        System.exit(0);
+      }
+
+      if (args.length != 1) {
+        System.out.println("Only support check help message of a single command type");
+        System.out.println(USAGE);
+        System.exit(0);
+      }
+
+      String type = args[0];
+
+      if (BACKUP_COMMAND.CREATE.name().equalsIgnoreCase(type)) {
+        System.out.println(CREATE_CMD_USAGE);
+      } // other commands will be supported in future jira
+      System.exit(0);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java
new file mode 100644
index 0000000..e8d6444
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java
@@ -0,0 +1,318 @@
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * An object to encapsulate the information for each backup request
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupContext implements Serializable {
+
+  private static final long serialVersionUID = 2401435114454300992L;
+
+  // backup id: a timestamp when we request the backup
+  private String backupId;
+
+  // backup type, full or incremental
+  private String type;
+
+  // target root directory for storing the backup files
+  private String targetRootDir;
+
+  // overall backup status
+  private BackupHandler.BACKUPSTATUS flag;
+
+  // overall backup phase
+  private BackupHandler.BACKUPPHASE phase;
+
+  // overall backup failure message
+  private String failedMsg;
+
+  // backup status map for all tables
+  private Map<String, BackupStatus> backupStatusMap;
+
+  // actual start timestamp of the backup process
+  private long startTs;
+
+  // actual end timestamp of the backup process, could be fail or complete
+  private long endTs;
+
+  // the total bytes of incremental logs copied
+  private long totalBytesCopied;
+
+  // for incremental backup, the location of the backed-up hlogs
+  private String hlogTargetDir = null;
+
+  // incremental backup file list
+  transient private List<String> incrBackupFileList;
+
+  // new region server log timestamps for table set after distributed log roll
+  // key - table name, value - map of RegionServer hostname -> last log rolled timestamp
+  transient private HashMap<String, HashMap<String, String>> tableSetTimestampMap;
+
+  // cancel flag
+  private boolean cancelled = false;
+  // backup progress string
+
+  private String progress;
+
+  public BackupContext() {
+  }
+
+  public BackupContext(String backupId, String type, String[] tables, String targetRootDir,
+      String snapshot) {
+    super();
+
+    if (backupStatusMap == null) {
+      backupStatusMap = new HashMap<String, BackupStatus>();
+    }
+
+    this.backupId = backupId;
+    this.type = type;
+    this.targetRootDir = targetRootDir;
+
+    this.addTables(tables);
+
+    if (type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+      setHlogTargetDir(HBackupFileSystem.getLogBackupDir(targetRootDir, backupId));
+    }
+
+    this.startTs = 0;
+    this.endTs = 0;
+
+  }
+
+  /**
+   * Set progress string
+   * @param msg progress message
+   */
+
+  public void setProgress(String msg) {
+    this.progress = msg;
+  }
+
+  /**
+   * Get current progress msg
+   */
+  public String getProgress() {
+    return progress;
+  }
+
+  /**
+   * Mark cancel flag.
+   */
+  public void markCancel() {
+    this.cancelled = true;
+  }
+
+  /**
+   * Has been marked as cancelled or not.
+   * @return True if marked as cancelled
+   */
+  public boolean isCancelled() {
+    return this.cancelled;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  public void setBackupId(String backupId) {
+    this.backupId = backupId;
+  }
+
+  public BackupStatus getBackupStatus(String table) {
+    return this.backupStatusMap.get(table);
+  }
+
+  public String getFailedMsg() {
+    return failedMsg;
+  }
+
+  public void setFailedMsg(String failedMsg) {
+    this.failedMsg = failedMsg;
+  }
+
+  public long getStartTs() {
+    return startTs;
+  }
+
+  public void setStartTs(long startTs) {
+    this.startTs = startTs;
+  }
+
+  public long getEndTs() {
+    return endTs;
+  }
+
+  public void setEndTs(long endTs) {
+    this.endTs = endTs;
+  }
+
+  public long getTotalBytesCopied() {
+    return totalBytesCopied;
+  }
+
+  public BackupHandler.BACKUPSTATUS getFlag() {
+    return flag;
+  }
+
+  public void setFlag(BackupHandler.BACKUPSTATUS flag) {
+    this.flag = flag;
+  }
+
+  public BackupHandler.BACKUPPHASE getPhase() {
+    return phase;
+  }
+
+  public void setPhase(BackupHandler.BACKUPPHASE phase) {
+    this.phase = phase;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setSnapshotName(String table, String snapshotName) {
+    this.backupStatusMap.get(table).setSnapshotName(snapshotName);
+  }
+
+  public String getSnapshotName(String table) {
+    return this.backupStatusMap.get(table).getSnapshotName();
+  }
+
+  public List<String> getSnapshotNames() {
+    List<String> snapshotNames = new ArrayList<String>();
+    for (BackupStatus backupStatus : this.backupStatusMap.values()) {
+      snapshotNames.add(backupStatus.getSnapshotName());
+    }
+    return snapshotNames;
+  }
+
+  public Set<String> getTables() {
+    return this.backupStatusMap.keySet();
+  }
+
+  public String getTableListAsString() {
+    return BackupUtil.concat(backupStatusMap.keySet(), ";");
+  }
+
+  public void addTables(String[] tables) {
+    for (String table : tables) {
+      BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId);
+      this.backupStatusMap.put(table, backupStatus);
+    }
+  }
+
+  public String getTargetRootDir() {
+    return targetRootDir;
+  }
+
+  public void setHlogTargetDir(String hlogTagetDir) {
+    this.hlogTargetDir = hlogTagetDir;
+  }
+
+  public String getHLogTargetDir() {
+    return hlogTargetDir;
+  }
+
+  public List<String> getIncrBackupFileList() {
+    return incrBackupFileList;
+  }
+
+  public List<String> setIncrBackupFileList(List<String> incrBackupFileList) {
+    this.incrBackupFileList = incrBackupFileList;
+    return this.incrBackupFileList;
+  }
+
+  /**
+   * Set the new region server log timestamps after distributed log roll
+   * @param newTableSetTimestampMap table timestamp map
+   */
+  public void setIncrTimestampMap(HashMap<String, 
+      HashMap<String, String>> newTableSetTimestampMap) {
+    this.tableSetTimestampMap = newTableSetTimestampMap;
+  }
+
+  /**
+   * Get new region server log timestamps after distributed log roll
+   * @return new region server log timestamps
+   */
+  public HashMap<String, HashMap<String, String>> getIncrTimestampMap() {
+    return this.tableSetTimestampMap;
+  }
+
+  /**
+   * Get existing snapshot if backing up from existing snapshot.
+   * @return The existing snapshot, null if not backing up from existing snapshot
+   */
+  public String getExistingSnapshot() {
+    // this feature will be supported in another Jira
+    return null;
+  }
+
+  /**
+   * Check whether this backup context are for backing up from existing snapshot or not.
+   * @return true if it is for backing up from existing snapshot, otherwise false
+   */
+  public boolean fromExistingSnapshot() {
+    // this feature will be supported in later jiras
+    return false;
+  }
+
+  public String getTableBySnapshot(String snapshotName) {
+    for (Entry<String, BackupStatus> entry : this.backupStatusMap.entrySet()) {
+      if (snapshotName.equals(entry.getValue().getSnapshotName())) {
+        return entry.getKey();
+      }
+    }
+    return null;
+  }
+
+  public byte[] toByteArray() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    ObjectOutputStream oos = new ObjectOutputStream(baos);
+    oos.writeObject(this);
+    return baos.toByteArray();
+  }
+
+  public static BackupContext fromByteArray(byte[] data) 
+      throws IOException, ClassNotFoundException {
+    ByteArrayInputStream bais = new ByteArrayInputStream(data);
+    ObjectInputStream ois = new ObjectInputStream(bais);
+    return (BackupContext) ois.readObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java
new file mode 100644
index 0000000..223746d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java
@@ -0,0 +1,37 @@
+/**
+ * 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 org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface BackupCopyService extends Configurable {
+  static enum Type {
+    FULL, INCREMENTAL
+  }
+
+  public int copy(BackupHandler backupHandler, Configuration conf, BackupCopyService.Type copyType,
+      String[] options) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java
new file mode 100644
index 0000000..fe0d0e2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java
@@ -0,0 +1,85 @@
+/**
+ * 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 org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Backup exception
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupException extends HBaseIOException {
+  private BackupContext description;
+
+  /**
+   * Some exception happened for a backup and don't even know the backup that it was about
+   * @param msg Full description of the failure
+   */
+  public BackupException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * Some exception happened for a backup with a cause
+   * @param cause the cause
+   */
+  public BackupException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * Exception for the given backup that has no previous root cause
+   * @param msg reason why the backup failed
+   * @param desc description of the backup that is being failed
+   */
+  public BackupException(String msg, BackupContext desc) {
+    super(msg);
+    this.description = desc;
+  }
+
+  /**
+   * Exception for the given backup due to another exception
+   * @param msg reason why the backup failed
+   * @param cause root cause of the failure
+   * @param desc description of the backup that is being failed
+   */
+  public BackupException(String msg, Throwable cause, BackupContext desc) {
+    super(msg, cause);
+    this.description = desc;
+  }
+
+  /**
+   * Exception when the description of the backup cannot be determined, due to some other root
+   * cause
+   * @param message description of what caused the failure
+   * @param e root cause
+   */
+  public BackupException(String message, Exception e) {
+    super(message, e);
+  }
+
+  public BackupContext getBackupContext() {
+    return this.description;
+  }
+
+}