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 2017/03/13 22:52:54 UTC

[01/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14123 [created] a4e03f60d


http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..299cd56
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.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;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+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.MobSnapshotTestingUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@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");
+    final CountDownLatch latch = new CountDownLatch(1);
+    final int NB_ROWS_IN_FAM3 = 6;
+    final byte[] fam3Name = Bytes.toBytes("f3");
+    final byte[] fam2Name = Bytes.toBytes("f2");
+    final Connection conn = ConnectionFactory.createConnection(conf1);
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        try {
+          latch.await();
+        } catch (InterruptedException ie) {
+        }
+        try {
+          HTable t1 = (HTable) conn.getTable(table1);
+          Put p1;
+          for (int i = 0; i < NB_ROWS_IN_FAM3; i++) {
+            p1 = new Put(Bytes.toBytes("row-t1" + i));
+            p1.addColumn(fam3Name, qualName, Bytes.toBytes("val" + i));
+            t1.put(p1);
+          }
+          LOG.debug("Wrote " + NB_ROWS_IN_FAM3 + " rows into family3");
+          t1.close();
+        } catch (IOException ioe) {
+          throw new RuntimeException(ioe);
+        }
+      }
+    };
+    t.start();
+
+    table1Desc.addFamily(new HColumnDescriptor(fam3Name));
+    // family 2 is MOB enabled
+    HColumnDescriptor hcd = new HColumnDescriptor(fam2Name);
+    hcd.setMobEnabled(true);
+    hcd.setMobThreshold(0L);
+    table1Desc.addFamily(hcd);
+    HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
+
+    SnapshotTestingUtils.loadData(TEST_UTIL, table1, 50, fam2Name);
+    HTable t1 = (HTable) conn.getTable(table1);
+    int rows0 = MobSnapshotTestingUtils.countMobRows(t1, fam2Name);
+
+    latch.countDown();
+    String backupId =
+        backupTables(BackupType.FULL, Lists.newArrayList(table1), BACKUP_REMOTE_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+
+    LOG.info("backup complete " + backupId);
+    Assert.assertEquals(TEST_UTIL.countRows(t1, famName), NB_ROWS_IN_BATCH);
+
+    t.join();
+    Assert.assertEquals(TEST_UTIL.countRows(t1, fam3Name), NB_ROWS_IN_FAM3);
+    t1.close();
+
+    TableName[] tablesRestoreFull = new TableName[] { table1 };
+
+    TableName[] tablesMapFull = new TableName[] { table1_restore };
+
+    BackupAdmin client = getBackupAdmin();
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_REMOTE_ROOT_DIR, backupId, false,
+      tablesRestoreFull, tablesMapFull, false));
+
+    // check tables for full restore
+    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hAdmin.tableExists(table1_restore));
+
+    // #5.2 - checking row count of tables for full restore
+    HTable hTable = (HTable) conn.getTable(table1_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable, famName), NB_ROWS_IN_BATCH);
+    int cnt3 = TEST_UTIL.countRows(hTable, fam3Name);
+    Assert.assertTrue(cnt3 >= 0 && cnt3 <= NB_ROWS_IN_FAM3);
+
+    int rows1 = MobSnapshotTestingUtils.countMobRows(t1, fam2Name);
+    Assert.assertEquals(rows0, rows1);
+    hTable.close();
+
+    hAdmin.close();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..c7ed954
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
@@ -0,0 +1,52 @@
+/**
+ * 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.backup.util.BackupUtils;
+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 =
+        backupTables(BackupType.FULL, toList(table1.getNameAsString()), BACKUP_REMOTE_ROOT_DIR);
+    LOG.info("backup complete");
+    TableName[] tableset = new TableName[] { table1 };
+    TableName[] tablemap = new TableName[] { table1_restore };
+    getBackupAdmin().restore(
+      BackupUtils.createRestoreRequest(BACKUP_REMOTE_ROOT_DIR, backupId, false, tableset,
+        tablemap, false));
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table1_restore));
+    TEST_UTIL.deleteTable(table1_restore);
+    hba.close();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..c61b018
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
@@ -0,0 +1,80 @@
+/**
+ * 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.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+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 = fullTableBackup(toList(table1.getNameAsString()));
+    LOG.info("backup complete");
+    TableName[] tableset = new TableName[] { table1 };
+    TableName[] tablemap = new TableName[] { table1_restore };
+    getBackupAdmin().restore(
+      BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, tablemap,
+        false));
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table1_restore));
+    TEST_UTIL.deleteTable(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");
+
+    List<TableName> tables = toList(table2.getNameAsString(), table3.getNameAsString());
+    String backupId = fullTableBackup(tables);
+    TableName[] restore_tableset = new TableName[] { table2, table3 };
+    TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
+    getBackupAdmin().restore(
+      BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, restore_tableset,
+        tablemap, false));
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table2_restore));
+    assertTrue(hba.tableExists(table3_restore));
+    TEST_UTIL.deleteTable(table2_restore);
+    TEST_UTIL.deleteTable(table3_restore);
+    hba.close();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
new file mode 100644
index 0000000..6212ff2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
@@ -0,0 +1,56 @@
+/**
+ * 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestSystemTableSnapshot extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestSystemTableSnapshot.class);
+
+  /**
+   * Verify that a single table is restored to a new table
+   * @throws Exception
+   */
+  //@Test - Disabled until we get resolution on system table snapshots
+
+  public void _testBackupRestoreSystemTable() throws Exception {
+
+    LOG.info("test snapshot system table");
+
+    TableName backupSystem = BackupSystemTable.getTableName(conf1);
+
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    String snapshotName = "sysTable";
+    hba.snapshot(snapshotName, backupSystem);
+
+    hba.disableTable(backupSystem);
+    hba.restoreSnapshot(snapshotName);
+    hba.enableTable(backupSystem);
+    hba.close();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
new file mode 100644
index 0000000..229597b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
@@ -0,0 +1,162 @@
+/**
+ * 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 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.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.TestBackupBase;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+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.AbstractFSWALProvider;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+@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");
+
+    List<TableName> tableSetFullList = Lists.newArrayList(table1, table2, table3, table4);
+
+    try (BackupSystemTable systemTable = new BackupSystemTable(TEST_UTIL.getConnection())) {
+      // 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());
+      cleaner.init(null);
+      cleaner.setConf(TEST_UTIL.getConfiguration());
+
+      Iterable<FileStatus> deletable = cleaner.getDeletableFiles(walFiles);
+      int size = Iterables.size(deletable);
+
+      // We can delete all files because we do not have yet recorded backup sessions
+      assertTrue(size == walFiles.size());
+
+      systemTable.addWALFiles(swalFiles, "backup", "root");
+      String backupIdFull = fullTableBackup(tableSetFullList);
+      assertTrue(checkSucceeded(backupIdFull));
+      // Check one more time
+      deletable = cleaner.getDeletableFiles(walFiles);
+      // We can delete wal files because they were saved into backup system table table
+      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());
+      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
+
+      List<TableName> tableSetIncList = Lists.newArrayList(table1, table2, table3);
+      String backupIdIncMultiple = backupTables(BackupType.INCREMENTAL, tableSetIncList,
+        BACKUP_ROOT_DIR);
+      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() && !AbstractFSWALProvider.isMetaFile(lfs.getPath())) {
+        logFiles.add(lfs);
+        LOG.info(lfs);
+      }
+    }
+    return logFiles;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java
index e71318b..296b38f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java
@@ -64,7 +64,7 @@ public class SimpleMasterProcedureManager extends MasterProcedureManager {
     // setup the default procedure coordinator
     String name = master.getServerName().toString();
     ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, 1);
-    ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinatorRpcs(
+    ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinator(
         master.getZooKeeper(), getProcedureSignature(), name);
 
     this.coordinator = new ProcedureCoordinator(comms, tpool);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
index 9a77ce5..f20c8a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedure.java
@@ -127,7 +127,7 @@ public class TestZKProcedure {
     String opDescription = "coordination test - " + members.length + " cohort members";
 
     // start running the controller
-    ZKProcedureCoordinatorRpcs coordinatorComms = new ZKProcedureCoordinatorRpcs(
+    ZKProcedureCoordinator coordinatorComms = new ZKProcedureCoordinator(
         coordZkw, opDescription, COORDINATOR_NODE_NAME);
     ThreadPoolExecutor pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE);
     ProcedureCoordinator coordinator = new ProcedureCoordinator(coordinatorComms, pool) {
@@ -208,7 +208,7 @@ public class TestZKProcedure {
 
     // start running the coordinator and its controller
     ZooKeeperWatcher coordinatorWatcher = newZooKeeperWatcher();
-    ZKProcedureCoordinatorRpcs coordinatorController = new ZKProcedureCoordinatorRpcs(
+    ZKProcedureCoordinator coordinatorController = new ZKProcedureCoordinator(
         coordinatorWatcher, opDescription, COORDINATOR_NODE_NAME);
     ThreadPoolExecutor pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, POOL_SIZE, KEEP_ALIVE);
     ProcedureCoordinator coordinator = spy(new ProcedureCoordinator(coordinatorController, pool));
@@ -393,7 +393,7 @@ public class TestZKProcedure {
 
   private void closeAll(
       ProcedureCoordinator coordinator,
-      ZKProcedureCoordinatorRpcs coordinatorController,
+      ZKProcedureCoordinator coordinatorController,
       List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> cohort)
       throws IOException {
     // make sure we close all the resources

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
index d864db2..e7e2b23 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestZKProcedureControllers.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hbase.procedure;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -31,11 +31,11 @@ import java.util.concurrent.CountDownLatch;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -188,9 +188,9 @@ public class TestZKProcedureControllers {
 
     ProcedureMember member = Mockito.mock(ProcedureMember.class);
 
-    Pair<ZKProcedureCoordinatorRpcs, List<ZKProcedureMemberRpcs>> pair = controllers
+    Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> pair = controllers
         .start(watcher, operationName, coordinator, CONTROLLER_NODE_NAME, member, expected);
-    ZKProcedureCoordinatorRpcs controller = pair.getFirst();
+    ZKProcedureCoordinator controller = pair.getFirst();
     List<ZKProcedureMemberRpcs> cohortControllers = pair.getSecond();
     // start the operation
     Procedure p = Mockito.mock(Procedure.class);
@@ -266,9 +266,9 @@ public class TestZKProcedureControllers {
     Procedure p = Mockito.mock(Procedure.class);
     Mockito.when(p.getName()).thenReturn(operationName);
 
-    Pair<ZKProcedureCoordinatorRpcs, List<ZKProcedureMemberRpcs>> pair = controllers
+    Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> pair = controllers
         .start(watcher, operationName, coordinator, CONTROLLER_NODE_NAME, member, expected);
-    ZKProcedureCoordinatorRpcs controller = pair.getFirst();
+    ZKProcedureCoordinator controller = pair.getFirst();
     List<ZKProcedureMemberRpcs> cohortControllers = pair.getSecond();
 
     // post 1/2 the prepare nodes early
@@ -383,7 +383,7 @@ public class TestZKProcedureControllers {
    * Specify how the controllers that should be started (not spy/mockable) for the test.
    */
   private abstract class StartControllers {
-    public abstract Pair<ZKProcedureCoordinatorRpcs, List<ZKProcedureMemberRpcs>> start(
+    public abstract Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
         ZooKeeperWatcher watcher, String operationName,
         ProcedureCoordinator coordinator, String controllerName,
         ProcedureMember member, List<String> cohortNames) throws Exception;
@@ -392,12 +392,12 @@ public class TestZKProcedureControllers {
   private final StartControllers startCoordinatorFirst = new StartControllers() {
 
     @Override
-    public Pair<ZKProcedureCoordinatorRpcs, List<ZKProcedureMemberRpcs>> start(
+    public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
         ZooKeeperWatcher watcher, String operationName,
         ProcedureCoordinator coordinator, String controllerName,
         ProcedureMember member, List<String> expected) throws Exception {
       // start the controller
-      ZKProcedureCoordinatorRpcs controller = new ZKProcedureCoordinatorRpcs(
+      ZKProcedureCoordinator controller = new ZKProcedureCoordinator(
           watcher, operationName, CONTROLLER_NODE_NAME);
       controller.start(coordinator);
 
@@ -420,7 +420,7 @@ public class TestZKProcedureControllers {
   private final StartControllers startCohortFirst = new StartControllers() {
 
     @Override
-    public Pair<ZKProcedureCoordinatorRpcs, List<ZKProcedureMemberRpcs>> start(
+    public Pair<ZKProcedureCoordinator, List<ZKProcedureMemberRpcs>> start(
         ZooKeeperWatcher watcher, String operationName,
         ProcedureCoordinator coordinator, String controllerName,
         ProcedureMember member, List<String> expected) throws Exception {
@@ -434,7 +434,7 @@ public class TestZKProcedureControllers {
       }
 
       // start the controller
-      ZKProcedureCoordinatorRpcs controller = new ZKProcedureCoordinatorRpcs(
+      ZKProcedureCoordinator controller = new ZKProcedureCoordinator(
           watcher, operationName, CONTROLLER_NODE_NAME);
       controller.start(coordinator);
 


[10/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

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


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

Branch: refs/heads/HBASE-14123
Commit: 3aaea8e041d8963cba4b2d62060bc7d5abf787e8
Parents: e0365df
Author: tedyu <yu...@gmail.com>
Authored: Fri Mar 10 15:37:25 2017 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Mar 10 15:37:25 2017 -0800

----------------------------------------------------------------------
 bin/hbase                                       |    6 +
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |    4 +-
 .../hbase/client/RpcRetryingCallerImpl.java     |    3 +-
 .../apache/hadoop/hbase/backup/BackupType.java  |   25 +
 .../hadoop/hbase/util/AbstractHBaseTool.java    |   18 +-
 .../hbase/IntegrationTestBackupRestore.java     |  311 +
 .../shaded/protobuf/generated/BackupProtos.java | 7013 ++++++++++++++++++
 .../shaded/protobuf/generated/MasterProtos.java |   18 +-
 .../src/main/protobuf/Backup.proto              |  117 +
 hbase-server/pom.xml                            |   10 +
 .../apache/hadoop/hbase/backup/BackupAdmin.java |  128 +
 .../hadoop/hbase/backup/BackupCopyJob.java      |   55 +
 .../hadoop/hbase/backup/BackupDriver.java       |  204 +
 .../apache/hadoop/hbase/backup/BackupInfo.java  |  545 ++
 .../hadoop/hbase/backup/BackupRequest.java      |  139 +
 .../hbase/backup/BackupRestoreConstants.java    |  115 +
 .../hbase/backup/BackupRestoreFactory.java      |   66 +
 .../hadoop/hbase/backup/BackupTableInfo.java    |   82 +
 .../hadoop/hbase/backup/HBackupFileSystem.java  |  141 +
 .../apache/hadoop/hbase/backup/LogUtils.java    |   50 +
 .../hadoop/hbase/backup/RestoreDriver.java      |  265 +
 .../apache/hadoop/hbase/backup/RestoreJob.java  |   46 +
 .../hadoop/hbase/backup/RestoreRequest.java     |  135 +
 .../hbase/backup/impl/BackupAdminImpl.java      |  524 ++
 .../hbase/backup/impl/BackupCommands.java       |  780 ++
 .../hbase/backup/impl/BackupException.java      |   84 +
 .../hadoop/hbase/backup/impl/BackupManager.java |  472 ++
 .../hbase/backup/impl/BackupManifest.java       |  666 ++
 .../hbase/backup/impl/BackupSystemTable.java    | 1376 ++++
 .../backup/impl/FullTableBackupClient.java      |  189 +
 .../backup/impl/IncrementalBackupManager.java   |  344 +
 .../impl/IncrementalTableBackupClient.java      |  216 +
 .../hbase/backup/impl/RestoreTablesClient.java  |  237 +
 .../hbase/backup/impl/TableBackupClient.java    |  387 +
 .../backup/mapreduce/HFileSplitterJob.java      |  181 +
 .../mapreduce/MapReduceBackupCopyJob.java       |  344 +
 .../backup/mapreduce/MapReduceRestoreJob.java   |  182 +
 .../hbase/backup/master/BackupLogCleaner.java   |  142 +
 .../master/LogRollMasterProcedureManager.java   |  155 +
 .../regionserver/LogRollBackupSubprocedure.java |  168 +
 .../LogRollBackupSubprocedurePool.java          |  139 +
 .../LogRollRegionServerProcedureManager.java    |  185 +
 .../hadoop/hbase/backup/util/BackupSet.java     |   58 +
 .../hadoop/hbase/backup/util/BackupUtils.java   |  702 ++
 .../hadoop/hbase/backup/util/RestoreTool.java   |  610 ++
 .../BaseCoordinatedStateManager.java            |   20 +-
 .../coordination/ZkCoordinatedStateManager.java |   23 +-
 .../hbase/mapreduce/HFileInputFormat.java       |  174 +
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |   25 +-
 .../hadoop/hbase/mapreduce/WALInputFormat.java  |   42 +-
 .../hadoop/hbase/mapreduce/WALPlayer.java       |   83 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |  129 +-
 .../hbase/master/snapshot/SnapshotManager.java  |    4 +-
 .../procedure/ZKProcedureCoordinatorRpcs.java   |    9 +-
 .../flush/MasterFlushTableProcedureManager.java |    4 +-
 .../hbase/regionserver/HRegionServer.java       |   17 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |    5 +
 .../hadoop/hbase/HBaseTestingUtility.java       |   41 +-
 .../hadoop/hbase/backup/TestBackupBase.java     |  293 +
 .../hbase/backup/TestBackupBoundaryTests.java   |   97 +
 .../hbase/backup/TestBackupCommandLineTool.java |  431 ++
 .../hadoop/hbase/backup/TestBackupDelete.java   |  102 +
 .../hbase/backup/TestBackupDeleteRestore.java   |   70 +
 .../hadoop/hbase/backup/TestBackupDescribe.java |  110 +
 .../hbase/backup/TestBackupMultipleDeletes.java |  159 +
 .../hbase/backup/TestBackupShowHistory.java     |  148 +
 .../hbase/backup/TestBackupStatusProgress.java  |   96 +
 .../hbase/backup/TestBackupSystemTable.java     |  511 ++
 .../hadoop/hbase/backup/TestFullBackup.java     |   59 +
 .../hadoop/hbase/backup/TestFullBackupSet.java  |  103 +
 .../backup/TestFullBackupSetRestoreSet.java     |  128 +
 .../hadoop/hbase/backup/TestFullRestore.java    |  345 +
 .../hbase/backup/TestIncrementalBackup.java     |  200 +
 .../TestIncrementalBackupDeleteTable.java       |  129 +
 .../hadoop/hbase/backup/TestRemoteBackup.java   |  129 +
 .../hadoop/hbase/backup/TestRemoteRestore.java  |   52 +
 .../hbase/backup/TestRestoreBoundaryTests.java  |   80 +
 .../hbase/backup/TestSystemTableSnapshot.java   |   56 +
 .../backup/master/TestBackupLogCleaner.java     |  162 +
 .../procedure/SimpleMasterProcedureManager.java |    2 +-
 .../hadoop/hbase/procedure/TestZKProcedure.java |    6 +-
 .../procedure/TestZKProcedureControllers.java   |   24 +-
 82 files changed, 21602 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index 1653c5a..f1114af 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -103,6 +103,8 @@ if [ $# = 0 ]; then
   echo "  ltt             Run LoadTestTool"
   echo "  canary          Run the Canary tool"
   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
@@ -315,6 +317,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.BackupDriver'
+elif [ "$COMMAND" = "restore" ] ; then
+  CLASS='org.apache.hadoop.hbase.backup.RestoreDriver'
 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/3aaea8e0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 1f143b5..5275dae 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -4098,11 +4098,11 @@ public class HBaseAdmin implements Admin {
     if (result == 0) {
       Iterator<HColumnDescriptor> remoteHCDIter = peerHtd.getFamilies().iterator();
       Iterator<HColumnDescriptor> localHCDIter = localHtd.getFamilies().iterator();
-          
+
       while (remoteHCDIter.hasNext() && localHCDIter.hasNext()) {
         HColumnDescriptor remoteHCD = remoteHCDIter.next();
         HColumnDescriptor localHCD = localHCDIter.next();
-        
+
         String remoteHCDName = remoteHCD.getNameAsString();
         String localHCDName = localHCD.getNameAsString();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
index 3f65e6e..c59b020 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
@@ -107,6 +107,7 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
       } catch (PreemptiveFastFailException e) {
         throw e;
       } catch (Throwable t) {
+        Throwable e = t.getCause();
         ExceptionUtil.rethrowIfInterrupt(t);
 
         // translateException throws exception when should not retry: i.e. when request is bad.
@@ -185,7 +186,7 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
       }
     }
   }
-  
+
   /**
    * Get the good or the remote exception if any, throws the DoNotRetryIOException.
    * @param t the throwable to analyze

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java
new file mode 100644
index 0000000..79f4636
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/backup/BackupType.java
@@ -0,0 +1,25 @@
+/**
+ * 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;
+
+@InterfaceAudience.Private
+public enum BackupType {
+  FULL, INCREMENTAL
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractHBaseTool.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractHBaseTool.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractHBaseTool.java
index a51a80f..29d10ae 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractHBaseTool.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/AbstractHBaseTool.java
@@ -22,7 +22,9 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 
+import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.DefaultParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.MissingOptionException;
@@ -32,9 +34,9 @@ import org.apache.commons.cli.ParseException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -47,12 +49,15 @@ public abstract class AbstractHBaseTool implements Tool, Configurable {
   protected static final int EXIT_SUCCESS = 0;
   protected static final int EXIT_FAILURE = 1;
 
+  public static final String SHORT_HELP_OPTION = "h";
+  public static final String LONG_HELP_OPTION = "help";
+
   private static final Option HELP_OPTION = new Option("h", "help", false,
       "Prints help for this tool.");
 
   private static final Log LOG = LogFactory.getLog(AbstractHBaseTool.class);
 
-  private final Options options = new Options();
+  protected final Options options = new Options();
 
   protected Configuration conf = null;
 
@@ -108,7 +113,7 @@ public abstract class AbstractHBaseTool implements Tool, Configurable {
   }
 
   @Override
-  public final int run(String[] args) throws IOException {
+  public int run(String[] args) throws IOException {
     cmdLineArgs = args;
     if (conf == null) {
       LOG.error("Tool configuration is not initialized");
@@ -161,6 +166,13 @@ public abstract class AbstractHBaseTool implements Tool, Configurable {
     return cl.getOptions().length != 0;
   }
 
+  protected CommandLine parseArgs(String[] args) throws ParseException {
+    options.addOption(SHORT_HELP_OPTION, LONG_HELP_OPTION, false, "Show usage");
+    addOptions();
+    CommandLineParser parser = new BasicParser();
+    return parser.parse(options, args);
+  }
+
   protected void printUsage() {
     printUsage("hbase " + getClass().getName() + " <options>", "Options:", "");
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
new file mode 100644
index 0000000..8596489
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
@@ -0,0 +1,311 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.cli.CommandLine;
+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.BackupAdmin;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.RestoreRequest;
+import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.IntegrationTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+
+/**
+ * An integration test to detect regressions in HBASE-7912. Create
+ * a table with many regions, load data, perform series backup/load operations,
+ * then restore and verify data
+ * @see <a href="https://issues.apache.org/jira/browse/HBASE-7912">HBASE-7912</a>
+ * @see <a href="https://issues.apache.org/jira/browse/HBASE-14123">HBASE-14123</a>
+ */
+@Category(IntegrationTests.class)
+public class IntegrationTestBackupRestore extends IntegrationTestBase {
+  private static final String CLASS_NAME = IntegrationTestBackupRestore.class.getSimpleName();
+  protected static final Log LOG = LogFactory.getLog(IntegrationTestBackupRestore.class);
+  protected static final TableName TABLE_NAME1 = TableName.valueOf(CLASS_NAME + ".table1");
+  protected static final TableName TABLE_NAME2 = TableName.valueOf(CLASS_NAME + ".table2");
+  protected static final String COLUMN_NAME = "f";
+  protected static final String REGION_COUNT_KEY = "regions_per_rs";
+  protected static final String REGIONSERVER_COUNT_KEY = "region_servers";
+  protected static final int DEFAULT_REGION_COUNT = 10;
+  protected static final int DEFAULT_REGIONSERVER_COUNT = 2;
+  protected static int regionsCountPerServer;
+  protected static int regionServerCount;
+  protected static final String NB_ROWS_IN_BATCH_KEY = "rows_in_batch";
+  protected static final int DEFAULT_NB_ROWS_IN_BATCH = 20000;
+  private static int rowsInBatch;
+  private static String BACKUP_ROOT_DIR = "backupIT";
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    util = new IntegrationTestingUtility();
+    Configuration conf = util.getConfiguration();
+    regionsCountPerServer = conf.getInt(REGION_COUNT_KEY, DEFAULT_REGION_COUNT);
+    regionServerCount =
+        conf.getInt(REGIONSERVER_COUNT_KEY, DEFAULT_REGIONSERVER_COUNT);
+    rowsInBatch = conf.getInt(NB_ROWS_IN_BATCH_KEY, DEFAULT_NB_ROWS_IN_BATCH);
+    enableBackup(conf);
+    LOG.info(String.format("Initializing cluster with %d region servers.", regionServerCount));
+    util.initializeCluster(regionServerCount);
+    LOG.info("Cluster initialized");
+    util.deleteTableIfAny(TABLE_NAME1);
+    util.deleteTableIfAny(TABLE_NAME2);
+    LOG.info("Cluster ready");
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    LOG.info("Cleaning up after test.");
+    if(util.isDistributedCluster()) {
+      util.deleteTableIfAny(TABLE_NAME1);
+      LOG.info("Cleaning up after test. TABLE1 done");
+      util.deleteTableIfAny(TABLE_NAME2);
+      LOG.info("Cleaning up after test. TABLE2 done");
+      cleanUpBackupDir();
+    }
+    LOG.info("Restoring cluster.");
+    util.restoreCluster();
+    LOG.info("Cluster restored.");
+  }
+
+  private void enableBackup(Configuration conf) {
+    // Enable backup
+    conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true);
+    BackupManager.decorateMasterConfiguration(conf);
+    BackupManager.decorateRegionServerConfiguration(conf);
+  }
+
+  private void cleanUpBackupDir() throws IOException {
+    FileSystem fs = FileSystem.get(util.getConfiguration());
+    fs.delete(new Path(BACKUP_ROOT_DIR), true);
+  }
+
+  @Test
+  public void testBackupRestore() throws Exception {
+    BACKUP_ROOT_DIR = util.getDataTestDirOnTestFS() + Path.SEPARATOR + BACKUP_ROOT_DIR;
+    createTable(TABLE_NAME1);
+    createTable(TABLE_NAME2);
+    runTest();
+  }
+
+
+  private void createTable(TableName tableName) throws Exception {
+    long startTime, endTime;
+    HTableDescriptor desc = new HTableDescriptor(tableName);
+    HColumnDescriptor[] columns =
+        new HColumnDescriptor[]{new HColumnDescriptor(COLUMN_NAME)};
+    LOG.info(String.format("Creating table %s with %d splits.", tableName,
+      regionsCountPerServer));
+    startTime = System.currentTimeMillis();
+    HBaseTestingUtility.createPreSplitLoadTestTable(util.getConfiguration(), desc, columns,
+      regionsCountPerServer);
+    util.waitTableAvailable(tableName);
+    endTime = System.currentTimeMillis();
+    LOG.info(String.format("Pre-split table created successfully in %dms.",
+      (endTime - startTime)));
+  }
+
+  private void loadData(TableName table, int numRows) throws IOException {
+    Connection conn = util.getConnection();
+    // #0- insert some data to a table
+    Table t1 = conn.getTable(table);
+    util.loadRandomRows(t1, new byte[]{'f'}, 100, numRows);
+    // flush table
+    conn.getAdmin().flush(TableName.valueOf(table.getName()));
+  }
+
+  private void runTest() throws IOException {
+
+    try (Connection conn = util.getConnection();
+         Admin admin = conn.getAdmin();
+         BackupAdmin client = new BackupAdminImpl(conn);) {
+
+      // #0- insert some data to table TABLE_NAME1, TABLE_NAME2
+      loadData(TABLE_NAME1, rowsInBatch);
+      loadData(TABLE_NAME2, rowsInBatch);
+      // #1 - create full backup for all tables
+      LOG.info("create full backup image for all tables");
+      List<TableName> tables = Lists.newArrayList(TABLE_NAME1, TABLE_NAME2);
+
+      BackupRequest.Builder builder = new BackupRequest.Builder();
+      BackupRequest request =
+          builder.withBackupType(BackupType.FULL).withTableList(tables)
+              .withTargetRootDir(BACKUP_ROOT_DIR).build();
+      String backupIdFull = client.backupTables(request);
+      assertTrue(checkSucceeded(backupIdFull));
+      // #2 - insert some data to table
+      loadData(TABLE_NAME1, rowsInBatch);
+      loadData(TABLE_NAME2, rowsInBatch);
+      HTable t1 = (HTable) conn.getTable(TABLE_NAME1);
+      Assert.assertEquals(util.countRows(t1), rowsInBatch * 2);
+      t1.close();
+      HTable t2 = (HTable) conn.getTable(TABLE_NAME2);
+      Assert.assertEquals(util.countRows(t2), rowsInBatch * 2);
+      t2.close();
+      // #3 - incremental backup for tables
+      tables = Lists.newArrayList(TABLE_NAME1, TABLE_NAME2);
+      builder = new BackupRequest.Builder();
+      request =
+          builder.withBackupType(BackupType.INCREMENTAL).withTableList(tables)
+              .withTargetRootDir(BACKUP_ROOT_DIR).build();
+      String backupIdIncMultiple = client.backupTables(request);
+      assertTrue(checkSucceeded(backupIdIncMultiple));
+      // #4 - restore full backup for all tables, without overwrite
+      TableName[] tablesRestoreFull = new TableName[] { TABLE_NAME1, TABLE_NAME2 };
+      client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false, tablesRestoreFull,
+        null, true));
+      // #5.1 - check tables for full restore
+      assertTrue(admin.tableExists(TABLE_NAME1));
+      assertTrue(admin.tableExists(TABLE_NAME2));
+      // #5.2 - checking row count of tables for full restore
+      HTable hTable = (HTable) conn.getTable(TABLE_NAME1);
+      Assert.assertEquals(util.countRows(hTable), rowsInBatch);
+      hTable.close();
+      hTable = (HTable) conn.getTable(TABLE_NAME2);
+      Assert.assertEquals(util.countRows(hTable), rowsInBatch);
+      hTable.close();
+      // #6 - restore incremental backup for multiple tables, with overwrite
+      TableName[] tablesRestoreIncMultiple = new TableName[] { TABLE_NAME1, TABLE_NAME2 };
+      client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple, false,
+        tablesRestoreIncMultiple, null, true));
+      hTable = (HTable) conn.getTable(TABLE_NAME1);
+      Assert.assertEquals(util.countRows(hTable), rowsInBatch * 2);
+      hTable.close();
+      hTable = (HTable) conn.getTable(TABLE_NAME2);
+      Assert.assertEquals(util.countRows(hTable), rowsInBatch * 2);
+      hTable.close();
+    }
+  }
+
+  protected boolean checkSucceeded(String backupId) throws IOException {
+    BackupInfo status = getBackupInfo(backupId);
+    if (status == null) return false;
+    return status.getState() == BackupState.COMPLETE;
+  }
+
+  private BackupInfo getBackupInfo(String backupId) throws IOException {
+    try (BackupSystemTable table = new BackupSystemTable(util.getConnection())) {
+      return table.readBackupInfo(backupId);
+    }
+  }
+
+  /**
+   * Get restore request.
+   */
+  public RestoreRequest createRestoreRequest(String backupRootDir, String backupId, boolean check,
+      TableName[] fromTables, TableName[] toTables, boolean isOverwrite) {
+    RestoreRequest.Builder builder = new RestoreRequest.Builder();
+    return builder.withBackupRootDir(backupRootDir)
+                                    .withBackupId(backupId)
+                                    .withCheck(check)
+                                    .withFromTables(fromTables)
+                                    .withToTables(toTables)
+                                    .withOvewrite(isOverwrite).build();
+  }
+
+  @Override
+  public void setUpCluster() throws Exception {
+    util = getTestingUtil(getConf());
+    enableBackup(getConf());
+    LOG.debug("Initializing/checking cluster has " + regionServerCount + " servers");
+    util.initializeCluster(regionServerCount);
+    LOG.debug("Done initializing/checking cluster");
+  }
+
+  @Override
+  public int runTestFromCommandLine() throws Exception {
+    testBackupRestore();
+    return 0;
+  }
+
+  @Override
+  public TableName getTablename() {
+    // That is only valid when Monkey is CALM (no monkey)
+    return null;
+  }
+
+  @Override
+  protected Set<String> getColumnFamilies() {
+    // That is only valid when Monkey is CALM (no monkey)
+    return null;
+  }
+
+  @Override
+  protected void addOptions() {
+    addOptWithArg(REGIONSERVER_COUNT_KEY, "Total number of region servers. Default: '"
+        + DEFAULT_REGIONSERVER_COUNT + "'");
+    addOptWithArg(REGION_COUNT_KEY, "Total number of regions. Default: " + DEFAULT_REGION_COUNT);
+    addOptWithArg(NB_ROWS_IN_BATCH_KEY, "Total number of data rows to be loaded (per table/batch."
+        + " Total number of batches=2). Default: " + DEFAULT_NB_ROWS_IN_BATCH);
+
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    super.processOptions(cmd);
+    regionsCountPerServer =
+        Integer.parseInt(cmd.getOptionValue(REGION_COUNT_KEY,
+          Integer.toString(DEFAULT_REGION_COUNT)));
+    regionServerCount =
+        Integer.parseInt(cmd.getOptionValue(REGIONSERVER_COUNT_KEY,
+          Integer.toString(DEFAULT_REGIONSERVER_COUNT)));
+    rowsInBatch =
+        Integer.parseInt(cmd.getOptionValue(NB_ROWS_IN_BATCH_KEY,
+          Integer.toString(DEFAULT_NB_ROWS_IN_BATCH)));
+    LOG.info(Objects.toStringHelper("Parsed Options").add(REGION_COUNT_KEY, regionsCountPerServer)
+        .add(REGIONSERVER_COUNT_KEY, regionServerCount).add(NB_ROWS_IN_BATCH_KEY, rowsInBatch)
+        .toString());
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    IntegrationTestingUtility.setUseDistributedCluster(conf);
+    int status = ToolRunner.run(conf, new IntegrationTestBackupRestore(), args);
+    System.exit(status);
+  }
+}


[02/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
new file mode 100644
index 0000000..38724fb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
@@ -0,0 +1,102 @@
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupDelete extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupDelete.class);
+
+  /**
+   * Verify that full backup is created on a single table with data correctly. Verify that history
+   * works as expected
+   * @throws Exception
+   */
+  @Test
+  public void testBackupDelete() throws Exception {
+    LOG.info("test backup delete on a single table with data");
+    List<TableName> tableList = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tableList);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+    String[] backupIds = new String[] { backupId };
+    BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection());
+    BackupInfo info = table.readBackupInfo(backupId);
+    Path path = new Path(info.getBackupRootDir(), backupId);
+    FileSystem fs = FileSystem.get(path.toUri(), conf1);
+    assertTrue(fs.exists(path));
+    int deleted = getBackupAdmin().deleteBackups(backupIds);
+
+    assertTrue(!fs.exists(path));
+    assertTrue(fs.exists(new Path(info.getBackupRootDir())));
+    assertTrue(1 == deleted);
+    table.close();
+    LOG.info("delete_backup");
+  }
+
+  /**
+   * Verify that full backup is created on a single table with data correctly. Verify that history
+   * works as expected
+   * @throws Exception
+   */
+  @Test
+  public void testBackupDeleteCommand() throws Exception {
+    LOG.info("test backup delete on a single table with data: command-line");
+    List<TableName> tableList = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tableList);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+
+    String[] args = new String[] { "delete", backupId };
+    // Run backup
+
+    try {
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+    } catch (Exception e) {
+      LOG.error("failed", e);
+    }
+    LOG.info("delete_backup");
+    String output = baos.toString();
+    LOG.info(baos.toString());
+    assertTrue(output.indexOf("Deleted 1 backups") >= 0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
new file mode 100644
index 0000000..0921871
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
@@ -0,0 +1,70 @@
+/**
+ * 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.assertTrue;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(MediumTests.class)
+public class TestBackupDeleteRestore extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupDeleteRestore.class);
+
+  /**
+   * Verify that load data- backup - delete some data - restore works as expected - deleted data get
+   * restored.
+   * @throws Exception
+   */
+  @Test
+  public void testBackupDeleteRestore() throws Exception {
+
+    LOG.info("test full restore on a single table empty table");
+
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+    int numRows = TEST_UTIL.countRows(table1);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    // delete row
+    try (Table table = TEST_UTIL.getConnection().getTable(table1);) {
+      Delete delete = new Delete("row0".getBytes());
+      table.delete(delete);
+      hba.flush(table1);
+    }
+
+    TableName[] tableset = new TableName[] { table1 };
+    TableName[] tablemap = null;// new TableName[] { table1_restore };
+    BackupAdmin client = getBackupAdmin();
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
+      tableset, tablemap, true));
+
+    int numRowsAfterRestore = TEST_UTIL.countRows(table1);
+    assertEquals(numRows, numRowsAfterRestore);
+    hba.close();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
new file mode 100644
index 0000000..38aa30d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
@@ -0,0 +1,110 @@
+/**
+ * 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.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupCommands;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupDescribe extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupDescribe.class);
+
+  /**
+   * Verify that describe works as expected if incorrect backup Id is supplied
+   * @throws Exception
+   */
+  @Test
+  public void testBackupDescribe() throws Exception {
+
+    LOG.info("test backup describe on a single table with data");
+
+    String[] args = new String[] { "describe", "backup_2" };
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret < 0);
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setErr(new PrintStream(baos));
+    args = new String[] { "progress" };
+    ToolRunner.run(TEST_UTIL.getConfiguration(), new BackupDriver(), args);
+
+    String output = baos.toString();
+    LOG.info("Output from progress: " + output);
+    assertTrue(output.indexOf(BackupCommands.NO_ACTIVE_SESSION_FOUND) >= 0);
+  }
+
+  @Test
+  public void testBackupSetCommandWithNonExistentTable() throws Exception {
+    String[] args = new String[] { "set", "add", "some_set", "table" };
+    // Run backup
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertNotEquals(ret, 0);
+  }
+
+  @Test
+  public void testBackupDescribeCommand() throws Exception {
+
+    LOG.info("test backup describe on a single table with data: command-line");
+
+    List<TableName> tableList = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tableList);
+
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+
+    BackupInfo info = getBackupAdmin().getBackupInfo(backupId);
+    assertTrue(info.getState() == BackupState.COMPLETE);
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+
+    String[] args = new String[] { "describe", backupId };
+    // Run backup
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    String response = baos.toString();
+    assertTrue(response.indexOf(backupId) > 0);
+    assertTrue(response.indexOf("COMPLETE") > 0);
+
+    BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection());
+    BackupInfo status = table.readBackupInfo(backupId);
+    String desc = status.getShortDescription();
+    table.close();
+    assertTrue(response.indexOf(desc) >= 0);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
new file mode 100644
index 0000000..d173e20
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
@@ -0,0 +1,159 @@
+/**
+ * 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.assertTrue;
+
+import java.util.HashSet;
+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.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+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.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Create multiple backups for two tables: table1, table2 then perform 1 delete
+ */
+@Category(LargeTests.class)
+public class TestBackupMultipleDeletes extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestBackupMultipleDeletes.class);
+
+  @Test
+  public void testBackupMultipleDeletes() throws Exception {
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+    List<TableName> tables = Lists.newArrayList(table1, table2);
+    HBaseAdmin admin = null;
+    Connection conn = ConnectionFactory.createConnection(conf1);
+    admin = (HBaseAdmin) conn.getAdmin();
+    BackupAdmin client = new BackupAdminImpl(conn);
+    BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    String backupIdFull = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdFull));
+    // #2 - insert some data to table table1
+    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.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH * 2);
+    t1.close();
+    // #3 - incremental backup for table1
+    tables = Lists.newArrayList(table1);
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdInc1 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdInc1));
+    // #4 - insert some data to table table2
+    HTable t2 = (HTable) conn.getTable(table2);
+    Put p2 = null;
+    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+      p2 = new Put(Bytes.toBytes("row-t2" + i));
+      p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t2.put(p2);
+    }
+    // #5 - incremental backup for table1, table2
+    tables = Lists.newArrayList(table1, table2);
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdInc2 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdInc2));
+    // #6 - insert some data to table table1
+    t1 = (HTable) conn.getTable(table1);
+    for (int i = NB_ROWS_IN_BATCH; i < 2 * NB_ROWS_IN_BATCH; i++) {
+      p1 = new Put(Bytes.toBytes("row-t1" + i));
+      p1.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t1.put(p1);
+    }
+    // #7 - incremental backup for table1
+    tables = Lists.newArrayList(table1);
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdInc3 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdInc3));
+    // #8 - insert some data to table table2
+    t2 = (HTable) conn.getTable(table2);
+    for (int i = NB_ROWS_IN_BATCH; i < 2 * NB_ROWS_IN_BATCH; i++) {
+      p2 = new Put(Bytes.toBytes("row-t1" + i));
+      p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t2.put(p2);
+    }
+    // #9 - incremental backup for table1, table2
+    tables = Lists.newArrayList(table1, table2);
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdInc4 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdInc4));
+    // #10 full backup for table3
+    tables = Lists.newArrayList(table3);
+    request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    String backupIdFull2 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdFull2));
+    // #11 - incremental backup for table3
+    tables = Lists.newArrayList(table3);
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdInc5 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdInc5));
+    LOG.error("Delete backupIdInc2");
+    client.deleteBackups(new String[] { backupIdInc2 });
+    LOG.error("Delete backupIdInc2 done");
+    List<BackupInfo> list = client.getHistory(100);
+    // First check number of backup images before and after
+    assertEquals(4, list.size());
+    // then verify that no backupIdInc2,3,4
+    Set<String> ids = new HashSet<String>();
+    ids.add(backupIdInc2);
+    ids.add(backupIdInc3);
+    ids.add(backupIdInc4);
+    for (BackupInfo info : list) {
+      String backupId = info.getBackupId();
+      if (ids.contains(backupId)) {
+        assertTrue(false);
+      }
+    }
+    // Verify that backupInc5 contains only table3
+    boolean found = false;
+    for (BackupInfo info : list) {
+      String backupId = info.getBackupId();
+      if (backupId.equals(backupIdInc5)) {
+        assertTrue(info.getTables().size() == 1);
+        assertEquals(table3, info.getTableNames().get(0));
+        found = true;
+      }
+    }
+    assertTrue(found);
+    admin.close();
+    conn.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
new file mode 100644
index 0000000..740c396
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
@@ -0,0 +1,148 @@
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.List;
+
+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.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupShowHistory extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupShowHistory.class);
+
+  private boolean findBackup(List<BackupInfo> history, String backupId) {
+    assertTrue(history.size() > 0);
+    boolean success = false;
+    for (BackupInfo info : history) {
+      if (info.getBackupId().equals(backupId)) {
+        success = true;
+        break;
+      }
+    }
+    return success;
+  }
+
+  /**
+   * Verify that full backup is created on a single table with data correctly. Verify that history
+   * works as expected
+   * @throws Exception
+   */
+  @Test
+  public void testBackupHistory() throws Exception {
+
+    LOG.info("test backup history on a single table with data");
+
+    List<TableName> tableList = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tableList);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+
+    List<BackupInfo> history = getBackupAdmin().getHistory(10);
+    assertTrue(findBackup(history, backupId));
+    BackupInfo.Filter nullFilter = new BackupInfo.Filter() {
+      @Override
+      public boolean apply(BackupInfo info) {
+        return true;
+      }
+    };
+    history = BackupUtils.getHistory(conf1, 10, new Path(BACKUP_ROOT_DIR), nullFilter);
+    assertTrue(findBackup(history, backupId));
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+
+    String[] args = new String[] { "history", "-n", "10", "-p", BACKUP_ROOT_DIR };
+    // Run backup
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    LOG.info("show_history");
+    String output = baos.toString();
+    LOG.info(output);
+    baos.close();
+    assertTrue(output.indexOf(backupId) > 0);
+
+    tableList = Lists.newArrayList(table2);
+    String backupId2 = fullTableBackup(tableList);
+    assertTrue(checkSucceeded(backupId2));
+    LOG.info("backup complete: " + table2);
+    BackupInfo.Filter tableNameFilter = new BackupInfo.Filter() {
+      @Override
+      public boolean apply(BackupInfo image) {
+        if (table1 == null) return true;
+        List<TableName> names = image.getTableNames();
+        return names.contains(table1);
+      }
+    };
+    BackupInfo.Filter tableSetFilter = new BackupInfo.Filter() {
+      @Override
+      public boolean apply(BackupInfo info) {
+        String backupId = info.getBackupId();
+        return backupId.startsWith("backup");
+      }
+    };
+
+    history = getBackupAdmin().getHistory(10, tableNameFilter, tableSetFilter);
+    assertTrue(history.size() > 0);
+    boolean success = true;
+    for (BackupInfo info : history) {
+      if (!info.getTableNames().contains(table1)) {
+        success = false;
+        break;
+      }
+    }
+    assertTrue(success);
+
+    history =
+        BackupUtils.getHistory(conf1, 10, new Path(BACKUP_ROOT_DIR), tableNameFilter,
+          tableSetFilter);
+    assertTrue(history.size() > 0);
+    success = true;
+    for (BackupInfo info : history) {
+      if (!info.getTableNames().contains(table1)) {
+        success = false;
+        break;
+      }
+    }
+    assertTrue(success);
+
+    args =
+        new String[] { "history", "-n", "10", "-p", BACKUP_ROOT_DIR,
+          "-t", "table1", "-s", "backup" };
+    // Run backup
+    ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    LOG.info("show_history");
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
new file mode 100644
index 0000000..a904d9b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
@@ -0,0 +1,96 @@
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupStatusProgress extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupStatusProgress.class);
+
+  /**
+   * Verify that full backup is created on a single table with data correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testBackupStatusProgress() throws Exception {
+
+    LOG.info("test backup status/progress on a single table with data");
+
+    List<TableName> tableList = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tableList);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+
+    BackupInfo info = getBackupAdmin().getBackupInfo(backupId);
+    assertTrue(info.getState() == BackupState.COMPLETE);
+
+    LOG.debug(info.getShortDescription());
+    assertTrue(info.getProgress() > 0);
+
+  }
+
+  @Test
+  public void testBackupStatusProgressCommand() throws Exception {
+
+    LOG.info("test backup status/progress on a single table with data: command-line");
+
+    List<TableName> tableList = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tableList);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+
+    String[] args = new String[] { "describe", backupId };
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    String responce = baos.toString();
+    assertTrue(responce.indexOf(backupId) > 0);
+    assertTrue(responce.indexOf("COMPLETE") > 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+
+    args = new String[] { "progress", backupId };
+    ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    responce = baos.toString();
+    assertTrue(responce.indexOf(backupId) >= 0);
+    assertTrue(responce.indexOf("progress") > 0);
+    assertTrue(responce.indexOf("100") > 0);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..5814d87
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.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 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.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test cases for backup system table API
+ */
+@Category(MediumTests.class)
+public class TestBackupSystemTable {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  protected static Configuration conf = UTIL.getConfiguration();
+  protected static MiniHBaseCluster cluster;
+  protected static Connection conn;
+  protected BackupSystemTable table;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true);
+    BackupManager.decorateMasterConfiguration(conf);
+    BackupManager.decorateRegionServerConfiguration(conf);
+    cluster = UTIL.startMiniCluster();
+    conn = UTIL.getConnection();
+  }
+
+  @Before
+  public void before() throws IOException {
+    table = new BackupSystemTable(conn);
+  }
+
+  @After
+  public void after() {
+    if (table != null) {
+      table.close();
+    }
+
+  }
+
+  @Test
+  public void testUpdateReadDeleteBackupStatus() throws IOException {
+    BackupInfo ctx = createBackupInfo();
+    table.updateBackupInfo(ctx);
+    BackupInfo readCtx = table.readBackupInfo(ctx.getBackupId());
+    assertTrue(compare(ctx, readCtx));
+    // try fake backup id
+    readCtx = table.readBackupInfo("fake");
+    assertNull(readCtx);
+    // delete backup info
+    table.deleteBackupInfo(ctx.getBackupId());
+    readCtx = table.readBackupInfo(ctx.getBackupId());
+    assertNull(readCtx);
+    cleanBackupTable();
+  }
+
+  @Test
+  public void testWriteReadBackupStartCode() throws IOException {
+    Long code = 100L;
+    table.writeBackupStartCode(code, "root");
+    String readCode = table.readBackupStartCode("root");
+    assertEquals(code, new Long(Long.parseLong(readCode)));
+    cleanBackupTable();
+  }
+
+  private void cleanBackupTable() throws IOException {
+    Admin admin = UTIL.getHBaseAdmin();
+    admin.disableTable(BackupSystemTable.getTableName(conf));
+    admin.truncateTable(BackupSystemTable.getTableName(conf), true);
+    if (admin.isTableDisabled(BackupSystemTable.getTableName(conf))) {
+      admin.enableTable(BackupSystemTable.getTableName(conf));
+    }
+  }
+
+  @Test
+  public void testBackupHistory() throws IOException {
+    int n = 10;
+    List<BackupInfo> list = createBackupInfoList(n);
+
+    // Load data
+    for (BackupInfo bc : list) {
+      // Make sure we set right status
+      bc.setState(BackupState.COMPLETE);
+      table.updateBackupInfo(bc);
+    }
+
+    // Reverse list for comparison
+    Collections.reverse(list);
+    List<BackupInfo> history = table.getBackupHistory();
+    assertTrue(history.size() == n);
+
+    for (int i = 0; i < n; i++) {
+      BackupInfo ctx = list.get(i);
+      BackupInfo data = history.get(i);
+      assertTrue(compare(ctx, data));
+    }
+
+    cleanBackupTable();
+
+  }
+
+  @Test
+  public void testBackupDelete() throws IOException {
+
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+
+      int n = 10;
+      List<BackupInfo> list = createBackupInfoList(n);
+
+      // Load data
+      for (BackupInfo bc : list) {
+        // Make sure we set right status
+        bc.setState(BackupState.COMPLETE);
+        table.updateBackupInfo(bc);
+      }
+
+      // Verify exists
+      for (BackupInfo bc : list) {
+        assertNotNull(table.readBackupInfo(bc.getBackupId()));
+      }
+
+      // Delete all
+      for (BackupInfo bc : list) {
+        table.deleteBackupInfo(bc.getBackupId());
+      }
+
+      // Verify do not exists
+      for (BackupInfo bc : list) {
+        assertNull(table.readBackupInfo(bc.getBackupId()));
+      }
+
+      cleanBackupTable();
+    }
+
+  }
+
+  @Test
+  public void testRegionServerLastLogRollResults() throws IOException {
+    String[] servers = new String[] { "server1", "server2", "server3" };
+    Long[] timestamps = new Long[] { 100L, 102L, 107L };
+
+    for (int i = 0; i < servers.length; i++) {
+      table.writeRegionServerLastLogRollResult(servers[i], timestamps[i], "root");
+    }
+
+    HashMap<String, Long> result = table.readRegionServerLastLogRollResult("root");
+    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]);
+      Long ts1 = timestamps[i];
+      Long ts2 = result.get(keysAsArray[i]);
+      assertEquals(ts1, ts2);
+    }
+
+    cleanBackupTable();
+  }
+
+  @Test
+  public void testIncrementalBackupTableSet() throws IOException {
+    TreeSet<TableName> tables1 = new TreeSet<>();
+
+    tables1.add(TableName.valueOf("t1"));
+    tables1.add(TableName.valueOf("t2"));
+    tables1.add(TableName.valueOf("t3"));
+
+    TreeSet<TableName> tables2 = new TreeSet<>();
+
+    tables2.add(TableName.valueOf("t3"));
+    tables2.add(TableName.valueOf("t4"));
+    tables2.add(TableName.valueOf("t5"));
+
+    table.addIncrementalBackupTableSet(tables1, "root");
+    BackupSystemTable table = new BackupSystemTable(conn);
+    TreeSet<TableName> res1 = (TreeSet<TableName>) table.getIncrementalBackupTableSet("root");
+    assertTrue(tables1.size() == res1.size());
+    Iterator<TableName> desc1 = tables1.descendingIterator();
+    Iterator<TableName> desc2 = res1.descendingIterator();
+    while (desc1.hasNext()) {
+      assertEquals(desc1.next(), desc2.next());
+    }
+
+    table.addIncrementalBackupTableSet(tables2, "root");
+    TreeSet<TableName> res2 = (TreeSet<TableName>) table.getIncrementalBackupTableSet("root");
+    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 {
+    TreeSet<TableName> tables = new TreeSet<>();
+
+    tables.add(TableName.valueOf("t1"));
+    tables.add(TableName.valueOf("t2"));
+    tables.add(TableName.valueOf("t3"));
+
+    HashMap<String, Long> rsTimestampMap = new HashMap<String, Long>();
+
+    rsTimestampMap.put("rs1:100", 100L);
+    rsTimestampMap.put("rs2:100", 101L);
+    rsTimestampMap.put("rs3:100", 103L);
+
+    table.writeRegionServerLogTimestamp(tables, rsTimestampMap, "root");
+
+    HashMap<TableName, HashMap<String, Long>> result = table.readLogTimestampMap("root");
+
+    assertTrue(tables.size() == result.size());
+
+    for (TableName t : tables) {
+      HashMap<String, Long> rstm = result.get(t);
+      assertNotNull(rstm);
+      assertEquals(rstm.get("rs1:100"), new Long(100L));
+      assertEquals(rstm.get("rs2:100"), new Long(101L));
+      assertEquals(rstm.get("rs3:100"), new Long(103L));
+    }
+
+    Set<TableName> tables1 = new TreeSet<>();
+
+    tables1.add(TableName.valueOf("t3"));
+    tables1.add(TableName.valueOf("t4"));
+    tables1.add(TableName.valueOf("t5"));
+
+    HashMap<String, Long> rsTimestampMap1 = new HashMap<String, Long>();
+
+    rsTimestampMap1.put("rs1:100", 200L);
+    rsTimestampMap1.put("rs2:100", 201L);
+    rsTimestampMap1.put("rs3:100", 203L);
+
+    table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1, "root");
+
+    result = table.readLogTimestampMap("root");
+
+    assertTrue(5 == result.size());
+
+    for (TableName t : tables) {
+      HashMap<String, Long> rstm = result.get(t);
+      assertNotNull(rstm);
+      if (t.equals(TableName.valueOf("t3")) == false) {
+        assertEquals(rstm.get("rs1:100"), new Long(100L));
+        assertEquals(rstm.get("rs2:100"), new Long(101L));
+        assertEquals(rstm.get("rs3:100"), new Long(103L));
+      } else {
+        assertEquals(rstm.get("rs1:100"), new Long(200L));
+        assertEquals(rstm.get("rs2:100"), new Long(201L));
+        assertEquals(rstm.get("rs3:100"), new Long(203L));
+      }
+    }
+
+    for (TableName t : tables1) {
+      HashMap<String, Long> rstm = result.get(t);
+      assertNotNull(rstm);
+      assertEquals(rstm.get("rs1:100"), new Long(200L));
+      assertEquals(rstm.get("rs2:100"), new Long(201L));
+      assertEquals(rstm.get("rs3:100"), new Long(203L));
+    }
+
+    cleanBackupTable();
+
+  }
+
+  @Test
+  public void testAddWALFiles() throws IOException {
+    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", "root");
+
+    assertTrue(table.isWALFileDeletable(files.get(0)));
+    assertTrue(table.isWALFileDeletable(files.get(1)));
+    assertTrue(table.isWALFileDeletable(files.get(2)));
+    assertFalse(table.isWALFileDeletable(newFile));
+
+    cleanBackupTable();
+  }
+
+  /**
+   * Backup set tests
+   */
+
+  @Test
+  public void testBackupSetAddNotExists() throws IOException {
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+
+      String[] tables = new String[] { "table1", "table2", "table3" };
+      String setName = "name";
+      table.addToBackupSet(setName, tables);
+      List<TableName> tnames = table.describeBackupSet(setName);
+      assertTrue(tnames != null);
+      assertTrue(tnames.size() == tables.length);
+      for (int i = 0; i < tnames.size(); i++) {
+        assertTrue(tnames.get(i).getNameAsString().equals(tables[i]));
+      }
+      cleanBackupTable();
+    }
+
+  }
+
+  @Test
+  public void testBackupSetAddExists() throws IOException {
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+
+      String[] tables = new String[] { "table1", "table2", "table3" };
+      String setName = "name";
+      table.addToBackupSet(setName, tables);
+      String[] addTables = new String[] { "table4", "table5", "table6" };
+      table.addToBackupSet(setName, addTables);
+
+      List<TableName> tnames = table.describeBackupSet(setName);
+      assertTrue(tnames != null);
+      assertTrue(tnames.size() == tables.length + addTables.length);
+      for (int i = 0; i < tnames.size(); i++) {
+        assertTrue(tnames.get(i).getNameAsString().equals("table" + (i + 1)));
+      }
+      cleanBackupTable();
+    }
+  }
+
+  @Test
+  public void testBackupSetAddExistsIntersects() throws IOException {
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+
+      String[] tables = new String[] { "table1", "table2", "table3" };
+      String setName = "name";
+      table.addToBackupSet(setName, tables);
+      String[] addTables = new String[] { "table3", "table4", "table5", "table6" };
+      table.addToBackupSet(setName, addTables);
+
+      List<TableName> tnames = table.describeBackupSet(setName);
+      assertTrue(tnames != null);
+      assertTrue(tnames.size() == tables.length + addTables.length - 1);
+      for (int i = 0; i < tnames.size(); i++) {
+        assertTrue(tnames.get(i).getNameAsString().equals("table" + (i + 1)));
+      }
+      cleanBackupTable();
+    }
+  }
+
+  @Test
+  public void testBackupSetRemoveSomeNotExists() throws IOException {
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+
+      String[] tables = new String[] { "table1", "table2", "table3", "table4" };
+      String setName = "name";
+      table.addToBackupSet(setName, tables);
+      String[] removeTables = new String[] { "table4", "table5", "table6" };
+      table.removeFromBackupSet(setName, removeTables);
+
+      List<TableName> tnames = table.describeBackupSet(setName);
+      assertTrue(tnames != null);
+      assertTrue(tnames.size() == tables.length - 1);
+      for (int i = 0; i < tnames.size(); i++) {
+        assertTrue(tnames.get(i).getNameAsString().equals("table" + (i + 1)));
+      }
+      cleanBackupTable();
+    }
+  }
+
+  @Test
+  public void testBackupSetRemove() throws IOException {
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+
+      String[] tables = new String[] { "table1", "table2", "table3", "table4" };
+      String setName = "name";
+      table.addToBackupSet(setName, tables);
+      String[] removeTables = new String[] { "table4", "table3" };
+      table.removeFromBackupSet(setName, removeTables);
+
+      List<TableName> tnames = table.describeBackupSet(setName);
+      assertTrue(tnames != null);
+      assertTrue(tnames.size() == tables.length - 2);
+      for (int i = 0; i < tnames.size(); i++) {
+        assertTrue(tnames.get(i).getNameAsString().equals("table" + (i + 1)));
+      }
+      cleanBackupTable();
+    }
+  }
+
+  @Test
+  public void testBackupSetDelete() throws IOException {
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+
+      String[] tables = new String[] { "table1", "table2", "table3", "table4" };
+      String setName = "name";
+      table.addToBackupSet(setName, tables);
+      table.deleteBackupSet(setName);
+
+      List<TableName> tnames = table.describeBackupSet(setName);
+      assertTrue(tnames == null);
+      cleanBackupTable();
+    }
+  }
+
+  @Test
+  public void testBackupSetList() throws IOException {
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+
+      String[] tables = new String[] { "table1", "table2", "table3", "table4" };
+      String setName1 = "name1";
+      String setName2 = "name2";
+      table.addToBackupSet(setName1, tables);
+      table.addToBackupSet(setName2, tables);
+
+      List<String> list = table.listBackupSets();
+
+      assertTrue(list.size() == 2);
+      assertTrue(list.get(0).equals(setName1));
+      assertTrue(list.get(1).equals(setName2));
+
+      cleanBackupTable();
+    }
+  }
+
+  private boolean compare(BackupInfo one, BackupInfo two) {
+    return one.getBackupId().equals(two.getBackupId()) && one.getType().equals(two.getType())
+        && one.getBackupRootDir().equals(two.getBackupRootDir())
+        && one.getStartTs() == two.getStartTs() && one.getCompleteTs() == two.getCompleteTs();
+  }
+
+  private BackupInfo createBackupInfo() {
+
+    BackupInfo ctxt =
+        new BackupInfo("backup_" + System.nanoTime(), BackupType.FULL, new TableName[] {
+            TableName.valueOf("t1"), TableName.valueOf("t2"), TableName.valueOf("t3") },
+            "/hbase/backup");
+    ctxt.setStartTs(System.currentTimeMillis());
+    ctxt.setCompleteTs(System.currentTimeMillis() + 1);
+    return ctxt;
+  }
+
+  private List<BackupInfo> createBackupInfoList(int size) {
+    List<BackupInfo> list = new ArrayList<BackupInfo>();
+    for (int i = 0; i < size; i++) {
+      list.add(createBackupInfo());
+      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/3aaea8e0/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..124d19f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
@@ -0,0 +1,59 @@
+/**
+ * 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.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+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);
+
+  @Test
+  public void testFullBackupMultipleCommand() throws Exception {
+    LOG.info("test full backup on a multiple tables with data: command-line");
+    try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
+      int before = table.getBackupHistory().size();
+      String[] args =
+          new String[] { "create", "full", BACKUP_ROOT_DIR, "-t",
+              table1.getNameAsString() + "," + table2.getNameAsString() };
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      List<BackupInfo> backups = table.getBackupHistory();
+      int after = table.getBackupHistory().size();
+      assertTrue(after == before + 1);
+      for (BackupInfo data : backups) {
+        String backupId = data.getBackupId();
+        assertTrue(checkSucceeded(backupId));
+      }
+    }
+    LOG.info("backup complete");
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
new file mode 100644
index 0000000..4dc894b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
@@ -0,0 +1,103 @@
+/**
+ * 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.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestFullBackupSet extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestFullBackupSet.class);
+
+  /**
+   * Verify that full backup is created on a single table with data correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupSetExist() throws Exception {
+
+    LOG.info("Test full backup, backup set exists");
+
+    // Create set
+    try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
+      String name = "name";
+      table.addToBackupSet(name, new String[] { table1.getNameAsString() });
+      List<TableName> names = table.describeBackupSet(name);
+
+      assertNotNull(names);
+      assertTrue(names.size() == 1);
+      assertTrue(names.get(0).equals(table1));
+
+      String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name };
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      List<BackupInfo> backups = table.getBackupHistory();
+      assertTrue(backups.size() == 1);
+      String backupId = backups.get(0).getBackupId();
+      assertTrue(checkSucceeded(backupId));
+
+      LOG.info("backup complete");
+
+      // Restore from set into other table
+      args =
+          new String[] { BACKUP_ROOT_DIR, backupId, "-s", name, "-m",
+              table1_restore.getNameAsString(), "-o" };
+      // Run backup
+      ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+      assertTrue(ret == 0);
+      HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+      assertTrue(hba.tableExists(table1_restore));
+      // Verify number of rows in both tables
+      assertEquals(TEST_UTIL.countRows(table1), TEST_UTIL.countRows(table1_restore));
+      TEST_UTIL.deleteTable(table1_restore);
+      LOG.info("restore into other table is complete");
+      hba.close();
+
+    }
+
+  }
+
+  @Test
+  public void testFullBackupSetDoesNotExist() throws Exception {
+
+    LOG.info("test full backup, backup set does not exist");
+    String name = "name1";
+    String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name };
+    // Run backup
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret != 0);
+
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
new file mode 100644
index 0000000..6b007f9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
@@ -0,0 +1,128 @@
+/**
+ * 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.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(LargeTests.class)
+public class TestFullBackupSetRestoreSet extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestFullBackupSetRestoreSet.class);
+
+  @Test
+  public void testFullRestoreSetToOtherTable() throws Exception {
+
+    LOG.info("Test full restore set");
+
+    // Create set
+    try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
+      String name = "name";
+      table.addToBackupSet(name, new String[] { table1.getNameAsString() });
+      List<TableName> names = table.describeBackupSet(name);
+
+      assertNotNull(names);
+      assertTrue(names.size() == 1);
+      assertTrue(names.get(0).equals(table1));
+
+      String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name };
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      List<BackupInfo> backups = table.getBackupHistory();
+      assertTrue(backups.size() == 1);
+      String backupId = backups.get(0).getBackupId();
+      assertTrue(checkSucceeded(backupId));
+
+      LOG.info("backup complete");
+
+      // Restore from set into other table
+      args =
+          new String[] { BACKUP_ROOT_DIR, backupId, "-s", name, "-m",
+              table1_restore.getNameAsString(), "-o" };
+      // Run backup
+      ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+      assertTrue(ret == 0);
+      HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+      assertTrue(hba.tableExists(table1_restore));
+      // Verify number of rows in both tables
+      assertEquals(TEST_UTIL.countRows(table1), TEST_UTIL.countRows(table1_restore));
+      TEST_UTIL.deleteTable(table1_restore);
+      LOG.info("restore into other table is complete");
+      hba.close();
+    }
+  }
+
+  @Test
+  public void testFullRestoreSetToSameTable() throws Exception {
+
+    LOG.info("Test full restore set to same table");
+
+    // Create set
+    try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
+      String name = "name1";
+      table.addToBackupSet(name, new String[] { table1.getNameAsString() });
+      List<TableName> names = table.describeBackupSet(name);
+
+      assertNotNull(names);
+      assertTrue(names.size() == 1);
+      assertTrue(names.get(0).equals(table1));
+
+      String[] args = new String[] { "create", "full", BACKUP_ROOT_DIR, "-s", name };
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      List<BackupInfo> backups = table.getBackupHistory();
+      String backupId = backups.get(0).getBackupId();
+      assertTrue(checkSucceeded(backupId));
+
+      LOG.info("backup complete");
+      int count = TEST_UTIL.countRows(table1);
+      TEST_UTIL.deleteTable(table1);
+
+      // Restore from set into other table
+      args = new String[] { BACKUP_ROOT_DIR, backupId, "-s", name, "-o" };
+      // Run backup
+      ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+      assertTrue(ret == 0);
+      HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+      assertTrue(hba.tableExists(table1));
+      // Verify number of rows in both tables
+      assertEquals(count, TEST_UTIL.countRows(table1));
+      LOG.info("restore into same table is complete");
+      hba.close();
+
+    }
+
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..61bad68
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
@@ -0,0 +1,345 @@
+/**
+ * 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.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@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");
+
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    LOG.info("backup complete");
+
+    TableName[] tableset = new TableName[] { table1 };
+    TableName[] tablemap = new TableName[] { table1_restore };
+    BackupAdmin client = getBackupAdmin();
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
+      tableset, tablemap, false));
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table1_restore));
+    TEST_UTIL.deleteTable(table1_restore);
+    hba.close();
+  }
+
+
+
+  @Test
+  public void testFullRestoreSingleCommand() throws Exception {
+
+    LOG.info("test full restore on a single table empty table: command-line");
+
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tables);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+    // restore <backup_root_path> <backup_id> <tables> [tableMapping]
+    String[] args =
+        new String[] { BACKUP_ROOT_DIR, backupId, "-t", table1.getNameAsString(), "-m",
+            table1_restore.getNameAsString() };
+    // Run backup
+    int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+
+    assertTrue(ret == 0);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table1_restore));
+    TEST_UTIL.deleteTable(table1_restore);
+    hba.close();
+  }
+
+  @Test
+  public void testFullRestoreCheckCommand() throws Exception {
+
+    LOG.info("test full restore on a single table: command-line, check only");
+
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tables);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+    // restore <backup_root_path> <backup_id> <tables> [tableMapping]
+    String[] args =
+        new String[] { BACKUP_ROOT_DIR, backupId, "-t", table1.getNameAsString(), "-m",
+            table1_restore.getNameAsString(), "-c" };
+    // Run backup
+    int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+    assertTrue(ret == 0);
+    //Verify that table has not been restored
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertFalse(hba.tableExists(table1_restore));
+  }
+
+  /**
+   * 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");
+    List<TableName> tables = Lists.newArrayList(table2, table3);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    TableName[] restore_tableset = new TableName[] { table2, table3 };
+    TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
+    BackupAdmin client = getBackupAdmin();
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
+      restore_tableset, tablemap, false));
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table2_restore));
+    assertTrue(hba.tableExists(table3_restore));
+    TEST_UTIL.deleteTable(table2_restore);
+    TEST_UTIL.deleteTable(table3_restore);
+    hba.close();
+  }
+
+  /**
+   * Verify that multiple tables are restored to new tables.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreMultipleCommand() throws Exception {
+    LOG.info("create full backup image on multiple tables: command-line");
+    List<TableName> tables = Lists.newArrayList(table2, table3);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    TableName[] restore_tableset = new TableName[] { table2, table3 };
+    TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
+
+    // restore <backup_root_path> <backup_id> <tables> [tableMapping]
+    String[] args =
+        new String[] { BACKUP_ROOT_DIR, backupId, "-t", StringUtils.join(restore_tableset, ","),
+          "-m", StringUtils.join(tablemap, ",") };
+    // Run backup
+    int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+
+    assertTrue(ret == 0);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table2_restore));
+    assertTrue(hba.tableExists(table3_restore));
+    TEST_UTIL.deleteTable(table2_restore);
+    TEST_UTIL.deleteTable(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");
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    LOG.info("backup complete");
+
+    TableName[] tableset = new TableName[] { table1 };
+    BackupAdmin client = getBackupAdmin();
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
+      tableset, null, true));
+  }
+
+  /**
+   * Verify that a single table is restored using overwrite
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreSingleOverwriteCommand() throws Exception {
+
+    LOG.info("test full restore on a single table empty table: command-line");
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+    TableName[] tableset = new TableName[] { table1 };
+    // restore <backup_root_path> <backup_id> <tables> [tableMapping]
+    String[] args =
+        new String[] { BACKUP_ROOT_DIR, backupId, "-t", StringUtils.join(tableset, ","), "-o" };
+    // Run restore
+    int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+    assertTrue(ret == 0);
+
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table1));
+    hba.close();
+
+  }
+
+  /**
+   * 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");
+
+    List<TableName> tables = Lists.newArrayList(table2, table3);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    TableName[] restore_tableset = new TableName[] { table2, table3 };
+    BackupAdmin client = getBackupAdmin();
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
+      restore_tableset, null, true));
+  }
+
+  /**
+   * Verify that multiple tables are restored to new tables using overwrite.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreMultipleOverwriteCommand() throws Exception {
+    LOG.info("create full backup image on multiple tables: command-line");
+
+    List<TableName> tables = Lists.newArrayList(table2, table3);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    TableName[] restore_tableset = new TableName[] { table2, table3 };
+    // restore <backup_root_path> <backup_id> <tables> [tableMapping]
+    String[] args =
+        new String[] { BACKUP_ROOT_DIR, backupId, "-t",
+        StringUtils.join(restore_tableset, ","), "-o" };
+    // Run backup
+    int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+
+    assertTrue(ret == 0);
+    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hba.tableExists(table2));
+    assertTrue(hba.tableExists(table3));
+    hba.close();
+  }
+
+  /**
+   * 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");
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    LOG.info("backup complete");
+
+    TableName[] tableset = new TableName[] { TableName.valueOf("faketable") };
+    TableName[] tablemap = new TableName[] { table1_restore };
+    BackupAdmin client = getBackupAdmin();
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
+      tableset, tablemap, false));
+  }
+
+  /**
+   * Verify that restore fails on a single table that does not exist.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreSingleDNECommand() throws Exception {
+
+    LOG.info("test restore fails on a single table that does not exist: command-line");
+    List<TableName> tables = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    LOG.info("backup complete");
+
+    TableName[] tableset = new TableName[] { TableName.valueOf("faketable") };
+    TableName[] tablemap = new TableName[] { table1_restore };
+    String[] args =
+        new String[] { BACKUP_ROOT_DIR, backupId, StringUtils.join(tableset, ","), "-m",
+            StringUtils.join(tablemap, ",") };
+    // Run restore
+    int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+    assertTrue(ret != 0);
+
+  }
+
+  /**
+   * 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");
+
+    List<TableName> tables = Lists.newArrayList(table2, table3);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    TableName[] restore_tableset =
+        new TableName[] { TableName.valueOf("faketable1"), TableName.valueOf("faketable2") };
+    TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
+    BackupAdmin client = getBackupAdmin();
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
+      restore_tableset, tablemap, false));
+  }
+
+  /**
+   * Verify that restore fails on multiple tables that do not exist.
+   * @throws Exception
+   */
+  @Test
+  public void testFullRestoreMultipleDNECommand() throws Exception {
+
+    LOG.info("test restore fails on multiple tables that do not exist: command-line");
+
+    List<TableName> tables = Lists.newArrayList(table2, table3);
+    String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+
+    TableName[] restore_tableset =
+        new TableName[] { TableName.valueOf("faketable1"), TableName.valueOf("faketable2") };
+    TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
+    String[] args =
+        new String[] { BACKUP_ROOT_DIR, backupId, StringUtils.join(restore_tableset, ","), "-m",
+            StringUtils.join(tablemap, ",") };
+    // Run restore
+    int ret = ToolRunner.run(conf1, new RestoreDriver(), args);
+    assertTrue(ret != 0);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..77bed43
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
@@ -0,0 +1,200 @@
+/**
+ * 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.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+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.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+@RunWith(Parameterized.class)
+public class TestIncrementalBackup extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackup.class);
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    provider = "multiwal";
+    List<Object[]> params = new ArrayList<Object[]>();
+    params.add(new Object[] { Boolean.TRUE });
+    return params;
+  }
+
+  public TestIncrementalBackup(Boolean b) {
+  }
+
+  // implement all test cases in 1 test since incremental backup/restore has dependencies
+  @Test
+  public void TestIncBackupRestore() throws Exception {
+
+    int ADD_ROWS = 99;
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+
+    List<TableName> tables = Lists.newArrayList(table1, table2);
+    final byte[] fam3Name = Bytes.toBytes("f3");
+    table1Desc.addFamily(new HColumnDescriptor(fam3Name));
+    HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
+
+    Connection conn = ConnectionFactory.createConnection(conf1);
+    int NB_ROWS_FAM3 = 6;
+    insertIntoTable(conn, table1, fam3Name, 3, NB_ROWS_FAM3).close();
+
+    HBaseAdmin admin = null;
+    admin = (HBaseAdmin) conn.getAdmin();
+    BackupAdminImpl client = new BackupAdminImpl(conn);
+
+    BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    String backupIdFull = client.backupTables(request);
+
+    assertTrue(checkSucceeded(backupIdFull));
+
+    // #2 - insert some data to table
+    HTable t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS);
+    LOG.debug("writing " + ADD_ROWS + " rows to " + table1);
+
+    Assert.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_FAM3);
+    t1.close();
+    LOG.debug("written " + ADD_ROWS + " rows to " + table1);
+
+    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.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + 5);
+    t2.close();
+    LOG.debug("written " + 5 + " rows to " + table2);
+    // split table1
+    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
+    List<HRegion> regions = cluster.getRegions(table1);
+
+    byte[] name = regions.get(0).getRegionInfo().getRegionName();
+    long startSplitTime = EnvironmentEdgeManager.currentTime();
+    admin.splitRegion(name);
+
+    while (!admin.isTableAvailable(table1)) {
+      Thread.sleep(100);
+    }
+
+    long endSplitTime = EnvironmentEdgeManager.currentTime();
+
+    // split finished
+    LOG.debug("split finished in =" + (endSplitTime - startSplitTime));
+
+    // #3 - incremental backup for multiple tables
+    tables = Lists.newArrayList(table1, table2);
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdIncMultiple = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdIncMultiple));
+
+    // add column family f2 to table1
+    final byte[] fam2Name = Bytes.toBytes("f2");
+    table1Desc.addFamily(new HColumnDescriptor(fam2Name));
+    // drop column family f3
+    table1Desc.removeFamily(fam3Name);
+    HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
+
+    int NB_ROWS_FAM2 = 7;
+    HTable t3 = insertIntoTable(conn, table1, fam2Name, 2, NB_ROWS_FAM2);
+    t3.close();
+
+    // #3 - incremental backup for multiple tables
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdIncMultiple2 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdIncMultiple2));
+
+    // #4 - restore full backup for all tables, without overwrite
+    TableName[] tablesRestoreFull = new TableName[] { table1, table2 };
+
+    TableName[] tablesMapFull = new TableName[] { table1_restore, table2_restore };
+
+    LOG.debug("Restoring full " + backupIdFull);
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false,
+      tablesRestoreFull, tablesMapFull, false));
+
+    // #5.1 - check tables for full restore
+    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hAdmin.tableExists(table1_restore));
+    assertTrue(hAdmin.tableExists(table2_restore));
+
+    hAdmin.close();
+
+    // #5.2 - checking row count of tables for full restore
+    HTable hTable = (HTable) conn.getTable(table1_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + NB_ROWS_FAM3);
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(table2_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH);
+    hTable.close();
+
+    // #6 - restore incremental backup for multiple tables, with overwrite
+    TableName[] tablesRestoreIncMultiple = new TableName[] { table1, table2 };
+    TableName[] tablesMapIncMultiple = new TableName[] { table1_restore, table2_restore };
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple2,
+      false, tablesRestoreIncMultiple, tablesMapIncMultiple, true));
+
+    hTable = (HTable) conn.getTable(table1_restore);
+    LOG.debug("After incremental restore: " + hTable.getTableDescriptor());
+    LOG.debug("f1 has " + TEST_UTIL.countRows(hTable, famName) + " rows");
+    Assert.assertEquals(TEST_UTIL.countRows(hTable, famName), NB_ROWS_IN_BATCH + ADD_ROWS);
+    LOG.debug("f2 has " + TEST_UTIL.countRows(hTable, fam2Name) + " rows");
+    Assert.assertEquals(TEST_UTIL.countRows(hTable, fam2Name), NB_ROWS_FAM2);
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(table2_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + 5);
+    hTable.close();
+
+    admin.close();
+    conn.close();
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
new file mode 100644
index 0000000..59d0908
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.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;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+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.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+/**
+ * 1. Create table t1, t2
+ * 2. Load data to t1, t2
+ * 3 Full backup t1, t2
+ * 4 Delete t2
+ * 5 Load data to t1
+ * 6 Incremental backup t1
+ */
+@Category(LargeTests.class)
+public class TestIncrementalBackupDeleteTable extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupDeleteTable.class);
+
+  // implement all test cases in 1 test since incremental backup/restore has dependencies
+  @Test
+  public void TestIncBackupDeleteTable() throws Exception {
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+
+    List<TableName> tables = Lists.newArrayList(table1, table2);
+    HBaseAdmin admin = null;
+    Connection conn = ConnectionFactory.createConnection(conf1);
+    admin = (HBaseAdmin) conn.getAdmin();
+    BackupAdminImpl client = new BackupAdminImpl(conn);
+
+    BackupRequest request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+    String backupIdFull = client.backupTables(request);
+
+    assertTrue(checkSucceeded(backupIdFull));
+
+    // #2 - insert some data to table table1
+    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.assertEquals(TEST_UTIL.countRows(t1), NB_ROWS_IN_BATCH * 2);
+    t1.close();
+
+    // Delete table table2
+    admin.disableTable(table2);
+    admin.deleteTable(table2);
+
+    // #3 - incremental backup for table1
+    tables = Lists.newArrayList(table1);
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdIncMultiple = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdIncMultiple));
+
+    // #4 - restore full backup for all tables, without overwrite
+    TableName[] tablesRestoreFull = new TableName[] { table1, table2 };
+
+    TableName[] tablesMapFull = new TableName[] { table1_restore, table2_restore };
+
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false,
+      tablesRestoreFull, tablesMapFull, false));
+
+    // #5.1 - check tables for full restore
+    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+    assertTrue(hAdmin.tableExists(table1_restore));
+    assertTrue(hAdmin.tableExists(table2_restore));
+
+    // #5.2 - checking row count of tables for full restore
+    HTable hTable = (HTable) conn.getTable(table1_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH);
+    hTable.close();
+
+    hTable = (HTable) conn.getTable(table2_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH);
+    hTable.close();
+
+    // #6 - restore incremental backup for table1
+    TableName[] tablesRestoreIncMultiple = new TableName[] { table1 };
+    TableName[] tablesMapIncMultiple = new TableName[] { table1_restore };
+    client.restore(BackupUtils.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple,
+      false, tablesRestoreIncMultiple, tablesMapIncMultiple, true));
+
+    hTable = (HTable) conn.getTable(table1_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH * 2);
+    hTable.close();
+    admin.close();
+    conn.close();
+  }
+
+}


[09/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java
new file mode 100644
index 0000000..4cad101
--- /dev/null
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/BackupProtos.java
@@ -0,0 +1,7013 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: Backup.proto
+
+package org.apache.hadoop.hbase.shaded.protobuf.generated;
+
+public final class BackupProtos {
+  private BackupProtos() {}
+  public static void registerAllExtensions(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite registry) {
+  }
+
+  public static void registerAllExtensions(
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistry registry) {
+    registerAllExtensions(
+        (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
+  }
+  /**
+   * Protobuf enum {@code hbase.pb.BackupType}
+   */
+  public enum BackupType
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>FULL = 0;</code>
+     */
+    FULL(0),
+    /**
+     * <code>INCREMENTAL = 1;</code>
+     */
+    INCREMENTAL(1),
+    ;
+
+    /**
+     * <code>FULL = 0;</code>
+     */
+    public static final int FULL_VALUE = 0;
+    /**
+     * <code>INCREMENTAL = 1;</code>
+     */
+    public static final int INCREMENTAL_VALUE = 1;
+
+
+    public final int getNumber() {
+      return value;
+    }
+
+    /**
+     * @deprecated Use {@link #forNumber(int)} instead.
+     */
+    @java.lang.Deprecated
+    public static BackupType valueOf(int value) {
+      return forNumber(value);
+    }
+
+    public static BackupType forNumber(int value) {
+      switch (value) {
+        case 0: return FULL;
+        case 1: return INCREMENTAL;
+        default: return null;
+      }
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<BackupType>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+        BackupType> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<BackupType>() {
+            public BackupType findValueByNumber(int number) {
+              return BackupType.forNumber(number);
+            }
+          };
+
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(ordinal());
+    }
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.getDescriptor().getEnumTypes().get(0);
+    }
+
+    private static final BackupType[] VALUES = values();
+
+    public static BackupType valueOf(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int value;
+
+    private BackupType(int value) {
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.BackupType)
+  }
+
+  public interface ServerTimestampOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.ServerTimestamp)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+     */
+    boolean hasServerName();
+    /**
+     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServerName();
+    /**
+     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder();
+
+    /**
+     * <code>optional uint64 timestamp = 2;</code>
+     */
+    boolean hasTimestamp();
+    /**
+     * <code>optional uint64 timestamp = 2;</code>
+     */
+    long getTimestamp();
+  }
+  /**
+   * <pre>
+   **
+   * ServerTimestamp keeps last WAL roll time per Region Server
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.ServerTimestamp}
+   */
+  public  static final class ServerTimestamp extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.ServerTimestamp)
+      ServerTimestampOrBuilder {
+    // Use ServerTimestamp.newBuilder() to construct.
+    private ServerTimestamp(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private ServerTimestamp() {
+      timestamp_ = 0L;
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ServerTimestamp(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = serverName_.toBuilder();
+              }
+              serverName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(serverName_);
+                serverName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              timestamp_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int SERVER_NAME_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName serverName_;
+    /**
+     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+     */
+    public boolean hasServerName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServerName() {
+      return serverName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : serverName_;
+    }
+    /**
+     * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
+      return serverName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : serverName_;
+    }
+
+    public static final int TIMESTAMP_FIELD_NUMBER = 2;
+    private long timestamp_;
+    /**
+     * <code>optional uint64 timestamp = 2;</code>
+     */
+    public boolean hasTimestamp() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional uint64 timestamp = 2;</code>
+     */
+    public long getTimestamp() {
+      return timestamp_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (hasServerName()) {
+        if (!getServerName().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, getServerName());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, timestamp_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getServerName());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, timestamp_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp other = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp) obj;
+
+      boolean result = true;
+      result = result && (hasServerName() == other.hasServerName());
+      if (hasServerName()) {
+        result = result && getServerName()
+            .equals(other.getServerName());
+      }
+      result = result && (hasTimestamp() == other.hasTimestamp());
+      if (hasTimestamp()) {
+        result = result && (getTimestamp()
+            == other.getTimestamp());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasServerName()) {
+        hash = (37 * hash) + SERVER_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getServerName().hashCode();
+      }
+      if (hasTimestamp()) {
+        hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getTimestamp());
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     **
+     * ServerTimestamp keeps last WAL roll time per Region Server
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.ServerTimestamp}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.ServerTimestamp)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getServerNameFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (serverNameBuilder_ == null) {
+          serverName_ = null;
+        } else {
+          serverNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        timestamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_ServerTimestamp_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp result = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (serverNameBuilder_ == null) {
+          result.serverName_ = serverName_;
+        } else {
+          result.serverName_ = serverNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.timestamp_ = timestamp_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance()) return this;
+        if (other.hasServerName()) {
+          mergeServerName(other.getServerName());
+        }
+        if (other.hasTimestamp()) {
+          setTimestamp(other.getTimestamp());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (hasServerName()) {
+          if (!getServerName().isInitialized()) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName serverName_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> serverNameBuilder_;
+      /**
+       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public boolean hasServerName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName getServerName() {
+        if (serverNameBuilder_ == null) {
+          return serverName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : serverName_;
+        } else {
+          return serverNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public Builder setServerName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) {
+        if (serverNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          serverName_ = value;
+          onChanged();
+        } else {
+          serverNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public Builder setServerName(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) {
+        if (serverNameBuilder_ == null) {
+          serverName_ = builderForValue.build();
+          onChanged();
+        } else {
+          serverNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public Builder mergeServerName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName value) {
+        if (serverNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              serverName_ != null &&
+              serverName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) {
+            serverName_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.newBuilder(serverName_).mergeFrom(value).buildPartial();
+          } else {
+            serverName_ = value;
+          }
+          onChanged();
+        } else {
+          serverNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public Builder clearServerName() {
+        if (serverNameBuilder_ == null) {
+          serverName_ = null;
+          onChanged();
+        } else {
+          serverNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder getServerNameBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getServerNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerNameOrBuilder() {
+        if (serverNameBuilder_ != null) {
+          return serverNameBuilder_.getMessageOrBuilder();
+        } else {
+          return serverName_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance() : serverName_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.ServerName server_name = 1;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
+          getServerNameFieldBuilder() {
+        if (serverNameBuilder_ == null) {
+          serverNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>(
+                  getServerName(),
+                  getParentForChildren(),
+                  isClean());
+          serverName_ = null;
+        }
+        return serverNameBuilder_;
+      }
+
+      private long timestamp_ ;
+      /**
+       * <code>optional uint64 timestamp = 2;</code>
+       */
+      public boolean hasTimestamp() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional uint64 timestamp = 2;</code>
+       */
+      public long getTimestamp() {
+        return timestamp_;
+      }
+      /**
+       * <code>optional uint64 timestamp = 2;</code>
+       */
+      public Builder setTimestamp(long value) {
+        bitField0_ |= 0x00000002;
+        timestamp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 timestamp = 2;</code>
+       */
+      public Builder clearTimestamp() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        timestamp_ = 0L;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.ServerTimestamp)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ServerTimestamp)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ServerTimestamp>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<ServerTimestamp>() {
+      public ServerTimestamp parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new ServerTimestamp(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ServerTimestamp> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<ServerTimestamp> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface TableServerTimestampOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.TableServerTimestamp)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp> 
+        getServerTimestampList();
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getServerTimestamp(int index);
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    int getServerTimestampCount();
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder> 
+        getServerTimestampOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder getServerTimestampOrBuilder(
+        int index);
+  }
+  /**
+   * <pre>
+   **
+   *  TableServerTimestamp keeps last WAL roll time per Region Server &amp; Table
+   *  Each table have different last WAL roll time stamps across cluster, on every RS
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.TableServerTimestamp}
+   */
+  public  static final class TableServerTimestamp extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.TableServerTimestamp)
+      TableServerTimestampOrBuilder {
+    // Use TableServerTimestamp.newBuilder() to construct.
+    private TableServerTimestamp(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private TableServerTimestamp() {
+      serverTimestamp_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private TableServerTimestamp(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                serverTimestamp_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              serverTimestamp_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          serverTimestamp_ = java.util.Collections.unmodifiableList(serverTimestamp_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int TABLE_NAME_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+    }
+
+    public static final int SERVER_TIMESTAMP_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp> serverTimestamp_;
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp> getServerTimestampList() {
+      return serverTimestamp_;
+    }
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder> 
+        getServerTimestampOrBuilderList() {
+      return serverTimestamp_;
+    }
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    public int getServerTimestampCount() {
+      return serverTimestamp_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getServerTimestamp(int index) {
+      return serverTimestamp_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder getServerTimestampOrBuilder(
+        int index) {
+      return serverTimestamp_.get(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (hasTableName()) {
+        if (!getTableName().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getServerTimestampCount(); i++) {
+        if (!getServerTimestamp(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, getTableName());
+      }
+      for (int i = 0; i < serverTimestamp_.size(); i++) {
+        output.writeMessage(2, serverTimestamp_.get(i));
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getTableName());
+      }
+      for (int i = 0; i < serverTimestamp_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, serverTimestamp_.get(i));
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp other = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp) obj;
+
+      boolean result = true;
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && getServerTimestampList()
+          .equals(other.getServerTimestampList());
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (getServerTimestampCount() > 0) {
+        hash = (37 * hash) + SERVER_TIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + getServerTimestampList().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     **
+     *  TableServerTimestamp keeps last WAL roll time per Region Server &amp; Table
+     *  Each table have different last WAL roll time stamps across cluster, on every RS
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.TableServerTimestamp}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.TableServerTimestamp)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.class, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getTableNameFieldBuilder();
+          getServerTimestampFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (tableNameBuilder_ == null) {
+          tableName_ = null;
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (serverTimestampBuilder_ == null) {
+          serverTimestamp_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          serverTimestampBuilder_.clear();
+        }
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.internal_static_hbase_pb_TableServerTimestamp_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp result = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (serverTimestampBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            serverTimestamp_ = java.util.Collections.unmodifiableList(serverTimestamp_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.serverTimestamp_ = serverTimestamp_;
+        } else {
+          result.serverTimestamp_ = serverTimestampBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.getDefaultInstance()) return this;
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (serverTimestampBuilder_ == null) {
+          if (!other.serverTimestamp_.isEmpty()) {
+            if (serverTimestamp_.isEmpty()) {
+              serverTimestamp_ = other.serverTimestamp_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureServerTimestampIsMutable();
+              serverTimestamp_.addAll(other.serverTimestamp_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.serverTimestamp_.isEmpty()) {
+            if (serverTimestampBuilder_.isEmpty()) {
+              serverTimestampBuilder_.dispose();
+              serverTimestampBuilder_ = null;
+              serverTimestamp_ = other.serverTimestamp_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              serverTimestampBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getServerTimestampFieldBuilder() : null;
+            } else {
+              serverTimestampBuilder_.addAllMessages(other.serverTimestamp_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (hasTableName()) {
+          if (!getTableName().isInitialized()) {
+            return false;
+          }
+        }
+        for (int i = 0; i < getServerTimestampCount(); i++) {
+          if (!getServerTimestamp(i).isInitialized()) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              tableName_ != null &&
+              tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = null;
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  getTableName(),
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp> serverTimestamp_ =
+        java.util.Collections.emptyList();
+      private void ensureServerTimestampIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          serverTimestamp_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp>(serverTimestamp_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder> serverTimestampBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp> getServerTimestampList() {
+        if (serverTimestampBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(serverTimestamp_);
+        } else {
+          return serverTimestampBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public int getServerTimestampCount() {
+        if (serverTimestampBuilder_ == null) {
+          return serverTimestamp_.size();
+        } else {
+          return serverTimestampBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp getServerTimestamp(int index) {
+        if (serverTimestampBuilder_ == null) {
+          return serverTimestamp_.get(index);
+        } else {
+          return serverTimestampBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public Builder setServerTimestamp(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp value) {
+        if (serverTimestampBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureServerTimestampIsMutable();
+          serverTimestamp_.set(index, value);
+          onChanged();
+        } else {
+          serverTimestampBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public Builder setServerTimestamp(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder builderForValue) {
+        if (serverTimestampBuilder_ == null) {
+          ensureServerTimestampIsMutable();
+          serverTimestamp_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          serverTimestampBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public Builder addServerTimestamp(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp value) {
+        if (serverTimestampBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureServerTimestampIsMutable();
+          serverTimestamp_.add(value);
+          onChanged();
+        } else {
+          serverTimestampBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public Builder addServerTimestamp(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp value) {
+        if (serverTimestampBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureServerTimestampIsMutable();
+          serverTimestamp_.add(index, value);
+          onChanged();
+        } else {
+          serverTimestampBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public Builder addServerTimestamp(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder builderForValue) {
+        if (serverTimestampBuilder_ == null) {
+          ensureServerTimestampIsMutable();
+          serverTimestamp_.add(builderForValue.build());
+          onChanged();
+        } else {
+          serverTimestampBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public Builder addServerTimestamp(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder builderForValue) {
+        if (serverTimestampBuilder_ == null) {
+          ensureServerTimestampIsMutable();
+          serverTimestamp_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          serverTimestampBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public Builder addAllServerTimestamp(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp> values) {
+        if (serverTimestampBuilder_ == null) {
+          ensureServerTimestampIsMutable();
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, serverTimestamp_);
+          onChanged();
+        } else {
+          serverTimestampBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public Builder clearServerTimestamp() {
+        if (serverTimestampBuilder_ == null) {
+          serverTimestamp_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          serverTimestampBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public Builder removeServerTimestamp(int index) {
+        if (serverTimestampBuilder_ == null) {
+          ensureServerTimestampIsMutable();
+          serverTimestamp_.remove(index);
+          onChanged();
+        } else {
+          serverTimestampBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder getServerTimestampBuilder(
+          int index) {
+        return getServerTimestampFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder getServerTimestampOrBuilder(
+          int index) {
+        if (serverTimestampBuilder_ == null) {
+          return serverTimestamp_.get(index);  } else {
+          return serverTimestampBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder> 
+           getServerTimestampOrBuilderList() {
+        if (serverTimestampBuilder_ != null) {
+          return serverTimestampBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(serverTimestamp_);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder addServerTimestampBuilder() {
+        return getServerTimestampFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder addServerTimestampBuilder(
+          int index) {
+        return getServerTimestampFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .hbase.pb.ServerTimestamp server_timestamp = 2;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder> 
+           getServerTimestampBuilderList() {
+        return getServerTimestampFieldBuilder().getBuilderList();
+      }
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder> 
+          getServerTimestampFieldBuilder() {
+        if (serverTimestampBuilder_ == null) {
+          serverTimestampBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestamp.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.ServerTimestampOrBuilder>(
+                  serverTimestamp_,
+                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  getParentForChildren(),
+                  isClean());
+          serverTimestamp_ = null;
+        }
+        return serverTimestampBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.TableServerTimestamp)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.TableServerTimestamp)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableServerTimestamp>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<TableServerTimestamp>() {
+      public TableServerTimestamp parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new TableServerTimestamp(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableServerTimestamp> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableServerTimestamp> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface BackupImageOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.BackupImage)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional string backup_id = 1;</code>
+     */
+    boolean hasBackupId();
+    /**
+     * <code>optional string backup_id = 1;</code>
+     */
+    java.lang.String getBackupId();
+    /**
+     * <code>optional string backup_id = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getBackupIdBytes();
+
+    /**
+     * <code>optional .hbase.pb.BackupType backup_type = 2;</code>
+     */
+    boolean hasBackupType();
+    /**
+     * <code>optional .hbase.pb.BackupType backup_type = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType getBackupType();
+
+    /**
+     * <code>optional string backup_root_dir = 3;</code>
+     */
+    boolean hasBackupRootDir();
+    /**
+     * <code>optional string backup_root_dir = 3;</code>
+     */
+    java.lang.String getBackupRootDir();
+    /**
+     * <code>optional string backup_root_dir = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getBackupRootDirBytes();
+
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName> 
+        getTableListList();
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableList(int index);
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    int getTableListCount();
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+        getTableListOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
+        int index);
+
+    /**
+     * <code>optional uint64 start_ts = 5;</code>
+     */
+    boolean hasStartTs();
+    /**
+     * <code>optional uint64 start_ts = 5;</code>
+     */
+    long getStartTs();
+
+    /**
+     * <code>optional uint64 complete_ts = 6;</code>
+     */
+    boolean hasCompleteTs();
+    /**
+     * <code>optional uint64 complete_ts = 6;</code>
+     */
+    long getCompleteTs();
+
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage> 
+        getAncestorsList();
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage getAncestors(int index);
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    int getAncestorsCount();
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
+        getAncestorsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
+        int index);
+
+    /**
+     * <code>repeated .hbase.pb.TableServerTimestamp tst_map = 8;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp> 
+        getTstMapList();
+    /**
+     * <code>repeated .hbase.pb.TableServerTimestamp tst_map = 8;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp getTstMap(int index);
+    /**
+     * <code>repeated .hbase.pb.TableServerTimestamp tst_map = 8;</code>
+     */
+    int getTstMapCount();
+    /**
+     * <code>repeated .hbase.pb.TableServerTimestamp tst_map = 8;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder> 
+        getTstMapOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableServerTimestamp tst_map = 8;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestampOrBuilder getTstMapOrBuilder(
+        int index);
+  }
+  /**
+   * <pre>
+   **
+   * Structure keeps relevant info for backup restore session
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.BackupImage}
+   */
+  public  static final class BackupImage extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.BackupImage)
+      BackupImageOrBuilder {
+    // Use BackupImage.newBuilder() to construct.
+    private BackupImage(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private BackupImage() {
+      backupId_ = "";
+      backupType_ = 0;
+      backupRootDir_ = "";
+      tableList_ = java.util.Collections.emptyList();
+      startTs_ = 0L;
+      completeTs_ = 0L;
+      ancestors_ = java.util.Collections.emptyList();
+      tstMap_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private BackupImage(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00000001;
+              backupId_ = bs;
+              break;
+            }
+            case 16: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType value = org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(2, rawValue);
+              } else {
+                bitField0_ |= 0x00000002;
+                backupType_ = rawValue;
+              }
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00000004;
+              backupRootDir_ = bs;
+              break;
+            }
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+                tableList_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName>();
+                mutable_bitField0_ |= 0x00000008;
+              }
+              tableList_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry));
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000008;
+              startTs_ = input.readUInt64();
+              break;
+            }
+            case 48: {
+              bitField0_ |= 0x00000010;
+              completeTs_ = input.readUInt64();
+              break;
+            }
+            case 58: {
+              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+                ancestors_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage>();
+                mutable_bitField0_ |= 0x00000040;
+              }
+              ancestors_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupImage.PARSER, extensionRegistry));
+              break;
+            }
+            case 66: {
+              if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+                tstMap_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp>();
+                mutable_bitField0_ |= 0x00000080;
+              }
+              tstMap_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.TableServerTimestamp.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+          tableList_ = java.util.Collections.unmodifiableList(tableList_);
+        }
+        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+          ancestors_ = java.util.Collections.unmodifiableList(ancestors_);
+        }
+        if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+          tstMap_ = java.util.Collections.unmodifiableList(tstMap_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Des

<TRUNCATED>

[07/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
new file mode 100644
index 0000000..c1d5258
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
@@ -0,0 +1,524 @@
+/**
+ * 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.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+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.BackupAdmin;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.RestoreRequest;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import com.google.common.collect.Lists;
+
+@InterfaceAudience.Private
+public class BackupAdminImpl implements BackupAdmin {
+  public final static String CHECK_OK = "Checking backup images: OK";
+  public final static String CHECK_FAILED =
+      "Checking backup images: Failed. Some dependencies are missing for restore";
+  private static final Log LOG = LogFactory.getLog(BackupAdminImpl.class);
+
+  private final Connection conn;
+
+  public BackupAdminImpl(Connection conn) {
+    this.conn = conn;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (conn != null) {
+      conn.close();
+    }
+  }
+
+
+  @Override
+  public BackupInfo getBackupInfo(String backupId) throws IOException {
+    BackupInfo backupInfo = null;
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      if (backupId == null) {
+        ArrayList<BackupInfo> recentSessions = table.getBackupInfos(BackupState.RUNNING);
+        if (recentSessions.isEmpty()) {
+          LOG.warn("No ongoing sessions found.");
+          return null;
+        }
+        // else show status for ongoing session
+        // must be one maximum
+        return recentSessions.get(0);
+      } else {
+        backupInfo = table.readBackupInfo(backupId);
+        return backupInfo;
+      }
+    }
+  }
+
+  @Override
+  public int deleteBackups(String[] backupIds) throws IOException {
+    // TODO: requires Fault tolerance support, failure will leave system
+    // in a non-consistent state
+    // see HBASE-15227
+    int totalDeleted = 0;
+    Map<String, HashSet<TableName>> allTablesMap = new HashMap<String, HashSet<TableName>>();
+
+    try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) {
+      for (int i = 0; i < backupIds.length; i++) {
+        BackupInfo info = sysTable.readBackupInfo(backupIds[i]);
+        if (info != null) {
+          String rootDir = info.getBackupRootDir();
+          HashSet<TableName> allTables = allTablesMap.get(rootDir);
+          if (allTables == null) {
+            allTables = new HashSet<TableName>();
+            allTablesMap.put(rootDir, allTables);
+          }
+          allTables.addAll(info.getTableNames());
+          totalDeleted += deleteBackup(backupIds[i], sysTable);
+        }
+      }
+      finalizeDelete(allTablesMap, sysTable);
+    }
+    return totalDeleted;
+  }
+
+  /**
+   * Updates incremental backup set for every backupRoot
+   * @param tablesMap map [backupRoot: Set<TableName>]
+   * @param table backup system table
+   * @throws IOException
+   */
+
+  private void finalizeDelete(Map<String, HashSet<TableName>> tablesMap, BackupSystemTable table)
+      throws IOException {
+    for (String backupRoot : tablesMap.keySet()) {
+      Set<TableName> incrTableSet = table.getIncrementalBackupTableSet(backupRoot);
+      Map<TableName, ArrayList<BackupInfo>> tableMap =
+          table.getBackupHistoryForTableSet(incrTableSet, backupRoot);
+      for (Map.Entry<TableName, ArrayList<BackupInfo>> entry : tableMap.entrySet()) {
+        if (entry.getValue() == null) {
+          // No more backups for a table
+          incrTableSet.remove(entry.getKey());
+        }
+      }
+      if (!incrTableSet.isEmpty()) {
+        table.addIncrementalBackupTableSet(incrTableSet, backupRoot);
+      } else { // empty
+        table.deleteIncrementalBackupTableSet(backupRoot);
+      }
+    }
+  }
+
+  /**
+   * Delete single backup and all related backups <br>
+   * Algorithm:<br>
+   *  Backup type: FULL or INCREMENTAL <br>
+   *  Is this last backup session for table T: YES or NO <br>
+   *  For every table T from table list 'tables':<br>
+   *  if(FULL, YES) deletes only physical data (PD) <br>
+   *  if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo,<br>
+   *  until we either reach the most recent backup for T in the system or FULL backup<br>
+   *  which includes T<br>
+   *  if(INCREMENTAL, YES) deletes only physical data (PD)
+   *  if(INCREMENTAL, NO) deletes physical data and for table T scans all backup images between last<br>
+   *  FULL backup, which is older than the backup being deleted and the next FULL backup (if exists) <br>
+   *  or last one for a particular table T and removes T from list of backup tables.
+   * @param backupId backup id
+   * @param sysTable backup system table
+   * @return total number of deleted backup images
+   * @throws IOException
+   */
+  private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException {
+
+    BackupInfo backupInfo = sysTable.readBackupInfo(backupId);
+
+    int totalDeleted = 0;
+    if (backupInfo != null) {
+      LOG.info("Deleting backup " + backupInfo.getBackupId() + " ...");
+      BackupUtils.cleanupBackupData(backupInfo, conn.getConfiguration());
+      // List of tables in this backup;
+      List<TableName> tables = backupInfo.getTableNames();
+      long startTime = backupInfo.getStartTs();
+      for (TableName tn : tables) {
+        boolean isLastBackupSession = isLastBackupSession(sysTable, tn, startTime);
+        if (isLastBackupSession) {
+          continue;
+        }
+        // else
+        List<BackupInfo> affectedBackups = getAffectedBackupInfos(backupInfo, tn, sysTable);
+        for (BackupInfo info : affectedBackups) {
+          if (info.equals(backupInfo)) {
+            continue;
+          }
+          removeTableFromBackupImage(info, tn, sysTable);
+        }
+      }
+      LOG.debug("Delete backup info " + backupInfo.getBackupId());
+
+      sysTable.deleteBackupInfo(backupInfo.getBackupId());
+      LOG.info("Delete backup " + backupInfo.getBackupId() + " completed.");
+      totalDeleted++;
+    } else {
+      LOG.warn("Delete backup failed: no information found for backupID=" + backupId);
+    }
+    return totalDeleted;
+  }
+
+  private void removeTableFromBackupImage(BackupInfo info, TableName tn,
+      BackupSystemTable sysTable) throws IOException {
+    List<TableName> tables = info.getTableNames();
+    LOG.debug("Remove " + tn + " from " + info.getBackupId() + " tables="
+        + info.getTableListAsString());
+    if (tables.contains(tn)) {
+      tables.remove(tn);
+
+      if (tables.isEmpty()) {
+        LOG.debug("Delete backup info " + info.getBackupId());
+
+        sysTable.deleteBackupInfo(info.getBackupId());
+        BackupUtils.cleanupBackupData(info, conn.getConfiguration());
+      } else {
+        info.setTables(tables);
+        sysTable.updateBackupInfo(info);
+        // Now, clean up directory for table
+        cleanupBackupDir(info, tn, conn.getConfiguration());
+      }
+    }
+  }
+
+  private List<BackupInfo> getAffectedBackupInfos(BackupInfo backupInfo, TableName tn,
+      BackupSystemTable table) throws IOException {
+    LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn);
+    long ts = backupInfo.getStartTs();
+    List<BackupInfo> list = new ArrayList<BackupInfo>();
+    List<BackupInfo> history = table.getBackupHistory(backupInfo.getBackupRootDir());
+    // Scan from most recent to backupInfo
+    // break when backupInfo reached
+    for (BackupInfo info : history) {
+      if (info.getStartTs() == ts) {
+        break;
+      }
+      List<TableName> tables = info.getTableNames();
+      if (tables.contains(tn)) {
+        BackupType bt = info.getType();
+        if (bt == BackupType.FULL) {
+          // Clear list if we encounter FULL backup
+          list.clear();
+        } else {
+          LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn
+              + " added " + info.getBackupId() + " tables=" + info.getTableListAsString());
+          list.add(info);
+        }
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Clean up the data at target directory
+   * @throws IOException
+   */
+  private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf)
+      throws IOException {
+    try {
+      // clean up the data at target directory
+      String targetDir = backupInfo.getBackupRootDir();
+      if (targetDir == null) {
+        LOG.warn("No target directory specified for " + backupInfo.getBackupId());
+        return;
+      }
+
+      FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf);
+
+      Path targetDirPath =
+          new Path(BackupUtils.getTableBackupDir(backupInfo.getBackupRootDir(),
+            backupInfo.getBackupId(), table));
+      if (outputFs.delete(targetDirPath, true)) {
+        LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done.");
+      } else {
+        LOG.info("No data has been found in " + targetDirPath.toString() + ".");
+      }
+
+    } catch (IOException e1) {
+      LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " for table " + table
+          + "at " + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + ".");
+      throw e1;
+    }
+  }
+
+  private boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime)
+      throws IOException {
+    List<BackupInfo> history = table.getBackupHistory();
+    for (BackupInfo info : history) {
+      List<TableName> tables = info.getTableNames();
+      if (!tables.contains(tn)) {
+        continue;
+      }
+      if (info.getStartTs() <= startTime) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public List<BackupInfo> getHistory(int n) throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<BackupInfo> history = table.getBackupHistory();
+      if (history.size() <= n) return history;
+      List<BackupInfo> list = new ArrayList<BackupInfo>();
+      for (int i = 0; i < n; i++) {
+        list.add(history.get(i));
+      }
+      return list;
+    }
+  }
+
+  @Override
+  public List<BackupInfo> getHistory(int n, BackupInfo.Filter... filters) throws IOException {
+    if (filters.length == 0) return getHistory(n);
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<BackupInfo> history = table.getBackupHistory();
+      List<BackupInfo> result = new ArrayList<BackupInfo>();
+      for (BackupInfo bi : history) {
+        if (result.size() == n) break;
+        boolean passed = true;
+        for (int i = 0; i < filters.length; i++) {
+          if (!filters[i].apply(bi)) {
+            passed = false;
+            break;
+          }
+        }
+        if (passed) {
+          result.add(bi);
+        }
+      }
+      return result;
+    }
+  }
+
+  @Override
+  public List<BackupSet> listBackupSets() throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<String> list = table.listBackupSets();
+      List<BackupSet> bslist = new ArrayList<BackupSet>();
+      for (String s : list) {
+        List<TableName> tables = table.describeBackupSet(s);
+        if (tables != null) {
+          bslist.add(new BackupSet(s, tables));
+        }
+      }
+      return bslist;
+    }
+  }
+
+  @Override
+  public BackupSet getBackupSet(String name) throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<TableName> list = table.describeBackupSet(name);
+      if (list == null) return null;
+      return new BackupSet(name, list);
+    }
+  }
+
+  @Override
+  public boolean deleteBackupSet(String name) throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      if (table.describeBackupSet(name) == null) {
+        return false;
+      }
+      table.deleteBackupSet(name);
+      return true;
+    }
+  }
+
+  @Override
+  public void addToBackupSet(String name, TableName[] tables) throws IOException {
+    String[] tableNames = new String[tables.length];
+    try (final BackupSystemTable table = new BackupSystemTable(conn);
+        final Admin admin = conn.getAdmin();) {
+      for (int i = 0; i < tables.length; i++) {
+        tableNames[i] = tables[i].getNameAsString();
+        if (!admin.tableExists(TableName.valueOf(tableNames[i]))) {
+          throw new IOException("Cannot add " + tableNames[i] + " because it doesn't exist");
+        }
+      }
+      table.addToBackupSet(name, tableNames);
+      LOG.info("Added tables [" + StringUtils.join(tableNames, " ") + "] to '" + name
+          + "' backup set");
+    }
+  }
+
+  @Override
+  public void removeFromBackupSet(String name, TableName[] tables) throws IOException {
+    LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + "'");
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      table.removeFromBackupSet(name, toStringArray(tables));
+      LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name
+          + "' completed.");
+    }
+  }
+
+  private String[] toStringArray(TableName[] list) {
+    String[] arr = new String[list.length];
+    for(int i=0; i < list.length; i++) {
+      arr[i] = list[i].toString();
+    }
+    return arr;
+  }
+
+  @Override
+  public void restore(RestoreRequest request) throws IOException {
+    if (request.isCheck()) {
+      HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
+      // check and load backup image manifest for the tables
+      Path rootPath = new Path(request.getBackupRootDir());
+      String backupId = request.getBackupId();
+      TableName[] sTableArray = request.getFromTables();
+      HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray,
+        conn.getConfiguration(), rootPath, backupId);
+
+      // Check and validate the backup image and its dependencies
+
+      if (BackupUtils.validate(backupManifestMap, conn.getConfiguration())) {
+        LOG.info(CHECK_OK);
+      } else {
+        LOG.error(CHECK_FAILED);
+      }
+      return;
+    }
+    // Execute restore request
+    new RestoreTablesClient(conn, request).execute();
+  }
+
+  @Override
+  public String backupTables(BackupRequest request) throws IOException {
+    BackupType type = request.getBackupType();
+    String targetRootDir = request.getTargetRootDir();
+    List<TableName> tableList = request.getTableList();
+
+    String backupId =BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime();
+    if (type == BackupType.INCREMENTAL) {
+      Set<TableName> incrTableSet = null;
+      try (BackupSystemTable table = new BackupSystemTable(conn)) {
+        incrTableSet = table.getIncrementalBackupTableSet(targetRootDir);
+      }
+
+      if (incrTableSet.isEmpty()) {
+        String msg = "Incremental backup table set contains no tables. "
+            + "You need to run full backup first " +
+            (tableList != null ? "on "+StringUtils.join(tableList, ","): "");
+
+        throw new IOException(msg);
+      }
+      if(tableList != null) {
+        tableList.removeAll(incrTableSet);
+        if (!tableList.isEmpty()) {
+          String extraTables = StringUtils.join(tableList, ",");
+          String msg = "Some tables (" + extraTables + ") haven't gone through full backup. "+
+            "Perform full backup on " + extraTables + " first, "
+            + "then retry the command";
+          throw new IOException(msg);
+        }
+      }
+      tableList = Lists.newArrayList(incrTableSet);
+    }
+    if (tableList != null && !tableList.isEmpty()) {
+      for (TableName table : tableList) {
+        String targetTableBackupDir =
+            HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
+        Path targetTableBackupDirPath = new Path(targetTableBackupDir);
+        FileSystem outputFs =
+            FileSystem.get(targetTableBackupDirPath.toUri(), conn.getConfiguration());
+        if (outputFs.exists(targetTableBackupDirPath)) {
+          throw new IOException("Target backup directory " + targetTableBackupDir
+              + " exists already.");
+        }
+      }
+      ArrayList<TableName> nonExistingTableList = null;
+      try (Admin admin = conn.getAdmin();) {
+        for (TableName tableName : tableList) {
+          if (!admin.tableExists(tableName)) {
+            if (nonExistingTableList == null) {
+              nonExistingTableList = new ArrayList<>();
+            }
+            nonExistingTableList.add(tableName);
+          }
+        }
+      }
+      if (nonExistingTableList != null) {
+        if (type == BackupType.INCREMENTAL) {
+          // Update incremental backup set
+          tableList = excludeNonExistingTables(tableList, nonExistingTableList);
+        } else {
+          // Throw exception only in full mode - we try to backup non-existing table
+          throw new IOException("Non-existing tables found in the table list: "
+              + nonExistingTableList);
+        }
+      }
+    }
+
+    // update table list
+    BackupRequest.Builder builder = new BackupRequest.Builder();
+    request = builder.withBackupType(request.getBackupType()).
+                      withTableList(tableList).
+                      withTargetRootDir(request.getTargetRootDir()).
+                      withBackupSetName(request.getBackupSetName()).
+                      withTotalTasks(request.getTotalTasks()).
+                      withBandwidthPerTasks((int)request.getBandwidth()).build();
+
+    if (type == BackupType.FULL) {
+      new FullTableBackupClient(conn, backupId, request).execute();
+    } else {
+      new IncrementalTableBackupClient(conn, backupId, request).execute();
+    }
+    return backupId;
+  }
+
+  private List<TableName> excludeNonExistingTables(List<TableName> tableList,
+      List<TableName> nonExistingTableList) {
+
+    for (TableName table : nonExistingTableList) {
+      tableList.remove(table);
+    }
+    return tableList;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
new file mode 100644
index 0000000..75e0ab7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
@@ -0,0 +1,780 @@
+/**
+ * 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.impl;
+
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_BACKUP_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * General backup commands, options and usage messages
+ */
+
+@InterfaceAudience.Private
+public final class BackupCommands  {
+
+  public final static String INCORRECT_USAGE = "Incorrect usage";
+
+  public static final String USAGE = "Usage: hbase backup COMMAND [command-specific arguments]\n"
+      + "where COMMAND is one of:\n" + "  create     create a new backup image\n"
+      + "  delete     delete an existing backup image\n"
+      + "  describe   show the detailed information of a backup image\n"
+      + "  history    show history of all successful backups\n"
+      + "  progress   show the progress of the latest backup request\n"
+      + "  set        backup set management\n"
+      + "Run \'hbase backup COMMAND -h\' to see help message for each command\n";
+
+  public static final String CREATE_CMD_USAGE =
+      "Usage: hbase backup create <type> <backup_path> [options]\n"
+          + "  type           \"full\" to create a full backup image\n"
+          + "                 \"incremental\" to create an incremental backup image\n"
+          + "  backup_path     Full path to store the backup image\n";
+
+  public static final String PROGRESS_CMD_USAGE = "Usage: hbase backup progress <backup_id>\n"
+      + "  backup_id       Backup image id (optional). If no id specified, the command will show\n"
+      + "                  progress for currently running backup session.";
+  public static final String NO_INFO_FOUND = "No info was found for backup id: ";
+  public static final String NO_ACTIVE_SESSION_FOUND = "No active backup sessions found.";
+
+  public static final String DESCRIBE_CMD_USAGE = "Usage: hbase backup describe <backup_id>\n"
+      + "  backup_id       Backup image id\n";
+
+  public static final String HISTORY_CMD_USAGE = "Usage: hbase backup history [options]";
+
+  public static final String DELETE_CMD_USAGE = "Usage: hbase backup delete <backup_id>\n"
+      + "  backup_id       Backup image id\n";
+
+  public static final String CANCEL_CMD_USAGE = "Usage: hbase backup cancel <backup_id>\n"
+      + "  backup_id       Backup image id\n";
+
+  public static final String SET_CMD_USAGE = "Usage: hbase backup set COMMAND [name] [tables]\n"
+      + "  name            Backup set name\n"
+      + "  tables          Comma separated list of tables.\n"
+      + "COMMAND is one of:\n" + "  add             add tables to a set, create a set if needed\n"
+      + "  remove          remove tables from a set\n"
+      + "  list            list all backup sets in the system\n"
+      + "  describe        describe set\n" + "  delete          delete backup set\n";
+
+  public static final String USAGE_FOOTER = "";
+
+  public static abstract class Command extends Configured {
+    CommandLine cmdline;
+
+    Command(Configuration conf) {
+      super(conf);
+    }
+
+    public void execute() throws IOException {
+      if (cmdline.hasOption("h") || cmdline.hasOption("help")) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    protected abstract void printUsage();
+  }
+
+  private BackupCommands() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  public static Command createCommand(Configuration conf, BackupCommand type, CommandLine cmdline) {
+    Command cmd = null;
+    switch (type) {
+    case CREATE:
+      cmd = new CreateCommand(conf, cmdline);
+      break;
+    case DESCRIBE:
+      cmd = new DescribeCommand(conf, cmdline);
+      break;
+    case PROGRESS:
+      cmd = new ProgressCommand(conf, cmdline);
+      break;
+    case DELETE:
+      cmd = new DeleteCommand(conf, cmdline);
+      break;
+    case CANCEL:
+      cmd = new CancelCommand(conf, cmdline);
+      break;
+    case HISTORY:
+      cmd = new HistoryCommand(conf, cmdline);
+      break;
+    case SET:
+      cmd = new BackupSetCommand(conf, cmdline);
+      break;
+    case HELP:
+    default:
+      cmd = new HelpCommand(conf, cmdline);
+      break;
+    }
+    return cmd;
+  }
+
+  static int numOfArgs(String[] args) {
+    if (args == null) return 0;
+    return args.length;
+  }
+
+  public static class CreateCommand extends Command {
+
+    CreateCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null || cmdline.getArgs() == null) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length !=3) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      if (!BackupType.FULL.toString().equalsIgnoreCase(args[1])
+          && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[1])) {
+        System.out.println("ERROR: invalid backup type: " + args[1]);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      if (!verifyPath(args[2])) {
+        System.out.println("ERROR: invalid backup destination: " + args[2]);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String tables = null;
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+
+      // Check if we have both: backup set and list of tables
+      if (cmdline.hasOption(OPTION_TABLE) && cmdline.hasOption(OPTION_SET)) {
+        System.out.println("ERROR: You can specify either backup set or list"
+            + " of tables, but not both");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      // Check backup set
+      String setName = null;
+      if (cmdline.hasOption(OPTION_SET)) {
+        setName = cmdline.getOptionValue(OPTION_SET);
+        tables = getTablesForSet(setName, conf);
+
+        if (tables == null) {
+          System.out.println("ERROR: Backup set '" + setName
+              + "' is either empty or does not exist");
+          printUsage();
+          throw new IOException(INCORRECT_USAGE);
+        }
+      } else {
+        tables = cmdline.getOptionValue(OPTION_TABLE);
+      }
+      int bandwidth =
+          cmdline.hasOption(OPTION_BANDWIDTH) ? Integer.parseInt(cmdline
+              .getOptionValue(OPTION_BANDWIDTH)) : -1;
+      int workers =
+          cmdline.hasOption(OPTION_WORKERS) ? Integer.parseInt(cmdline
+              .getOptionValue(OPTION_WORKERS)) : -1;
+
+      try (Connection conn = ConnectionFactory.createConnection(getConf());
+          BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+
+       BackupRequest.Builder builder = new BackupRequest.Builder();
+       BackupRequest request = builder.withBackupType(BackupType.valueOf(args[1].toUpperCase()))
+            .withTableList(tables != null ?
+                          Lists.newArrayList(BackupUtils.parseTableNames(tables)) : null)
+            .withTargetRootDir(args[2])
+            .withTotalTasks(workers)
+            .withBandwidthPerTasks(bandwidth)
+            .withBackupSetName(setName).build();
+        String backupId = admin.backupTables(request);
+        System.out.println("Backup session " + backupId + " finished. Status: SUCCESS");
+      } catch (IOException e) {
+        System.out.println("Backup session finished. Status: FAILURE");
+        throw e;
+      }
+    }
+
+    private boolean verifyPath(String path) {
+      try {
+        Path p = new Path(path);
+        Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+        URI uri = p.toUri();
+        if (uri.getScheme() == null) return false;
+        FileSystem.get(uri, conf);
+        return true;
+      } catch (Exception e) {
+        return false;
+      }
+    }
+
+    private String getTablesForSet(String name, Configuration conf) throws IOException {
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupSystemTable table = new BackupSystemTable(conn)) {
+        List<TableName> tables = table.describeBackupSet(name);
+        if (tables == null) return null;
+        return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(CREATE_CMD_USAGE);
+      Options options = new Options();
+      options.addOption(OPTION_WORKERS, true, OPTION_WORKERS_DESC);
+      options.addOption(OPTION_BANDWIDTH, true, OPTION_BANDWIDTH_DESC);
+      options.addOption(OPTION_SET, true, OPTION_SET_BACKUP_DESC);
+      options.addOption(OPTION_TABLE, true, OPTION_TABLE_LIST_DESC);
+
+      HelpFormatter helpFormatter = new HelpFormatter();
+      helpFormatter.setLeftPadding(2);
+      helpFormatter.setDescPadding(8);
+      helpFormatter.setWidth(100);
+      helpFormatter.setSyntaxPrefix("Options:");
+      helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
+
+    }
+  }
+
+  private static class HelpCommand extends Command {
+
+    HelpCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String[] args = cmdline.getArgs();
+      if (args == null || args.length == 0) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      if (args.length != 2) {
+        System.out.println("ERROR: Only supports help message of a single command type");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String type = args[1];
+
+      if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) {
+        System.out.println(CREATE_CMD_USAGE);
+      } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(type)) {
+        System.out.println(DESCRIBE_CMD_USAGE);
+      } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(type)) {
+        System.out.println(HISTORY_CMD_USAGE);
+      } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(type)) {
+        System.out.println(PROGRESS_CMD_USAGE);
+      } else if (BackupCommand.DELETE.name().equalsIgnoreCase(type)) {
+        System.out.println(DELETE_CMD_USAGE);
+      } else if (BackupCommand.CANCEL.name().equalsIgnoreCase(type)) {
+        System.out.println(CANCEL_CMD_USAGE);
+      } else if (BackupCommand.SET.name().equalsIgnoreCase(type)) {
+        System.out.println(SET_CMD_USAGE);
+      } else {
+        System.out.println("Unknown command : " + type);
+        printUsage();
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(USAGE);
+    }
+  }
+
+  private static class DescribeCommand extends Command {
+
+    DescribeCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null || cmdline.getArgs() == null) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length != 2) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String backupId = args[1];
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+        BackupInfo info = sysTable.readBackupInfo(backupId);
+        if (info == null) {
+          System.out.println("ERROR: " + backupId + " does not exist");
+          printUsage();
+          throw new IOException(INCORRECT_USAGE);
+        }
+        System.out.println(info.getShortDescription());
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(DESCRIBE_CMD_USAGE);
+    }
+  }
+
+  private static class ProgressCommand extends Command {
+
+    ProgressCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length == 1) {
+        System.out.println("No backup id was specified, "
+            + "will retrieve the most recent (ongoing) session");
+      }
+      String[] args = cmdline == null ? null : cmdline.getArgs();
+      if (args != null && args.length > 2) {
+        System.err.println("ERROR: wrong number of arguments: " + args.length);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String backupId = (args == null || args.length <= 1) ? null : args[1];
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+        BackupInfo info = null;
+
+        if (backupId != null) {
+          info = sysTable.readBackupInfo(backupId);
+        } else {
+          List<BackupInfo> infos = sysTable.getBackupInfos(BackupState.RUNNING);
+          if (infos != null && infos.size() > 0) {
+            info = infos.get(0);
+            backupId = info.getBackupId();
+            System.out.println("Found ongoing session with backupId=" + backupId);
+          } else {
+          }
+        }
+        int progress = info == null ? -1 : info.getProgress();
+        if (progress < 0) {
+          if (backupId != null) {
+            System.out.println(NO_INFO_FOUND + backupId);
+          } else {
+            System.err.println(NO_ACTIVE_SESSION_FOUND);
+          }
+        } else {
+          System.out.println(backupId + " progress=" + progress + "%");
+        }
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(PROGRESS_CMD_USAGE);
+    }
+  }
+
+  private static class DeleteCommand extends Command {
+
+    DeleteCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String[] args = cmdline.getArgs();
+      String[] backupIds = new String[args.length - 1];
+      System.arraycopy(args, 1, backupIds, 0, backupIds.length);
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        int deleted = admin.deleteBackups(backupIds);
+        System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
+      }
+
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(DELETE_CMD_USAGE);
+    }
+  }
+
+  // TODO Cancel command
+
+  private static class CancelCommand extends Command {
+
+    CancelCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      throw new UnsupportedOperationException("Cancel command is not supported yet.");
+    }
+
+    @Override
+    protected void printUsage() {
+    }
+  }
+
+  private static class HistoryCommand extends Command {
+
+    private final static int DEFAULT_HISTORY_LENGTH = 10;
+
+    HistoryCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+
+      super.execute();
+
+      int n = parseHistoryLength();
+      final TableName tableName = getTableName();
+      final String setName = getTableSetName();
+      BackupInfo.Filter tableNameFilter = new BackupInfo.Filter() {
+        @Override
+        public boolean apply(BackupInfo info) {
+          if (tableName == null) return true;
+          List<TableName> names = info.getTableNames();
+          return names.contains(tableName);
+        }
+      };
+      BackupInfo.Filter tableSetFilter = new BackupInfo.Filter() {
+        @Override
+        public boolean apply(BackupInfo info) {
+          if (setName == null) return true;
+          String backupId = info.getBackupId();
+          return backupId.startsWith(setName);
+        }
+      };
+      Path backupRootPath = getBackupRootPath();
+      List<BackupInfo> history = null;
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      if (backupRootPath == null) {
+        // Load from backup system table
+        try (final Connection conn = ConnectionFactory.createConnection(conf);
+            final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+
+          history = sysTable.getBackupHistory(n, tableNameFilter, tableSetFilter);
+        }
+      } else {
+        // load from backup FS
+        history =
+            BackupUtils.getHistory(conf, n, backupRootPath, tableNameFilter, tableSetFilter);
+      }
+      for (BackupInfo info : history) {
+        System.out.println(info.getShortDescription());
+      }
+    }
+
+    private Path getBackupRootPath() throws IOException {
+      String value = null;
+      try {
+        value = cmdline.getOptionValue(OPTION_PATH);
+        if (value == null) return null;
+        return new Path(value);
+      } catch (IllegalArgumentException e) {
+        System.out.println("ERROR: Illegal argument for backup root path: " + value);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    private TableName getTableName() throws IOException {
+      String value = cmdline.getOptionValue(OPTION_TABLE);
+      if (value == null) return null;
+      try {
+        return TableName.valueOf(value);
+      } catch (IllegalArgumentException e) {
+        System.out.println("Illegal argument for table name: " + value);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    private String getTableSetName() throws IOException {
+      String value = cmdline.getOptionValue(OPTION_SET);
+      return value;
+    }
+
+    private int parseHistoryLength() throws IOException {
+      String value = cmdline.getOptionValue(OPTION_RECORD_NUMBER);
+      try {
+        if (value == null) return DEFAULT_HISTORY_LENGTH;
+        return Integer.parseInt(value);
+      } catch (NumberFormatException e) {
+        System.out.println("Illegal argument for history length: " + value);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(HISTORY_CMD_USAGE);
+      Options options = new Options();
+      options.addOption(OPTION_RECORD_NUMBER, true, OPTION_RECORD_NUMBER_DESC);
+      options.addOption(OPTION_PATH, true, OPTION_PATH_DESC);
+      options.addOption(OPTION_TABLE, true, OPTION_TABLE_DESC);
+      options.addOption(OPTION_SET, true, OPTION_SET_DESC);
+
+      HelpFormatter helpFormatter = new HelpFormatter();
+      helpFormatter.setLeftPadding(2);
+      helpFormatter.setDescPadding(8);
+      helpFormatter.setWidth(100);
+      helpFormatter.setSyntaxPrefix("Options:");
+      helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
+    }
+  }
+
+  private static class BackupSetCommand extends Command {
+    private final static String SET_ADD_CMD = "add";
+    private final static String SET_REMOVE_CMD = "remove";
+    private final static String SET_DELETE_CMD = "delete";
+    private final static String SET_DESCRIBE_CMD = "describe";
+    private final static String SET_LIST_CMD = "list";
+
+    BackupSetCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      // Command-line must have at least one element
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String[] args = cmdline.getArgs();
+      String cmdStr = args[1];
+      BackupCommand cmd = getCommand(cmdStr);
+
+      switch (cmd) {
+      case SET_ADD:
+        processSetAdd(args);
+        break;
+      case SET_REMOVE:
+        processSetRemove(args);
+        break;
+      case SET_DELETE:
+        processSetDelete(args);
+        break;
+      case SET_DESCRIBE:
+        processSetDescribe(args);
+        break;
+      case SET_LIST:
+        processSetList(args);
+        break;
+      default:
+        break;
+
+      }
+    }
+
+    private void processSetList(String[] args) throws IOException {
+      // List all backup set names
+      // does not expect any args
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        List<BackupSet> list = admin.listBackupSets();
+        for (BackupSet bs : list) {
+          System.out.println(bs);
+        }
+      }
+    }
+
+    private void processSetDescribe(String[] args) throws IOException {
+      if (args == null || args.length != 3) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String setName = args[2];
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+        List<TableName> tables = sysTable.describeBackupSet(setName);
+        BackupSet set = tables == null ? null : new BackupSet(setName, tables);
+        if (set == null) {
+          System.out.println("Set '" + setName + "' does not exist.");
+        } else {
+          System.out.println(set);
+        }
+      }
+    }
+
+    private void processSetDelete(String[] args) throws IOException {
+      if (args == null || args.length != 3) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String setName = args[2];
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        boolean result = admin.deleteBackupSet(setName);
+        if (result) {
+          System.out.println("Delete set " + setName + " OK.");
+        } else {
+          System.out.println("Set " + setName + " does not exist");
+        }
+      }
+    }
+
+    private void processSetRemove(String[] args) throws IOException {
+      if (args == null || args.length != 4) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String setName = args[2];
+      String[] tables = args[3].split(",");
+      TableName[] tableNames = toTableNames(tables);
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        admin.removeFromBackupSet(setName, tableNames);
+      }
+    }
+
+    private TableName[] toTableNames(String[] tables) {
+      TableName[] arr = new TableName[tables.length];
+      for (int i=0; i < tables.length; i++) {
+        arr[i] = TableName.valueOf(tables[i]);
+      }
+      return arr;
+    }
+
+    private void processSetAdd(String[] args) throws IOException {
+      if (args == null || args.length != 4) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String setName = args[2];
+      String[] tables = args[3].split(",");
+      TableName[] tableNames = new TableName[tables.length];
+      for (int i = 0; i < tables.length; i++) {
+        tableNames[i] = TableName.valueOf(tables[i]);
+      }
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        admin.addToBackupSet(setName, tableNames);
+      }
+
+    }
+
+    private BackupCommand getCommand(String cmdStr) throws IOException {
+      if (cmdStr.equals(SET_ADD_CMD)) {
+        return BackupCommand.SET_ADD;
+      } else if (cmdStr.equals(SET_REMOVE_CMD)) {
+        return BackupCommand.SET_REMOVE;
+      } else if (cmdStr.equals(SET_DELETE_CMD)) {
+        return BackupCommand.SET_DELETE;
+      } else if (cmdStr.equals(SET_DESCRIBE_CMD)) {
+        return BackupCommand.SET_DESCRIBE;
+      } else if (cmdStr.equals(SET_LIST_CMD)) {
+        return BackupCommand.SET_LIST;
+      } else {
+        System.out.println("ERROR: Unknown command for 'set' :" + cmdStr);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(SET_CMD_USAGE);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
new file mode 100644
index 0000000..2c7d35f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
@@ -0,0 +1,84 @@
+/**
+ * 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.impl;
+
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Backup exception
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+public class BackupException extends HBaseIOException {
+  private BackupInfo info;
+
+  /**
+   * 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, BackupInfo desc) {
+    super(msg);
+    this.info = 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, BackupInfo desc) {
+    super(msg, cause);
+    this.info = 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 BackupInfo getBackupInfo() {
+    return this.info;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
new file mode 100644
index 0000000..1d27e79
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -0,0 +1,472 @@
+/**
+ * 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.impl;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+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.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.procedure.ProcedureManagerHost;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Handles backup requests, creates backup info records in backup system table to
+ * keep track of backup sessions, dispatches backup request.
+ */
+@InterfaceAudience.Private
+public class BackupManager implements Closeable {
+  private static final Log LOG = LogFactory.getLog(BackupManager.class);
+
+  protected Configuration conf = null;
+  protected BackupInfo backupInfo = null;
+  protected BackupSystemTable systemTable;
+  protected final Connection conn;
+
+  /**
+   * Backup manager constructor.
+   * @param conn connection
+   * @param conf configuration
+   * @throws IOException exception
+   */
+  public BackupManager(Connection conn, Configuration conf) throws IOException {
+    if (!conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY,
+      BackupRestoreConstants.BACKUP_ENABLE_DEFAULT)) {
+      throw new BackupException("HBase backup is not enabled. Check your "
+          + BackupRestoreConstants.BACKUP_ENABLE_KEY + " setting.");
+    }
+    this.conf = conf;
+    this.conn = conn;
+    this.systemTable = new BackupSystemTable(conn);
+
+  }
+
+  /**
+   * Returns backup info
+   */
+  protected BackupInfo getBackupInfo() {
+    return backupInfo;
+  }
+
+  /**
+   * This method modifies the master's configuration in order to inject backup-related features
+   * (TESTs only)
+   * @param conf configuration
+   */
+  @VisibleForTesting
+  public static void decorateMasterConfiguration(Configuration conf) {
+    if (!isBackupEnabled(conf)) {
+      return;
+    }
+    // Add WAL archive cleaner plug-in
+    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);
+    }
+
+    String classes = conf.get(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY);
+    String masterProcedureClass = LogRollMasterProcedureManager.class.getName();
+    if (classes == null) {
+      conf.set(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY, masterProcedureClass);
+    } else if (!classes.contains(masterProcedureClass)) {
+      conf.set(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY, classes + "," + masterProcedureClass);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Added log cleaner: " + cleanerClass +"\n" +
+                "Added master procedure manager: " + masterProcedureClass);
+    }
+
+  }
+
+  /**
+   * This method modifies the Region Server configuration in order to inject backup-related features
+   * TESTs only.
+   * @param conf configuration
+   */
+  @VisibleForTesting
+  public static void decorateRegionServerConfiguration(Configuration conf) {
+    if (!isBackupEnabled(conf)) {
+      return;
+    }
+
+    String classes = conf.get(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY);
+    String regionProcedureClass = LogRollRegionServerProcedureManager.class.getName();
+    if (classes == null) {
+      conf.set(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY, regionProcedureClass);
+    } else if (!classes.contains(regionProcedureClass)) {
+      conf.set(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY, classes + ","
+          + regionProcedureClass);
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Added region procedure manager: " + regionProcedureClass);
+    }
+
+  }
+
+  public static boolean isBackupEnabled(Configuration conf) {
+    return conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY,
+      BackupRestoreConstants.BACKUP_ENABLE_DEFAULT);
+  }
+
+  /**
+   * Get configuration
+   * @return configuration
+   */
+  Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * Stop all the work of backup.
+   */
+  @Override
+  public void close() {
+
+    if (systemTable != null) {
+      try {
+        systemTable.close();
+      } catch (Exception e) {
+        LOG.error(e);
+      }
+    }
+  }
+
+  /**
+   * Creates a backup info based on input backup request.
+   * @param backupId backup id
+   * @param type type
+   * @param tableList table list
+   * @param targetRootDir root dir
+   * @param workers number of parallel workers
+   * @param bandwidth bandwidth per worker in MB per sec
+   * @return BackupInfo
+   * @throws BackupException exception
+   */
+  public BackupInfo createBackupInfo(String backupId, BackupType type,
+      List<TableName> tableList, String targetRootDir, int workers, long bandwidth)
+      throws BackupException {
+    if (targetRootDir == null) {
+      throw new BackupException("Wrong backup request parameter: target backup root directory");
+    }
+
+    if (type == BackupType.FULL && (tableList == null || tableList.isEmpty())) {
+      // 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 (Admin admin = conn.getAdmin()) {
+        htds = admin.listTables();
+      } catch (Exception e) {
+        throw new BackupException(e);
+      }
+
+      if (htds == null) {
+        throw new BackupException("No table exists for full backup of all tables.");
+      } else {
+        tableList = new ArrayList<>();
+        for (HTableDescriptor hTableDescriptor : htds) {
+          TableName tn = hTableDescriptor.getTableName();
+          if (tn.equals(BackupSystemTable.getTableName(conf))) {
+            // skip backup system table
+            continue;
+          }
+          tableList.add(hTableDescriptor.getTableName());
+        }
+
+        LOG.info("Full backup all the tables available in the cluster: " + tableList);
+      }
+    }
+
+    // there are one or more tables in the table list
+    backupInfo =
+        new BackupInfo(backupId, type, tableList.toArray(new TableName[tableList.size()]),
+            targetRootDir);
+    backupInfo.setBandwidth(bandwidth);
+    backupInfo.setWorkers(workers);
+    return backupInfo;
+  }
+
+  /**
+   * Check if any ongoing backup. Currently, we only reply on checking status in backup system
+   * table. 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<BackupInfo> sessions = systemTable.getBackupInfos(BackupState.RUNNING);
+    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.");
+    }
+  }
+
+  public void setBackupInfo(BackupInfo backupInfo) {
+    this.backupInfo = backupInfo;
+  }
+
+  /**
+   * Get direct ancestors of the current backup.
+   * @param backupInfo The backup info for the current backup
+   * @return The ancestors for the current backup
+   * @throws IOException exception
+   * @throws BackupException exception
+   */
+  public ArrayList<BackupImage> getAncestors(BackupInfo backupInfo) throws IOException,
+      BackupException {
+    LOG.debug("Getting the direct ancestors of the current backup " + backupInfo.getBackupId());
+
+    ArrayList<BackupImage> ancestors = new ArrayList<BackupImage>();
+
+    // full backup does not have ancestor
+    if (backupInfo.getType() == BackupType.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<BackupInfo> allHistoryList = getBackupHistory(true);
+    for (BackupInfo backup : allHistoryList) {
+
+      BackupImage.Builder builder = BackupImage.newBuilder();
+
+      BackupImage image =
+          builder.withBackupId(backup.getBackupId()).withType(backup.getType())
+              .withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames())
+              .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build();
+
+      // add the full backup image as an ancestor until the last incremental backup
+      if (backup.getType().equals(BackupType.FULL)) {
+        // 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:");
+          for (BackupImage image1 : ancestors) {
+            LOG.debug("  BackupID=" + image1.getBackupId() + ", BackupDir=" + image1.getRootDir());
+          }
+        } else {
+          Path logBackupPath =
+              HBackupFileSystem.getLogBackupPath(backup.getBackupRootDir(), backup.getBackupId());
+          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: " + "{BackupID="
+              + lastIncrImage.getBackupId() + "," + "BackupDir=" + 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 backupInfo backup info
+   * @param table table
+   * @return backupImages on the dependency list
+   * @throws BackupException exception
+   * @throws IOException exception
+   */
+  public ArrayList<BackupImage> getAncestors(BackupInfo backupInfo, TableName table)
+      throws BackupException, IOException {
+    ArrayList<BackupImage> ancestors = getAncestors(backupInfo);
+    ArrayList<BackupImage> tableAncestors = new ArrayList<BackupImage>();
+    for (BackupImage image : ancestors) {
+      if (image.hasTable(table)) {
+        tableAncestors.add(image);
+        if (image.getType() == BackupType.FULL) {
+          break;
+        }
+      }
+    }
+    return tableAncestors;
+  }
+
+  /*
+   * backup system table operations
+   */
+
+  /**
+   * Updates status (state) of a backup session in a persistent store
+   * @param context context
+   * @throws IOException exception
+   */
+  public void updateBackupInfo(BackupInfo context) throws IOException {
+    systemTable.updateBackupInfo(context);
+  }
+
+  /**
+   * Read the last backup start code (timestamp) of last successful backup. Will return null if
+   * there is no startcode stored in backup system table 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(backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Write the start code (timestamp) to backup system table. If passed in null, then write 0 byte.
+   * @param startCode start code
+   * @throws IOException exception
+   */
+  public void writeBackupStartCode(Long startCode) throws IOException {
+    systemTable.writeBackupStartCode(startCode, backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Get the RS log information after the last log roll from backup system table.
+   * @return RS log info
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> readRegionServerLastLogRollResult() throws IOException {
+    return systemTable.readRegionServerLastLogRollResult(backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Get all completed backup information (in desc order by time)
+   * @return history info of BackupCompleteData
+   * @throws IOException exception
+   */
+  public List<BackupInfo> getBackupHistory() throws IOException {
+    return systemTable.getBackupHistory();
+  }
+
+  public ArrayList<BackupInfo> getBackupHistory(boolean completed) throws IOException {
+    return systemTable.getBackupHistory(completed);
+  }
+
+  /**
+   * Write the current timestamps for each regionserver to backup system table 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<TableName> tables,
+      HashMap<String, Long> newTimestamps) throws IOException {
+    systemTable.writeRegionServerLogTimestamp(tables, newTimestamps,
+      backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Read the timestamp for each region server log after the last successful backup. Each table has
+   * its own set of the timestamps.
+   * @return the timestamp for each region server. key: tableName value:
+   *         RegionServer,PreviousTimeStamp
+   * @throws IOException exception
+   */
+  public HashMap<TableName, HashMap<String, Long>> readLogTimestampMap() throws IOException {
+    return systemTable.readLogTimestampMap(backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Return the current tables covered by incremental backup.
+   * @return set of tableNames
+   * @throws IOException exception
+   */
+  public Set<TableName> getIncrementalBackupTableSet() throws IOException {
+    return systemTable.getIncrementalBackupTableSet(backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Adds set of tables to overall incremental backup table set
+   * @param tables tables
+   * @throws IOException exception
+   */
+  public void addIncrementalBackupTableSet(Set<TableName> tables) throws IOException {
+    systemTable.addIncrementalBackupTableSet(tables, backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * 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, backupInfo.getBackupId(), backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Get WAL files iterator
+   * @return WAL files iterator from backup system table
+   * @throws IOException
+   */
+  public Iterator<BackupSystemTable.WALItem> getWALFilesFromBackupSystem() throws IOException {
+    return systemTable.getWALFilesIterator(backupInfo.getBackupRootDir());
+  }
+
+  public Connection getConnection() {
+    return conn;
+  }
+}


[12/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov) - add ZKProcedureCoordinator

Posted by te...@apache.org.
HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov) - add ZKProcedureCoordinator


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

Branch: refs/heads/HBASE-14123
Commit: a4e03f60ded2765ce18e2412bde0bf01fbfa7be0
Parents: 063f539
Author: tedyu <yu...@gmail.com>
Authored: Fri Mar 10 17:13:47 2017 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Mar 10 17:13:47 2017 -0800

----------------------------------------------------------------------
 .../hbase/procedure/ZKProcedureCoordinator.java | 328 +++++++++++++++++++
 1 file changed, 328 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a4e03f60/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
new file mode 100644
index 0000000..b656894
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
@@ -0,0 +1,328 @@
+/**
+ * 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.procedure;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.errorhandling.ForeignException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * ZooKeeper based {@link ProcedureCoordinatorRpcs} for a {@link ProcedureCoordinator}
+ */
+@InterfaceAudience.Private
+public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
+  private static final Log LOG = LogFactory.getLog(ZKProcedureCoordinator.class);
+  private ZKProcedureUtil zkProc = null;
+  protected ProcedureCoordinator coordinator = null;  // if started this should be non-null
+
+  ZooKeeperWatcher watcher;
+  String procedureType;
+  String coordName;
+
+  /**
+   * @param watcher zookeeper watcher. Owned by <tt>this</tt> and closed via {@link #close()}
+   * @param procedureClass procedure type name is a category for when there are multiple kinds of
+   *    procedures.-- this becomes a znode so be aware of the naming restrictions
+   * @param coordName name of the node running the coordinator
+   * @throws KeeperException if an unexpected zk error occurs
+   */
+  public ZKProcedureCoordinator(ZooKeeperWatcher watcher,
+      String procedureClass, String coordName) {
+    this.watcher = watcher;
+    this.procedureType = procedureClass;
+    this.coordName = coordName;
+  }
+
+  /**
+   * The "acquire" phase.  The coordinator creates a new procType/acquired/ znode dir. If znodes
+   * appear, first acquire to relevant listener or sets watch waiting for notification of
+   * the acquire node
+   *
+   * @param proc the Procedure
+   * @param info data to be stored in the acquire node
+   * @param nodeNames children of the acquire phase
+   * @throws IOException if any failure occurs.
+   */
+  @Override
+  final public void sendGlobalBarrierAcquire(Procedure proc, byte[] info, List<String> nodeNames)
+      throws IOException, IllegalArgumentException {
+    String procName = proc.getName();
+    // start watching for the abort node
+    String abortNode = zkProc.getAbortZNode(procName);
+    try {
+      // check to see if the abort node already exists
+      if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), abortNode)) {
+        abort(abortNode);
+      }
+      // If we get an abort node watch triggered here, we'll go complete creating the acquired
+      // znode but then handle the acquire znode and bail out
+    } catch (KeeperException e) {
+      String msg = "Failed while watching abort node:" + abortNode;
+      LOG.error(msg, e);
+      throw new IOException(msg, e);
+    }
+
+    // create the acquire barrier
+    String acquire = zkProc.getAcquiredBarrierNode(procName);
+    LOG.debug("Creating acquire znode:" + acquire);
+    try {
+      // notify all the procedure listeners to look for the acquire node
+      byte[] data = ProtobufUtil.prependPBMagic(info);
+      ZKUtil.createWithParents(zkProc.getWatcher(), acquire, data);
+      // loop through all the children of the acquire phase and watch for them
+      for (String node : nodeNames) {
+        String znode = ZKUtil.joinZNode(acquire, node);
+        LOG.debug("Watching for acquire node:" + znode);
+        if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
+          coordinator.memberAcquiredBarrier(procName, node);
+        }
+      }
+    } catch (KeeperException e) {
+      String msg = "Failed while creating acquire node:" + acquire;
+      LOG.error(msg, e);
+      throw new IOException(msg, e);
+    }
+  }
+
+  @Override
+  public void sendGlobalBarrierReached(Procedure proc, List<String> nodeNames) throws IOException {
+    String procName = proc.getName();
+    String reachedNode = zkProc.getReachedBarrierNode(procName);
+    LOG.debug("Creating reached barrier zk node:" + reachedNode);
+    try {
+      // create the reached znode and watch for the reached znodes
+      ZKUtil.createWithParents(zkProc.getWatcher(), reachedNode);
+      // loop through all the children of the acquire phase and watch for them
+      for (String node : nodeNames) {
+        String znode = ZKUtil.joinZNode(reachedNode, node);
+        if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
+          byte[] dataFromMember = ZKUtil.getData(zkProc.getWatcher(), znode);
+          // ProtobufUtil.isPBMagicPrefix will check null
+          if (dataFromMember != null && dataFromMember.length > 0) {
+            if (!ProtobufUtil.isPBMagicPrefix(dataFromMember)) {
+              String msg =
+                "Failed to get data from finished node or data is illegally formatted: " + znode;
+              LOG.error(msg);
+              throw new IOException(msg);
+            } else {
+              dataFromMember = Arrays.copyOfRange(dataFromMember, ProtobufUtil.lengthOfPBMagic(),
+                dataFromMember.length);
+              coordinator.memberFinishedBarrier(procName, node, dataFromMember);
+            }
+          } else {
+            coordinator.memberFinishedBarrier(procName, node, dataFromMember);
+          }
+        }
+      }
+    } catch (KeeperException e) {
+      String msg = "Failed while creating reached node:" + reachedNode;
+      LOG.error(msg, e);
+      throw new IOException(msg, e);
+    } catch (InterruptedException e) {
+      String msg = "Interrupted while creating reached node:" + reachedNode;
+      LOG.error(msg, e);
+      throw new InterruptedIOException(msg);
+    }
+  }
+
+
+  /**
+   * Delete znodes that are no longer in use.
+   */
+  @Override
+  final public void resetMembers(Procedure proc) throws IOException {
+    String procName = proc.getName();
+    boolean stillGettingNotifications = false;
+    do {
+      try {
+        LOG.debug("Attempting to clean out zk node for op:" + procName);
+        zkProc.clearZNodes(procName);
+        stillGettingNotifications = false;
+      } catch (KeeperException.NotEmptyException e) {
+        // recursive delete isn't transactional (yet) so we need to deal with cases where we get
+        // children trickling in
+        stillGettingNotifications = true;
+      } catch (KeeperException e) {
+        String msg = "Failed to complete reset procedure " + procName;
+        LOG.error(msg, e);
+        throw new IOException(msg, e);
+      }
+    } while (stillGettingNotifications);
+  }
+
+  /**
+   * Start monitoring znodes in ZK - subclass hook to start monitoring znodes they are about.
+   * @return true if succeed, false if encountered initialization errors.
+   */
+  @Override
+  final public boolean start(final ProcedureCoordinator coordinator) {
+    if (this.coordinator != null) {
+      throw new IllegalStateException(
+        "ZKProcedureCoordinator already started and already has listener installed");
+    }
+    this.coordinator = coordinator;
+
+    try {
+      this.zkProc = new ZKProcedureUtil(watcher, procedureType) {
+        @Override
+        public void nodeCreated(String path) {
+          if (!isInProcedurePath(path)) return;
+          LOG.debug("Node created: " + path);
+          logZKTree(this.baseZNode);
+          if (isAcquiredPathNode(path)) {
+            // node wasn't present when we created the watch so zk event triggers acquire
+            coordinator.memberAcquiredBarrier(ZKUtil.getNodeName(ZKUtil.getParent(path)),
+              ZKUtil.getNodeName(path));
+          } else if (isReachedPathNode(path)) {
+            // node was absent when we created the watch so zk event triggers the finished barrier.
+
+            // TODO Nothing enforces that acquire and reached znodes from showing up in wrong order.
+            String procName = ZKUtil.getNodeName(ZKUtil.getParent(path));
+            String member = ZKUtil.getNodeName(path);
+            // get the data from the procedure member
+            try {
+              byte[] dataFromMember = ZKUtil.getData(watcher, path);
+              // ProtobufUtil.isPBMagicPrefix will check null
+              if (dataFromMember != null && dataFromMember.length > 0) {
+                if (!ProtobufUtil.isPBMagicPrefix(dataFromMember)) {
+                  ForeignException ee = new ForeignException(coordName,
+                    "Failed to get data from finished node or data is illegally formatted:"
+                        + path);
+                  coordinator.abortProcedure(procName, ee);
+                } else {
+                  dataFromMember = Arrays.copyOfRange(dataFromMember, ProtobufUtil.lengthOfPBMagic(),
+                    dataFromMember.length);
+                  LOG.debug("Finished data from procedure '" + procName
+                    + "' member '" + member + "': " + new String(dataFromMember));
+                  coordinator.memberFinishedBarrier(procName, member, dataFromMember);
+                }
+              } else {
+                coordinator.memberFinishedBarrier(procName, member, dataFromMember);
+              }
+            } catch (KeeperException e) {
+              ForeignException ee = new ForeignException(coordName, e);
+              coordinator.abortProcedure(procName, ee);
+            } catch (InterruptedException e) {
+              ForeignException ee = new ForeignException(coordName, e);
+              coordinator.abortProcedure(procName, ee);
+            }
+          } else if (isAbortPathNode(path)) {
+            abort(path);
+          } else {
+            LOG.debug("Ignoring created notification for node:" + path);
+          }
+        }
+      };
+      zkProc.clearChildZNodes();
+    } catch (KeeperException e) {
+      LOG.error("Unable to start the ZK-based Procedure Coordinator rpcs.", e);
+      return false;
+    }
+
+    LOG.debug("Starting the controller for procedure member:" + coordName);
+    return true;
+  }
+
+  /**
+   * This is the abort message being sent by the coordinator to member
+   *
+   * TODO this code isn't actually used but can be used to issue a cancellation from the
+   * coordinator.
+   */
+  @Override
+  final public void sendAbortToMembers(Procedure proc, ForeignException ee) {
+    String procName = proc.getName();
+    LOG.debug("Aborting procedure '" + procName + "' in zk");
+    String procAbortNode = zkProc.getAbortZNode(procName);
+    try {
+      LOG.debug("Creating abort znode:" + procAbortNode);
+      String source = (ee.getSource() == null) ? coordName : ee.getSource();
+      byte[] errorInfo = ProtobufUtil.prependPBMagic(ForeignException.serialize(source, ee));
+      // first create the znode for the procedure
+      ZKUtil.createAndFailSilent(zkProc.getWatcher(), procAbortNode, errorInfo);
+      LOG.debug("Finished creating abort node:" + procAbortNode);
+    } catch (KeeperException e) {
+      // possible that we get this error for the procedure if we already reset the zk state, but in
+      // that case we should still get an error for that procedure anyways
+      zkProc.logZKTree(zkProc.baseZNode);
+      coordinator.rpcConnectionFailure("Failed to post zk node:" + procAbortNode
+          + " to abort procedure '" + procName + "'", new IOException(e));
+    }
+  }
+
+  /**
+   * Receive a notification and propagate it to the local coordinator
+   * @param abortNode full znode path to the failed procedure information
+   */
+  protected void abort(String abortNode) {
+    String procName = ZKUtil.getNodeName(abortNode);
+    ForeignException ee = null;
+    try {
+      byte[] data = ZKUtil.getData(zkProc.getWatcher(), abortNode);
+      if (data == null || data.length == 0) {
+        // ignore
+        return;
+      } else if (!ProtobufUtil.isPBMagicPrefix(data)) {
+        LOG.warn("Got an error notification for op:" + abortNode
+            + " but we can't read the information. Killing the procedure.");
+        // we got a remote exception, but we can't describe it
+        ee = new ForeignException(coordName,
+          "Data in abort node is illegally formatted.  ignoring content.");
+      } else {
+
+        data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length);
+        ee = ForeignException.deserialize(data);
+      }
+    } catch (IOException e) {
+      LOG.warn("Got an error notification for op:" + abortNode
+          + " but we can't read the information. Killing the procedure.");
+      // we got a remote exception, but we can't describe it
+      ee = new ForeignException(coordName, e);
+    } catch (KeeperException e) {
+      coordinator.rpcConnectionFailure("Failed to get data for abort node:" + abortNode
+          + zkProc.getAbortZnode(), new IOException(e));
+    } catch (InterruptedException e) {
+      coordinator.rpcConnectionFailure("Failed to get data for abort node:" + abortNode
+          + zkProc.getAbortZnode(), new IOException(e));
+      Thread.currentThread().interrupt();
+    }
+    coordinator.abortProcedure(procName, ee);
+  }
+
+  @Override
+  final public void close() throws IOException {
+    zkProc.close();
+  }
+
+  /**
+   * Used in testing
+   */
+  final ZKProcedureUtil getZkProcedureUtil() {
+    return zkProc;
+  }
+}


[11/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov) - remove ZKProcedureCoordinatorRpcs

Posted by te...@apache.org.
HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov) - remove ZKProcedureCoordinatorRpcs


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

Branch: refs/heads/HBASE-14123
Commit: 063f539c7bb5a92074c370f1144d91eae7a549e8
Parents: 3aaea8e
Author: tedyu <yu...@gmail.com>
Authored: Fri Mar 10 17:08:20 2017 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Mar 10 17:08:20 2017 -0800

----------------------------------------------------------------------
 .../procedure/ZKProcedureCoordinatorRpcs.java   | 328 -------------------
 1 file changed, 328 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/063f539c/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
deleted file mode 100644
index b656894..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
+++ /dev/null
@@ -1,328 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.procedure;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.errorhandling.ForeignException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * ZooKeeper based {@link ProcedureCoordinatorRpcs} for a {@link ProcedureCoordinator}
- */
-@InterfaceAudience.Private
-public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
-  private static final Log LOG = LogFactory.getLog(ZKProcedureCoordinator.class);
-  private ZKProcedureUtil zkProc = null;
-  protected ProcedureCoordinator coordinator = null;  // if started this should be non-null
-
-  ZooKeeperWatcher watcher;
-  String procedureType;
-  String coordName;
-
-  /**
-   * @param watcher zookeeper watcher. Owned by <tt>this</tt> and closed via {@link #close()}
-   * @param procedureClass procedure type name is a category for when there are multiple kinds of
-   *    procedures.-- this becomes a znode so be aware of the naming restrictions
-   * @param coordName name of the node running the coordinator
-   * @throws KeeperException if an unexpected zk error occurs
-   */
-  public ZKProcedureCoordinator(ZooKeeperWatcher watcher,
-      String procedureClass, String coordName) {
-    this.watcher = watcher;
-    this.procedureType = procedureClass;
-    this.coordName = coordName;
-  }
-
-  /**
-   * The "acquire" phase.  The coordinator creates a new procType/acquired/ znode dir. If znodes
-   * appear, first acquire to relevant listener or sets watch waiting for notification of
-   * the acquire node
-   *
-   * @param proc the Procedure
-   * @param info data to be stored in the acquire node
-   * @param nodeNames children of the acquire phase
-   * @throws IOException if any failure occurs.
-   */
-  @Override
-  final public void sendGlobalBarrierAcquire(Procedure proc, byte[] info, List<String> nodeNames)
-      throws IOException, IllegalArgumentException {
-    String procName = proc.getName();
-    // start watching for the abort node
-    String abortNode = zkProc.getAbortZNode(procName);
-    try {
-      // check to see if the abort node already exists
-      if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), abortNode)) {
-        abort(abortNode);
-      }
-      // If we get an abort node watch triggered here, we'll go complete creating the acquired
-      // znode but then handle the acquire znode and bail out
-    } catch (KeeperException e) {
-      String msg = "Failed while watching abort node:" + abortNode;
-      LOG.error(msg, e);
-      throw new IOException(msg, e);
-    }
-
-    // create the acquire barrier
-    String acquire = zkProc.getAcquiredBarrierNode(procName);
-    LOG.debug("Creating acquire znode:" + acquire);
-    try {
-      // notify all the procedure listeners to look for the acquire node
-      byte[] data = ProtobufUtil.prependPBMagic(info);
-      ZKUtil.createWithParents(zkProc.getWatcher(), acquire, data);
-      // loop through all the children of the acquire phase and watch for them
-      for (String node : nodeNames) {
-        String znode = ZKUtil.joinZNode(acquire, node);
-        LOG.debug("Watching for acquire node:" + znode);
-        if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
-          coordinator.memberAcquiredBarrier(procName, node);
-        }
-      }
-    } catch (KeeperException e) {
-      String msg = "Failed while creating acquire node:" + acquire;
-      LOG.error(msg, e);
-      throw new IOException(msg, e);
-    }
-  }
-
-  @Override
-  public void sendGlobalBarrierReached(Procedure proc, List<String> nodeNames) throws IOException {
-    String procName = proc.getName();
-    String reachedNode = zkProc.getReachedBarrierNode(procName);
-    LOG.debug("Creating reached barrier zk node:" + reachedNode);
-    try {
-      // create the reached znode and watch for the reached znodes
-      ZKUtil.createWithParents(zkProc.getWatcher(), reachedNode);
-      // loop through all the children of the acquire phase and watch for them
-      for (String node : nodeNames) {
-        String znode = ZKUtil.joinZNode(reachedNode, node);
-        if (ZKUtil.watchAndCheckExists(zkProc.getWatcher(), znode)) {
-          byte[] dataFromMember = ZKUtil.getData(zkProc.getWatcher(), znode);
-          // ProtobufUtil.isPBMagicPrefix will check null
-          if (dataFromMember != null && dataFromMember.length > 0) {
-            if (!ProtobufUtil.isPBMagicPrefix(dataFromMember)) {
-              String msg =
-                "Failed to get data from finished node or data is illegally formatted: " + znode;
-              LOG.error(msg);
-              throw new IOException(msg);
-            } else {
-              dataFromMember = Arrays.copyOfRange(dataFromMember, ProtobufUtil.lengthOfPBMagic(),
-                dataFromMember.length);
-              coordinator.memberFinishedBarrier(procName, node, dataFromMember);
-            }
-          } else {
-            coordinator.memberFinishedBarrier(procName, node, dataFromMember);
-          }
-        }
-      }
-    } catch (KeeperException e) {
-      String msg = "Failed while creating reached node:" + reachedNode;
-      LOG.error(msg, e);
-      throw new IOException(msg, e);
-    } catch (InterruptedException e) {
-      String msg = "Interrupted while creating reached node:" + reachedNode;
-      LOG.error(msg, e);
-      throw new InterruptedIOException(msg);
-    }
-  }
-
-
-  /**
-   * Delete znodes that are no longer in use.
-   */
-  @Override
-  final public void resetMembers(Procedure proc) throws IOException {
-    String procName = proc.getName();
-    boolean stillGettingNotifications = false;
-    do {
-      try {
-        LOG.debug("Attempting to clean out zk node for op:" + procName);
-        zkProc.clearZNodes(procName);
-        stillGettingNotifications = false;
-      } catch (KeeperException.NotEmptyException e) {
-        // recursive delete isn't transactional (yet) so we need to deal with cases where we get
-        // children trickling in
-        stillGettingNotifications = true;
-      } catch (KeeperException e) {
-        String msg = "Failed to complete reset procedure " + procName;
-        LOG.error(msg, e);
-        throw new IOException(msg, e);
-      }
-    } while (stillGettingNotifications);
-  }
-
-  /**
-   * Start monitoring znodes in ZK - subclass hook to start monitoring znodes they are about.
-   * @return true if succeed, false if encountered initialization errors.
-   */
-  @Override
-  final public boolean start(final ProcedureCoordinator coordinator) {
-    if (this.coordinator != null) {
-      throw new IllegalStateException(
-        "ZKProcedureCoordinator already started and already has listener installed");
-    }
-    this.coordinator = coordinator;
-
-    try {
-      this.zkProc = new ZKProcedureUtil(watcher, procedureType) {
-        @Override
-        public void nodeCreated(String path) {
-          if (!isInProcedurePath(path)) return;
-          LOG.debug("Node created: " + path);
-          logZKTree(this.baseZNode);
-          if (isAcquiredPathNode(path)) {
-            // node wasn't present when we created the watch so zk event triggers acquire
-            coordinator.memberAcquiredBarrier(ZKUtil.getNodeName(ZKUtil.getParent(path)),
-              ZKUtil.getNodeName(path));
-          } else if (isReachedPathNode(path)) {
-            // node was absent when we created the watch so zk event triggers the finished barrier.
-
-            // TODO Nothing enforces that acquire and reached znodes from showing up in wrong order.
-            String procName = ZKUtil.getNodeName(ZKUtil.getParent(path));
-            String member = ZKUtil.getNodeName(path);
-            // get the data from the procedure member
-            try {
-              byte[] dataFromMember = ZKUtil.getData(watcher, path);
-              // ProtobufUtil.isPBMagicPrefix will check null
-              if (dataFromMember != null && dataFromMember.length > 0) {
-                if (!ProtobufUtil.isPBMagicPrefix(dataFromMember)) {
-                  ForeignException ee = new ForeignException(coordName,
-                    "Failed to get data from finished node or data is illegally formatted:"
-                        + path);
-                  coordinator.abortProcedure(procName, ee);
-                } else {
-                  dataFromMember = Arrays.copyOfRange(dataFromMember, ProtobufUtil.lengthOfPBMagic(),
-                    dataFromMember.length);
-                  LOG.debug("Finished data from procedure '" + procName
-                    + "' member '" + member + "': " + new String(dataFromMember));
-                  coordinator.memberFinishedBarrier(procName, member, dataFromMember);
-                }
-              } else {
-                coordinator.memberFinishedBarrier(procName, member, dataFromMember);
-              }
-            } catch (KeeperException e) {
-              ForeignException ee = new ForeignException(coordName, e);
-              coordinator.abortProcedure(procName, ee);
-            } catch (InterruptedException e) {
-              ForeignException ee = new ForeignException(coordName, e);
-              coordinator.abortProcedure(procName, ee);
-            }
-          } else if (isAbortPathNode(path)) {
-            abort(path);
-          } else {
-            LOG.debug("Ignoring created notification for node:" + path);
-          }
-        }
-      };
-      zkProc.clearChildZNodes();
-    } catch (KeeperException e) {
-      LOG.error("Unable to start the ZK-based Procedure Coordinator rpcs.", e);
-      return false;
-    }
-
-    LOG.debug("Starting the controller for procedure member:" + coordName);
-    return true;
-  }
-
-  /**
-   * This is the abort message being sent by the coordinator to member
-   *
-   * TODO this code isn't actually used but can be used to issue a cancellation from the
-   * coordinator.
-   */
-  @Override
-  final public void sendAbortToMembers(Procedure proc, ForeignException ee) {
-    String procName = proc.getName();
-    LOG.debug("Aborting procedure '" + procName + "' in zk");
-    String procAbortNode = zkProc.getAbortZNode(procName);
-    try {
-      LOG.debug("Creating abort znode:" + procAbortNode);
-      String source = (ee.getSource() == null) ? coordName : ee.getSource();
-      byte[] errorInfo = ProtobufUtil.prependPBMagic(ForeignException.serialize(source, ee));
-      // first create the znode for the procedure
-      ZKUtil.createAndFailSilent(zkProc.getWatcher(), procAbortNode, errorInfo);
-      LOG.debug("Finished creating abort node:" + procAbortNode);
-    } catch (KeeperException e) {
-      // possible that we get this error for the procedure if we already reset the zk state, but in
-      // that case we should still get an error for that procedure anyways
-      zkProc.logZKTree(zkProc.baseZNode);
-      coordinator.rpcConnectionFailure("Failed to post zk node:" + procAbortNode
-          + " to abort procedure '" + procName + "'", new IOException(e));
-    }
-  }
-
-  /**
-   * Receive a notification and propagate it to the local coordinator
-   * @param abortNode full znode path to the failed procedure information
-   */
-  protected void abort(String abortNode) {
-    String procName = ZKUtil.getNodeName(abortNode);
-    ForeignException ee = null;
-    try {
-      byte[] data = ZKUtil.getData(zkProc.getWatcher(), abortNode);
-      if (data == null || data.length == 0) {
-        // ignore
-        return;
-      } else if (!ProtobufUtil.isPBMagicPrefix(data)) {
-        LOG.warn("Got an error notification for op:" + abortNode
-            + " but we can't read the information. Killing the procedure.");
-        // we got a remote exception, but we can't describe it
-        ee = new ForeignException(coordName,
-          "Data in abort node is illegally formatted.  ignoring content.");
-      } else {
-
-        data = Arrays.copyOfRange(data, ProtobufUtil.lengthOfPBMagic(), data.length);
-        ee = ForeignException.deserialize(data);
-      }
-    } catch (IOException e) {
-      LOG.warn("Got an error notification for op:" + abortNode
-          + " but we can't read the information. Killing the procedure.");
-      // we got a remote exception, but we can't describe it
-      ee = new ForeignException(coordName, e);
-    } catch (KeeperException e) {
-      coordinator.rpcConnectionFailure("Failed to get data for abort node:" + abortNode
-          + zkProc.getAbortZnode(), new IOException(e));
-    } catch (InterruptedException e) {
-      coordinator.rpcConnectionFailure("Failed to get data for abort node:" + abortNode
-          + zkProc.getAbortZnode(), new IOException(e));
-      Thread.currentThread().interrupt();
-    }
-    coordinator.abortProcedure(procName, ee);
-  }
-
-  @Override
-  final public void close() throws IOException {
-    zkProc.close();
-  }
-
-  /**
-   * Used in testing
-   */
-  final ZKProcedureUtil getZkProcedureUtil() {
-    return zkProc;
-  }
-}


[04/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..47e428c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
@@ -0,0 +1,155 @@
+/**
+ * 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.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+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.procedure.RegionServerProcedureManager;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Master procedure manager for coordinated cluster-wide WAL roll operation, which is run during
+ * backup operation, see {@link MasterProcedureManager} and and {@link RegionServerProcedureManager}
+ */
+@InterfaceAudience.Private
+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 {
+    if (!isBackupEnabled()) {
+      LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY
+          + " setting");
+      return;
+    }
+    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());
+    }
+
+    List<NameStringPair> conf = desc.getConfigurationList();
+    byte[] data = new byte[0];
+    if (conf.size() > 0) {
+      // Get backup root path
+      data = conf.get(0).getValue().getBytes();
+    }
+    Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), data, 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();
+  }
+
+  private boolean isBackupEnabled() {
+    return BackupManager.isBackupEnabled(master.getConfiguration());
+  }
+
+  @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/3aaea8e0/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..8fc644c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.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.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.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+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.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.wal.WAL;
+
+/**
+ * This backup sub-procedure implementation forces a WAL rolling on a RS.
+ */
+@InterfaceAudience.Private
+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;
+  private String backupRoot;
+
+  public LogRollBackupSubprocedure(RegionServerServices rss, ProcedureMember member,
+      ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
+      LogRollBackupSubprocedurePool taskManager, byte[] data) {
+
+    super(member, LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, errorListener,
+        wakeFrequency, timeout);
+    LOG.info("Constructing a LogRollBackupSubprocedure.");
+    this.rss = rss;
+    this.taskManager = taskManager;
+    if (data != null) {
+      backupRoot = new String(data);
+    }
+  }
+
+  /**
+   * 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();
+      List<WAL> wals = rss.getWALs();
+      long highest = -1;
+      for (WAL wal : wals) {
+        if (wal == null) continue;
+        if (((AbstractFSWAL) wal).getFilenum() > highest) {
+          highest = ((AbstractFSWAL) wal).getFilenum();
+        }
+      }
+
+      LOG.info("Trying to roll log in backup subprocedure, current log number: " + filenum
+          + " highest: " + highest + " on " + rss.getServerName());
+      ((HRegionServer) rss).getWalRoller().requestRollAll();
+      long start = EnvironmentEdgeManager.currentTime();
+      while (!((HRegionServer) rss).getWalRoller().walRollFinished()) {
+        Thread.sleep(20);
+      }
+      LOG.debug("log roll took " + (EnvironmentEdgeManager.currentTime() - start));
+      LOG.info("After roll log in backup subprocedure, current log number: " + hlog.getFilenum()
+          + " on " + rss.getServerName());
+
+      Connection connection = rss.getConnection();
+      try (final BackupSystemTable table = new BackupSystemTable(connection)) {
+        // sanity check, good for testing
+        HashMap<String, Long> serverTimestampMap =
+            table.readRegionServerLastLogRollResult(backupRoot);
+        String host = rss.getServerName().getHostname();
+        int port = rss.getServerName().getPort();
+        String server = host + ":" + port;
+        Long sts = serverTimestampMap.get(host);
+        if (sts != null && sts > highest) {
+          LOG.warn("Won't update server's last roll log result: current=" + sts + " new=" + highest);
+          return null;
+        }
+        // write the log number to backup system table.
+        table.writeRegionServerLastLogRollResult(server, highest, backupRoot);
+        return null;
+      } catch (Exception e) {
+        LOG.error(e);
+        throw e;
+      }
+    }
+  }
+
+  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();
+    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/3aaea8e0/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..65a1fa3
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
@@ -0,0 +1,139 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.errorhandling.ForeignException;
+
+/**
+ * Handle running each of the individual tasks for completing a backup procedure on a region
+ * server.
+ */
+@InterfaceAudience.Private
+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/3aaea8e0/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..9d5a858
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
@@ -0,0 +1,185 @@
+/**
+ * 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.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+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.RegionServerServices;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * This manager class handles the work dealing with distributed WAL roll request.
+ * <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 sub-procedure, 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
+ */
+@InterfaceAudience.Private
+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 region servers */
+  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;
+  private boolean started = false;
+
+  /**
+   * Create a default backup procedure manager
+   */
+  public LogRollRegionServerProcedureManager() {
+  }
+
+  /**
+   * Start accepting backup procedure requests.
+   */
+  @Override
+  public void start() {
+    if (!BackupManager.isBackupEnabled(rss.getConfiguration())) {
+      LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY
+          + " setting");
+      return;
+    }
+    this.memberRpcs.start(rss.getServerName().toString(), member);
+    started = true;
+    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 {
+    if (!started) {
+      return;
+    }
+    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(byte[] data) {
+
+    // 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, data);
+
+  }
+
+  /**
+   * 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(data);
+    }
+  }
+
+  @Override
+  public void initialize(RegionServerServices rss) throws KeeperException {
+    this.rss = rss;
+    if (!BackupManager.isBackupEnabled(rss.getConfiguration())) {
+      LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY
+          + " setting");
+      return;
+    }
+    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/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
new file mode 100644
index 0000000..0da6fc4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
@@ -0,0 +1,58 @@
+/**
+ * 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.util;
+
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Backup set is a named group of HBase tables, which are managed together by Backup/Restore
+ * framework. Instead of using list of tables in backup or restore operation, one can use set's name
+ * instead.
+ */
+@InterfaceAudience.Private
+public class BackupSet {
+  private final String name;
+  private final List<TableName> tables;
+
+  public BackupSet(String name, List<TableName> tables) {
+    this.name = name;
+    this.tables = tables;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public List<TableName> getTables() {
+    return tables;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(name).append("={");
+    sb.append(StringUtils.join(tables, ','));
+    sb.append("}");
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
new file mode 100644
index 0000000..e32853d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
@@ -0,0 +1,702 @@
+/**
+ * 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.util;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+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 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.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.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.RestoreRequest;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+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.AbstractFSWALProvider;
+
+/**
+ * A collection for methods used by multiple classes to backup HBase tables.
+ */
+@InterfaceAudience.Private
+public final class BackupUtils {
+  protected static final Log LOG = LogFactory.getLog(BackupUtils.class);
+  public static final String LOGNAME_SEPARATOR = ".";
+
+  private BackupUtils() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * 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
+   */
+  public static HashMap<String, Long> getRSLogTimestampMins(
+      HashMap<TableName, HashMap<String, Long>> rsLogTimestampMap) {
+
+    if (rsLogTimestampMap == null || rsLogTimestampMap.isEmpty()) {
+      return null;
+    }
+
+    HashMap<String, Long> rsLogTimestampMins = new HashMap<String, Long>();
+    HashMap<String, HashMap<TableName, Long>> rsLogTimestampMapByRS =
+        new HashMap<String, HashMap<TableName, Long>>();
+
+    for (Entry<TableName, HashMap<String, Long>> tableEntry : rsLogTimestampMap.entrySet()) {
+      TableName table = tableEntry.getKey();
+      HashMap<String, Long> rsLogTimestamp = tableEntry.getValue();
+      for (Entry<String, Long> rsEntry : rsLogTimestamp.entrySet()) {
+        String rs = rsEntry.getKey();
+        Long ts = rsEntry.getValue();
+        if (!rsLogTimestampMapByRS.containsKey(rs)) {
+          rsLogTimestampMapByRS.put(rs, new HashMap<TableName, Long>());
+          rsLogTimestampMapByRS.get(rs).put(table, ts);
+        } else {
+          rsLogTimestampMapByRS.get(rs).put(table, ts);
+        }
+      }
+    }
+
+    for (Entry<String, HashMap<TableName, Long>> entry : rsLogTimestampMapByRS.entrySet()) {
+      String rs = entry.getKey();
+      rsLogTimestampMins.put(rs, BackupUtils.getMinValue(entry.getValue()));
+    }
+
+    return rsLogTimestampMins;
+  }
+
+  /**
+   * copy out Table RegionInfo into incremental backup image need to consider move this
+   * logic into HBackupFileSystem
+   * @param conn connection
+   * @param backupInfo backup info
+   * @param conf configuration
+   * @throws IOException exception
+   * @throws InterruptedException exception
+   */
+  public static void copyTableRegionInfo(Connection conn, BackupInfo backupInfo,
+      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 (TableName table : backupInfo.getTables()) {
+
+      if (!MetaTableAccessor.tableExists(conn, table)) {
+        LOG.warn("Table " + table + " does not exists, skipping it.");
+        continue;
+      }
+      HTableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table);
+
+      // write a copy of descriptor to the target directory
+      Path target = new Path(backupInfo.getTableBackupDir(table));
+      FileSystem targetFs = target.getFileSystem(conf);
+      FSTableDescriptors descriptors =
+          new FSTableDescriptors(conf, targetFs, FSUtils.getRootDir(conf));
+      descriptors.createTableDescriptorForTableDirectory(target, orig, false);
+      LOG.debug("Attempting to copy table info for:" + table + " target: " + target
+          + " descriptor: " + orig);
+      LOG.debug("Finished copying tableinfo.");
+      List<HRegionInfo> regions = null;
+      regions = MetaTableAccessor.getTableRegions(conn, table);
+      // 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(backupInfo.getTableBackupDir(table)),
+              regionInfo);
+        regionDir =
+            new Path(backupInfo.getTableBackupDir(table), 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, "." + HConstants.REGIONINFO_QUALIFIER_STR);
+    // 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();
+    }
+  }
+
+  /**
+   * Parses hostname:port from WAL file path
+   * @param p path to WAL file
+   * @return hostname:port
+   */
+  public static String parseHostNameFromLogFile(Path p) {
+    try {
+      if (AbstractFSWALProvider.isArchivedLogFile(p)) {
+        return BackupUtils.parseHostFromOldLog(p);
+      } else {
+        ServerName sname = AbstractFSWALProvider.getServerNameFromWALDirectoryName(p);
+        if (sname != null) {
+          return sname.getAddress().toString();
+        } else {
+          LOG.error("Skip log file (can't parse): " + p);
+          return null;
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Skip log file (can't parse): " + p, e);
+      return null;
+    }
+  }
+
+  /**
+   * Returns 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 getUniqueWALFileNamePart(new Path(walFileName));
+  }
+
+  /**
+   * Returns 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();
+  }
+
+  /**
+   * 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.isDirectory()) {
+          totalLength += getFilesLength(fs, fileStatus.getPath());
+        } else {
+          totalLength += fileStatus.getLen();
+        }
+      }
+    }
+    return totalLength;
+  }
+
+  /**
+   * Get list of all old WAL files (WALs and archive)
+   * @param c configuration
+   * @param hostTimestampMap {host,timestamp} map
+   * @return list of WAL files
+   * @throws IOException exception
+   */
+  public static List<String> getWALFilesOlderThan(final Configuration c,
+      final HashMap<String, Long> 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 (AbstractFSWALProvider.isMetaFile(p)) {
+            return false;
+          }
+          String host = parseHostNameFromLogFile(p);
+          if (host == null) {
+            return false;
+          }
+          Long oldTimestamp = hostTimestampMap.get(host);
+          Long currentLogTS = BackupUtils.getCreationTime(p);
+          return currentLogTS <= oldTimestamp;
+        } catch (Exception e) {
+          LOG.warn("Can not parse" + p, e);
+          return false;
+        }
+      }
+    };
+    FileSystem fs = FileSystem.get(c);
+    logFiles = BackupUtils.getFiles(fs, logDir, logFiles, filter);
+    logFiles = BackupUtils.getFiles(fs, oldLogDir, logFiles, filter);
+    return logFiles;
+  }
+
+  public static TableName[] parseTableNames(String tables) {
+    if (tables == null) {
+      return null;
+    }
+    String[] tableArray = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+
+    TableName[] ret = new TableName[tableArray.length];
+    for (int i = 0; i < tableArray.length; i++) {
+      ret[i] = TableName.valueOf(tableArray[i]);
+    }
+    return ret;
+  }
+
+
+  /**
+   * Check whether the backup path exist
+   * @param backupStr backup
+   * @param conf configuration
+   * @return Yes if path exists
+   * @throws IOException exception
+   */
+  public 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();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Schema of given url: " + backupStr + " is: " + targetFsScheme);
+    }
+    if (fileSys.exists(backupPath)) {
+      isExist = true;
+    }
+    return isExist;
+  }
+
+  /**
+   * Check target path first, confirm it doesn't exist before backup
+   * @param backupRootPath backup destination path
+   * @param conf configuration
+   * @throws IOException exception
+   */
+  public static void checkTargetDir(String backupRootPath, Configuration conf) throws IOException {
+    boolean targetExists = false;
+    try {
+      targetExists = 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;
+      }
+    }
+
+    if (targetExists) {
+      LOG.info("Using existing backup root dir: " + backupRootPath);
+    } else {
+      LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created.");
+    }
+  }
+
+  /**
+   * Get the min value for all the Values a map.
+   * @param map map
+   * @return the min value
+   */
+  public static <T> Long getMinValue(HashMap<T, Long> map) {
+    Long minTimestamp = null;
+    if (map != null) {
+      ArrayList<Long> timestampList = new ArrayList<Long>(map.values());
+      Collections.sort(timestampList);
+      // The min among all the RS log timestamps will be kept in backup system table table.
+      minTimestamp = timestampList.get(0);
+    }
+    return minTimestamp;
+  }
+
+  /**
+   * Parses host name:port from archived WAL path
+   * @param p path
+   * @return host name
+   * @throws IOException exception
+   */
+  public static String parseHostFromOldLog(Path p) {
+    try {
+      String n = p.getName();
+      int idx = n.lastIndexOf(LOGNAME_SEPARATOR);
+      String s = URLDecoder.decode(n.substring(0, idx), "UTF8");
+      return ServerName.parseHostname(s) + ":" + ServerName.parsePort(s);
+    } catch (Exception e) {
+      LOG.warn("Skip log file (can't parse): " + p);
+      return null;
+    }
+  }
+
+  /**
+   * 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
+   */
+  public static Long getCreationTime(Path p) 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 Long.parseLong(ts);
+  }
+
+  public 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());
+      }
+    }
+    return files;
+  }
+
+  public static void cleanupBackupData(BackupInfo context, Configuration conf) throws IOException {
+    cleanupHLogDir(context, conf);
+    cleanupTargetDir(context, conf);
+  }
+
+  /**
+   * Clean up directories which are generated when DistCp copying hlogs
+   * @param backupInfo backup info
+   * @param conf configuration
+   * @throws IOException exception
+   */
+  private static void cleanupHLogDir(BackupInfo backupInfo, Configuration conf)
+      throws IOException {
+
+    String logDir = backupInfo.getHLogTargetDir();
+    if (logDir == null) {
+      LOG.warn("No log directory specified for " + backupInfo.getBackupId());
+      return;
+    }
+
+    Path rootPath = new Path(logDir).getParent();
+    FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
+    FileStatus[] files = listStatus(fs, rootPath, null);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      LOG.debug("Delete log files: " + file.getPath().getName());
+      fs.delete(file.getPath(), true);
+    }
+  }
+
+
+  private static void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) {
+    try {
+      // clean up the data at target directory
+      LOG.debug("Trying to cleanup up target dir : " + backupInfo.getBackupId());
+      String targetDir = backupInfo.getBackupRootDir();
+      if (targetDir == null) {
+        LOG.warn("No target directory specified for " + backupInfo.getBackupId());
+        return;
+      }
+
+      FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf);
+
+      for (TableName table : backupInfo.getTables()) {
+        Path targetDirPath =
+            new Path(getTableBackupDir(backupInfo.getBackupRootDir(), backupInfo.getBackupId(),
+              table));
+        if (outputFs.delete(targetDirPath, true)) {
+          LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done.");
+        } else {
+          LOG.info("No data has been found in " + targetDirPath.toString() + ".");
+        }
+
+        Path tableDir = targetDirPath.getParent();
+        FileStatus[] backups = listStatus(outputFs, tableDir, null);
+        if (backups == null || backups.length == 0) {
+          outputFs.delete(tableDir, true);
+          LOG.debug(tableDir.toString() + " is empty, remove it.");
+        }
+      }
+      outputFs.delete(new Path(targetDir, backupInfo.getBackupId()), true);
+    } catch (IOException e1) {
+      LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " at "
+          + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + ".");
+    }
+  }
+
+  /**
+   * 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/backup_1396650096738/default/t1_dn/"
+   * @param backupRootDir backup root directory
+   * @param backupId backup id
+   * @param tableName table name
+   * @return backupPath String for the particular table
+   */
+  public static String getTableBackupDir(String backupRootDir, String backupId,
+      TableName tableName) {
+    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
+        + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString()
+        + Path.SEPARATOR;
+  }
+
+  /**
+   * Sort history list by start time in descending order.
+   * @param historyList history list
+   * @return sorted list of BackupCompleteData
+   */
+  public static ArrayList<BackupInfo> sortHistoryListDesc(ArrayList<BackupInfo> historyList) {
+    ArrayList<BackupInfo> list = new ArrayList<BackupInfo>();
+    TreeMap<String, BackupInfo> map = new TreeMap<String, BackupInfo>();
+    for (BackupInfo h : historyList) {
+      map.put(Long.toString(h.getStartTs()), h);
+    }
+    Iterator<String> i = map.descendingKeySet().iterator();
+    while (i.hasNext()) {
+      list.add(map.get(i.next()));
+    }
+    return list;
+  }
+
+
+  /**
+   * Calls fs.listStatus() and treats FileNotFoundException as non-fatal This accommodates
+   * differences between hadoop versions, where hadoop 1 does not throw a FileNotFoundException, and
+   * return an empty FileStatus[] while Hadoop 2 will throw FileNotFoundException.
+   * @param fs file system
+   * @param dir directory
+   * @param filter path filter
+   * @return null if dir is empty or doesn't exist, otherwise FileStatus array
+   */
+  public static FileStatus[]
+      listStatus(final FileSystem fs, final Path dir, final PathFilter filter) throws IOException {
+    FileStatus[] status = null;
+    try {
+      status = filter == null ? fs.listStatus(dir) : fs.listStatus(dir, filter);
+    } catch (FileNotFoundException fnfe) {
+      // if directory doesn't exist, return null
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(dir + " doesn't exist");
+      }
+    }
+    if (status == null || status.length < 1) return null;
+    return status;
+  }
+
+  /**
+   * Return the 'path' component of a Path. In Hadoop, Path is an URI. This method returns the
+   * 'path' component of a Path's URI: e.g. If a Path is
+   * <code>hdfs://example.org:9000/hbase_trunk/TestTable/compaction.dir</code>, this method returns
+   * <code>/hbase_trunk/TestTable/compaction.dir</code>. This method is useful if you want to print
+   * out a Path without qualifying Filesystem instance.
+   * @param p file system Path whose 'path' component we are to return.
+   * @return Path portion of the Filesystem
+   */
+  public static String getPath(Path p) {
+    return p.toUri().getPath();
+  }
+
+  /**
+   * 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"
+   */
+  public static String getLogBackupDir(String backupRootDir, String backupId) {
+    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
+        + HConstants.HREGION_LOGDIR_NAME;
+  }
+
+  private static List<BackupInfo> getHistory(Configuration conf, Path backupRootPath)
+      throws IOException {
+    // Get all (n) history from backup root destination
+    FileSystem fs = FileSystem.get(conf);
+    RemoteIterator<LocatedFileStatus> it = fs.listLocatedStatus(backupRootPath);
+
+    List<BackupInfo> infos = new ArrayList<BackupInfo>();
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (!lfs.isDirectory()) continue;
+      String backupId = lfs.getPath().getName();
+      try {
+        BackupInfo info = loadBackupInfo(backupRootPath, backupId, fs);
+        infos.add(info);
+      } catch (IOException e) {
+        LOG.error("Can not load backup info from: " + lfs.getPath(), e);
+      }
+    }
+    // Sort
+    Collections.sort(infos, new Comparator<BackupInfo>() {
+
+      @Override
+      public int compare(BackupInfo o1, BackupInfo o2) {
+        long ts1 = getTimestamp(o1.getBackupId());
+        long ts2 = getTimestamp(o2.getBackupId());
+        if (ts1 == ts2) return 0;
+        return ts1 < ts2 ? 1 : -1;
+      }
+
+      private long getTimestamp(String backupId) {
+        String[] split = backupId.split("_");
+        return Long.parseLong(split[1]);
+      }
+    });
+    return infos;
+  }
+
+  public static List<BackupInfo> getHistory(Configuration conf, int n, Path backupRootPath,
+      BackupInfo.Filter... filters) throws IOException {
+    List<BackupInfo> infos = getHistory(conf, backupRootPath);
+    List<BackupInfo> ret = new ArrayList<BackupInfo>();
+    for (BackupInfo info : infos) {
+      if (ret.size() == n) {
+        break;
+      }
+      boolean passed = true;
+      for (int i = 0; i < filters.length; i++) {
+        if (!filters[i].apply(info)) {
+          passed = false;
+          break;
+        }
+      }
+      if (passed) {
+        ret.add(info);
+      }
+    }
+    return ret;
+  }
+
+  public static BackupInfo loadBackupInfo(Path backupRootPath, String backupId, FileSystem fs)
+      throws IOException {
+    Path backupPath = new Path(backupRootPath, backupId);
+
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(backupPath, true);
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (lfs.getPath().getName().equals(BackupManifest.MANIFEST_FILE_NAME)) {
+        // Load BackupManifest
+        BackupManifest manifest = new BackupManifest(fs, lfs.getPath().getParent());
+        BackupInfo info = manifest.toBackupInfo();
+        return info;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Create restore request.
+   * @param backupRootDir backup root dir
+   * @param backupId backup id
+   * @param check check only
+   * @param fromTables table list from
+   * @param toTables   table list to
+   * @param isOverwrite overwrite data
+   * @return request obkect
+   */
+  public static RestoreRequest createRestoreRequest(String backupRootDir, String backupId,
+      boolean check, TableName[] fromTables, TableName[] toTables, boolean isOverwrite) {
+    RestoreRequest.Builder builder = new RestoreRequest.Builder();
+    RestoreRequest request = builder.withBackupRootDir(backupRootDir)
+                                    .withBackupId(backupId)
+                                    .withCheck(check)
+                                    .withFromTables(fromTables)
+                                    .withToTables(toTables)
+                                    .withOvewrite(isOverwrite).build();
+    return request;
+  }
+
+  public static boolean validate(HashMap<TableName, BackupManifest> backupManifestMap,
+      Configuration conf) throws IOException {
+    boolean isValid = true;
+
+    for (Entry<TableName, BackupManifest> manifestEntry : backupManifestMap.entrySet()) {
+      TableName table = manifestEntry.getKey();
+      TreeSet<BackupImage> imageSet = new TreeSet<BackupImage>();
+
+      ArrayList<BackupImage> depList = manifestEntry.getValue().getDependentListByTable(table);
+      if (depList != null && !depList.isEmpty()) {
+        imageSet.addAll(depList);
+      }
+
+      LOG.info("Dependent image(s) from old to new:");
+      for (BackupImage image : imageSet) {
+        String imageDir =
+            HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table);
+        if (!BackupUtils.checkPathExist(imageDir, conf)) {
+          LOG.error("ERROR: backup image does not exist: " + imageDir);
+          isValid = false;
+          break;
+        }
+        LOG.info("Backup image: " + image.getBackupId() + " for '" + table + "' is available");
+      }
+    }
+    return isValid;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
new file mode 100644
index 0000000..a130c21
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -0,0 +1,610 @@
+/**
+ * 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.util;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+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.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.BackupRestoreFactory;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.RestoreJob;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+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.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
+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.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+
+/**
+ * A collection for methods used by multiple classes to restore HBase tables.
+ */
+@InterfaceAudience.Private
+public class RestoreTool {
+
+  public static final Log LOG = LogFactory.getLog(BackupUtils.class);
+
+  private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR };
+
+  private final static long TABLE_AVAILABILITY_WAIT_TIME = 180000;
+
+  protected Configuration conf = null;
+
+  protected Path backupRootPath;
+
+  protected String backupId;
+
+  protected FileSystem fs;
+  private final Path restoreTmpPath;
+
+  // store table name and snapshot dir mapping
+  private final HashMap<TableName, Path> snapshotMap = new HashMap<>();
+
+  public RestoreTool(Configuration conf, final Path backupRootPath, final String backupId)
+      throws IOException {
+    this.conf = conf;
+    this.backupRootPath = backupRootPath;
+    this.backupId = backupId;
+    this.fs = backupRootPath.getFileSystem(conf);
+    this.restoreTmpPath =
+        new Path(conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY,
+          HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY), "restore");
+  }
+
+  /**
+   * 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
+   */
+  Path getTableArchivePath(TableName tableName) throws IOException {
+
+    Path baseDir =
+        new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, backupId),
+            HConstants.HFILE_ARCHIVE_DIRECTORY);
+    Path dataDir = new Path(baseDir, HConstants.BASE_NAMESPACE_DIR);
+    Path archivePath = new Path(dataDir, tableName.getNamespaceAsString());
+    Path tableArchivePath = new Path(archivePath, 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;
+  }
+
+  /**
+   * Gets region list
+   * @param tableName table name
+   * @return RegionList region list
+   * @throws FileNotFoundException exception
+   * @throws IOException exception
+   */
+  ArrayList<Path> getRegionList(TableName tableName) throws FileNotFoundException, IOException {
+    Path tableArchivePath = 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;
+  }
+
+
+  void modifyTableSync(Connection conn, HTableDescriptor desc) throws IOException {
+
+    try (Admin admin = conn.getAdmin();) {
+      admin.modifyTable(desc.getTableName(), desc);
+      int attempt = 0;
+      int maxAttempts = 600;
+      while (!admin.isTableAvailable(desc.getTableName())) {
+        Thread.sleep(100);
+        attempt++;
+        if (attempt++ > maxAttempts) {
+          throw new IOException("Timeout expired " + (maxAttempts * 100) + "ms");
+        }
+      }
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * 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 conn HBase connection
+   * @param tableBackupPath backup path
+   * @param logDirs : incremental backup folders, which contains WAL
+   * @param tableNames : source tableNames(table names were backuped)
+   * @param newTableNames : target tableNames(table names to be restored to)
+   * @param incrBackupId incremental backup Id
+   * @throws IOException exception
+   */
+  public void incrementalRestoreTable(Connection conn, Path tableBackupPath, Path[] logDirs,
+      TableName[] tableNames, TableName[] newTableNames, String incrBackupId) throws IOException {
+
+    try (Admin admin = conn.getAdmin();) {
+      if (tableNames.length != newTableNames.length) {
+        throw new IOException("Number of source tables and target tables does not match!");
+      }
+      FileSystem fileSys = tableBackupPath.getFileSystem(this.conf);
+
+      // for incremental backup image, expect the table already created either by user or previous
+      // full backup. Here, check that all new tables exists
+      for (TableName tableName : newTableNames) {
+        if (!admin.tableExists(tableName)) {
+          throw new IOException("HBase table " + tableName
+              + " does not exist. Create the table first, e.g. by restoring a full backup.");
+        }
+      }
+      // adjust table schema
+      for (int i = 0; i < tableNames.length; i++) {
+        TableName tableName = tableNames[i];
+        HTableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, incrBackupId);
+        LOG.debug("Found descriptor " + tableDescriptor + " through " + incrBackupId);
+
+        TableName newTableName = newTableNames[i];
+        HTableDescriptor newTableDescriptor = admin.getTableDescriptor(newTableName);
+        List<HColumnDescriptor> families = Arrays.asList(tableDescriptor.getColumnFamilies());
+        List<HColumnDescriptor> existingFamilies =
+            Arrays.asList(newTableDescriptor.getColumnFamilies());
+        boolean schemaChangeNeeded = false;
+        for (HColumnDescriptor family : families) {
+          if (!existingFamilies.contains(family)) {
+            newTableDescriptor.addFamily(family);
+            schemaChangeNeeded = true;
+          }
+        }
+        for (HColumnDescriptor family : existingFamilies) {
+          if (!families.contains(family)) {
+            newTableDescriptor.removeFamily(family.getName());
+            schemaChangeNeeded = true;
+          }
+        }
+        if (schemaChangeNeeded) {
+          modifyTableSync(conn, newTableDescriptor);
+          LOG.info("Changed " + newTableDescriptor.getTableName() + " to: " + newTableDescriptor);
+        }
+      }
+      RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf);
+
+      restoreService.run(logDirs, tableNames, newTableNames, false);
+    }
+  }
+
+  public void fullRestoreTable(Connection conn, Path tableBackupPath, TableName tableName,
+      TableName newTableName, boolean truncateIfExists, String lastIncrBackupId)
+          throws IOException {
+    restoreTableAndCreate(conn, tableName, newTableName, tableBackupPath, truncateIfExists,
+      lastIncrBackupId);
+  }
+
+  /**
+   * Returns value represent path for path to backup table snapshot directory:
+   * "/$USER/SBACKUP_ROOT/backup_id/namespace/table/.hbase-snapshot"
+   * @param backupRootPath backup root path
+   * @param tableName table name
+   * @param backupId backup Id
+   * @return path for snapshot
+   */
+  Path getTableSnapshotPath(Path backupRootPath, TableName tableName, String backupId) {
+    return new Path(HBackupFileSystem.getTableBackupPath(tableName, backupRootPath, backupId),
+        HConstants.SNAPSHOT_DIR_NAME);
+  }
+
+  /**
+   * Returns value represent path for:
+   * ""/$USER/SBACKUP_ROOT/backup_id/namespace/table/.hbase-snapshot/snapshot_1396650097621_namespace_table"
+   * 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
+   */
+  Path getTableInfoPath(TableName tableName) throws FileNotFoundException, IOException {
+    Path tableSnapShotPath = getTableSnapshotPath(backupRootPath, tableName, backupId);
+    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")) {
+        break;
+      }
+    }
+    return tableInfoPath;
+  }
+
+  /**
+   * Get table descriptor
+   * @param tableName is the table backed up
+   * @return {@link HTableDescriptor} saved in backup image of the table
+   */
+  HTableDescriptor getTableDesc(TableName tableName) throws FileNotFoundException, IOException {
+    Path tableInfoPath = this.getTableInfoPath(tableName);
+    SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath);
+    SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc);
+    HTableDescriptor tableDescriptor = manifest.getTableDescriptor();
+    if (!tableDescriptor.getTableName().equals(tableName)) {
+      LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: "
+          + tableInfoPath.toString());
+      LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString());
+      throw new FileNotFoundException("couldn't find Table Desc for table: " + tableName
+          + " under tableInfoPath: " + tableInfoPath.toString());
+    }
+    return tableDescriptor;
+  }
+
+  /**
+   * 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
+   */
+  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");
+      if (desFs.exists(restoreTmpPath)) {
+        try {
+          desFs.delete(restoreTmpPath, true);
+        } catch (IOException e) {
+          LOG.debug("Failed to delete path: " + restoreTmpPath
+              + ", need to check whether restore target DFS cluster is healthy");
+        }
+      }
+      FileUtil.copy(srcFs, tableArchivePath, desFs, restoreTmpPath, false, conf);
+      LOG.debug("Copied to temporary path on local cluster: " + restoreTmpPath);
+      tableArchivePath = restoreTmpPath;
+    }
+    return tableArchivePath;
+  }
+
+  private HTableDescriptor getTableDescriptor(FileSystem fileSys, TableName tableName,
+      String lastIncrBackupId) throws IOException {
+    if (lastIncrBackupId != null) {
+      String target =
+          BackupUtils.getTableBackupDir(backupRootPath.toString(),
+            lastIncrBackupId, tableName);
+      return FSTableDescriptors.getTableDescriptorFromFs(fileSys, new Path(target));
+    }
+    return null;
+  }
+
+  private void restoreTableAndCreate(Connection conn, TableName tableName, TableName newTableName,
+      Path tableBackupPath, boolean truncateIfExists, String lastIncrBackupId) throws IOException {
+    if (newTableName == null) {
+      newTableName = tableName;
+    }
+    FileSystem fileSys = tableBackupPath.getFileSystem(this.conf);
+
+    // get table descriptor first
+    HTableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, lastIncrBackupId);
+    if (tableDescriptor != null) {
+      LOG.debug("Retrieved descriptor: " + tableDescriptor + " thru " + lastIncrBackupId);
+    }
+
+    if (tableDescriptor == null) {
+      Path tableSnapshotPath = getTableSnapshotPath(backupRootPath, tableName, backupId);
+      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();
+        } else {
+          tableDescriptor = getTableDesc(tableName);
+          snapshotMap.put(tableName, getTableInfoPath(tableName));
+        }
+        if (tableDescriptor == null) {
+          LOG.debug("Found no table descriptor in the snapshot dir, previous schema would be lost");
+        }
+      } else {
+        throw new IOException("Table snapshot directory: " +
+            tableSnapshotPath + " does not exist.");
+      }
+    }
+
+    Path tableArchivePath = getTableArchivePath(tableName);
+    if (tableArchivePath == null) {
+      if (tableDescriptor != null) {
+        // find table descriptor but no archive dir means the table is empty, create table and exit
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("find table descriptor but no archive dir for table " + tableName
+              + ", will only create table");
+        }
+        tableDescriptor.setName(newTableName);
+        checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, null, tableDescriptor,
+          truncateIfExists);
+        return;
+      } else {
+        throw new IllegalStateException("Cannot restore hbase table because directory '"
+            + " tableArchivePath is null.");
+      }
+    }
+
+    if (tableDescriptor == null) {
+      tableDescriptor = new HTableDescriptor(newTableName);
+    } else {
+      tableDescriptor.setName(newTableName);
+    }
+
+    // record all region dirs:
+    // load all files in dir
+    try {
+      ArrayList<Path> regionPathList = getRegionList(tableName);
+
+      // should only try to create the table with all region informations, so we could pre-split
+      // the regions in fine grain
+      checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, regionPathList,
+        tableDescriptor, truncateIfExists);
+      if (tableArchivePath != null) {
+        // start real restore through bulkload
+        // if the backup target is on local cluster, special action needed
+        Path tempTableArchivePath = checkLocalAndBackup(tableArchivePath);
+        if (tempTableArchivePath.equals(tableArchivePath)) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("TableArchivePath for bulkload using existPath: " + tableArchivePath);
+          }
+        } else {
+          regionPathList = getRegionList(tempTableArchivePath); // point to the tempDir
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("TableArchivePath for bulkload using tempPath: " + tempTableArchivePath);
+          }
+        }
+
+        LoadIncrementalHFiles loader = createLoader(tempTableArchivePath, false);
+        for (Path regionPath : regionPathList) {
+          String regionName = regionPath.toString();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Restoring HFiles from directory " + regionName);
+          }
+          String[] args = { regionName, newTableName.getNameAsString() };
+          loader.run(args);
+        }
+      }
+      // we do not recovered edits
+    } catch (Exception e) {
+      throw new IllegalStateException("Cannot restore hbase table", e);
+    }
+  }
+
+  /**
+   * Gets region list
+   * @param tableArchivePath table archive path
+   * @return RegionList region list
+   * @throws FileNotFoundException exception
+   * @throws IOException exception
+   */
+  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;
+  }
+
+  /**
+   * 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 {
+
+    // By default, it is 32 and loader will fail if # of files in any region exceed this
+    // limit. Bad for snapshot restore.
+    this.conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
+    this.conf.set(LoadIncrementalHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes");
+    LoadIncrementalHFiles loader = null;
+    try {
+      loader = new LoadIncrementalHFiles(this.conf);
+    } catch (Exception e1) {
+      throw new IOException(e1);
+    }
+    return loader;
+  }
+
+  /**
+   * 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
+   */
+  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
+    // 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, 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();
+          }
+        }
+      }
+    }
+    return LoadIncrementalHFiles.inferBoundaries(map);
+  }
+
+  /**
+   * Prepare the table for bulkload, most codes copied from
+   * {@link LoadIncrementalHFiles#createTable(String, String)}
+   * @param conn connection
+   * @param tableBackupPath path
+   * @param tableName table name
+   * @param targetTableName target table name
+   * @param regionDirList region directory list
+   * @param htd table descriptor
+   * @param truncateIfExists truncates table if exists
+   * @throws IOException exception
+   */
+  private void checkAndCreateTable(Connection conn, Path tableBackupPath, TableName tableName,
+      TableName targetTableName, ArrayList<Path> regionDirList, HTableDescriptor htd,
+      boolean truncateIfExists) throws IOException {
+    try (Admin admin = conn.getAdmin();) {
+      boolean createNew = false;
+      if (admin.tableExists(targetTableName)) {
+        if (truncateIfExists) {
+          LOG.info("Truncating exising target table '" + targetTableName
+              + "', preserving region splits");
+          admin.disableTable(targetTableName);
+          admin.truncateTable(targetTableName, true);
+        } else {
+          LOG.info("Using exising target table '" + targetTableName + "'");
+        }
+      } else {
+        createNew = true;
+      }
+      if (createNew) {
+        LOG.info("Creating target table '" + targetTableName + "'");
+        byte[][] keys = null;
+        if (regionDirList == null || regionDirList.size() == 0) {
+          admin.createTable(htd, null);
+        } else {
+          keys = generateBoundaryKeys(regionDirList);
+          // create table using table descriptor and region boundaries
+          admin.createTable(htd, keys);
+        }
+        long startTime = EnvironmentEdgeManager.currentTime();
+        while (!admin.isTableAvailable(targetTableName, keys)) {
+          try {
+            Thread.sleep(100);
+          } catch (InterruptedException ie) {
+            Thread.currentThread().interrupt();
+          }
+          if (EnvironmentEdgeManager.currentTime() - startTime > TABLE_AVAILABILITY_WAIT_TIME) {
+            throw new IOException("Time out " + TABLE_AVAILABILITY_WAIT_TIME + "ms expired, table "
+                + targetTableName + " is still not available");
+          }
+        }
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..0e3755b 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,9 +17,14 @@
  */
 package org.apache.hadoop.hbase.coordination;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
+import org.apache.zookeeper.KeeperException;
 
 /**
  * Base class for {@link org.apache.hadoop.hbase.CoordinatedStateManager} implementations.
@@ -51,8 +56,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.ProcedureMemberRpcs}
+   */
+  public abstract ProcedureMemberRpcs
+    getProcedureMemberRpcs(String procType) throws KeeperException;
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..8ce5f9c 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,10 +17,17 @@
  */
 package org.apache.hadoop.hbase.coordination;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import java.io.IOException;
+
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.Server;
+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.procedure.ZKProcedureCoordinator;
+import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.zookeeper.KeeperException;
 
 /**
  * ZooKeeper-based implementation of {@link org.apache.hadoop.hbase.CoordinatedStateManager}.
@@ -49,9 +56,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 ZKProcedureCoordinator(watcher, procType, coordNode);
+  }
+
+  @Override
+  public ProcedureMemberRpcs getProcedureMemberRpcs(String procType) throws KeeperException {
+    return new ZKProcedureMemberRpcs(watcher, procType);
+  }
 }


[08/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 52b0ce5..bb6b40e 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -42223,7 +42223,7 @@ public final class MasterProtos {
        * <code>required .hbase.pb.SnapshotDescription snapshot = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
           snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -70510,7 +70510,7 @@ public final class MasterProtos {
 
       /**
        * <pre>
-       ** Get a run of the CleanerChore
+       ** Get a run of the CleanerChore 
        * </pre>
        *
        * <code>rpc RunCleanerChore(.hbase.pb.RunCleanerChoreRequest) returns (.hbase.pb.RunCleanerChoreResponse);</code>
@@ -72424,7 +72424,7 @@ public final class MasterProtos {
 
     /**
      * <pre>
-     ** Get a run of the CleanerChore
+     ** Get a run of the CleanerChore 
      * </pre>
      *
      * <code>rpc RunCleanerChore(.hbase.pb.RunCleanerChoreRequest) returns (.hbase.pb.RunCleanerChoreResponse);</code>
@@ -76186,32 +76186,32 @@ public final class MasterProtos {
       internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RunCleanerChoreRequest_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RunCleanerChoreRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RunCleanerChoreResponse_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RunCleanerChoreResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetCleanerChoreRunningRequest_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetCleanerChoreRunningRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetCleanerChoreRunningResponse_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetCleanerChoreRunningResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsCleanerChoreEnabledRequest_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsCleanerChoreEnabledRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsCleanerChoreEnabledResponse_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsCleanerChoreEnabledResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-protocol-shaded/src/main/protobuf/Backup.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto b/hbase-protocol-shaded/src/main/protobuf/Backup.proto
new file mode 100644
index 0000000..241d42a
--- /dev/null
+++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto
@@ -0,0 +1,117 @@
+/**
+ * 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 optional 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.
+ */
+
+// This file contains Backup manifest
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
+option java_outer_classname = "BackupProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+/**
+ * Backup type enum: FULL or INCREMENTAL
+ */
+
+enum BackupType {
+  FULL = 0;
+  INCREMENTAL = 1;
+}
+/**
+ * ServerTimestamp keeps last WAL roll time per Region Server
+ */
+message ServerTimestamp {
+  optional ServerName server_name = 1;
+  optional uint64 timestamp = 2;
+}
+
+/**
+ *  TableServerTimestamp keeps last WAL roll time per Region Server & Table
+ *  Each table have different last WAL roll time stamps across cluster, on every RS
+ */
+message TableServerTimestamp {
+  optional TableName table_name = 1;
+  repeated ServerTimestamp server_timestamp = 2;
+}
+
+/**
+ * Structure keeps relevant info for backup restore session
+ */
+message BackupImage {
+  optional string backup_id = 1;
+  optional BackupType backup_type = 2;
+  optional string backup_root_dir = 3;
+  repeated TableName table_list = 4;
+  optional uint64 start_ts = 5;
+  optional uint64 complete_ts = 6;
+  repeated BackupImage ancestors = 7;
+  repeated TableServerTimestamp tst_map = 8;
+
+}
+
+/**
+ * Internal structure used during backup
+ */
+message BackupTableInfo {
+  optional TableName table_name = 1;
+  optional string snapshot_name = 2;
+}
+
+/**
+ * Backup session information
+ */
+message BackupInfo {
+  optional string backup_id = 1;
+  optional BackupType backup_type = 2;
+  optional string backup_root_dir = 3;
+  optional BackupState backup_state = 4;
+  optional BackupPhase backup_phase = 5;
+  optional string failed_message = 6;
+  repeated BackupTableInfo backup_table_info = 7;
+  optional uint64  start_ts = 8;
+  optional uint64  complete_ts = 9;
+  optional uint32 progress = 10;
+  optional uint32 workers_number = 11;
+  optional uint64 bandwidth = 12;
+
+  /**
+   * Backup session states
+   */
+  enum BackupState {
+    RUNNING = 0;
+    COMPLETE = 1;
+    FAILED = 2;
+    CANCELLED = 3;
+  }
+
+  /**
+   * Phases of a backup session in RUNNING state
+   *
+   */
+  enum BackupPhase {
+    REQUEST = 0;
+    SNAPSHOT = 1;
+    PREPARE_INCREMENTAL = 2;
+    SNAPSHOTCOPY = 3;
+    INCREMENTAL_COPY = 4;
+    STORE_MANIFEST = 5;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 09a4512..1eba784 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -674,6 +674,11 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-distcp</artifactId>
+          <version>${hadoop-two.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
         </dependency>
         <dependency>
@@ -759,6 +764,11 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-distcp</artifactId>
+          <version>${hadoop-three.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
new file mode 100644
index 0000000..6f642a4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
@@ -0,0 +1,128 @@
+/**
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * The administrative API for HBase Backup. Construct an instance and call {@link #close()}
+ * afterwards.
+ * <p>
+ * BackupAdmin can be used to create backups, restore data from backups and for other
+ * backup-related operations.
+ * @since 2.0
+ */
+@InterfaceAudience.Private
+public interface BackupAdmin extends Closeable {
+
+  /**
+   * Backup given list of tables fully. This is a synchronous operation.
+   * It returns backup id on success or throw exception on failure.
+   * @param userRequest BackupRequest instance
+   * @return the backup Id
+   */
+
+  String backupTables(final BackupRequest userRequest) throws IOException;
+
+  /**
+   * Restore backup
+   * @param request restore request
+   * @throws IOException exception
+   */
+  void restore(RestoreRequest request) throws IOException;
+
+  /**
+   * Describe backup image command
+   * @param backupId backup id
+   * @return backup info
+   * @throws IOException exception
+   */
+  BackupInfo getBackupInfo(String backupId) throws IOException;
+
+
+  /**
+   * Delete backup image command
+   * @param backupIds backup id list
+   * @return total number of deleted sessions
+   * @throws IOException exception
+   */
+  int deleteBackups(String[] backupIds) throws IOException;
+
+  /**
+   * Show backup history command
+   * @param n last n backup sessions
+   * @return list of backup info objects
+   * @throws IOException exception
+   */
+  List<BackupInfo> getHistory(int n) throws IOException;
+
+  /**
+   * Show backup history command with filters
+   * @param n last n backup sessions
+   * @param f list of filters
+   * @return list of backup info objects
+   * @throws IOException exception
+   */
+  List<BackupInfo> getHistory(int n, BackupInfo.Filter... f) throws IOException;
+
+  /**
+   * Backup sets list command - list all backup sets. Backup set is a named group of tables.
+   * @return all registered backup sets
+   * @throws IOException exception
+   */
+  List<BackupSet> listBackupSets() throws IOException;
+
+  /**
+   * Backup set describe command. Shows list of tables in this particular backup set.
+   * @param name set name
+   * @return backup set description or null
+   * @throws IOException exception
+   */
+  BackupSet getBackupSet(String name) throws IOException;
+
+  /**
+   * Delete backup set command
+   * @param name backup set name
+   * @return true, if success, false - otherwise
+   * @throws IOException exception
+   */
+  boolean deleteBackupSet(String name) throws IOException;
+
+  /**
+   * Add tables to backup set command
+   * @param name name of backup set.
+   * @param tables list of tables to be added to this set.
+   * @throws IOException exception
+   */
+  void addToBackupSet(String name, TableName[] tables) throws IOException;
+
+  /**
+   * Remove tables from backup set
+   * @param name name of backup set.
+   * @param tables list of tables to be removed from this set.
+   * @throws IOException exception
+   */
+  void removeFromBackupSet(String name, TableName[] tables) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
new file mode 100644
index 0000000..007e4c1
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
@@ -0,0 +1,55 @@
+/**
+ * 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.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Backup copy job is a part of a backup process. The concrete implementation is responsible for
+ * copying data from a cluster to backup destination. Concrete implementation is provided by backup
+ * provider, see {@link BackupRestoreFactory}
+ */
+@InterfaceAudience.Private
+public interface BackupCopyJob extends Configurable {
+
+  /**
+   * Copy backup data to destination
+   * @param backupInfo context object
+   * @param backupManager backup manager
+   * @param conf configuration
+   * @param backupType backup type (FULL or INCREMENTAL)
+   * @param options array of options (implementation-specific)
+   * @return result (0 - success, -1 failure )
+   * @throws IOException exception
+   */
+  int copy(BackupInfo backupInfo, BackupManager backupManager, Configuration conf,
+      BackupType backupType, String[] options) throws IOException;
+
+  /**
+   * Cancel copy job
+   * @param jobHandler backup copy job handler
+   * @throws IOException
+   */
+  void cancel(String jobHandler) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
new file mode 100644
index 0000000..5794fce
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
@@ -0,0 +1,204 @@
+/**
+ * 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.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.cli.CommandLine;
+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.backup.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.backup.impl.BackupCommands;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ *
+ * Command-line entry point for backup operation
+ *
+ */
+@InterfaceAudience.Private
+public class BackupDriver extends AbstractHBaseTool {
+
+  private static final Log LOG = LogFactory.getLog(BackupDriver.class);
+  private CommandLine cmd;
+
+  public BackupDriver() throws IOException {
+    init();
+  }
+
+  protected void init() throws IOException {
+    // disable irrelevant loggers to avoid it mess up command output
+    LogUtils.disableZkAndClientLoggers(LOG);
+  }
+
+  private int parseAndRun(String[] args) throws IOException {
+
+    // Check if backup is enabled
+    if (!BackupManager.isBackupEnabled(getConf())) {
+      System.err.println(BackupRestoreConstants.ENABLE_BACKUP);
+      return -1;
+    }
+
+    System.out.println(BackupRestoreConstants.VERIFY_BACKUP);
+
+    String cmd = null;
+    String[] remainArgs = null;
+    if (args == null || args.length == 0) {
+      printToolUsage();
+      return -1;
+    } else {
+      cmd = args[0];
+      remainArgs = new String[args.length - 1];
+      if (args.length > 1) {
+        System.arraycopy(args, 1, remainArgs, 0, args.length - 1);
+      }
+    }
+
+    BackupCommand type = BackupCommand.HELP;
+    if (BackupCommand.CREATE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.CREATE;
+    } else if (BackupCommand.HELP.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.HELP;
+    } else if (BackupCommand.DELETE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.DELETE;
+    } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.DESCRIBE;
+    } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.HISTORY;
+    } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.PROGRESS;
+    } else if (BackupCommand.SET.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.SET;
+    } else {
+      System.out.println("Unsupported command for backup: " + cmd);
+      printToolUsage();
+      return -1;
+    }
+
+    // enable debug logging
+    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+    if (this.cmd.hasOption(OPTION_DEBUG)) {
+      backupClientLogger.setLevel(Level.DEBUG);
+    } else {
+      backupClientLogger.setLevel(Level.INFO);
+    }
+
+    BackupCommands.Command command = BackupCommands.createCommand(getConf(), type, this.cmd);
+    if (type == BackupCommand.CREATE && conf != null) {
+      ((BackupCommands.CreateCommand) command).setConf(conf);
+    }
+    try {
+      command.execute();
+    } catch (IOException e) {
+      if (e.getMessage().equals(BackupCommands.INCORRECT_USAGE)) {
+        return -1;
+      }
+      throw e;
+    }
+    return 0;
+  }
+
+  @Override
+  protected void addOptions() {
+    // define supported options
+    addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC);
+    addOptWithArg(OPTION_TABLE, OPTION_TABLE_DESC);
+    addOptWithArg(OPTION_BANDWIDTH, OPTION_BANDWIDTH_DESC);
+    addOptWithArg(OPTION_WORKERS, OPTION_WORKERS_DESC);
+    addOptWithArg(OPTION_RECORD_NUMBER, OPTION_RECORD_NUMBER_DESC);
+    addOptWithArg(OPTION_SET, OPTION_SET_DESC);
+    addOptWithArg(OPTION_PATH, OPTION_PATH_DESC);
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    this.cmd = cmd;
+  }
+
+  @Override
+  protected int doWork() throws Exception {
+    return parseAndRun(cmd.getArgs());
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    Path hbasedir = FSUtils.getRootDir(conf);
+    URI defaultFs = hbasedir.getFileSystem(conf).getUri();
+    FSUtils.setFsDefault(conf, new Path(defaultFs));
+    int ret = ToolRunner.run(conf, new BackupDriver(), args);
+    System.exit(ret);
+  }
+
+  @Override
+  public int run(String[] args) throws IOException {
+    if (conf == null) {
+      LOG.error("Tool configuration is not initialized");
+      throw new NullPointerException("conf");
+    }
+
+    CommandLine cmd;
+    try {
+      // parse the command line arguments
+      cmd = parseArgs(args);
+      cmdLineArgs = args;
+    } catch (Exception e) {
+      System.err.println("Error when parsing command-line arguments: " + e.getMessage());
+      printToolUsage();
+      return EXIT_FAILURE;
+    }
+    processOptions(cmd);
+
+    int ret = EXIT_FAILURE;
+    try {
+      ret = doWork();
+    } catch (Exception e) {
+      LOG.error("Error running command-line tool", e);
+      return EXIT_FAILURE;
+    }
+    return ret;
+  }
+
+  protected void printToolUsage() throws IOException {
+    System.out.println(BackupCommands.USAGE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
new file mode 100644
index 0000000..c7d2848
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
@@ -0,0 +1,545 @@
+/**
+ * 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.io.InputStream;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.Builder;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * An object to encapsulate the information for each backup session
+ */
+@InterfaceAudience.Private
+public class BackupInfo implements Comparable<BackupInfo> {
+  private static final Log LOG = LogFactory.getLog(BackupInfo.class);
+
+  public static interface Filter {
+
+    /**
+     * Filter interface
+     * @param info backup info
+     * @return true if info passes filter, false otherwise
+     */
+    public boolean apply(BackupInfo info);
+  }
+
+  /**
+   * Backup session states
+   */
+  public static enum BackupState {
+    RUNNING, COMPLETE, FAILED, ANY;
+  }
+
+  /**
+   * BackupPhase - phases of an ACTIVE backup session (running), when state of a backup session is
+   * BackupState.RUNNING
+   */
+  public static enum BackupPhase {
+    REQUEST, SNAPSHOT, PREPARE_INCREMENTAL, SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
+  }
+
+  /**
+   * Backup id
+   */
+  private String backupId;
+
+  /**
+   * Backup type, full or incremental
+   */
+  private BackupType type;
+
+  /**
+   * Target root directory for storing the backup files
+   */
+  private String backupRootDir;
+
+  /**
+   * Backup state
+   */
+  private BackupState state;
+
+  /**
+   * Backup phase
+   */
+  private BackupPhase phase;
+
+  /**
+   * Backup failure message
+   */
+  private String failedMsg;
+
+  /**
+   * Backup status map for all tables
+   */
+  private Map<TableName, BackupTableInfo> backupTableInfoMap;
+
+  /**
+   * Actual start timestamp of a backup process
+   */
+  private long startTs;
+
+  /**
+   * Actual end timestamp of the backup process
+   */
+  private long completeTs;
+
+  /**
+   * Total bytes of incremental logs copied
+   */
+  private long totalBytesCopied;
+
+  /**
+   * For incremental backup, a location of a backed-up hlogs
+   */
+  private String hlogTargetDir = null;
+
+  /**
+   * Incremental backup file list
+   */
+  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
+   */
+  private HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap;
+
+  /**
+   * Backup progress in %% (0-100)
+   */
+  private int progress;
+
+  /**
+   * Number of parallel workers. -1 - system defined
+   */
+  private int workers = -1;
+
+  /**
+   * Bandwidth per worker in MB per sec. -1 - unlimited
+   */
+  private long bandwidth = -1;
+
+  public BackupInfo() {
+    backupTableInfoMap = new HashMap<TableName, BackupTableInfo>();
+  }
+
+  public BackupInfo(String backupId, BackupType type, TableName[] tables, String targetRootDir) {
+    this();
+    this.backupId = backupId;
+    this.type = type;
+    this.backupRootDir = targetRootDir;
+    this.addTables(tables);
+    if (type == BackupType.INCREMENTAL) {
+      setHLogTargetDir(BackupUtils.getLogBackupDir(targetRootDir, backupId));
+    }
+    this.startTs = 0;
+    this.completeTs = 0;
+  }
+
+  public int getWorkers() {
+    return workers;
+  }
+
+  public void setWorkers(int workers) {
+    this.workers = workers;
+  }
+
+  public long getBandwidth() {
+    return bandwidth;
+  }
+
+  public void setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+  }
+
+  public void setBackupTableInfoMap(Map<TableName, BackupTableInfo> backupTableInfoMap) {
+    this.backupTableInfoMap = backupTableInfoMap;
+  }
+
+  public HashMap<TableName, HashMap<String, Long>> getTableSetTimestampMap() {
+    return tableSetTimestampMap;
+  }
+
+  public void setTableSetTimestampMap(HashMap<TableName,
+                                      HashMap<String, Long>> tableSetTimestampMap) {
+    this.tableSetTimestampMap = tableSetTimestampMap;
+  }
+
+  public void setType(BackupType type) {
+    this.type = type;
+  }
+
+  public void setBackupRootDir(String targetRootDir) {
+    this.backupRootDir = targetRootDir;
+  }
+
+  public void setTotalBytesCopied(long totalBytesCopied) {
+    this.totalBytesCopied = totalBytesCopied;
+  }
+
+  /**
+   * Set progress (0-100%)
+   * @param p progress value
+   */
+
+  public void setProgress(int p) {
+    this.progress = p;
+  }
+
+  /**
+   * Get current progress
+   */
+  public int getProgress() {
+    return progress;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  public void setBackupId(String backupId) {
+    this.backupId = backupId;
+  }
+
+  public BackupTableInfo getBackupTableInfo(TableName table) {
+    return this.backupTableInfoMap.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 getCompleteTs() {
+    return completeTs;
+  }
+
+  public void setCompleteTs(long endTs) {
+    this.completeTs = endTs;
+  }
+
+  public long getTotalBytesCopied() {
+    return totalBytesCopied;
+  }
+
+  public BackupState getState() {
+    return state;
+  }
+
+  public void setState(BackupState flag) {
+    this.state = flag;
+  }
+
+  public BackupPhase getPhase() {
+    return phase;
+  }
+
+  public void setPhase(BackupPhase phase) {
+    this.phase = phase;
+  }
+
+  public BackupType getType() {
+    return type;
+  }
+
+  public void setSnapshotName(TableName table, String snapshotName) {
+    this.backupTableInfoMap.get(table).setSnapshotName(snapshotName);
+  }
+
+  public String getSnapshotName(TableName table) {
+    return this.backupTableInfoMap.get(table).getSnapshotName();
+  }
+
+  public List<String> getSnapshotNames() {
+    List<String> snapshotNames = new ArrayList<String>();
+    for (BackupTableInfo backupStatus : this.backupTableInfoMap.values()) {
+      snapshotNames.add(backupStatus.getSnapshotName());
+    }
+    return snapshotNames;
+  }
+
+  public Set<TableName> getTables() {
+    return this.backupTableInfoMap.keySet();
+  }
+
+  public List<TableName> getTableNames() {
+    return new ArrayList<TableName>(backupTableInfoMap.keySet());
+  }
+
+  public void addTables(TableName[] tables) {
+    for (TableName table : tables) {
+      BackupTableInfo backupStatus = new BackupTableInfo(table, this.backupRootDir, this.backupId);
+      this.backupTableInfoMap.put(table, backupStatus);
+    }
+  }
+
+  public void setTables(List<TableName> tables) {
+    this.backupTableInfoMap.clear();
+    for (TableName table : tables) {
+      BackupTableInfo backupStatus = new BackupTableInfo(table, this.backupRootDir, this.backupId);
+      this.backupTableInfoMap.put(table, backupStatus);
+    }
+  }
+
+  public String getBackupRootDir() {
+    return backupRootDir;
+  }
+
+  public String getTableBackupDir(TableName tableName) {
+    return BackupUtils.getTableBackupDir(backupRootDir, backupId, tableName);
+  }
+
+  public void setHLogTargetDir(String hlogTagetDir) {
+    this.hlogTargetDir = hlogTagetDir;
+  }
+
+  public String getHLogTargetDir() {
+    return hlogTargetDir;
+  }
+
+  public List<String> getIncrBackupFileList() {
+    return incrBackupFileList;
+  }
+
+  public void setIncrBackupFileList(List<String> incrBackupFileList) {
+    this.incrBackupFileList = incrBackupFileList;
+  }
+
+  /**
+   * Set the new region server log timestamps after distributed log roll
+   * @param newTableSetTimestampMap table timestamp map
+   */
+  public void
+      setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap) {
+    this.tableSetTimestampMap = newTableSetTimestampMap;
+  }
+
+  /**
+   * Get new region server log timestamps after distributed log roll
+   * @return new region server log timestamps
+   */
+  public HashMap<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+    return this.tableSetTimestampMap;
+  }
+
+  public TableName getTableBySnapshot(String snapshotName) {
+    for (Entry<TableName, BackupTableInfo> entry : this.backupTableInfoMap.entrySet()) {
+      if (snapshotName.equals(entry.getValue().getSnapshotName())) {
+        return entry.getKey();
+      }
+    }
+    return null;
+  }
+
+  public BackupProtos.BackupInfo toProtosBackupInfo() {
+    BackupProtos.BackupInfo.Builder builder = BackupProtos.BackupInfo.newBuilder();
+    builder.setBackupId(getBackupId());
+    setBackupTableInfoMap(builder);
+    builder.setCompleteTs(getCompleteTs());
+    if (getFailedMsg() != null) {
+      builder.setFailedMessage(getFailedMsg());
+    }
+    if (getState() != null) {
+      builder.setBackupState(BackupProtos.BackupInfo.BackupState.valueOf(getState().name()));
+    }
+    if (getPhase() != null) {
+      builder.setBackupPhase(BackupProtos.BackupInfo.BackupPhase.valueOf(getPhase().name()));
+    }
+
+    builder.setProgress(getProgress());
+    builder.setStartTs(getStartTs());
+    builder.setBackupRootDir(getBackupRootDir());
+    builder.setBackupType(BackupProtos.BackupType.valueOf(getType().name()));
+    builder.setWorkersNumber(workers);
+    builder.setBandwidth(bandwidth);
+    return builder.build();
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = 33 * type.hashCode() + backupId != null ? backupId.hashCode() : 0;
+    if (backupRootDir != null) {
+      hash = 33 * hash + backupRootDir.hashCode();
+    }
+    hash = 33 * hash + state.hashCode();
+    hash = 33 * hash + phase.hashCode();
+    hash = 33 * hash + (int) (startTs ^ (startTs >>> 32));
+    hash = 33 * hash + (int) (completeTs ^ (completeTs >>> 32));
+    hash = 33 * hash + (int) (totalBytesCopied ^ (totalBytesCopied >>> 32));
+    if (hlogTargetDir != null) {
+      hash = 33 * hash + hlogTargetDir.hashCode();
+    }
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof BackupInfo) {
+      BackupInfo other = (BackupInfo) obj;
+      try {
+        return Bytes.equals(toByteArray(), other.toByteArray());
+      } catch (IOException e) {
+        LOG.error(e);
+        return false;
+      }
+    } else {
+      return false;
+    }
+  }
+
+  public byte[] toByteArray() throws IOException {
+    return toProtosBackupInfo().toByteArray();
+  }
+
+  private void setBackupTableInfoMap(Builder builder) {
+    for (Entry<TableName, BackupTableInfo> entry : backupTableInfoMap.entrySet()) {
+      builder.addBackupTableInfo(entry.getValue().toProto());
+    }
+  }
+
+  public static BackupInfo fromByteArray(byte[] data) throws IOException {
+    return fromProto(BackupProtos.BackupInfo.parseFrom(data));
+  }
+
+  public static BackupInfo fromStream(final InputStream stream) throws IOException {
+    return fromProto(BackupProtos.BackupInfo.parseDelimitedFrom(stream));
+  }
+
+  public static BackupInfo fromProto(BackupProtos.BackupInfo proto) {
+    BackupInfo context = new BackupInfo();
+    context.setBackupId(proto.getBackupId());
+    context.setBackupTableInfoMap(toMap(proto.getBackupTableInfoList()));
+    context.setCompleteTs(proto.getCompleteTs());
+    if (proto.hasFailedMessage()) {
+      context.setFailedMsg(proto.getFailedMessage());
+    }
+    if (proto.hasBackupState()) {
+      context.setState(BackupInfo.BackupState.valueOf(proto.getBackupState().name()));
+    }
+
+    context.setHLogTargetDir(BackupUtils.getLogBackupDir(proto.getBackupRootDir(),
+      proto.getBackupId()));
+
+    if (proto.hasBackupPhase()) {
+      context.setPhase(BackupPhase.valueOf(proto.getBackupPhase().name()));
+    }
+    if (proto.hasProgress()) {
+      context.setProgress(proto.getProgress());
+    }
+    context.setStartTs(proto.getStartTs());
+    context.setBackupRootDir(proto.getBackupRootDir());
+    context.setType(BackupType.valueOf(proto.getBackupType().name()));
+    context.setWorkers(proto.getWorkersNumber());
+    context.setBandwidth(proto.getBandwidth());
+    return context;
+  }
+
+  private static Map<TableName, BackupTableInfo> toMap(List<BackupProtos.BackupTableInfo> list) {
+    HashMap<TableName, BackupTableInfo> map = new HashMap<>();
+    for (BackupProtos.BackupTableInfo tbs : list) {
+      map.put(ProtobufUtil.toTableName(tbs.getTableName()), BackupTableInfo.convert(tbs));
+    }
+    return map;
+  }
+
+  public String getShortDescription() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("{");
+    sb.append("ID=" + backupId).append(",");
+    sb.append("Type=" + getType()).append(",");
+    sb.append("Tables=" + getTableListAsString()).append(",");
+    sb.append("State=" + getState()).append(",");
+    Date date = null;
+    Calendar cal = Calendar.getInstance();
+    cal.setTimeInMillis(getStartTs());
+    date = cal.getTime();
+    sb.append("Start time=" + date).append(",");
+    if (state == BackupState.FAILED) {
+      sb.append("Failed message=" + getFailedMsg()).append(",");
+    } else if (state == BackupState.RUNNING) {
+      sb.append("Phase=" + getPhase()).append(",");
+    } else if (state == BackupState.COMPLETE) {
+      cal = Calendar.getInstance();
+      cal.setTimeInMillis(getCompleteTs());
+      date = cal.getTime();
+      sb.append("End time=" + date).append(",");
+    }
+    sb.append("Progress=" + getProgress() + "%");
+    sb.append("}");
+
+    return sb.toString();
+  }
+
+  public String getStatusAndProgressAsString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("id: ").append(getBackupId()).append(" state: ").append(getState())
+        .append(" progress: ").append(getProgress());
+    return sb.toString();
+  }
+
+  public String getTableListAsString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("{");
+    sb.append(StringUtils.join(backupTableInfoMap.keySet(), ","));
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * We use only time stamps to compare objects during sort operation
+   */
+  @Override
+  public int compareTo(BackupInfo o) {
+    Long thisTS =
+        Long.valueOf(this.getBackupId().substring(this.getBackupId().lastIndexOf("_") + 1));
+    Long otherTS = Long.valueOf(o.getBackupId().substring(o.getBackupId().lastIndexOf("_") + 1));
+    return thisTS.compareTo(otherTS);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
new file mode 100644
index 0000000..dadd861
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
@@ -0,0 +1,139 @@
+/**
+ * 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.util.List;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * POJO class for backup request
+ */
+@InterfaceAudience.Private
+public final class BackupRequest {
+
+  public static class Builder {
+
+    BackupRequest request;
+
+    public Builder() {
+      request = new BackupRequest();
+    }
+
+    public Builder withBackupType(BackupType type) {
+      request.setBackupType(type);
+      return this;
+    }
+
+    public Builder withTableList(List<TableName> tables) {
+      request.setTableList(tables);
+      return this;
+    }
+
+    public Builder withTargetRootDir(String backupDir) {
+      request.setTargetRootDir(backupDir);
+      return this;
+    }
+
+    public Builder withBackupSetName(String setName) {
+      request.setBackupSetName(setName);
+      return this;
+    }
+
+    public Builder withTotalTasks(int numTasks) {
+      request.setTotalTasks(numTasks);
+      return this;
+    }
+
+    public Builder withBandwidthPerTasks(int bandwidth) {
+      request.setBandwidth(bandwidth);
+      return this;
+    }
+
+    public BackupRequest build() {
+      return request;
+    }
+
+  }
+
+  private BackupType type;
+  private List<TableName> tableList;
+  private String targetRootDir;
+  private int totalTasks = -1;
+  private long bandwidth = -1L;
+  private String backupSetName;
+
+  private BackupRequest() {
+  }
+
+  private BackupRequest setBackupType(BackupType type) {
+    this.type = type;
+    return this;
+  }
+
+  public BackupType getBackupType() {
+    return this.type;
+  }
+
+  private BackupRequest setTableList(List<TableName> tableList) {
+    this.tableList = tableList;
+    return this;
+  }
+
+  public List<TableName> getTableList() {
+    return this.tableList;
+  }
+
+  private BackupRequest setTargetRootDir(String targetRootDir) {
+    this.targetRootDir = targetRootDir;
+    return this;
+  }
+
+  public String getTargetRootDir() {
+    return this.targetRootDir;
+  }
+
+  private BackupRequest setTotalTasks(int totalTasks) {
+    this.totalTasks = totalTasks;
+    return this;
+  }
+
+  public int getTotalTasks() {
+    return this.totalTasks;
+  }
+
+  private BackupRequest setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+    return this;
+  }
+
+  public long getBandwidth() {
+    return this.bandwidth;
+  }
+
+  public String getBackupSetName() {
+    return backupSetName;
+  }
+
+  private BackupRequest setBackupSetName(String backupSetName) {
+    this.backupSetName = backupSetName;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..770ccce
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
@@ -0,0 +1,115 @@
+/**
+ * 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.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * BackupRestoreConstants holds a bunch of HBase Backup and Restore constants
+ */
+@InterfaceAudience.Private
+public interface BackupRestoreConstants {
+  /*
+   * Backup/Restore constants
+   */
+  public final static String BACKUP_SYSTEM_TABLE_NAME_KEY = "hbase.backup.system.table.name";
+  public final static String BACKUP_SYSTEM_TABLE_NAME_DEFAULT = "hbase:backup";
+
+  public final static String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl";
+
+  public final static int BACKUP_SYSTEM_TTL_DEFAULT = HConstants.FOREVER;
+  public final static String BACKUP_ENABLE_KEY = "hbase.backup.enable";
+  public final static boolean BACKUP_ENABLE_DEFAULT = false;
+  /*
+   *  Drivers option list
+   */
+  public static final String OPTION_OVERWRITE = "o";
+  public static final String OPTION_OVERWRITE_DESC =
+      "Overwrite data if any of the restore target tables exists";
+
+  public static final String OPTION_CHECK = "c";
+  public static final String OPTION_CHECK_DESC =
+      "Check restore sequence and dependencies only (does not execute the command)";
+
+  public static final String OPTION_SET = "s";
+  public static final String OPTION_SET_DESC = "Backup set name";
+  public static final String OPTION_SET_RESTORE_DESC =
+      "Backup set to restore, mutually exclusive with -t (table list)";
+  public static final String OPTION_SET_BACKUP_DESC =
+      "Backup set to backup, mutually exclusive with -t (table list)";
+  public static final String OPTION_DEBUG = "d";
+  public static final String OPTION_DEBUG_DESC = "Enable debug loggings";
+
+  public static final String OPTION_TABLE = "t";
+  public static final String OPTION_TABLE_DESC = "Table name. If specified, only backup images,"
+      + " which contain this table will be listed.";
+
+  public static final String OPTION_TABLE_LIST = "l";
+  public static final String OPTION_TABLE_LIST_DESC = "Table name list, comma-separated.";
+
+  public static final String OPTION_BANDWIDTH = "b";
+  public static final String OPTION_BANDWIDTH_DESC = "Bandwidth per task (MapReduce task) in MB/s";
+
+  public static final String OPTION_WORKERS = "w";
+  public static final String OPTION_WORKERS_DESC = "Number of parallel MapReduce tasks to execute";
+
+  public static final String OPTION_RECORD_NUMBER = "n";
+  public static final String OPTION_RECORD_NUMBER_DESC =
+      "Number of records of backup history. Default: 10";
+
+  public static final String OPTION_PATH = "p";
+  public static final String OPTION_PATH_DESC = "Backup destination root directory path";
+
+  public static final String OPTION_TABLE_MAPPING = "m";
+  public static final String OPTION_TABLE_MAPPING_DESC =
+      "A comma separated list of target tables. "
+          + "If specified, each table in <tables> must have a mapping";
+
+  public static final String BACKUP_CONFIG_STRING =  BackupRestoreConstants.BACKUP_ENABLE_KEY + "=true\n"
+      + "hbase.master.logcleaner.plugins="
+      +"YOUR_PLUGINS,org.apache.hadoop.hbase.backup.master.BackupLogCleaner\n"
+      + "hbase.procedure.master.classes=YOUR_CLASSES,"
+      +"org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager\n"
+      + "hbase.procedure.regionserver.classes=YOUR_CLASSES,"
+      + "org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager\n"
+      + "and restart the cluster";
+  public static final String ENABLE_BACKUP = "Backup is not enabled. To enable backup, "+
+      "in hbase-site.xml, set:\n "
+      + BACKUP_CONFIG_STRING;
+
+  public static final String VERIFY_BACKUP = "Please make sure that backup is enabled on the cluster. To enable backup, "+
+      "in hbase-site.xml, set:\n "
+      + BACKUP_CONFIG_STRING;
+
+  /*
+   *  Delimiter in table name list in restore command
+   */
+  public static final String TABLENAME_DELIMITER_IN_COMMAND = ",";
+
+  public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root";
+
+  public static final String BACKUPID_PREFIX = "backup_";
+
+  public static enum BackupCommand {
+    CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, PROGRESS,
+    SET, SET_ADD, SET_REMOVE, SET_DELETE, SET_DESCRIBE, SET_LIST
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
new file mode 100644
index 0000000..6d8967a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.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.conf.Configuration;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreJob;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Factory implementation for backup/restore related jobs
+ *
+ */
+@InterfaceAudience.Private
+public final class BackupRestoreFactory {
+
+  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 BackupRestoreFactory() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Gets backup restore job
+   * @param conf configuration
+   * @return backup restore task instance
+   */
+  public static RestoreJob getRestoreJob(Configuration conf) {
+    Class<? extends RestoreJob> cls =
+        conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreJob.class, RestoreJob.class);
+    RestoreJob service = ReflectionUtils.newInstance(cls, conf);
+    service.setConf(conf);
+    return service;
+  }
+
+  /**
+   * Gets backup copy job
+   * @param conf configuration
+   * @return backup copy task
+   */
+  public static BackupCopyJob getBackupCopyJob(Configuration conf) {
+    Class<? extends BackupCopyJob> cls =
+        conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, MapReduceBackupCopyJob.class,
+          BackupCopyJob.class);
+    BackupCopyJob service = ReflectionUtils.newInstance(cls, conf);
+    service.setConf(conf);
+    return service;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
new file mode 100644
index 0000000..a5390a6
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
@@ -0,0 +1,82 @@
+/**
+ * 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.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
+
+/**
+ * Backup related information encapsulated for a table. At this moment only target directory,
+ * snapshot name and table name are encapsulated here.
+ */
+
+@InterfaceAudience.Private
+public class BackupTableInfo  {
+
+  /*
+   *  Table name for backup
+   */
+  private TableName table;
+
+  /*
+   *  Snapshot name for offline/online snapshot
+   */
+
+  private String snapshotName = null;
+
+  public BackupTableInfo() {
+
+  }
+
+  public BackupTableInfo(TableName table, String targetRootDir, String backupId) {
+    this.table = table;
+  }
+
+  public String getSnapshotName() {
+    return snapshotName;
+  }
+
+  public void setSnapshotName(String snapshotName) {
+    this.snapshotName = snapshotName;
+  }
+
+  public TableName getTable() {
+    return table;
+  }
+
+  public static BackupTableInfo convert(BackupProtos.BackupTableInfo proto) {
+    BackupTableInfo bs = new BackupTableInfo();
+    bs.table = ProtobufUtil.toTableName(proto.getTableName());
+    if (proto.hasSnapshotName()) {
+      bs.snapshotName = proto.getSnapshotName();
+    }
+    return bs;
+  }
+
+  public BackupProtos.BackupTableInfo toProto() {
+    BackupProtos.BackupTableInfo.Builder builder = BackupProtos.BackupTableInfo.newBuilder();
+    if (snapshotName != null) {
+      builder.setSnapshotName(snapshotName);
+    }
+    builder.setTableName(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toProtoTableName(table));
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..46044db
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
@@ -0,0 +1,141 @@
+/**
+ *
+ * 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.HashMap;
+
+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.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * 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
+public class HBackupFileSystem {
+  public static final Log LOG = LogFactory.getLog(HBackupFileSystem.class);
+
+  /**
+   * This is utility class.
+   */
+  private HBackupFileSystem() {
+  }
+
+  /**
+   * 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/backup/backup_1396650096738/default/t1_dn/",
+   * where "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
+   * @param backupRootDir backup root directory
+   * @param backupId backup id
+   * @param tableName table name
+   * @return backupPath String for the particular table
+   */
+  public static String
+      getTableBackupDir(String backupRootDir, String backupId, TableName tableName) {
+    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
+        + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString()
+        + Path.SEPARATOR;
+  }
+
+  /**
+   * 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/backup/backup_1396650096738/default/t1_dn/",
+   * where "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
+   * @param backupRootPath backup root path
+   * @param tableName table name
+   * @param backupId backup Id
+   * @return backupPath for the particular table
+   */
+  public static Path getTableBackupPath(TableName tableName,
+      Path backupRootPath, String backupId) {
+    return new Path(getTableBackupDir(backupRootPath.toString(), backupId, tableName));
+  }
+
+  /**
+   * 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/backup/WALs/backup_1396650096738"
+   */
+  public static String getLogBackupDir(String backupRootDir, String backupId) {
+    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
+        + HConstants.HREGION_LOGDIR_NAME;
+  }
+
+  public static Path getLogBackupPath(String backupRootDir, String backupId) {
+    return new Path(getLogBackupDir(backupRootDir, backupId));
+  }
+
+  private static Path getManifestPath(TableName tableName, Configuration conf, Path backupRootPath,
+      String backupId) throws IOException {
+    Path manifestPath =
+        new Path(getTableBackupPath(tableName, backupRootPath, backupId),
+            BackupManifest.MANIFEST_FILE_NAME);
+
+    FileSystem fs = backupRootPath.getFileSystem(conf);
+    if (!fs.exists(manifestPath)) {
+      // check log dir for incremental backup case
+      manifestPath =
+          new Path(getLogBackupDir(backupRootPath.toString(), backupId) + Path.SEPARATOR
+              + BackupManifest.MANIFEST_FILE_NAME);
+      if (!fs.exists(manifestPath)) {
+        String errorMsg =
+            "Could not find backup manifest " + BackupManifest.MANIFEST_FILE_NAME + " for "
+                + backupId + ". File " + manifestPath + " does not exists. Did " + backupId
+                + " correspond to previously taken backup ?";
+        throw new IOException(errorMsg);
+      }
+    }
+    return manifestPath;
+  }
+
+  public static BackupManifest getManifest(TableName tableName, Configuration conf,
+      Path backupRootPath, String backupId) throws IOException {
+    BackupManifest manifest =
+        new BackupManifest(conf, getManifestPath(tableName, conf, backupRootPath, backupId));
+    return manifest;
+  }
+
+  /**
+   * 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
+   */
+  public static void checkImageManifestExist(HashMap<TableName, BackupManifest> backupManifestMap,
+      TableName[] tableArray, Configuration conf, Path backupRootPath, String backupId)
+      throws IOException {
+    for (TableName tableName : tableArray) {
+      BackupManifest manifest = getManifest(tableName, conf, backupRootPath, backupId);
+      backupManifestMap.put(tableName, manifest);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
new file mode 100644
index 0000000..1becb75
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
@@ -0,0 +1,50 @@
+/**
+  * 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.commons.logging.Log;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ * Utility class for disabling Zk and client logging
+ *
+ */
+@InterfaceAudience.Private
+final class LogUtils {
+
+  private LogUtils() {
+  }
+
+  /**
+   * Disables Zk- and HBase client logging
+   * @param log
+   */
+  static void disableZkAndClientLoggers(Log log) {
+    // disable zookeeper log to avoid it mess up command output
+    Logger zkLogger = Logger.getLogger("org.apache.zookeeper");
+    zkLogger.setLevel(Level.OFF);
+    // disable hbase zookeeper tool log to avoid it mess up command output
+    Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper");
+    hbaseZkLogger.setLevel(Level.OFF);
+    // disable hbase client log to avoid it mess up command output
+    Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client");
+    hbaseClientLogger.setLevel(Level.OFF);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
new file mode 100644
index 0000000..82a1b56
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
@@ -0,0 +1,265 @@
+/**
+ * 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.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_RESTORE_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING_DESC;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.lang.StringUtils;
+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.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ *
+ * Command-line entry point for restore operation
+ *
+ */
+@InterfaceAudience.Private
+public class RestoreDriver extends AbstractHBaseTool {
+
+  private static final Log LOG = LogFactory.getLog(RestoreDriver.class);
+  private CommandLine cmd;
+
+  private static final String USAGE_STRING =
+      "Usage: hbase restore <backup_path> <backup_id> [options]\n"
+          + "  backup_path     Path to a backup destination root\n"
+          + "  backup_id       Backup image ID to restore\n"
+          + "  table(s)        Comma-separated list of tables to restore\n";
+
+  private static final String USAGE_FOOTER = "";
+
+  protected RestoreDriver() throws IOException {
+    init();
+  }
+
+  protected void init() throws IOException {
+    // disable irrelevant loggers to avoid it mess up command output
+    LogUtils.disableZkAndClientLoggers(LOG);
+  }
+
+  private int parseAndRun(String[] args) throws IOException {
+    // Check if backup is enabled
+    if (!BackupManager.isBackupEnabled(getConf())) {
+      System.err.println(BackupRestoreConstants.ENABLE_BACKUP);
+      return -1;
+    }
+
+    System.out.println(BackupRestoreConstants.VERIFY_BACKUP);
+
+    // enable debug logging
+    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+    if (cmd.hasOption(OPTION_DEBUG)) {
+      backupClientLogger.setLevel(Level.DEBUG);
+    }
+
+    // whether to overwrite to existing table if any, false by default
+    boolean overwrite = cmd.hasOption(OPTION_OVERWRITE);
+    if (overwrite) {
+      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");
+    }
+
+    if (cmd.hasOption(OPTION_SET) && cmd.hasOption(OPTION_TABLE)) {
+      System.err.println("Options -s and -t are mutaully exclusive,"+
+          " you can not specify both of them.");
+      printToolUsage();
+      return -1;
+    }
+
+    if (!cmd.hasOption(OPTION_SET) && !cmd.hasOption(OPTION_TABLE)) {
+      System.err.println("You have to specify either set name or table list to restore");
+      printToolUsage();
+      return -1;
+    }
+    // parse main restore command options
+    String[] remainArgs = cmd.getArgs();
+    if (remainArgs.length != 2) {
+      printToolUsage();
+      return -1;
+    }
+
+    String backupRootDir = remainArgs[0];
+    String backupId = remainArgs[1];
+    String tables = null;
+    String tableMapping =
+        cmd.hasOption(OPTION_TABLE_MAPPING) ? cmd.getOptionValue(OPTION_TABLE_MAPPING) : null;
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        BackupAdmin client = new BackupAdminImpl(conn);) {
+      // Check backup set
+      if (cmd.hasOption(OPTION_SET)) {
+        String setName = cmd.getOptionValue(OPTION_SET);
+        try {
+          tables = getTablesForSet(conn, setName, conf);
+        } catch (IOException e) {
+          System.out.println("ERROR: " + e.getMessage() + " for setName=" + setName);
+          printToolUsage();
+          return -2;
+        }
+        if (tables == null) {
+          System.out.println("ERROR: Backup set '" + setName
+              + "' is either empty or does not exist");
+          printToolUsage();
+          return -3;
+        }
+      } else {
+        tables = cmd.getOptionValue(OPTION_TABLE);
+      }
+
+      TableName[] sTableArray = BackupUtils.parseTableNames(tables);
+      TableName[] tTableArray = BackupUtils.parseTableNames(tableMapping);
+
+      if (sTableArray != null && tTableArray != null &&
+          (sTableArray.length != tTableArray.length)) {
+        System.out.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping);
+        printToolUsage();
+        return -4;
+      }
+
+      client.restore(BackupUtils.createRestoreRequest(backupRootDir, backupId, check,
+        sTableArray, tTableArray, overwrite));
+    } catch (Exception e) {
+      e.printStackTrace();
+      return -5;
+    }
+    return 0;
+  }
+
+  private String getTablesForSet(Connection conn, String name, Configuration conf)
+      throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<TableName> tables = table.describeBackupSet(name);
+      if (tables == null) return null;
+      return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+    }
+  }
+
+  @Override
+  protected void addOptions() {
+    // define supported options
+    addOptNoArg(OPTION_OVERWRITE, OPTION_OVERWRITE_DESC);
+    addOptNoArg(OPTION_CHECK, OPTION_CHECK_DESC);
+    addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC);
+    addOptWithArg(OPTION_SET, OPTION_SET_RESTORE_DESC);
+    addOptWithArg(OPTION_TABLE, OPTION_TABLE_LIST_DESC);
+
+    addOptWithArg(OPTION_TABLE_MAPPING, OPTION_TABLE_MAPPING_DESC);
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    this.cmd = cmd;
+  }
+
+  @Override
+  protected int doWork() throws Exception {
+    return parseAndRun(cmd.getArgs());
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    Path hbasedir = FSUtils.getRootDir(conf);
+    URI defaultFs = hbasedir.getFileSystem(conf).getUri();
+    FSUtils.setFsDefault(conf, new Path(defaultFs));
+    int ret = ToolRunner.run(conf, new RestoreDriver(), args);
+    System.exit(ret);
+  }
+
+  @Override
+  public int run(String[] args) throws IOException {
+    if (conf == null) {
+      LOG.error("Tool configuration is not initialized");
+      throw new NullPointerException("conf");
+    }
+
+    CommandLine cmd;
+    try {
+      // parse the command line arguments
+      cmd = parseArgs(args);
+      cmdLineArgs = args;
+    } catch (Exception e) {
+      System.out.println("Error when parsing command-line arguments: " + e.getMessage());
+      printToolUsage();
+      return EXIT_FAILURE;
+    }
+
+    if (cmd.hasOption(SHORT_HELP_OPTION) || cmd.hasOption(LONG_HELP_OPTION)) {
+      printToolUsage();
+      return EXIT_FAILURE;
+    }
+
+    processOptions(cmd);
+
+    int ret = EXIT_FAILURE;
+    try {
+      ret = doWork();
+    } catch (Exception e) {
+      LOG.error("Error running command-line tool", e);
+      return EXIT_FAILURE;
+    }
+    return ret;
+  }
+
+  protected void printToolUsage() throws IOException {
+    System.out.println(USAGE_STRING);
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.setLeftPadding(2);
+    helpFormatter.setDescPadding(8);
+    helpFormatter.setWidth(100);
+    helpFormatter.setSyntaxPrefix("Options:");
+    helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
new file mode 100644
index 0000000..86fb963
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
@@ -0,0 +1,46 @@
+/**
+ * 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.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Restore operation job interface Concrete implementation is provided by backup provider, see
+ * {@link BackupRestoreFactory}
+ */
+
+@InterfaceAudience.Private
+public interface RestoreJob extends Configurable {
+
+  /**
+   * Run restore operation
+   * @param dirPaths path array of WAL log directories
+   * @param fromTables from tables
+   * @param toTables to tables
+   * @param fullBackupRestore full backup restore
+   * @throws IOException
+   */
+  void run(Path[] dirPaths, TableName[] fromTables, TableName[] toTables,
+      boolean fullBackupRestore) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
new file mode 100644
index 0000000..de3ad5a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
@@ -0,0 +1,135 @@
+/**
+ * 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.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * POJO class for restore request
+ */
+@InterfaceAudience.Private
+public class RestoreRequest {
+
+  public static class Builder {
+    RestoreRequest request;
+
+    public Builder() {
+      request = new RestoreRequest();
+    }
+
+    public Builder withBackupRootDir(String backupRootDir) {
+      request.setBackupRootDir(backupRootDir);
+      return this;
+    }
+
+    public Builder withBackupId(String backupId) {
+      request.setBackupId(backupId);
+      return this;
+    }
+
+    public Builder withCheck(boolean check) {
+      request.setCheck(check);
+      return this;
+    }
+
+    public Builder withFromTables(TableName[] fromTables) {
+      request.setFromTables(fromTables);
+      return this;
+    }
+
+    public Builder withToTables(TableName[] toTables) {
+      request.setToTables(toTables);
+      return this;
+    }
+
+    public Builder withOvewrite(boolean overwrite) {
+      request.setOverwrite(overwrite);
+      return this;
+    }
+
+
+    public RestoreRequest build() {
+      return request;
+    }
+  }
+
+  private String backupRootDir;
+  private String backupId;
+  private boolean check = false;
+  private TableName[] fromTables;
+  private TableName[] toTables;
+  private boolean overwrite = false;
+
+  private RestoreRequest() {
+  }
+
+  public String getBackupRootDir() {
+    return backupRootDir;
+  }
+
+  private RestoreRequest setBackupRootDir(String backupRootDir) {
+    this.backupRootDir = backupRootDir;
+    return this;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  private RestoreRequest setBackupId(String backupId) {
+    this.backupId = backupId;
+    return this;
+  }
+
+  public boolean isCheck() {
+    return check;
+  }
+
+  private RestoreRequest setCheck(boolean check) {
+    this.check = check;
+    return this;
+  }
+
+  public TableName[] getFromTables() {
+    return fromTables;
+  }
+
+  private RestoreRequest setFromTables(TableName[] fromTables) {
+    this.fromTables = fromTables;
+    return this;
+  }
+
+  public TableName[] getToTables() {
+    return toTables;
+  }
+
+  private RestoreRequest setToTables(TableName[] toTables) {
+    this.toTables = toTables;
+    return this;
+  }
+
+  public boolean isOverwrite() {
+    return overwrite;
+  }
+
+  private RestoreRequest setOverwrite(boolean overwrite) {
+    this.overwrite = overwrite;
+    return this;
+  }
+}


[03/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileInputFormat.java
new file mode 100644
index 0000000..e90d5c1
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileInputFormat.java
@@ -0,0 +1,174 @@
+/**
+ * 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.mapreduce;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+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.Cell;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simple MR input format for HFiles.
+ * This code was borrowed from Apache Crunch project.
+ * Updated to the recent version of HBase.
+ */
+public class HFileInputFormat extends FileInputFormat<NullWritable, Cell> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HFileInputFormat.class);
+
+  /**
+   * File filter that removes all "hidden" files. This might be something worth removing from
+   * a more general purpose utility; it accounts for the presence of metadata files created
+   * in the way we're doing exports.
+   */
+  static final PathFilter HIDDEN_FILE_FILTER = new PathFilter() {
+    @Override
+    public boolean accept(Path p) {
+      String name = p.getName();
+      return !name.startsWith("_") && !name.startsWith(".");
+    }
+  };
+
+  /**
+   * Record reader for HFiles.
+   */
+  private static class HFileRecordReader extends RecordReader<NullWritable, Cell> {
+
+    private Reader in;
+    protected Configuration conf;
+    private HFileScanner scanner;
+
+    /**
+     * A private cache of the key value so it doesn't need to be loaded twice from the scanner.
+     */
+    private Cell value = null;
+    private long count;
+    private boolean seeked = false;
+
+    @Override
+    public void initialize(InputSplit split, TaskAttemptContext context)
+        throws IOException, InterruptedException {
+      FileSplit fileSplit = (FileSplit) split;
+      conf = context.getConfiguration();
+      Path path = fileSplit.getPath();
+      FileSystem fs = path.getFileSystem(conf);
+      LOG.info("Initialize HFileRecordReader for {}", path);
+      this.in = HFile.createReader(fs, path, conf);
+
+      // The file info must be loaded before the scanner can be used.
+      // This seems like a bug in HBase, but it's easily worked around.
+      this.in.loadFileInfo();
+      this.scanner = in.getScanner(false, false);
+
+    }
+
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+      boolean hasNext;
+      if (!seeked) {
+        LOG.info("Seeking to start");
+        hasNext = scanner.seekTo();
+        seeked = true;
+      } else {
+        hasNext = scanner.next();
+      }
+      if (!hasNext) {
+        return false;
+      }
+      value = scanner.getCell();
+      count++;
+      return true;
+    }
+
+    @Override
+    public NullWritable getCurrentKey() throws IOException, InterruptedException {
+      return NullWritable.get();
+    }
+
+    @Override
+    public Cell getCurrentValue() throws IOException, InterruptedException {
+      return value;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+      // This would be inaccurate if KVs are not uniformly-sized or we have performed a seek to
+      // the start row, but better than nothing anyway.
+      return 1.0f * count / in.getEntries();
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (in != null) {
+        in.close();
+        in = null;
+      }
+    }
+  }
+
+  @Override
+  protected List<FileStatus> listStatus(JobContext job) throws IOException {
+    List<FileStatus> result = new ArrayList<FileStatus>();
+
+    // Explode out directories that match the original FileInputFormat filters
+    // since HFiles are written to directories where the
+    // directory name is the column name
+    for (FileStatus status : super.listStatus(job)) {
+      if (status.isDirectory()) {
+        FileSystem fs = status.getPath().getFileSystem(job.getConfiguration());
+        for (FileStatus match : fs.listStatus(status.getPath(), HIDDEN_FILE_FILTER)) {
+          result.add(match);
+        }
+      } else {
+        result.add(status);
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public RecordReader<NullWritable, Cell> createRecordReader(InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    return new HFileRecordReader();
+  }
+
+  @Override
+  protected boolean isSplitable(JobContext context, Path filename) {
+    // This file isn't splittable.
+    return false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 718e88b..f59e24c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -100,7 +100,6 @@ import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Multimaps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 /**
  * Tool to load the output of HFileOutputFormat into an existing table.
  */
@@ -116,7 +115,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     = "hbase.mapreduce.bulkload.max.hfiles.perRegion.perFamily";
   private static final String ASSIGN_SEQ_IDS = "hbase.mapreduce.bulkload.assign.sequenceNumbers";
   public final static String CREATE_TABLE_CONF_KEY = "create.table";
-  public final static String SILENCE_CONF_KEY = "ignore.unmatched.families";
+  public final static String IGNORE_UNMATCHED_CF_CONF_KEY = "ignore.unmatched.families";
   public final static String ALWAYS_COPY_FILES = "always.copy.files";
 
   // We use a '.' prefix which is ignored when walking directory trees
@@ -168,7 +167,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     System.err.println("usage: " + NAME + " /path/to/hfileoutputformat-output tablename" + "\n -D"
         + CREATE_TABLE_CONF_KEY + "=no - can be used to avoid creation of table by this tool\n"
         + "  Note: if you set this to 'no', then the target table must already exist in HBase\n -D"
-        + SILENCE_CONF_KEY + "=yes - can be used to ignore unmatched column families\n"
+        + IGNORE_UNMATCHED_CF_CONF_KEY + "=yes - can be used to ignore unmatched column families\n"
         + "\n");
   }
 
@@ -530,7 +529,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
         boolean success = false;
         try {
           LOG.debug("Going to connect to server " + getLocation() + " for row "
-              + Bytes.toStringBinary(getRow()) + " with hfile group " + famPaths);
+              + Bytes.toStringBinary(getRow()) + " with hfile group " +
+              LoadIncrementalHFiles.this.toString( famPaths));
           byte[] regionName = getLocation().getRegionInfo().getRegionName();
           try (Table table = conn.getTable(getTableName())) {
             secureClient = new SecureBulkLoadClient(getConf(), table);
@@ -1047,6 +1047,21 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
     }
   }
 
+  private final String toString(List<Pair<byte[], String>> list) {
+    StringBuffer sb = new StringBuffer();
+    sb.append("[");
+    if(list != null){
+      for(Pair<byte[], String> pair: list) {
+        sb.append("{");
+        sb.append(Bytes.toStringBinary(pair.getFirst()));
+        sb.append(",");
+        sb.append(pair.getSecond());
+        sb.append("}");
+      }
+    }
+    sb.append("]");
+    return sb.toString();
+  }
   private boolean isSecureBulkLoadEndpointAvailable() {
     String classes = getConf().get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
     return classes.contains("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint");
@@ -1245,7 +1260,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
       try (Table table = connection.getTable(tableName);
         RegionLocator locator = connection.getRegionLocator(tableName)) {
-        boolean silence = "yes".equalsIgnoreCase(getConf().get(SILENCE_CONF_KEY, ""));
+        boolean silence = "yes".equalsIgnoreCase(getConf().get(IGNORE_UNMATCHED_CF_CONF_KEY, ""));
         boolean copyFiles = "yes".equalsIgnoreCase(getConf().get(ALWAYS_COPY_FILES, ""));
         if (dirPath != null) {
           doBulkLoad(hfofDir, admin, table, locator, silence, copyFiles);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
index 8514ace..0ca78b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
@@ -27,22 +27,26 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.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.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WAL.Reader;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * Simple {@link InputFormat} for {@link org.apache.hadoop.hbase.wal.WAL} files.
@@ -169,7 +173,7 @@ public class WALInputFormat extends InputFormat<WALKey, WALEdit> {
           temp = reader.next(currentEntry);
           i++;
         } catch (EOFException x) {
-          LOG.info("Corrupted entry detected. Ignoring the rest of the file."
+          LOG.warn("Corrupted entry detected. Ignoring the rest of the file."
               + " (This is normal when a RegionServer crashed.)");
           return false;
         }
@@ -231,29 +235,37 @@ public class WALInputFormat extends InputFormat<WALKey, WALEdit> {
   List<InputSplit> getSplits(final JobContext context, final String startKey, final String endKey)
       throws IOException, InterruptedException {
     Configuration conf = context.getConfiguration();
-    Path inputDir = new Path(conf.get("mapreduce.input.fileinputformat.inputdir"));
-
+    Path[] inputPaths = getInputPaths(conf);
     long startTime = conf.getLong(startKey, Long.MIN_VALUE);
     long endTime = conf.getLong(endKey, Long.MAX_VALUE);
 
-    FileSystem fs = inputDir.getFileSystem(conf);
-    List<FileStatus> files = getFiles(fs, inputDir, startTime, endTime);
-
-    List<InputSplit> splits = new ArrayList<>(files.size());
-    for (FileStatus file : files) {
+    List<FileStatus> allFiles = new ArrayList<FileStatus>();
+    for(Path inputPath: inputPaths){
+      FileSystem fs = inputPath.getFileSystem(conf);
+      List<FileStatus> files = getFiles(fs, inputPath, startTime, endTime);
+      allFiles.addAll(files);
+    }
+    List<InputSplit> splits = new ArrayList<InputSplit>(allFiles.size());
+    for (FileStatus file : allFiles) {
       splits.add(new WALSplit(file.getPath().toString(), file.getLen(), startTime, endTime));
     }
     return splits;
   }
 
+  private Path[] getInputPaths(Configuration conf) {
+    String inpDirs = conf.get("mapreduce.input.fileinputformat.inputdir");
+    return StringUtils.stringToPath(inpDirs.split(","));
+  }
+
   private List<FileStatus> getFiles(FileSystem fs, Path dir, long startTime, long endTime)
       throws IOException {
     List<FileStatus> result = new ArrayList<>();
     LOG.debug("Scanning " + dir.toString() + " for WAL files");
 
-    FileStatus[] files = fs.listStatus(dir);
-    if (files == null) return Collections.emptyList();
-    for (FileStatus file : files) {
+    RemoteIterator<LocatedFileStatus> iter = fs.listLocatedStatus(dir);
+    if (!iter.hasNext()) return Collections.emptyList();
+    while (iter.hasNext()) {
+      LocatedFileStatus file = iter.next();
       if (file.isDirectory()) {
         // recurse into sub directories
         result.addAll(getFiles(fs, file.getPath(), startTime, endTime));
@@ -264,7 +276,7 @@ public class WALInputFormat extends InputFormat<WALKey, WALEdit> {
           try {
             long fileStartTime = Long.parseLong(name.substring(idx+1));
             if (fileStartTime <= endTime) {
-              LOG.info("Found: " + name);
+              LOG.info("Found: " + file);
               result.add(file);
             }
           } catch (NumberFormatException x) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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 cca2041..d16dcf5 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
@@ -70,9 +70,9 @@ import org.apache.hadoop.util.ToolRunner;
 public class WALPlayer extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(WALPlayer.class);
   final static String NAME = "WALPlayer";
-  final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
-  final static String TABLES_KEY = "wal.input.tables";
-  final static String TABLE_MAP_KEY = "wal.input.tablesmap";
+  public final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
+  public final static String TABLES_KEY = "wal.input.tables";
+  public final static String TABLE_MAP_KEY = "wal.input.tablesmap";
 
   // This relies on Hadoop Configuration to handle warning about deprecated configs and
   // to set the correct non-deprecated configs when an old one shows up.
@@ -84,6 +84,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);
   }
@@ -93,7 +96,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
@@ -105,7 +108,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);
           }
         }
@@ -148,7 +153,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)) {
@@ -159,8 +166,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 {
@@ -176,31 +187,41 @@ 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();
       }
     }
 
-    /**
-     * @param cell
-     * @return Return true if we are to emit this cell.
-     */
     protected boolean filter(Context context, final Cell cell) {
       return true;
     }
 
     @Override
+    protected void
+        cleanup(Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation>.Context context)
+            throws IOException, InterruptedException {
+      super.cleanup(context);
+    }
+
+    @Override
     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) {
+      if (tableMap == null) {
+        tableMap = tablesToUse;
+      }
+      if (tablesToUse == null) {
         // Then user wants all tables.
-      } else if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
+      } else if (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.");
+        throw new IOException("Incorrect table mapping specified .");
       }
       int i = 0;
       if (tablesToUse != null) {
@@ -214,7 +235,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
@@ -243,7 +266,7 @@ public class WALPlayer extends Configured implements Tool {
     Configuration conf = getConf();
     setupTime(conf, WALInputFormat.START_TIME_KEY);
     setupTime(conf, WALInputFormat.END_TIME_KEY);
-    Path inputDir = new Path(args[0]);
+    String inputDirs = args[0];
     String[] tables = args[1].split(",");
     String[] tableMap;
     if (args.length > 2) {
@@ -257,13 +280,18 @@ public class WALPlayer extends Configured implements Tool {
     }
     conf.setStrings(TABLES_KEY, tables);
     conf.setStrings(TABLE_MAP_KEY, tableMap);
-    Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + inputDir));
+    Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + System.currentTimeMillis()));
     job.setJarByClass(WALPlayer.class);
-    FileInputFormat.setInputPaths(job, inputDir);
+
+    FileInputFormat.addInputPaths(job, inputDirs);
+
     job.setInputFormatClass(WALInputFormat.class);
     job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+
     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
     if (hfileOutPath != null) {
+      LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs);
+
       // the bulk HFile case
       if (tables.length != 1) {
         throw new IOException("Exactly one table must be specified for the bulk export option");
@@ -299,7 +327,9 @@ public class WALPlayer extends Configured implements Tool {
     return job;
   }
 
-  /*
+
+  /**
+   * Print usage
    * @param errorMsg Error message.  Can be null.
    */
   private void usage(final String errorMsg) {
@@ -309,7 +339,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 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.");
@@ -322,10 +353,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/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 177ee32..f86f800 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -74,10 +74,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegi
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.*;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
@@ -88,9 +86,129 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockH
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -233,6 +351,7 @@ public class MasterRpcServices extends RSRpcServices
   /**
    * @return list of blocking services and their security info classes that this server supports
    */
+  @Override
   protected List<BlockingServiceAndInterface> getServices() {
     List<BlockingServiceAndInterface> bssi = new ArrayList<>(5);
     bssi.add(new BlockingServiceAndInterface(

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index 54b68d3..55d58e0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -62,7 +62,7 @@ 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.procedure.ZKProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
@@ -1114,7 +1114,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
     // setup the default procedure coordinator
     String name = master.getServerName().toString();
     ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, opThreads);
-    ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinatorRpcs(
+    ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinator(
         master.getZooKeeper(), SnapshotManager.ONLINE_SNAPSHOT_CONTROLLER_DESCRIPTION, name);
 
     this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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 4632d23..b656894 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
@@ -35,8 +35,8 @@ import org.apache.zookeeper.KeeperException;
  * ZooKeeper based {@link ProcedureCoordinatorRpcs} for a {@link ProcedureCoordinator}
  */
 @InterfaceAudience.Private
-public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs {
-  private static final Log LOG = LogFactory.getLog(ZKProcedureCoordinatorRpcs.class);
+public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
+  private static final Log LOG = LogFactory.getLog(ZKProcedureCoordinator.class);
   private ZKProcedureUtil zkProc = null;
   protected ProcedureCoordinator coordinator = null;  // if started this should be non-null
 
@@ -51,8 +51,8 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs {
    * @param coordName name of the node running the coordinator
    * @throws KeeperException if an unexpected zk error occurs
    */
-  public ZKProcedureCoordinatorRpcs(ZooKeeperWatcher watcher,
-      String procedureClass, String coordName) throws KeeperException {
+  public ZKProcedureCoordinator(ZooKeeperWatcher watcher,
+      String procedureClass, String coordName) {
     this.watcher = watcher;
     this.procedureType = procedureClass;
     this.coordName = coordName;
@@ -179,6 +179,7 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs {
    * Start monitoring znodes in ZK - subclass hook to start monitoring znodes they are about.
    * @return true if succeed, false if encountered initialization errors.
    */
+  @Override
   final public boolean start(final ProcedureCoordinator coordinator) {
     if (this.coordinator != null) {
       throw new IllegalStateException(

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
index 7b624a5..3092114 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
@@ -42,7 +42,7 @@ 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.procedure.ZKProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -98,7 +98,7 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager {
     // setup the procedure coordinator
     String name = master.getServerName().toString();
     ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, threads);
-    ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinatorRpcs(
+    ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinator(
         master.getZooKeeper(), getProcedureSignature(), name);
 
     this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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 be4cca0..b3b5113 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
@@ -199,13 +199,13 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.data.Stat;
 
+import sun.misc.Signal;
+import sun.misc.SignalHandler;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
-import sun.misc.Signal;
-import sun.misc.SignalHandler;
-
 /**
  * HRegionServer makes a set of HRegions available to clients. It checks in with
  * the HMaster. There are many HRegionServers in a single HBase deployment.
@@ -385,7 +385,7 @@ public class HRegionServer extends HasThread implements
 
   // WAL roller. log is protected rather than private to avoid
   // eclipse warning when accessed by inner classes
-  final LogRoller walRoller;
+  protected final LogRoller walRoller;
 
   // flag set after we're done setting up server threads
   final AtomicBoolean online = new AtomicBoolean(false);
@@ -535,7 +535,6 @@ public class HRegionServer extends HasThread implements
 
     // Disable usage of meta replicas in the regionserver
     this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
-
     // Config'ed params
     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
@@ -640,7 +639,7 @@ public class HRegionServer extends HasThread implements
     int cleanerInterval =
         conf.getInt("hbase.hfile.compaction.discharger.interval", 2 * 60 * 1000);
     this.compactedFileDischarger =
-        new CompactedHFilesDischarger(cleanerInterval, (Stoppable)this, (RegionServerServices)this);
+        new CompactedHFilesDischarger(cleanerInterval, this, this);
     choreService.scheduleChore(compactedFileDischarger);
   }
 
@@ -859,7 +858,7 @@ public class HRegionServer extends HasThread implements
       rspmHost.loadProcedures(conf);
       rspmHost.initialize(this);
     } catch (KeeperException e) {
-      this.abort("Failed to reach zk cluster when creating procedure handler.", 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);
@@ -1948,6 +1947,10 @@ public class HRegionServer extends HasThread implements
     return wal;
   }
 
+  public LogRoller getWalRoller() {
+    return walRoller;
+  }
+
   @Override
   public Connection getConnection() {
     return getClusterConnection();

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index 9dd85d8..bf14933 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -368,6 +368,11 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
     return false;
   }
 
+  public static boolean isArchivedLogFile(Path p) {
+    String oldLog = Path.SEPARATOR + HConstants.HREGION_OLDLOGDIR_NAME + Path.SEPARATOR;
+    return p.toString().contains(oldLog);
+  }
+
   /**
    * Get prefix of the log from its name, assuming WAL name in format of
    * log_prefix.filenumber.log_suffix

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index a6c7f68..8a4ed72 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -49,7 +49,6 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
-import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.Log;
@@ -117,6 +116,7 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.RegionSplitter;
+import org.apache.hadoop.hbase.util.RegionSplitter.SplitAlgorithm;
 import org.apache.hadoop.hbase.util.RetryCounter;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL;
@@ -137,6 +137,8 @@ import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
+
 /**
  * Facility for testing HBase. Replacement for
  * old HBaseTestCase and HBaseClusterTestCase functionality.
@@ -2171,6 +2173,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     }
   }
 
+  public void loadRandomRows(final Table t, final byte[] f, int rowSize, int totalRows)
+      throws IOException {
+    Random r = new Random();
+    byte[] row = new byte[rowSize];
+    for (int i = 0; i < totalRows; i++) {
+      r.nextBytes(row);
+      Put put = new Put(row);
+      put.addColumn(f, new byte[]{0}, new byte[]{0});
+      t.put(put);
+    }
+  }
+
   public void verifyNumericRows(Table table, final byte[] f, int startRow, int endRow,
       int replicaId)
       throws IOException {
@@ -3337,6 +3351,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   }
 
   /**
+   * Waith until all system table's regions get assigned
+   * @throws IOException
+   */
+  public void waitUntilAllSystemRegionsAssigned() throws IOException {
+    waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME);
+    waitUntilAllRegionsAssigned(TableName.NAMESPACE_TABLE_NAME);
+  }
+
+  /**
    * Wait until all regions for a table in hbase:meta have a non-empty
    * info:server, or until timeout.  This means all regions have been deployed,
    * master has been informed and updated hbase:meta with the regions deployed
@@ -3801,12 +3824,24 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public static int createPreSplitLoadTestTable(Configuration conf,
       HTableDescriptor desc, HColumnDescriptor[] hcds, int numRegionsPerServer) throws IOException {
+
+    return createPreSplitLoadTestTable(conf, desc, hcds,
+      new RegionSplitter.HexStringSplit(), numRegionsPerServer);
+  }
+
+  /**
+   * Creates a pre-split table for load testing. If the table already exists,
+   * logs a warning and continues.
+   * @return the number of regions the table was split into
+   */
+  public static int createPreSplitLoadTestTable(Configuration conf,
+      HTableDescriptor desc, HColumnDescriptor[] hcds,
+      SplitAlgorithm splitter, int numRegionsPerServer) throws IOException {
     for (HColumnDescriptor hcd : hcds) {
       if (!desc.hasFamily(hcd.getName())) {
         desc.addFamily(hcd);
       }
     }
-
     int totalNumberOfRegions = 0;
     Connection unmanagedConnection = ConnectionFactory.createConnection(conf);
     Admin admin = unmanagedConnection.getAdmin();
@@ -3825,7 +3860,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
           "pre-splitting table into " + totalNumberOfRegions + " regions " +
           "(regions per server: " + numRegionsPerServer + ")");
 
-      byte[][] splits = new RegionSplitter.HexStringSplit().split(
+      byte[][] splits = splitter.split(
           totalNumberOfRegions);
 
       admin.createTable(desc, splits);

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..ec88549
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
@@ -0,0 +1,293 @@
+/*
+ *
+ * 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.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+
+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.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+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.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+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.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALFactory;
+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 = TableName.valueOf("table1");
+  protected static HTableDescriptor table1Desc;
+  protected static TableName table2 = TableName.valueOf("table2");
+  protected static TableName table3 = TableName.valueOf("table3");
+  protected static TableName table4 = TableName.valueOf("table4");
+
+  protected static TableName table1_restore = TableName.valueOf("ns1:table1_restore");
+  protected static TableName table2_restore = TableName.valueOf("ns2:table2_restore");
+  protected static TableName table3_restore = TableName.valueOf("ns3:table3_restore");
+  protected static TableName table4_restore = TableName.valueOf("ns4:table4_restore");
+
+  protected static final int NB_ROWS_IN_BATCH = 99;
+  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 String provider = "defaultProvider";
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL = new HBaseTestingUtility();
+    conf1 = TEST_UTIL.getConfiguration();
+    conf1.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true);
+    BackupManager.decorateMasterConfiguration(conf1);
+    BackupManager.decorateRegionServerConfiguration(conf1);
+    conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
+    // Set MultiWAL (with 2 default WAL files per RS)
+    conf1.set(WALFactory.WAL_PROVIDER, provider);
+    TEST_UTIL.startMiniZKCluster();
+    MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
+
+    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);
+    createTables();
+    populateFromMasterConfig(TEST_UTIL.getHBaseCluster().getMaster().getConfiguration(), conf1);
+  }
+
+  private static void populateFromMasterConfig(Configuration masterConf, Configuration conf) {
+    Iterator<Entry<String, String>> it = masterConf.iterator();
+    while (it.hasNext()) {
+      Entry<String, String> e = it.next();
+      conf.set(e.getKey(), e.getValue());
+    }
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
+    TEST_UTIL2.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniMapReduceCluster();
+  }
+
+  HTable insertIntoTable(Connection conn, TableName table, byte[] family, int id, int numRows)
+      throws IOException {
+    HTable t = (HTable) conn.getTable(table);
+    Put p1;
+    for (int i = 0; i < numRows; i++) {
+      p1 = new Put(Bytes.toBytes("row-" + table + "-" + id + "-" + i));
+      p1.addColumn(family, qualName, Bytes.toBytes("val" + i));
+      t.put(p1);
+    }
+    return t;
+  }
+
+
+  protected BackupRequest createBackupRequest(BackupType type,
+      List<TableName> tables, String path) {
+    BackupRequest.Builder builder = new BackupRequest.Builder();
+    BackupRequest request = builder.withBackupType(type)
+                                    .withTableList(tables)
+                                    .withTargetRootDir(path).build();
+    return request;
+  }
+
+  protected String backupTables(BackupType type, List<TableName> tables, String path)
+      throws IOException {
+    Connection conn = null;
+    BackupAdmin badmin = null;
+    String backupId;
+    try {
+      conn = ConnectionFactory.createConnection(conf1);
+      badmin = new BackupAdminImpl(conn);
+      BackupRequest request = createBackupRequest(type, tables, path);
+      backupId = badmin.backupTables(request);
+    } finally {
+      if (badmin != null) {
+        badmin.close();
+      }
+      if (conn != null) {
+        conn.close();
+      }
+    }
+    return backupId;
+  }
+
+  protected String fullTableBackup(List<TableName> tables) throws IOException {
+    return backupTables(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+  }
+
+  protected String incrementalTableBackup(List<TableName> tables) throws IOException {
+    return backupTables(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+  }
+
+  protected static void loadTable(Table 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.setDurability(Durability.SKIP_WAL);
+      p.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      table.put(p);
+    }
+  }
+
+  protected static void createTables() throws Exception {
+
+    long tid = System.currentTimeMillis();
+    table1 = TableName.valueOf("ns1:test-" + tid);
+    HBaseAdmin ha = TEST_UTIL.getHBaseAdmin();
+
+    // Create namespaces
+    NamespaceDescriptor desc1 = NamespaceDescriptor.create("ns1").build();
+    NamespaceDescriptor desc2 = NamespaceDescriptor.create("ns2").build();
+    NamespaceDescriptor desc3 = NamespaceDescriptor.create("ns3").build();
+    NamespaceDescriptor desc4 = NamespaceDescriptor.create("ns4").build();
+
+    ha.createNamespace(desc1);
+    ha.createNamespace(desc2);
+    ha.createNamespace(desc3);
+    ha.createNamespace(desc4);
+
+    HTableDescriptor desc = new HTableDescriptor(table1);
+    HColumnDescriptor fam = new HColumnDescriptor(famName);
+    desc.addFamily(fam);
+    ha.createTable(desc);
+    table1Desc = desc;
+    Connection conn = ConnectionFactory.createConnection(conf1);
+    Table table = conn.getTable(table1);
+    loadTable(table);
+    table.close();
+    table2 = TableName.valueOf("ns2:test-" + tid + 1);
+    desc = new HTableDescriptor(table2);
+    desc.addFamily(fam);
+    ha.createTable(desc);
+    table = conn.getTable(table2);
+    loadTable(table);
+    table.close();
+    table3 = TableName.valueOf("ns3:test-" + tid + 2);
+    table = TEST_UTIL.createTable(table3, famName);
+    table.close();
+    table4 = TableName.valueOf("ns4:test-" + tid + 3);
+    table = TEST_UTIL.createTable(table4, famName);
+    table.close();
+    ha.close();
+    conn.close();
+  }
+
+  protected boolean checkSucceeded(String backupId) throws IOException {
+    BackupInfo status = getBackupInfo(backupId);
+    if (status == null) return false;
+    return status.getState() == BackupState.COMPLETE;
+  }
+
+  protected boolean checkFailed(String backupId) throws IOException {
+    BackupInfo status = getBackupInfo(backupId);
+    if (status == null) return false;
+    return status.getState() == BackupState.FAILED;
+  }
+
+  private BackupInfo getBackupInfo(String backupId) throws IOException {
+    try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
+      BackupInfo status = table.readBackupInfo(backupId);
+      return status;
+    }
+  }
+
+  protected BackupAdmin getBackupAdmin() throws IOException {
+    return new BackupAdminImpl(TEST_UTIL.getConnection());
+  }
+
+  /**
+   * Helper method
+   */
+  protected List<TableName> toList(String... args) {
+    List<TableName> ret = new ArrayList<>();
+    for (int i = 0; i < args.length; i++) {
+      ret.add(TableName.valueOf(args[i]));
+    }
+    return ret;
+  }
+
+  protected void dumpBackupDir() throws IOException {
+    // Dump Backup Dir
+    FileSystem fs = FileSystem.get(conf1);
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(new Path(BACKUP_ROOT_DIR), true);
+    while (it.hasNext()) {
+      LOG.debug(it.next().getPath());
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..280314b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
@@ -0,0 +1,97 @@
+/**
+ * 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.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@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");
+    List<TableName> tables = Lists.newArrayList(table3);
+    LOG.info("Finished Backup " + fullTableBackup(tables));
+  }
+
+  /**
+   * 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");
+
+    List<TableName> tables = Lists.newArrayList(table3, table4);
+    fullTableBackup(tables);
+  }
+
+  /**
+   * Verify that full backup fails on a single table that does not exist.
+   * @throws Exception
+   */
+  @Test(expected = IOException.class)
+  public void testFullBackupSingleDNE() throws Exception {
+
+    LOG.info("test full backup fails on a single table that does not exist");
+    List<TableName> tables = toList("tabledne");
+    fullTableBackup(tables);
+  }
+
+  /**
+   * Verify that full backup fails on multiple tables that do not exist.
+   * @throws Exception
+   */
+  @Test(expected = IOException.class)
+  public void testFullBackupMultipleDNE() throws Exception {
+
+    LOG.info("test full backup fails on multiple tables that do not exist");
+    List<TableName> tables = toList("table1dne", "table2dne");
+    fullTableBackup(tables);
+  }
+
+  /**
+   * Verify that full backup fails on tableset containing real and fake tables.
+   * @throws Exception
+   */
+  @Test(expected = IOException.class)
+  public void testFullBackupMixExistAndDNE() throws Exception {
+    LOG.info("create full backup fails on tableset containing real and fake table");
+
+    List<TableName> tables = toList(table1.getNameAsString(), "tabledne");
+    fullTableBackup(tables);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java
new file mode 100644
index 0000000..08002fb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java
@@ -0,0 +1,431 @@
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestBackupCommandLineTool {
+
+  private final static String USAGE_DESCRIBE = "Usage: hbase backup describe <backup_id>";
+  private final static String USAGE_CREATE = "Usage: hbase backup create";
+  private final static String USAGE_HISTORY = "Usage: hbase backup history";
+  private final static String USAGE_BACKUP = "Usage: hbase backup";
+  private final static String USAGE_DELETE = "Usage: hbase backup delete";
+  private final static String USAGE_PROGRESS = "Usage: hbase backup progress";
+  private final static String USAGE_SET = "Usage: hbase backup set";
+  private final static String USAGE_RESTORE = "Usage: hbase restore";
+
+  Configuration conf;
+
+  @Before
+  public void setUpBefore() throws Exception {
+    conf = HBaseConfiguration.create();
+    conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true);
+  }
+
+  @Test
+  public void testBackupDriverDescribeHelp() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "describe", "-help" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "describe", "-h" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "describe" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_DESCRIBE) >= 0);
+  }
+
+  @Test
+  public void testBackupDriverCreateHelp() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "create", "-help" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_CREATE) >= 0);
+    assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0);
+
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "create", "-h" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_CREATE) >= 0);
+    assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "create" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_CREATE) >= 0);
+    assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0);
+
+  }
+
+  @Test
+  public void testBackupDriverHistoryHelp() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "history", "-help" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_HISTORY) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "history", "-h" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_HISTORY) >= 0);
+
+  }
+
+  @Test
+  public void testBackupDriverDeleteHelp() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "delete", "-help" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_DELETE) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "delete", "-h" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_DELETE) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "delete" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_DELETE) >= 0);
+  }
+
+  @Test
+  public void testBackupDriverProgressHelp() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "progress", "-help" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_PROGRESS) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "progress", "-h" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_PROGRESS) >= 0);
+  }
+
+  @Test
+  public void testBackupDriverSetHelp() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "set", "-help" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_SET) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "set", "-h" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_SET) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "set" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_SET) >= 0);
+
+  }
+
+  @Test
+  public void testBackupDriverHelp() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "-help" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_BACKUP) >= 0);
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "-h" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_BACKUP) >= 0);
+
+  }
+
+  @Test
+  public void testRestoreDriverHelp() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "-help" };
+    ToolRunner.run(conf, new RestoreDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_RESTORE) >= 0);
+    assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "-h" };
+    ToolRunner.run(conf, new RestoreDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_RESTORE) >= 0);
+    assertTrue(output.indexOf(BackupRestoreConstants.OPTION_TABLE_LIST_DESC) > 0);
+
+  }
+
+  @Test
+  public void testBackupDriverUnrecognizedCommand() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "command" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_BACKUP) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "command" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_BACKUP) >= 0);
+  }
+
+  @Test
+  public void testBackupDriverUnrecognizedOption() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "create", "-xx" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_BACKUP) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "describe", "-xx" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_BACKUP) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "history", "-xx" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_BACKUP) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "delete", "-xx" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_BACKUP) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "set", "-xx" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_BACKUP) >= 0);
+  }
+
+  @Test
+  public void testRestoreDriverUnrecognizedOption() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "-xx" };
+    ToolRunner.run(conf, new RestoreDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_RESTORE) >= 0);
+
+  }
+
+  @Test
+  public void testBackupDriverCreateWrongArgNumber() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "create" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_CREATE) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "create", "22" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_CREATE) >= 0);
+
+    baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    args = new String[] { "create", "22", "22", "22", "22", "22" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_CREATE) >= 0);
+  }
+
+  @Test
+  public void testBackupDriverDeleteWrongArgNumber() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "delete" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_DELETE) >= 0);
+
+  }
+
+  @Test
+  public void testBackupDriverHistoryWrongArgs() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "history", "-n", "xx" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf(USAGE_HISTORY) >= 0);
+
+  }
+
+  @Test
+  public void testBackupDriverWrongBackupDestination() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "create", "full", "clicks" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf("ERROR: invalid backup destination") >= 0);
+
+  }
+
+  @Test
+  public void testBackupDriverBackupSetAndList() throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+    String[] args = new String[] { "create", "full", "file:/", "-t", "clicks", "-s", "s" };
+    ToolRunner.run(conf, new BackupDriver(), args);
+
+    String output = baos.toString();
+    System.out.println(baos.toString());
+    assertTrue(output.indexOf("ERROR: You can specify either backup set or list") >= 0);
+
+  }
+
+}


[06/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
new file mode 100644
index 0000000..b8adac9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
@@ -0,0 +1,666 @@
+/**
+ * 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.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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.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.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+
+/**
+ * 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. BackupManifest is a storage container for BackupImage.
+ * It is responsible for storing/reading backup image data and has some additional utility methods.
+ *
+ */
+@InterfaceAudience.Private
+public class BackupManifest {
+
+  private static final Log LOG = LogFactory.getLog(BackupManifest.class);
+
+  // manifest file name
+  public static final String MANIFEST_FILE_NAME = ".backup.manifest";
+
+  /**
+   *  Backup image, the dependency graph is made up by series of backup images
+   *  BackupImage contains all the relevant information to restore the backup and
+   *  is used during restore operation
+   */
+
+  public static class BackupImage implements Comparable<BackupImage> {
+
+    static class Builder {
+      BackupImage image;
+
+      Builder() {
+        image = new BackupImage();
+      }
+
+      Builder withBackupId(String backupId) {
+        image.setBackupId(backupId);
+        return this;
+      }
+
+      Builder withType(BackupType type) {
+        image.setType(type);
+        return this;
+      }
+
+      Builder withRootDir(String rootDir) {
+        image.setRootDir(rootDir);
+        return this;
+      }
+
+      Builder withTableList(List<TableName> tableList) {
+        image.setTableList(tableList);
+        return this;
+      }
+
+      Builder withStartTime(long startTime) {
+        image.setStartTs(startTime);
+        return this;
+      }
+
+      Builder withCompleteTime(long completeTime) {
+        image.setCompleteTs(completeTime);
+        return this;
+      }
+
+      BackupImage build() {
+        return image;
+      }
+
+    }
+
+    private String backupId;
+    private BackupType type;
+    private String rootDir;
+    private List<TableName> tableList;
+    private long startTs;
+    private long completeTs;
+    private ArrayList<BackupImage> ancestors;
+    private HashMap<TableName, HashMap<String, Long>> incrTimeRanges;
+
+    static Builder newBuilder() {
+      return new Builder();
+    }
+
+    public BackupImage() {
+      super();
+    }
+
+    private BackupImage(String backupId, BackupType type, String rootDir,
+        List<TableName> tableList, long startTs, long completeTs) {
+      this.backupId = backupId;
+      this.type = type;
+      this.rootDir = rootDir;
+      this.tableList = tableList;
+      this.startTs = startTs;
+      this.completeTs = completeTs;
+    }
+
+    static BackupImage fromProto(BackupProtos.BackupImage im) {
+      String backupId = im.getBackupId();
+      String rootDir = im.getBackupRootDir();
+      long startTs = im.getStartTs();
+      long completeTs = im.getCompleteTs();
+      List<HBaseProtos.TableName> tableListList = im.getTableListList();
+      List<TableName> tableList = new ArrayList<TableName>();
+      for (HBaseProtos.TableName tn : tableListList) {
+        tableList.add(ProtobufUtil.toTableName(tn));
+      }
+
+      List<BackupProtos.BackupImage> ancestorList = im.getAncestorsList();
+
+      BackupType type =
+          im.getBackupType() == BackupProtos.BackupType.FULL ? BackupType.FULL
+              : BackupType.INCREMENTAL;
+
+      BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs);
+      for (BackupProtos.BackupImage img : ancestorList) {
+        image.addAncestor(fromProto(img));
+      }
+      image.setIncrTimeRanges(loadIncrementalTimestampMap(im));
+      return image;
+    }
+
+    BackupProtos.BackupImage toProto() {
+      BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder();
+      builder.setBackupId(backupId);
+      builder.setCompleteTs(completeTs);
+      builder.setStartTs(startTs);
+      builder.setBackupRootDir(rootDir);
+      if (type == BackupType.FULL) {
+        builder.setBackupType(BackupProtos.BackupType.FULL);
+      } else {
+        builder.setBackupType(BackupProtos.BackupType.INCREMENTAL);
+      }
+
+      for (TableName name : tableList) {
+        builder.addTableList(ProtobufUtil.toProtoTableName(name));
+      }
+
+      if (ancestors != null) {
+        for (BackupImage im : ancestors) {
+          builder.addAncestors(im.toProto());
+        }
+      }
+
+      setIncrementalTimestampMap(builder);
+      return builder.build();
+    }
+
+    private static HashMap<TableName, HashMap<String, Long>> loadIncrementalTimestampMap(
+        BackupProtos.BackupImage proto) {
+      List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList();
+
+      HashMap<TableName, HashMap<String, Long>> incrTimeRanges =
+          new HashMap<TableName, HashMap<String, Long>>();
+      if (list == null || list.size() == 0) return incrTimeRanges;
+      for (BackupProtos.TableServerTimestamp tst : list) {
+        TableName tn = ProtobufUtil.toTableName(tst.getTableName());
+        HashMap<String, Long> map = incrTimeRanges.get(tn);
+        if (map == null) {
+          map = new HashMap<String, Long>();
+          incrTimeRanges.put(tn, map);
+        }
+        List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList();
+        for (BackupProtos.ServerTimestamp stm : listSt) {
+          ServerName sn = ProtobufUtil.toServerName(stm.getServerName());
+          map.put(sn.getHostname() + ":" + sn.getPort(), stm.getTimestamp());
+        }
+      }
+      return incrTimeRanges;
+    }
+
+    private void setIncrementalTimestampMap(BackupProtos.BackupImage.Builder builder) {
+      if (this.incrTimeRanges == null) {
+        return;
+      }
+      for (Entry<TableName, HashMap<String, Long>> entry : this.incrTimeRanges.entrySet()) {
+        TableName key = entry.getKey();
+        HashMap<String, Long> value = entry.getValue();
+        BackupProtos.TableServerTimestamp.Builder tstBuilder =
+            BackupProtos.TableServerTimestamp.newBuilder();
+        tstBuilder.setTableName(ProtobufUtil.toProtoTableName(key));
+
+        for (Map.Entry<String, Long> entry2 : value.entrySet()) {
+          String s = entry2.getKey();
+          BackupProtos.ServerTimestamp.Builder stBuilder =
+              BackupProtos.ServerTimestamp.newBuilder();
+          HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder();
+          ServerName sn = ServerName.parseServerName(s);
+          snBuilder.setHostName(sn.getHostname());
+          snBuilder.setPort(sn.getPort());
+          stBuilder.setServerName(snBuilder.build());
+          stBuilder.setTimestamp(entry2.getValue());
+          tstBuilder.addServerTimestamp(stBuilder.build());
+        }
+        builder.addTstMap(tstBuilder.build());
+      }
+    }
+
+    public String getBackupId() {
+      return backupId;
+    }
+
+    private void setBackupId(String backupId) {
+      this.backupId = backupId;
+    }
+
+    public BackupType getType() {
+      return type;
+    }
+
+    private void setType(BackupType type) {
+      this.type = type;
+    }
+
+    public String getRootDir() {
+      return rootDir;
+    }
+
+    private void setRootDir(String rootDir) {
+      this.rootDir = rootDir;
+    }
+
+    public List<TableName> getTableNames() {
+      return tableList;
+    }
+
+    private void setTableList(List<TableName> tableList) {
+      this.tableList = tableList;
+    }
+
+    public long getStartTs() {
+      return startTs;
+    }
+
+    private void setStartTs(long startTs) {
+      this.startTs = startTs;
+    }
+
+    public long getCompleteTs() {
+      return completeTs;
+    }
+
+    private void setCompleteTs(long completeTs) {
+      this.completeTs = completeTs;
+    }
+
+    public ArrayList<BackupImage> getAncestors() {
+      if (this.ancestors == null) {
+        this.ancestors = new ArrayList<BackupImage>();
+      }
+      return this.ancestors;
+    }
+
+    private 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(TableName table) {
+      return tableList.contains(table);
+    }
+
+    @Override
+    public int compareTo(BackupImage other) {
+      String thisBackupId = this.getBackupId();
+      String otherBackupId = other.getBackupId();
+      int index1 = thisBackupId.lastIndexOf("_");
+      int index2 = otherBackupId.lastIndexOf("_");
+      String name1 = thisBackupId.substring(0, index1);
+      String name2 = otherBackupId.substring(0, index2);
+      if (name1.equals(name2)) {
+        Long thisTS = Long.valueOf(thisBackupId.substring(index1 + 1));
+        Long otherTS = Long.valueOf(otherBackupId.substring(index2 + 1));
+        return thisTS.compareTo(otherTS);
+      } else {
+        return name1.compareTo(name2);
+      }
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj instanceof BackupImage) {
+        return this.compareTo((BackupImage) obj) == 0;
+      }
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      int hash = 33 * this.getBackupId().hashCode() + type.hashCode();
+      hash = 33 * hash + rootDir.hashCode();
+      hash = 33 * hash + Long.valueOf(startTs).hashCode();
+      hash = 33 * hash + Long.valueOf(completeTs).hashCode();
+      for (TableName table : tableList) {
+        hash = 33 * hash + table.hashCode();
+      }
+      return hash;
+    }
+
+    public HashMap<TableName, HashMap<String, Long>> getIncrTimeRanges() {
+      return incrTimeRanges;
+    }
+
+    private void setIncrTimeRanges(HashMap<TableName, HashMap<String, Long>> incrTimeRanges) {
+      this.incrTimeRanges = incrTimeRanges;
+    }
+  }
+
+  // backup image directory
+  private String tableBackupDir = null;
+  private BackupImage backupImage;
+
+  /**
+   * Construct manifest for a ongoing backup.
+   * @param backup The ongoing backup info
+   */
+  public BackupManifest(BackupInfo backup) {
+
+    BackupImage.Builder builder = BackupImage.newBuilder();
+    this.backupImage =
+        builder.withBackupId(backup.getBackupId()).withType(backup.getType())
+            .withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames())
+            .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build();
+  }
+
+  /**
+   * Construct a table level manifest for a backup of the named table.
+   * @param backup The ongoing backup session info
+   */
+  public BackupManifest(BackupInfo backup, TableName table) {
+    this.tableBackupDir = backup.getTableBackupDir(table);
+    List<TableName> tables = new ArrayList<TableName>();
+    tables.add(table);
+    BackupImage.Builder builder = BackupImage.newBuilder();
+    this.backupImage =
+        builder.withBackupId(backup.getBackupId()).withType(backup.getType())
+            .withRootDir(backup.getBackupRootDir()).withTableList(tables)
+            .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build();
+  }
+
+  /**
+   * Construct manifest from a backup directory.
+   * @param conf configuration
+   * @param backupPath backup path
+   * @throws IOException
+   */
+
+  public BackupManifest(Configuration conf, Path backupPath) throws IOException {
+    this(backupPath.getFileSystem(conf), backupPath);
+  }
+
+  /**
+   * Construct manifest from a backup directory.
+   * @param fs the FileSystem
+   * @param backupPath backup path
+   * @throws BackupException exception
+   */
+
+  public BackupManifest(FileSystem fs, Path backupPath) throws BackupException {
+    if (LOG.isDebugEnabled()) {
+      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.
+    try {
+
+      FileStatus[] subFiles = BackupUtils.listStatus(fs, backupPath, null);
+      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(MANIFEST_FILE_NAME)) {
+
+          // load and set manifest field from file content
+          FSDataInputStream in = fs.open(subFile.getPath());
+          long len = subFile.getLen();
+          byte[] pbBytes = new byte[(int) len];
+          in.readFully(pbBytes);
+          BackupProtos.BackupImage proto = null;
+          try {
+            proto = BackupProtos.BackupImage.parseFrom(pbBytes);
+          } catch (Exception e) {
+            throw new BackupException(e);
+          }
+          this.backupImage = BackupImage.fromProto(proto);
+          LOG.debug("Loaded manifest instance from manifest file: "
+              + BackupUtils.getPath(subFile.getPath()));
+          return;
+        }
+      }
+      String errorMsg = "No manifest file found in: " + backupPath.toString();
+      throw new IOException(errorMsg);
+
+    } catch (IOException e) {
+      throw new BackupException(e.getMessage());
+    }
+  }
+
+  public BackupType getType() {
+    return backupImage.getType();
+  }
+
+  /**
+   * Get the table set of this image.
+   * @return The table set list
+   */
+  public List<TableName> getTableList() {
+    return backupImage.getTableNames();
+  }
+
+  /**
+   * Persist the manifest file.
+   * @throws IOException IOException when storing the manifest file.
+   */
+
+  public void store(Configuration conf) throws BackupException {
+    byte[] data = backupImage.toProto().toByteArray();
+    // write the file, overwrite if already exist
+    String logBackupDir =
+        BackupUtils.getLogBackupDir(backupImage.getRootDir(), backupImage.getBackupId());
+    Path manifestFilePath =
+        new Path(new Path((tableBackupDir != null ? tableBackupDir : logBackupDir)),
+            MANIFEST_FILE_NAME);
+    try (FSDataOutputStream out =
+        manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);) {
+      out.write(data);
+    } catch (IOException e) {
+      throw new BackupException(e.getMessage());
+    }
+
+    LOG.info("Manifest file stored to " + manifestFilePath);
+  }
+
+  /**
+   * Get this backup image.
+   * @return the backup image.
+   */
+  public BackupImage getBackupImage() {
+    return backupImage;
+  }
+
+  /**
+   * Add dependent backup image for this backup.
+   * @param image The direct dependent backup image
+   */
+  public void addDependentImage(BackupImage image) {
+    this.backupImage.addAncestor(image);
+  }
+
+  /**
+   * Set the incremental timestamp map directly.
+   * @param incrTimestampMap timestamp map
+   */
+  public void setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> incrTimestampMap) {
+    this.backupImage.setIncrTimeRanges(incrTimestampMap);
+  }
+
+  public Map<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+    return backupImage.getIncrTimeRanges();
+  }
+
+  /**
+   * 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>();
+    restoreImages.put(backupImage.startTs, backupImage);
+    for (BackupImage image : backupImage.getAncestors()) {
+      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(TableName 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() == BackupType.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(TableName 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;
+  }
+
+  /**
+   * 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() == BackupType.INCREMENTAL) {
+      return false;
+    }
+    if (image1.getStartTs() < image2.getStartTs()) {
+      return false;
+    }
+    List<TableName> image1TableList = image1.getTableNames();
+    List<TableName> image2TableList = image2.getTableNames();
+    boolean found = false;
+    for (int i = 0; i < image2TableList.size(); i++) {
+      found = false;
+      for (int j = 0; j < image1TableList.size(); j++) {
+        if (image2TableList.get(i).equals(image1TableList.get(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() == BackupType.INCREMENTAL) {
+        return false;
+      }
+      if (image1.getStartTs() < image.getStartTs()) {
+        return false;
+      }
+    }
+
+    ArrayList<String> image1TableList = new ArrayList<String>();
+    for (BackupImage image1 : fullImages) {
+      List<TableName> tableList = image1.getTableNames();
+      for (TableName table : tableList) {
+        image1TableList.add(table.getNameAsString());
+      }
+    }
+    ArrayList<String> image2TableList = new ArrayList<String>();
+    List<TableName> tableList = image.getTableNames();
+    for (TableName table : tableList) {
+      image2TableList.add(table.getNameAsString());
+    }
+
+    for (int i = 0; i < image2TableList.size(); i++) {
+      if (image1TableList.contains(image2TableList.get(i)) == false) {
+        return false;
+      }
+    }
+
+    LOG.debug("Full image set can cover image " + image.getBackupId());
+    return true;
+  }
+
+  public BackupInfo toBackupInfo() {
+    BackupInfo info = new BackupInfo();
+    info.setType(backupImage.getType());
+    List<TableName> list = backupImage.getTableNames();
+    TableName[] tables = new TableName[list.size()];
+    info.addTables(list.toArray(tables));
+    info.setBackupId(backupImage.getBackupId());
+    info.setStartTs(backupImage.getStartTs());
+    info.setBackupRootDir(backupImage.getRootDir());
+    if (backupImage.getType() == BackupType.INCREMENTAL) {
+      info.setHLogTargetDir(BackupUtils.getLogBackupDir(backupImage.getRootDir(),
+        backupImage.getBackupId()));
+    }
+    return info;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
new file mode 100644
index 0000000..6362f8e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -0,0 +1,1376 @@
+/**
+ * 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.impl;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.lang.StringUtils;
+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.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+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;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+/**
+ * This class provides API to access backup system table<br>
+ *
+ * Backup system table schema:<br>
+ * <p><ul>
+ * <li>1. Backup sessions rowkey= "session:"+backupId; value =serialized BackupInfo</li>
+ * <li>2. Backup start code rowkey = "startcode:"+backupRoot; value = startcode</li>
+ * <li>3. Incremental backup set rowkey="incrbackupset:"+backupRoot; value=[list of tables]</li>
+ * <li>4. Table-RS-timestamp map rowkey="trslm:"+backupRoot+table_name;
+ * value = map[RS-> last WAL timestamp]</li>
+ * <li>5. RS - WAL ts map rowkey="rslogts:"+backupRoot +server; value = last WAL timestamp</li>
+ * <li>6. WALs recorded rowkey="wals:"+WAL unique file name;
+ * value = backupId and full WAL file name</li>
+ * </ul></p>
+ */
+@InterfaceAudience.Private
+public final class BackupSystemTable implements Closeable {
+
+  static class WALItem {
+    String backupId;
+    String walFile;
+    String backupRoot;
+
+    WALItem(String backupId, String walFile, String backupRoot) {
+      this.backupId = backupId;
+      this.walFile = walFile;
+      this.backupRoot = backupRoot;
+    }
+
+    public String getBackupId() {
+      return backupId;
+    }
+
+    public String getWalFile() {
+      return walFile;
+    }
+
+    public String getBackupRoot() {
+      return backupRoot;
+    }
+
+    @Override
+    public String toString() {
+      return Path.SEPARATOR + backupRoot + Path.SEPARATOR + backupId + Path.SEPARATOR + walFile;
+    }
+
+  }
+
+  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
+
+  private TableName tableName;
+  /**
+   *  Stores backup sessions (contexts)
+   */
+  final static byte[] SESSIONS_FAMILY = "session".getBytes();
+  /**
+   * Stores other meta
+   */
+  final static byte[] META_FAMILY = "meta".getBytes();
+  /**
+   *  Connection to HBase cluster, shared among all instances
+   */
+  private final Connection connection;
+
+
+  private final static String BACKUP_INFO_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 String SET_KEY_PREFIX = "backupset:";
+
+  private final static byte[] EMPTY_VALUE = new byte[] {};
+
+  // Safe delimiter in a string
+  private final static String NULL = "\u0000";
+
+  public BackupSystemTable(Connection conn) throws IOException {
+    this.connection = conn;
+    tableName = BackupSystemTable.getTableName(conn.getConfiguration());
+    checkSystemTable();
+  }
+
+  private void checkSystemTable() throws IOException {
+    try (Admin admin = connection.getAdmin();) {
+
+      if (!admin.tableExists(tableName)) {
+        HTableDescriptor backupHTD =
+            BackupSystemTable.getSystemTableDescriptor(connection.getConfiguration());
+        admin.createTable(backupHTD);
+      }
+      waitForSystemTable(admin);
+    }
+  }
+
+  private void waitForSystemTable(Admin admin) throws IOException {
+    long TIMEOUT = 60000;
+    long startTime = EnvironmentEdgeManager.currentTime();
+    while (!admin.tableExists(tableName) || !admin.isTableAvailable(tableName)) {
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException e) {
+      }
+      if (EnvironmentEdgeManager.currentTime() - startTime > TIMEOUT) {
+        throw new IOException("Failed to create backup system table after "+ TIMEOUT+"ms");
+      }
+    }
+    LOG.debug("Backup table exists and available");
+
+  }
+
+
+
+  @Override
+  public void close() {
+    // do nothing
+  }
+
+  /**
+   * Updates status (state) of a backup session in backup system table table
+   * @param info backup info
+   * @throws IOException exception
+   */
+  public void updateBackupInfo(BackupInfo info) throws IOException {
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("update backup status in backup system table for: " + info.getBackupId()
+          + " set status=" + info.getState());
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = createPutForBackupInfo(info);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Deletes backup status from backup system table table
+   * @param backupId backup id
+   * @throws IOException exception
+   */
+
+  public void deleteBackupInfo(String backupId) throws IOException {
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("delete backup status in backup system table for " + backupId);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Delete del = createDeleteForBackupInfo(backupId);
+      table.delete(del);
+    }
+  }
+
+  /**
+   * Reads backup status object (instance of backup info) from backup system table table
+   * @param backupId backup id
+   * @return Current status of backup session or null
+   */
+
+  public BackupInfo readBackupInfo(String backupId) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read backup status from backup system table for: " + backupId);
+    }
+
+    try (Table table = connection.getTable(tableName)) {
+      Get get = createGetForBackupInfo(backupId);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return null;
+      }
+      return resultToBackupInfo(res);
+    }
+  }
+
+  /**
+   * 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.
+   * @param backupRoot directory path to backup destination
+   * @return the timestamp of last successful backup
+   * @throws IOException exception
+   */
+  public String readBackupStartCode(String backupRoot) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read backup start code from backup system table");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Get get = createGetForStartCode(backupRoot);
+      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);
+    }
+  }
+
+  /**
+   * Write the start code (timestamp) to backup system table. If passed in null, then write 0 byte.
+   * @param startCode start code
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public void writeBackupStartCode(Long startCode, String backupRoot) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("write backup start code to backup system table " + startCode);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = createPutForStartCode(startCode.toString(), backupRoot);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Get the Region Servers log information after the last log roll from backup system table.
+   * @param backupRoot root directory path to backup
+   * @return RS log info
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> readRegionServerLastLogRollResult(String backupRoot)
+      throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read region server last roll log result to backup system table");
+    }
+
+    Scan scan = createScanForReadRegionServerLastLogRollResult(backupRoot);
+
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      HashMap<String, Long> rsTimestampMap = new HashMap<String, Long>();
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        Cell cell = res.current();
+        byte[] row = CellUtil.cloneRow(cell);
+        String server =
+            getServerNameForReadRegionServerLastLogRollResult(row);
+        byte[] data = CellUtil.cloneValue(cell);
+        rsTimestampMap.put(server, Bytes.toLong(data));
+      }
+      return rsTimestampMap;
+    }
+  }
+
+  /**
+   * Writes Region Server last roll log result (timestamp) to backup system table table
+   * @param server Region Server name
+   * @param ts last log timestamp
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public void writeRegionServerLastLogRollResult(String server, Long ts, String backupRoot)
+      throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("write region server last roll log result to backup system table");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put =
+          createPutForRegionServerLastLogRollResult(server, ts, backupRoot);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Get all completed backup information (in desc order by time)
+   * @param onlyCompleted true, if only successfully completed sessions
+   * @return history info of BackupCompleteData
+   * @throws IOException exception
+   */
+  public ArrayList<BackupInfo> getBackupHistory(boolean onlyCompleted) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("get backup history from backup system table");
+    }
+    ArrayList<BackupInfo> list;
+    BackupState state = onlyCompleted ? BackupState.COMPLETE : BackupState.ANY;
+    list = getBackupInfos(state);
+    return BackupUtils.sortHistoryListDesc(list);
+  }
+
+  /**
+   * Get all backups history
+   * @return list of backup info
+   * @throws IOException
+   */
+  public List<BackupInfo> getBackupHistory() throws IOException {
+    return getBackupHistory(false);
+  }
+
+  /**
+   * Get first n backup history records
+   * @param n number of records
+   * @return list of records
+   * @throws IOException
+   */
+  public List<BackupInfo> getHistory(int n) throws IOException {
+
+    List<BackupInfo> history = getBackupHistory();
+    if (history.size() <= n) return history;
+    List<BackupInfo> list = new ArrayList<BackupInfo>();
+    for (int i = 0; i < n; i++) {
+      list.add(history.get(i));
+    }
+    return list;
+
+  }
+
+  /**
+   * Get backup history records filtered by list of filters.
+   * @param n max number of records
+   * @param filters list of filters
+   * @return backup records
+   * @throws IOException
+   */
+  public List<BackupInfo> getBackupHistory(int n, BackupInfo.Filter... filters) throws IOException {
+    if (filters.length == 0) return getHistory(n);
+
+    List<BackupInfo> history = getBackupHistory();
+    List<BackupInfo> result = new ArrayList<BackupInfo>();
+    for (BackupInfo bi : history) {
+      if (result.size() == n) break;
+      boolean passed = true;
+      for (int i = 0; i < filters.length; i++) {
+        if (!filters[i].apply(bi)) {
+          passed = false;
+          break;
+        }
+      }
+      if (passed) {
+        result.add(bi);
+      }
+    }
+    return result;
+
+  }
+
+  /**
+   * Get history for backup destination
+   * @param backupRoot backup destination path
+   * @return List of backup info
+   * @throws IOException
+   */
+  public List<BackupInfo> getBackupHistory(String backupRoot) throws IOException {
+    ArrayList<BackupInfo> history = getBackupHistory(false);
+    for (Iterator<BackupInfo> iterator = history.iterator(); iterator.hasNext();) {
+      BackupInfo info = iterator.next();
+      if (!backupRoot.equals(info.getBackupRootDir())) {
+        iterator.remove();
+      }
+    }
+    return history;
+  }
+
+  /**
+   * Get history for a table
+   * @param name table name
+   * @return history for a table
+   * @throws IOException
+   */
+  public List<BackupInfo> getBackupHistoryForTable(TableName name) throws IOException {
+    List<BackupInfo> history = getBackupHistory();
+    List<BackupInfo> tableHistory = new ArrayList<BackupInfo>();
+    for (BackupInfo info : history) {
+      List<TableName> tables = info.getTableNames();
+      if (tables.contains(name)) {
+        tableHistory.add(info);
+      }
+    }
+    return tableHistory;
+  }
+
+  public Map<TableName, ArrayList<BackupInfo>> getBackupHistoryForTableSet(Set<TableName> set,
+      String backupRoot) throws IOException {
+    List<BackupInfo> history = getBackupHistory(backupRoot);
+    Map<TableName, ArrayList<BackupInfo>> tableHistoryMap =
+        new HashMap<TableName, ArrayList<BackupInfo>>();
+    for (Iterator<BackupInfo> iterator = history.iterator(); iterator.hasNext();) {
+      BackupInfo info = iterator.next();
+      if (!backupRoot.equals(info.getBackupRootDir())) {
+        continue;
+      }
+      List<TableName> tables = info.getTableNames();
+      for (TableName tableName : tables) {
+        if (set.contains(tableName)) {
+          ArrayList<BackupInfo> list = tableHistoryMap.get(tableName);
+          if (list == null) {
+            list = new ArrayList<BackupInfo>();
+            tableHistoryMap.put(tableName, list);
+          }
+          list.add(info);
+        }
+      }
+    }
+    return tableHistoryMap;
+  }
+
+  /**
+   * Get all backup sessions with a given state (in descending order by time)
+   * @param state backup session state
+   * @return history info of backup info objects
+   * @throws IOException exception
+   */
+  public ArrayList<BackupInfo> getBackupInfos(BackupState state) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("get backup infos from backup system table");
+    }
+
+    Scan scan = createScanForBackupHistory();
+    ArrayList<BackupInfo> list = new ArrayList<BackupInfo>();
+
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        BackupInfo context = cellToBackupInfo(res.current());
+        if (state != BackupState.ANY && context.getState() != state) {
+          continue;
+        }
+        list.add(context);
+      }
+      return list;
+    }
+  }
+
+  /**
+   * Write the current timestamps for each regionserver to backup system table 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
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public void writeRegionServerLogTimestamp(Set<TableName> tables,
+      HashMap<String, Long> newTimestamps, String backupRoot) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("write RS log time stamps to backup system table for tables ["
+          + StringUtils.join(tables, ",") + "]");
+    }
+    List<Put> puts = new ArrayList<Put>();
+    for (TableName table : tables) {
+      byte[] smapData = toTableServerTimestampProto(table, newTimestamps).toByteArray();
+      Put put =
+          createPutForWriteRegionServerLogTimestamp(table, smapData,
+            backupRoot);
+      puts.add(put);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      table.put(puts);
+    }
+  }
+
+  /**
+   * 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
+   * @param backupRoot root directory path to backup
+   * @return the timestamp for each region server. key: tableName value:
+   *         RegionServer,PreviousTimeStamp
+   * @throws IOException exception
+   */
+  public HashMap<TableName, HashMap<String, Long>> readLogTimestampMap(String backupRoot)
+      throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read RS log ts from backup system table for root=" + backupRoot);
+    }
+
+    HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
+        new HashMap<TableName, HashMap<String, Long>>();
+
+    Scan scan = createScanForReadLogTimestampMap(backupRoot);
+    try (Table table = connection.getTable(tableName);
+        ResultScanner 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 = getTableNameForReadLogTimestampMap(row);
+        TableName tn = TableName.valueOf(tabName);
+        byte[] data = CellUtil.cloneValue(cell);
+        if (data == null) {
+          throw new IOException("Data of last backup data from backup system table "
+              + "is empty. Create a backup first.");
+        }
+        if (data != null && data.length > 0) {
+          HashMap<String, Long> lastBackup =
+              fromTableServerTimestampProto(BackupProtos.TableServerTimestamp.parseFrom(data));
+          tableTimestampMap.put(tn, lastBackup);
+        }
+      }
+      return tableTimestampMap;
+    }
+  }
+
+  private BackupProtos.TableServerTimestamp toTableServerTimestampProto(TableName table,
+      Map<String, Long> map) {
+    BackupProtos.TableServerTimestamp.Builder tstBuilder =
+        BackupProtos.TableServerTimestamp.newBuilder();
+    tstBuilder.setTableName(org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil
+        .toProtoTableName(table));
+
+    for (Entry<String, Long> entry : map.entrySet()) {
+      BackupProtos.ServerTimestamp.Builder builder = BackupProtos.ServerTimestamp.newBuilder();
+      HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder();
+      ServerName sn = ServerName.parseServerName(entry.getKey());
+      snBuilder.setHostName(sn.getHostname());
+      snBuilder.setPort(sn.getPort());
+      builder.setServerName(snBuilder.build());
+      builder.setTimestamp(entry.getValue());
+      tstBuilder.addServerTimestamp(builder.build());
+    }
+
+    return tstBuilder.build();
+  }
+
+  private HashMap<String, Long> fromTableServerTimestampProto(
+      BackupProtos.TableServerTimestamp proto) {
+    HashMap<String, Long> map = new HashMap<String, Long>();
+    List<BackupProtos.ServerTimestamp> list = proto.getServerTimestampList();
+    for (BackupProtos.ServerTimestamp st : list) {
+      ServerName sn =
+          org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(st.getServerName());
+      map.put(sn.getHostname() + ":" + sn.getPort(), st.getTimestamp());
+    }
+    return map;
+  }
+
+  /**
+   * Return the current tables covered by incremental backup.
+   * @param backupRoot root directory path to backup
+   * @return set of tableNames
+   * @throws IOException exception
+   */
+  public Set<TableName> getIncrementalBackupTableSet(String backupRoot) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("get incremental backup table set from backup system table");
+    }
+    TreeSet<TableName> set = new TreeSet<>();
+
+    try (Table table = connection.getTable(tableName)) {
+      Get get = createGetForIncrBackupTableSet(backupRoot);
+      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
+        set.add(TableName.valueOf(CellUtil.cloneQualifier(cell)));
+      }
+      return set;
+    }
+  }
+
+  /**
+   * Add tables to global incremental backup set
+   * @param tables set of tables
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public void addIncrementalBackupTableSet(Set<TableName> tables, String backupRoot)
+      throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Add incremental backup table set to backup system table. ROOT=" + backupRoot
+          + " tables [" + StringUtils.join(tables, " ") + "]");
+      for (TableName table : tables) {
+        LOG.debug(table);
+      }
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = createPutForIncrBackupTableSet(tables, backupRoot);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Deletes incremental backup set for a backup destination
+   * @param backupRoot backup root
+   */
+
+  public void deleteIncrementalBackupTableSet(String backupRoot) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Delete incremental backup table set to backup system table. ROOT=" + backupRoot);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Delete delete = createDeleteForIncrBackupTableSet(backupRoot);
+      table.delete(delete);
+    }
+  }
+
+  /**
+   * Register WAL files as eligible for deletion
+   * @param files files
+   * @param backupId backup id
+   * @param backupRoot root directory path to backup destination
+   * @throws IOException exception
+   */
+  public void addWALFiles(List<String> files, String backupId, String backupRoot)
+      throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("add WAL files to backup system table: " + backupId + " " + backupRoot + " files ["
+          + StringUtils.join(files, ",") + "]");
+      for (String f : files) {
+        LOG.debug("add :" + f);
+      }
+    }
+    try (Table table = connection.getTable(tableName)) {
+      List<Put> puts =
+          createPutsForAddWALFiles(files, backupId, backupRoot);
+      table.put(puts);
+    }
+  }
+
+  /**
+   * Register WAL files as eligible for deletion
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public Iterator<WALItem> getWALFilesIterator(String backupRoot) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("get WAL files from backup system table");
+    }
+    final Table table = connection.getTable(tableName);
+    Scan scan = createScanForGetWALs(backupRoot);
+    final ResultScanner scanner = table.getScanner(scan);
+    final Iterator<Result> it = scanner.iterator();
+    return new Iterator<WALItem>() {
+
+      @Override
+      public boolean hasNext() {
+        boolean next = it.hasNext();
+        if (!next) {
+          // close all
+          try {
+            scanner.close();
+            table.close();
+          } catch (IOException e) {
+            LOG.error("Close WAL Iterator", e);
+          }
+        }
+        return next;
+      }
+
+      @Override
+      public WALItem next() {
+        Result next = it.next();
+        List<Cell> cells = next.listCells();
+        byte[] buf = cells.get(0).getValueArray();
+        int len = cells.get(0).getValueLength();
+        int offset = cells.get(0).getValueOffset();
+        String backupId = new String(buf, offset, len);
+        buf = cells.get(1).getValueArray();
+        len = cells.get(1).getValueLength();
+        offset = cells.get(1).getValueOffset();
+        String walFile = new String(buf, offset, len);
+        buf = cells.get(2).getValueArray();
+        len = cells.get(2).getValueLength();
+        offset = cells.get(2).getValueOffset();
+        String backupRoot = new String(buf, offset, len);
+        return new WALItem(backupId, walFile, backupRoot);
+      }
+
+      @Override
+      public void remove() {
+        // not implemented
+        throw new RuntimeException("remove is not supported");
+      }
+    };
+
+  }
+
+  /**
+   * Check if WAL file is eligible for deletion Future: to support all backup destinations
+   * @param file name of a file to check
+   * @return true, if deletable, false otherwise.
+   * @throws IOException exception
+   */
+  public boolean isWALFileDeletable(String file) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Check if WAL file has been already backed up in backup system table " + file);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Get get = createGetForCheckWALFile(file);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  /**
+   * Checks if we have at least one backup session in backup system table This API is used by
+   * BackupLogCleaner
+   * @return true, if - at least one session exists in backup system table table
+   * @throws IOException exception
+   */
+  public boolean hasBackupSessions() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Has backup sessions from backup system table");
+    }
+    boolean result = false;
+    Scan scan = createScanForBackupHistory();
+    scan.setCaching(1);
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      if (scanner.next() != null) {
+        result = true;
+      }
+      return result;
+    }
+  }
+
+  /**
+   * BACKUP SETS
+   */
+
+  /**
+   * Get backup set list
+   * @return backup set list
+   * @throws IOException
+   */
+  public List<String> listBackupSets() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(" Backup set list");
+    }
+    List<String> list = new ArrayList<String>();
+    Table table = null;
+    ResultScanner scanner = null;
+    try {
+      table = connection.getTable(tableName);
+      Scan scan = createScanForBackupSetList();
+      scan.setMaxVersions(1);
+      scanner = table.getScanner(scan);
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        list.add(cellKeyToBackupSetName(res.current()));
+      }
+      return list;
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Get backup set description (list of tables)
+   * @param name set's name
+   * @return list of tables in a backup set
+   * @throws IOException
+   */
+  public List<TableName> describeBackupSet(String name) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(" Backup set describe: " + name);
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Get get = createGetForBackupSet(name);
+      Result res = table.get(get);
+      if (res.isEmpty()) return null;
+      res.advance();
+      String[] tables = cellValueToBackupSet(res.current());
+      return toList(tables);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  private List<TableName> toList(String[] tables) {
+    List<TableName> list = new ArrayList<TableName>(tables.length);
+    for (String name : tables) {
+      list.add(TableName.valueOf(name));
+    }
+    return list;
+  }
+
+  /**
+   * Add backup set (list of tables)
+   * @param name set name
+   * @param newTables list of tables, comma-separated
+   * @throws IOException
+   */
+  public void addToBackupSet(String name, String[] newTables) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Backup set add: " + name + " tables [" + StringUtils.join(newTables, " ") + "]");
+    }
+    Table table = null;
+    String[] union = null;
+    try {
+      table = connection.getTable(tableName);
+      Get get = createGetForBackupSet(name);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        union = newTables;
+      } else {
+        res.advance();
+        String[] tables = cellValueToBackupSet(res.current());
+        union = merge(tables, newTables);
+      }
+      Put put = createPutForBackupSet(name, union);
+      table.put(put);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  private String[] merge(String[] tables, String[] newTables) {
+    List<String> list = new ArrayList<String>();
+    // Add all from tables
+    for (String t : tables) {
+      list.add(t);
+    }
+    for (String nt : newTables) {
+      if (list.contains(nt)) continue;
+      list.add(nt);
+    }
+    String[] arr = new String[list.size()];
+    list.toArray(arr);
+    return arr;
+  }
+
+  /**
+   * Remove tables from backup set (list of tables)
+   * @param name set name
+   * @param toRemove list of tables
+   * @throws IOException
+   */
+  public void removeFromBackupSet(String name, String[] toRemove) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(" Backup set remove from : " + name + " tables [" + StringUtils.join(toRemove, " ")
+          + "]");
+    }
+    Table table = null;
+    String[] disjoint = null;
+    String[] tables = null;
+    try {
+      table = connection.getTable(tableName);
+      Get get = createGetForBackupSet(name);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        LOG.warn("Backup set '" + name + "' not found.");
+        return;
+      } else {
+        res.advance();
+        tables = cellValueToBackupSet(res.current());
+        disjoint = disjoin(tables, toRemove);
+      }
+      if (disjoint.length > 0 && disjoint.length != tables.length) {
+        Put put = createPutForBackupSet(name, disjoint);
+        table.put(put);
+      } else if(disjoint.length == tables.length) {
+        LOG.warn("Backup set '" + name + "' does not contain tables ["
+            + StringUtils.join(toRemove, " ") + "]");
+      } else { // disjoint.length == 0 and tables.length >0
+        // Delete  backup set
+        LOG.info("Backup set '"+name+"' is empty. Deleting.");
+        deleteBackupSet(name);
+      }
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  private String[] disjoin(String[] tables, String[] toRemove) {
+    List<String> list = new ArrayList<String>();
+    // Add all from tables
+    for (String t : tables) {
+      list.add(t);
+    }
+    for (String nt : toRemove) {
+      if (list.contains(nt)) {
+        list.remove(nt);
+      }
+    }
+    String[] arr = new String[list.size()];
+    list.toArray(arr);
+    return arr;
+  }
+
+  /**
+   * Delete backup set
+   * @param name set's name
+   * @throws IOException
+   */
+  public void deleteBackupSet(String name) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(" Backup set delete: " + name);
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Delete del = createDeleteForBackupSet(name);
+      table.delete(del);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Get backup system table descriptor
+   * @return table's descriptor
+   */
+  public static HTableDescriptor getSystemTableDescriptor(Configuration conf) {
+
+    HTableDescriptor tableDesc = new HTableDescriptor(getTableName(conf));
+    HColumnDescriptor colSessionsDesc = new HColumnDescriptor(SESSIONS_FAMILY);
+    colSessionsDesc.setMaxVersions(1);
+    // Time to keep backup sessions (secs)
+    Configuration config = HBaseConfiguration.create();
+    int ttl =
+        config.getInt(BackupRestoreConstants.BACKUP_SYSTEM_TTL_KEY,
+          BackupRestoreConstants.BACKUP_SYSTEM_TTL_DEFAULT);
+    colSessionsDesc.setTimeToLive(ttl);
+    tableDesc.addFamily(colSessionsDesc);
+    HColumnDescriptor colMetaDesc = new HColumnDescriptor(META_FAMILY);
+    tableDesc.addFamily(colMetaDesc);
+    return tableDesc;
+  }
+
+  public static TableName getTableName(Configuration conf) {
+    String name =
+        conf.get(BackupRestoreConstants.BACKUP_SYSTEM_TABLE_NAME_KEY,
+          BackupRestoreConstants.BACKUP_SYSTEM_TABLE_NAME_DEFAULT);
+    return TableName.valueOf(name);
+  }
+
+  public static String getTableNameAsString(Configuration conf) {
+    return getTableName(conf).getNameAsString();
+  }
+
+
+
+
+
+  /**
+   * Creates Put operation for a given backup info object
+   * @param context backup info
+   * @return put operation
+   * @throws IOException exception
+   */
+  private Put createPutForBackupInfo(BackupInfo context) throws IOException {
+    Put put = new Put(rowkey(BACKUP_INFO_PREFIX, context.getBackupId()));
+    put.addColumn(BackupSystemTable.SESSIONS_FAMILY, Bytes.toBytes("context"),
+      context.toByteArray());
+    return put;
+  }
+
+  /**
+   * Creates Get operation for a given backup id
+   * @param backupId backup's ID
+   * @return get operation
+   * @throws IOException exception
+   */
+  private Get createGetForBackupInfo(String backupId) throws IOException {
+    Get get = new Get(rowkey(BACKUP_INFO_PREFIX, backupId));
+    get.addFamily(BackupSystemTable.SESSIONS_FAMILY);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Delete operation for a given backup id
+   * @param backupId backup's ID
+   * @return delete operation
+   * @throws IOException exception
+   */
+  private Delete createDeleteForBackupInfo(String backupId) {
+    Delete del = new Delete(rowkey(BACKUP_INFO_PREFIX, backupId));
+    del.addFamily(BackupSystemTable.SESSIONS_FAMILY);
+    return del;
+  }
+
+  /**
+   * Converts Result to BackupInfo
+   * @param res HBase result
+   * @return backup info instance
+   * @throws IOException exception
+   */
+  private BackupInfo resultToBackupInfo(Result res) throws IOException {
+    res.advance();
+    Cell cell = res.current();
+    return cellToBackupInfo(cell);
+  }
+
+  /**
+   * Creates Get operation to retrieve start code from backup system table
+   * @return get operation
+   * @throws IOException exception
+   */
+  private Get createGetForStartCode(String rootPath) throws IOException {
+    Get get = new Get(rowkey(START_CODE_ROW, rootPath));
+    get.addFamily(BackupSystemTable.META_FAMILY);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Put operation to store start code to backup system table
+   * @return put operation
+   * @throws IOException exception
+   */
+  private Put createPutForStartCode(String startCode, String rootPath) {
+    Put put = new Put(rowkey(START_CODE_ROW, rootPath));
+    put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("startcode"),
+      Bytes.toBytes(startCode));
+    return put;
+  }
+
+  /**
+   * Creates Get to retrieve incremental backup table set from backup system table
+   * @return get operation
+   * @throws IOException exception
+   */
+  private Get createGetForIncrBackupTableSet(String backupRoot) throws IOException {
+    Get get = new Get(rowkey(INCR_BACKUP_SET, backupRoot));
+    get.addFamily(BackupSystemTable.META_FAMILY);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Put to store incremental backup table set
+   * @param tables tables
+   * @return put operation
+   */
+  private Put createPutForIncrBackupTableSet(Set<TableName> tables, String backupRoot) {
+    Put put = new Put(rowkey(INCR_BACKUP_SET, backupRoot));
+    for (TableName table : tables) {
+      put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes(table.getNameAsString()),
+        EMPTY_VALUE);
+    }
+    return put;
+  }
+
+  /**
+   * Creates Delete for incremental backup table set
+   * @param backupRoot backup root
+   * @return delete operation
+   */
+  private Delete createDeleteForIncrBackupTableSet(String backupRoot) {
+    Delete delete = new Delete(rowkey(INCR_BACKUP_SET, backupRoot));
+    delete.addFamily(BackupSystemTable.META_FAMILY);
+    return delete;
+  }
+
+  /**
+   * Creates Scan operation to load backup history
+   * @return scan operation
+   */
+  private Scan createScanForBackupHistory() {
+    Scan scan = new Scan();
+    byte[] startRow = Bytes.toBytes(BACKUP_INFO_PREFIX);
+    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.SESSIONS_FAMILY);
+    scan.setMaxVersions(1);
+    return scan;
+  }
+
+  /**
+   * Converts cell to backup info instance.
+   * @param current current cell
+   * @return backup backup info instance
+   * @throws IOException exception
+   */
+  private BackupInfo cellToBackupInfo(Cell current) throws IOException {
+    byte[] data = CellUtil.cloneValue(current);
+    return BackupInfo.fromByteArray(data);
+  }
+
+  /**
+   * Creates Put to write RS last roll log timestamp map
+   * @param table table
+   * @param smap map, containing RS:ts
+   * @return put operation
+   */
+  private Put createPutForWriteRegionServerLogTimestamp(TableName table, byte[] smap,
+      String backupRoot) {
+    Put put = new Put(rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot, NULL, table.getNameAsString()));
+    put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("log-roll-map"), smap);
+    return put;
+  }
+
+  /**
+   * Creates Scan to load table-> { RS -> ts} map of maps
+   * @return scan operation
+   */
+  private Scan createScanForReadLogTimestampMap(String backupRoot) {
+    Scan scan = new Scan();
+    byte[] startRow = rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot);
+    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.META_FAMILY);
+
+    return scan;
+  }
+
+  /**
+   * Get table name from rowkey
+   * @param cloneRow rowkey
+   * @return table name
+   */
+  private String getTableNameForReadLogTimestampMap(byte[] cloneRow) {
+    String s = Bytes.toString(cloneRow);
+    int index = s.lastIndexOf(NULL);
+    return s.substring(index + 1);
+  }
+
+  /**
+   * Creates Put to store RS last log result
+   * @param server server name
+   * @param timestamp log roll result (timestamp)
+   * @return put operation
+   */
+  private Put createPutForRegionServerLastLogRollResult(String server, Long timestamp,
+      String backupRoot) {
+    Put put = new Put(rowkey(RS_LOG_TS_PREFIX, backupRoot, NULL, server));
+    put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("rs-log-ts"),
+      Bytes.toBytes(timestamp));
+    return put;
+  }
+
+  /**
+   * Creates Scan operation to load last RS log roll results
+   * @return scan operation
+   */
+  private Scan createScanForReadRegionServerLastLogRollResult(String backupRoot) {
+    Scan scan = new Scan();
+    byte[] startRow = rowkey(RS_LOG_TS_PREFIX, backupRoot);
+    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.META_FAMILY);
+    scan.setMaxVersions(1);
+
+    return scan;
+  }
+
+  /**
+   * Get server's name from rowkey
+   * @param row rowkey
+   * @return server's name
+   */
+  private String getServerNameForReadRegionServerLastLogRollResult(byte[] row) {
+    String s = Bytes.toString(row);
+    int index = s.lastIndexOf(NULL);
+    return s.substring(index + 1);
+  }
+
+  /**
+   * 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
+   */
+  private List<Put> createPutsForAddWALFiles(List<String> files, String backupId,
+      String backupRoot) throws IOException {
+
+    List<Put> puts = new ArrayList<Put>();
+    for (String file : files) {
+      Put put = new Put(rowkey(WALS_PREFIX, BackupUtils.getUniqueWALFileNamePart(file)));
+      put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("backupId"),
+        Bytes.toBytes(backupId));
+      put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("file"), Bytes.toBytes(file));
+      put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("root"), Bytes.toBytes(backupRoot));
+      puts.add(put);
+    }
+    return puts;
+  }
+
+  /**
+   * Creates Scan operation to load WALs
+   * @param backupRoot path to backup destination
+   * @return scan operation
+   */
+  private Scan createScanForGetWALs(String backupRoot) {
+    // TODO: support for backupRoot
+    Scan scan = new Scan();
+    byte[] startRow = Bytes.toBytes(WALS_PREFIX);
+    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.META_FAMILY);
+    return scan;
+  }
+
+  /**
+   * Creates Get operation for a given wal file name TODO: support for backup destination
+   * @param file file
+   * @return get operation
+   * @throws IOException exception
+   */
+  private Get createGetForCheckWALFile(String file) throws IOException {
+    Get get = new Get(rowkey(WALS_PREFIX, BackupUtils.getUniqueWALFileNamePart(file)));
+    // add backup root column
+    get.addFamily(BackupSystemTable.META_FAMILY);
+    return get;
+  }
+
+  /**
+   * Creates Scan operation to load backup set list
+   * @return scan operation
+   */
+  private Scan createScanForBackupSetList() {
+    Scan scan = new Scan();
+    byte[] startRow = Bytes.toBytes(SET_KEY_PREFIX);
+    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.META_FAMILY);
+    return scan;
+  }
+
+  /**
+   * Creates Get operation to load backup set content
+   * @return get operation
+   */
+  private Get createGetForBackupSet(String name) {
+    Get get = new Get(rowkey(SET_KEY_PREFIX, name));
+    get.addFamily(BackupSystemTable.META_FAMILY);
+    return get;
+  }
+
+  /**
+   * Creates Delete operation to delete backup set content
+   * @param name backup set's name
+   * @return delete operation
+   */
+  private Delete createDeleteForBackupSet(String name) {
+    Delete del = new Delete(rowkey(SET_KEY_PREFIX, name));
+    del.addFamily(BackupSystemTable.META_FAMILY);
+    return del;
+  }
+
+  /**
+   * Creates Put operation to update backup set content
+   * @param name backup set's name
+   * @param tables list of tables
+   * @return put operation
+   */
+  private Put createPutForBackupSet(String name, String[] tables) {
+    Put put = new Put(rowkey(SET_KEY_PREFIX, name));
+    byte[] value = convertToByteArray(tables);
+    put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes("tables"), value);
+    return put;
+  }
+
+  private byte[] convertToByteArray(String[] tables) {
+    return StringUtils.join(tables, ",").getBytes();
+  }
+
+  /**
+   * Converts cell to backup set list.
+   * @param current current cell
+   * @return backup set as array of table names
+   * @throws IOException
+   */
+  private String[] cellValueToBackupSet(Cell current) throws IOException {
+    byte[] data = CellUtil.cloneValue(current);
+    if (data != null && data.length > 0) {
+      return Bytes.toString(data).split(",");
+    } else {
+      return new String[0];
+    }
+  }
+
+  /**
+   * Converts cell key to backup set name.
+   * @param current current cell
+   * @return backup set name
+   * @throws IOException
+   */
+  private String cellKeyToBackupSetName(Cell current) throws IOException {
+    byte[] data = CellUtil.cloneRow(current);
+    return Bytes.toString(data).substring(SET_KEY_PREFIX.length());
+  }
+
+  private byte[] rowkey(String s, String... other) {
+    StringBuilder sb = new StringBuilder(s);
+    for (String ss : other) {
+      sb.append(ss);
+    }
+    return sb.toString().getBytes();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
new file mode 100644
index 0000000..77d1184
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
@@ -0,0 +1,189 @@
+/**
+ * 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.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupCopyJob;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+/**
+ * Full table backup implementation
+ *
+ */
+@InterfaceAudience.Private
+public class FullTableBackupClient extends TableBackupClient {
+  private static final Log LOG = LogFactory.getLog(FullTableBackupClient.class);
+
+  public FullTableBackupClient(final Connection conn, final String backupId, BackupRequest request)
+      throws IOException {
+    super(conn, backupId, request);
+  }
+
+  /**
+   * Do snapshot copy.
+   * @param backupInfo backup info
+   * @throws Exception exception
+   */
+  private void snapshotCopy(BackupInfo backupInfo) throws Exception {
+    LOG.info("Snapshot copy is starting.");
+
+    // set overall backup phase: snapshot_copy
+    backupInfo.setPhase(BackupPhase.SNAPSHOTCOPY);
+
+    // call ExportSnapshot to copy files based on hbase snapshot for backup
+    // ExportSnapshot only support single snapshot export, need loop for multiple tables case
+    BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf);
+
+    // number of snapshots matches number of tables
+    float numOfSnapshots = backupInfo.getSnapshotNames().size();
+
+    LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied.");
+
+    for (TableName table : backupInfo.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.
+      // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots);
+      int res = 0;
+      String[] args = new String[4];
+      args[0] = "-snapshot";
+      args[1] = backupInfo.getSnapshotName(table);
+      args[2] = "-copy-to";
+      args[3] = backupInfo.getTableBackupDir(table);
+
+      LOG.debug("Copy snapshot " + args[1] + " to " + args[3]);
+      res = copyService.copy(backupInfo, backupManager, conf, BackupType.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.");
+    }
+  }
+
+  /**
+   * Backup request execution
+   * @throws IOException
+   */
+  @Override
+  public void execute() throws IOException {
+
+    try (Admin admin = conn.getAdmin();) {
+
+      // Begin BACKUP
+      beginBackup(backupManager, backupInfo);
+      String savedStartCode = null;
+      boolean firstBackup = false;
+      // do snapshot for full table backup
+
+      savedStartCode = backupManager.readBackupStartCode();
+      firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L;
+      if (firstBackup) {
+        // This is our first backup. Let's put some marker to system table so that we can hold the logs
+        // while we do the backup.
+        backupManager.writeBackupStartCode(0L);
+      }
+      // 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 ...");
+
+      Map<String, String> props = new HashMap<String, String>();
+      props.put("backupRoot", backupInfo.getBackupRootDir());
+      admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
+        LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props);
+
+      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 = BackupUtils.getWALFilesOlderThan(conf, newTimestamps);
+        backupManager.recordWALFiles(logFiles);
+      }
+
+      // SNAPSHOT_TABLES:
+      backupInfo.setPhase(BackupPhase.SNAPSHOT);
+      for (TableName tableName : tableList) {
+        String snapshotName =
+            "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_"
+                + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString();
+
+        admin.snapshot(snapshotName, tableName);
+
+        backupInfo.setSnapshotName(tableName, snapshotName);
+      }
+
+      // SNAPSHOT_COPY:
+      // do snapshot copy
+      LOG.debug("snapshot copy for " + backupId);
+      snapshotCopy(backupInfo);
+      // Updates incremental backup table set
+      backupManager.addIncrementalBackupTableSet(backupInfo.getTables());
+
+      // BACKUP_COMPLETE:
+      // 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
+      backupInfo.setState(BackupState.COMPLETE);
+      // The table list in backupInfo is good for both full backup and incremental backup.
+      // For incremental backup, it contains the incremental backup table set.
+      backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps);
+
+      HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
+          backupManager.readLogTimestampMap();
+
+      Long newStartCode =
+          BackupUtils.getMinValue(BackupUtils
+              .getRSLogTimestampMins(newTableSetTimestampMap));
+      backupManager.writeBackupStartCode(newStartCode);
+
+      // backup complete
+      completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf);
+    } catch (Exception e) {
+      failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ",
+        BackupType.FULL, conf);
+      throw new IOException(e);
+    }
+
+  }
+
+}


[05/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
new file mode 100644
index 0000000..0f1453e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
@@ -0,0 +1,344 @@
+/**
+ * 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.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+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.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable.WALItem;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+
+/**
+ * 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.Private
+public class IncrementalBackupManager extends BackupManager {
+  public static final Log LOG = LogFactory.getLog(IncrementalBackupManager.class);
+
+  public IncrementalBackupManager(Connection conn, Configuration conf) throws IOException {
+    super(conn, conf);
+  }
+
+  /**
+   * Obtain the list of logs that need to be copied out for this incremental backup. The list is set
+   * in BackupInfo.
+   * @param conn the Connection
+   * @param backupInfo backup info
+   * @return The new HashMap of RS log timestamps after the log roll for this incremental backup.
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> getIncrBackupLogFileList(Connection conn, BackupInfo backupInfo)
+      throws IOException {
+    List<String> logList;
+    HashMap<String, Long> newTimestamps;
+    HashMap<String, Long> previousTimestampMins;
+
+    String savedStartCode = readBackupStartCode();
+
+    // key: tableName
+    // value: <RegionServer,PreviousTimeStamp>
+    HashMap<TableName, HashMap<String, Long>> previousTimestampMap = readLogTimestampMap();
+
+    previousTimestampMins = BackupUtils.getRSLogTimestampMins(previousTimestampMap);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("StartCode " + savedStartCode + "for backupID " + backupInfo.getBackupId());
+    }
+    // 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 backup system table. "
+              + "In order to create an incremental backup, at least one full backup is needed.");
+    }
+
+    LOG.info("Execute roll log procedure for incremental backup ...");
+    HashMap<String, String> props = new HashMap<String, String>();
+    props.put("backupRoot", backupInfo.getBackupRootDir());
+
+    try (Admin admin = conn.getAdmin();) {
+
+      admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
+        LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props);
+
+    }
+    newTimestamps = readRegionServerLastLogRollResult();
+
+    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
+    List<WALItem> logFromSystemTable =
+        getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps, getBackupInfo()
+            .getBackupRootDir());
+    addLogsFromBackupSystemToContext(logFromSystemTable);
+
+    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
+    backupInfo.setIncrBackupFileList(logList);
+
+    return newTimestamps;
+  }
+
+  private List<String> excludeAlreadyBackedUpWALs(List<String> logList,
+      List<WALItem> logFromSystemTable) {
+
+    List<String> backupedWALList = toWALList(logFromSystemTable);
+    logList.removeAll(backupedWALList);
+    return logList;
+  }
+
+  private List<String> toWALList(List<WALItem> logFromSystemTable) {
+
+    List<String> list = new ArrayList<String>(logFromSystemTable.size());
+    for (WALItem item : logFromSystemTable) {
+      list.add(item.getWalFile());
+    }
+    return list;
+  }
+
+  private void addLogsFromBackupSystemToContext(List<WALItem> logFromSystemTable) {
+    List<String> walFiles = new ArrayList<String>();
+    for (WALItem item : logFromSystemTable) {
+      Path p = new Path(item.getWalFile());
+      String walFileName = p.getName();
+      String backupId = item.getBackupId();
+      String relWALPath = backupId + Path.SEPARATOR + walFileName;
+      walFiles.add(relWALPath);
+    }
+  }
+
+  /**
+   * For each region server: get all log files newer than the last timestamps, but not newer than
+   * the newest timestamps.
+   * @param olderTimestamps timestamp map for each region server of the last backup.
+   * @param newestTimestamps timestamp map for each region server that the backup should lead to.
+   * @return list of log files which needs to be added to this backup
+   * @throws IOException
+   */
+  private List<WALItem> getLogFilesFromBackupSystem(HashMap<String, Long> olderTimestamps,
+      HashMap<String, Long> newestTimestamps, String backupRoot) throws IOException {
+    List<WALItem> logFiles = new ArrayList<WALItem>();
+    Iterator<WALItem> it = getWALFilesFromBackupSystem();
+    while (it.hasNext()) {
+      WALItem item = it.next();
+      String rootDir = item.getBackupRoot();
+      if (!rootDir.equals(backupRoot)) {
+        continue;
+      }
+      String walFileName = item.getWalFile();
+      String server = BackupUtils.parseHostNameFromLogFile(new Path(walFileName));
+      if (server == null) {
+        continue;
+      }
+      Long tss = getTimestamp(walFileName);
+      Long oldTss = olderTimestamps.get(server);
+      Long newTss = newestTimestamps.get(server);
+      if (oldTss == null) {
+        logFiles.add(item);
+        continue;
+      }
+      if (newTss == null) {
+        newTss = Long.MAX_VALUE;
+      }
+      if (tss > oldTss && tss < newTss) {
+        logFiles.add(item);
+      }
+    }
+    return logFiles;
+  }
+
+  private Long getTimestamp(String walFileName) {
+    int index = walFileName.lastIndexOf(BackupUtils.LOGNAME_SEPARATOR);
+    return Long.parseLong(walFileName.substring(index + 1));
+  }
+
+  /**
+   * 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, Long> olderTimestamps,
+      HashMap<String, Long> 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();
+
+    List<String> resultLogFiles = new ArrayList<String>();
+    List<String> newestLogs = new ArrayList<String>();
+
+    /*
+     * The old region servers and timestamps info we kept in backup system table 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 backup system table 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 backup system table at the end of successful backup.
+     */
+
+    FileStatus[] rss;
+    Path p;
+    String host;
+    Long oldTimeStamp;
+    String currentLogFile;
+    long currentLogTS;
+
+    // Get the files in .logs.
+    rss = fs.listStatus(logDir);
+    for (FileStatus rs : rss) {
+      p = rs.getPath();
+      host = BackupUtils.parseHostNameFromLogFile(p);
+      if (host == null) {
+        continue;
+      }
+      FileStatus[] logs;
+      oldTimeStamp = olderTimestamps.get(host);
+      // It is possible that there is no old timestamp in backup system table 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 (AbstractFSWALProvider.isMetaFile(log.getPath())) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Skip hbase:meta log file: " + log.getPath().getName());
+          }
+          continue;
+        }
+        currentLogFile = log.getPath().toString();
+        resultLogFiles.add(currentLogFile);
+        currentLogTS = BackupUtils.getCreationTime(log.getPath());
+        // newestTimestamps is up-to-date with the current list of hosts
+        // so newestTimestamps.get(host) will not be null.
+        if (currentLogTS > 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 (AbstractFSWALProvider.isMetaFile(p)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Skip .meta log file: " + currentLogFile);
+        }
+        continue;
+      }
+      host = BackupUtils.parseHostFromOldLog(p);
+      if (host == null) {
+        continue;
+      }
+      currentLogTS = BackupUtils.getCreationTime(p);
+      oldTimeStamp = olderTimestamps.get(host);
+      /*
+       * It is possible that there is no old timestamp in backup system table 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 (currentLogTS < Long.parseLong(savedStartCode)) {
+          // This log file is really old, its region server was before our last backup.
+          continue;
+        } else {
+          resultLogFiles.add(currentLogFile);
+        }
+      } else if (currentLogTS > oldTimeStamp) {
+        resultLogFiles.add(currentLogFile);
+      }
+
+      // 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.
+      Long newTimestamp = newestTimestamps.get(host);
+      if (newTimestamp != null && currentLogTS > newTimestamp) {
+        newestLogs.add(currentLogFile);
+      }
+    }
+    // remove newest log per host because they are still in use
+    resultLogFiles.removeAll(newestLogs);
+    return resultLogFiles;
+  }
+
+  static class NewestLogFilter implements PathFilter {
+    private Long lastBackupTS = 0L;
+
+    public NewestLogFilter() {
+    }
+
+    protected void setLastBackupTS(Long ts) {
+      this.lastBackupTS = ts;
+    }
+
+    @Override
+    public boolean accept(Path path) {
+      // skip meta table log -- ts.meta file
+      if (AbstractFSWALProvider.isMetaFile(path)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Skip .meta log file: " + path.getName());
+        }
+        return false;
+      }
+      long timestamp;
+      try {
+        timestamp = BackupUtils.getCreationTime(path);
+        return timestamp > lastBackupTS;
+      } catch (Exception e) {
+        LOG.warn("Cannot read timestamp of log file " + path);
+        return false;
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
new file mode 100644
index 0000000..395ed6d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
@@ -0,0 +1,216 @@
+/**
+ * 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.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupCopyJob;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+
+/**
+ * Incremental backup implementation.
+ * See the {@link #execute() execute} method.
+ *
+ */
+@InterfaceAudience.Private
+public class IncrementalTableBackupClient extends TableBackupClient {
+  private static final Log LOG = LogFactory.getLog(IncrementalTableBackupClient.class);
+
+  public IncrementalTableBackupClient(final Connection conn, final String backupId,
+      BackupRequest request) throws IOException {
+    super(conn, backupId, request);
+  }
+
+  private List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    List<String> list = new ArrayList<String>();
+    for (String file : incrBackupFileList) {
+      if (fs.exists(new Path(file))) {
+        list.add(file);
+      } else {
+        LOG.warn("Can't find file: " + file);
+      }
+    }
+    return list;
+  }
+
+  private List<String> getMissingFiles(List<String> incrBackupFileList) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    List<String> list = new ArrayList<String>();
+    for (String file : incrBackupFileList) {
+      if (!fs.exists(new Path(file))) {
+        list.add(file);
+      }
+    }
+    return list;
+
+  }
+
+  /**
+   * Do incremental copy.
+   * @param backupInfo backup info
+   */
+  private void incrementalCopy(BackupInfo backupInfo) throws Exception {
+
+    LOG.info("Incremental copy is starting.");
+    // set overall backup phase: incremental_copy
+    backupInfo.setPhase(BackupPhase.INCREMENTAL_COPY);
+    // get incremental backup file list and prepare parms for DistCp
+    List<String> incrBackupFileList = backupInfo.getIncrBackupFileList();
+    // filter missing files out (they have been copied by previous backups)
+    incrBackupFileList = filterMissingFiles(incrBackupFileList);
+    String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
+    strArr[strArr.length - 1] = backupInfo.getHLogTargetDir();
+
+    BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf);
+    int counter = 0;
+    int MAX_ITERAIONS = 2;
+    while (counter++ < MAX_ITERAIONS) {
+      // We run DistCp maximum 2 times
+      // If it fails on a second time, we throw Exception
+      int res =
+          copyService.copy(backupInfo, backupManager, conf, BackupType.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 "
+            + StringUtils.join(incrBackupFileList, ",") + " to "
+            + backupInfo.getHLogTargetDir());
+      }
+      List<String> missingFiles = getMissingFiles(incrBackupFileList);
+
+      if (missingFiles.isEmpty()) {
+        break;
+      } else {
+        // Repeat DistCp, some files have been moved from WALs to oldWALs during previous run
+        // update backupInfo and strAttr
+        if (counter == MAX_ITERAIONS) {
+          String msg =
+              "DistCp could not finish the following files: " + StringUtils.join(missingFiles, ",");
+          LOG.error(msg);
+          throw new IOException(msg);
+        }
+        List<String> converted = convertFilesFromWALtoOldWAL(missingFiles);
+        incrBackupFileList.removeAll(missingFiles);
+        incrBackupFileList.addAll(converted);
+        backupInfo.setIncrBackupFileList(incrBackupFileList);
+
+        // Run DistCp only for missing files (which have been moved from WALs to oldWALs
+        // during previous run)
+        strArr = converted.toArray(new String[converted.size() + 1]);
+        strArr[strArr.length - 1] = backupInfo.getHLogTargetDir();
+      }
+    }
+
+    LOG.info("Incremental copy from " + StringUtils.join(incrBackupFileList, ",") + " to "
+        + backupInfo.getHLogTargetDir() + " finished.");
+  }
+
+  private List<String> convertFilesFromWALtoOldWAL(List<String> missingFiles) throws IOException {
+    List<String> list = new ArrayList<String>();
+    for (String path : missingFiles) {
+      if (path.indexOf(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME) < 0) {
+        LOG.error("Copy incremental log files failed, file is missing : " + path);
+        throw new IOException("Failed of Hadoop Distributed Copy to "
+            + backupInfo.getHLogTargetDir() + ", file is missing " + path);
+      }
+      list.add(path.replace(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME, Path.SEPARATOR
+          + HConstants.HREGION_OLDLOGDIR_NAME));
+    }
+    return list;
+  }
+
+  @Override
+  public void execute() throws IOException {
+
+    // case PREPARE_INCREMENTAL:
+    beginBackup(backupManager, backupInfo);
+    backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
+    LOG.debug("For incremental backup, current table set is "
+        + backupManager.getIncrementalBackupTableSet());
+    try {
+      newTimestamps =
+          ((IncrementalBackupManager) backupManager).getIncrBackupLogFileList(conn, backupInfo);
+    } catch (Exception e) {
+      // fail the overall backup and return
+      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
+        BackupType.INCREMENTAL, conf);
+    }
+
+    // case INCREMENTAL_COPY:
+    try {
+      // copy out the table and region info files for each table
+      BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
+      incrementalCopy(backupInfo);
+      // Save list of WAL files copied
+      backupManager.recordWALFiles(backupInfo.getIncrBackupFileList());
+    } catch (Exception e) {
+      String msg = "Unexpected exception in incremental-backup: incremental copy " + backupId;
+      // fail the overall backup and return
+      failBackup(conn, backupInfo, backupManager, e, msg, BackupType.INCREMENTAL, conf);
+    }
+    // case INCR_BACKUP_COMPLETE:
+    // 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
+    try {
+      backupInfo.setState(BackupState.COMPLETE);
+      // Set the previousTimestampMap which is before this current log roll to the manifest.
+      HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
+          backupManager.readLogTimestampMap();
+      backupInfo.setIncrTimestampMap(previousTimestampMap);
+
+      // The table list in backupInfo is good for both full backup and incremental backup.
+      // For incremental backup, it contains the incremental backup table set.
+      backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps);
+
+      HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
+          backupManager.readLogTimestampMap();
+
+      Long newStartCode =
+          BackupUtils.getMinValue(BackupUtils
+              .getRSLogTimestampMins(newTableSetTimestampMap));
+      backupManager.writeBackupStartCode(newStartCode);
+      // backup complete
+      completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
+
+    } catch (IOException e) {
+      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
+        BackupType.INCREMENTAL, conf);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
new file mode 100644
index 0000000..f418305
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
@@ -0,0 +1,237 @@
+/**
+ * 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.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.commons.lang.StringUtils;
+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.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.RestoreRequest;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.util.RestoreTool;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+
+/**
+ * Restore table implementation
+ *
+ */
+@InterfaceAudience.Private
+public class RestoreTablesClient {
+  private static final Log LOG = LogFactory.getLog(RestoreTablesClient.class);
+
+  private Configuration conf;
+  private Connection conn;
+  private String backupId;
+  private TableName[] sTableArray;
+  private TableName[] tTableArray;
+  private String targetRootDir;
+  private boolean isOverwrite;
+
+  public RestoreTablesClient(Connection conn, RestoreRequest request) throws IOException {
+    this.targetRootDir = request.getBackupRootDir();
+    this.backupId = request.getBackupId();
+    this.sTableArray = request.getFromTables();
+    this.tTableArray = request.getToTables();
+    if (tTableArray == null || tTableArray.length == 0) {
+      this.tTableArray = sTableArray;
+    }
+    this.isOverwrite = request.isOverwrite();
+    this.conn = conn;
+    this.conf = conn.getConfiguration();
+
+  }
+
+  /**
+   * Validate target tables
+   * @param conn connection
+   * @param mgr table state manager
+   * @param tTableArray: target tables
+   * @param isOverwrite overwrite existing table
+   * @throws IOException exception
+   */
+  private void checkTargetTables(TableName[] tTableArray, boolean isOverwrite) throws IOException {
+    ArrayList<TableName> existTableList = new ArrayList<>();
+    ArrayList<TableName> disabledTableList = new ArrayList<>();
+
+    // check if the tables already exist
+    try (Admin admin = conn.getAdmin();) {
+      for (TableName tableName : tTableArray) {
+        if (admin.tableExists(tableName)) {
+          existTableList.add(tableName);
+          if (admin.isTableDisabled(tableName)) {
+            disabledTableList.add(tableName);
+          }
+        } else {
+          LOG.info("HBase table " + tableName
+              + " does not exist. It will be created during restore process");
+        }
+      }
+    }
+
+    if (existTableList.size() > 0) {
+      if (!isOverwrite) {
+        LOG.error("Existing table ("
+            + existTableList
+            + ") found in the restore target, please add "
+            + "\"-overwrite\" option in the command if you mean"
+            + " to restore to these existing tables");
+        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 handle each backupImage in array
+   * @param svc: master services
+   * @param images: array BackupImage
+   * @param sTable: table to be restored
+   * @param tTable: table to be restored to
+   * @param truncateIfExists: truncate table
+   * @throws IOException exception
+   */
+
+  private void restoreImages(BackupImage[] images, TableName sTable, TableName tTable,
+      boolean truncateIfExists) throws IOException {
+
+    // First image MUST be image of a FULL backup
+    BackupImage image = images[0];
+    String rootDir = image.getRootDir();
+    String backupId = image.getBackupId();
+    Path backupRoot = new Path(rootDir);
+    RestoreTool restoreTool = new RestoreTool(conf, backupRoot, backupId);
+    Path tableBackupPath = HBackupFileSystem.getTableBackupPath(sTable, backupRoot, backupId);
+    String lastIncrBackupId = images.length == 1 ? null : images[images.length - 1].getBackupId();
+    // We need hFS only for full restore (see the code)
+    BackupManifest manifest = HBackupFileSystem.getManifest(sTable, conf, backupRoot, backupId);
+    if (manifest.getType() == BackupType.FULL) {
+      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from full" + " backup image "
+          + tableBackupPath.toString());
+      restoreTool.fullRestoreTable(conn, tableBackupPath, sTable, tTable, truncateIfExists,
+        lastIncrBackupId);
+    } else { // incremental Backup
+      throw new IOException("Unexpected backup type " + image.getType());
+    }
+
+    if (images.length == 1) {
+      // full backup restore done
+      return;
+    }
+
+    List<Path> dirList = new ArrayList<Path>();
+    // add full backup path
+    // full backup path comes first
+    for (int i = 1; i < images.length; i++) {
+      BackupImage im = images[i];
+      String logBackupDir = HBackupFileSystem.getLogBackupDir(im.getRootDir(), im.getBackupId());
+      dirList.add(new Path(logBackupDir));
+    }
+
+    String dirs = StringUtils.join(dirList, ",");
+    LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from log dirs: " + dirs);
+    Path[] paths = new Path[dirList.size()];
+    dirList.toArray(paths);
+    restoreTool.incrementalRestoreTable(conn, tableBackupPath, paths, new TableName[] { sTable },
+      new TableName[] { tTable }, lastIncrBackupId);
+    LOG.info(sTable + " has been successfully restored to " + tTable);
+
+  }
+
+  /**
+   * Restore operation. Stage 2: resolved Backup Image dependency
+   * @param backupManifestMap : tableName, Manifest
+   * @param sTableArray The array of tables to be restored
+   * @param tTableArray The array of mapping tables to restore to
+   * @return set of BackupImages restored
+   * @throws IOException exception
+   */
+  private void restore(HashMap<TableName, BackupManifest> backupManifestMap,
+      TableName[] sTableArray, TableName[] tTableArray, boolean isOverwrite) throws IOException {
+    TreeSet<BackupImage> restoreImageSet = new TreeSet<BackupImage>();
+    boolean truncateIfExists = isOverwrite;
+    try {
+      for (int i = 0; i < sTableArray.length; i++) {
+        TableName table = sTableArray[i];
+        BackupManifest manifest = backupManifestMap.get(table);
+        // Get the image list of this backup for restore in time order from old
+        // to new.
+        List<BackupImage> list = new ArrayList<BackupImage>();
+        list.add(manifest.getBackupImage());
+        TreeSet<BackupImage> set = new TreeSet<BackupImage>(list);
+        List<BackupImage> depList = manifest.getDependentListByTable(table);
+        set.addAll(depList);
+        BackupImage[] arr = new BackupImage[set.size()];
+        set.toArray(arr);
+        restoreImages(arr, table, tTableArray[i], truncateIfExists);
+        restoreImageSet.addAll(list);
+        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));
+          }
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Failed", e);
+      throw new IOException(e);
+    }
+    LOG.debug("restoreStage finished");
+  }
+
+  public void execute() throws IOException {
+
+    // case VALIDATION:
+    // check the target tables
+    checkTargetTables(tTableArray, isOverwrite);
+    // case RESTORE_IMAGES:
+    HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
+    // check and load backup image manifest for the tables
+    Path rootPath = new Path(targetRootDir);
+    HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, conf, rootPath,
+      backupId);
+    restore(backupManifestMap, sTableArray, tTableArray, isOverwrite);
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
new file mode 100644
index 0000000..42a8076
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
@@ -0,0 +1,387 @@
+/**
+ * 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.impl;
+
+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.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * Base class for backup operation. Concrete implementation for
+ * full and incremental backup are delegated to corresponding sub-classes:
+ * {@link FullTableBackupClient} and {@link IncrementalTableBackupClient}
+ *
+ */
+@InterfaceAudience.Private
+public abstract class TableBackupClient {
+  private static final Log LOG = LogFactory.getLog(TableBackupClient.class);
+
+  protected Configuration conf;
+  protected Connection conn;
+  protected String backupId;
+  protected List<TableName> tableList;
+  protected HashMap<String, Long> newTimestamps = null;
+
+  protected BackupManager backupManager;
+  protected BackupInfo backupInfo;
+
+  public TableBackupClient(final Connection conn, final String backupId, BackupRequest request)
+      throws IOException {
+    if (request.getBackupType() == BackupType.FULL) {
+      backupManager = new BackupManager(conn, conn.getConfiguration());
+    } else {
+      backupManager = new IncrementalBackupManager(conn, conn.getConfiguration());
+    }
+    this.backupId = backupId;
+    this.tableList = request.getTableList();
+    this.conn = conn;
+    this.conf = conn.getConfiguration();
+    backupInfo =
+        backupManager.createBackupInfo(backupId, request.getBackupType(), tableList,
+          request.getTargetRootDir(), request.getTotalTasks(), request.getBandwidth());
+    if (tableList == null || tableList.isEmpty()) {
+      this.tableList = new ArrayList<>(backupInfo.getTables());
+    }
+  }
+
+  /**
+   * Begin the overall backup.
+   * @param backupInfo backup info
+   * @throws IOException exception
+   */
+  protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo)
+      throws IOException {
+    backupManager.setBackupInfo(backupInfo);
+    // set the start timestamp of the overall backup
+    long startTs = EnvironmentEdgeManager.currentTime();
+    backupInfo.setStartTs(startTs);
+    // set overall backup status: ongoing
+    backupInfo.setState(BackupState.RUNNING);
+    backupInfo.setPhase(BackupPhase.REQUEST);
+    LOG.info("Backup " + backupInfo.getBackupId() + " started at " + startTs + ".");
+
+    backupManager.updateBackupInfo(backupInfo);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup session " + backupInfo.getBackupId() + " has been started.");
+    }
+  }
+
+  private String getMessage(Exception e) {
+    String msg = e.getMessage();
+    if (msg == null || msg.equals("")) {
+      msg = e.getClass().getName();
+    }
+    return msg;
+  }
+
+  /**
+   * Delete HBase snapshot for backup.
+   * @param backupInfo backup info
+   * @throws Exception exception
+   */
+  private void deleteSnapshot(final Connection conn, BackupInfo backupInfo, Configuration conf)
+      throws IOException {
+    LOG.debug("Trying to delete snapshot for full backup.");
+    for (String snapshotName : backupInfo.getSnapshotNames()) {
+      if (snapshotName == null) {
+        continue;
+      }
+      LOG.debug("Trying to delete snapshot: " + snapshotName);
+
+      try (Admin admin = conn.getAdmin();) {
+        admin.deleteSnapshot(snapshotName);
+      } catch (IOException ioe) {
+        LOG.debug("when deleting snapshot " + snapshotName, ioe);
+      }
+      LOG.debug("Deleting the snapshot " + snapshotName + " for backup " + backupInfo.getBackupId()
+          + " succeeded.");
+    }
+  }
+
+  /**
+   * Clean up directories with prefix "exportSnapshot-", which are generated when exporting
+   * snapshots.
+   * @throws IOException exception
+   */
+  private void cleanupExportSnapshotLog(Configuration conf) 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 the uncompleted data at target directory if the ongoing backup has already entered
+   * the copy phase.
+   */
+  private void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) {
+    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: "
+          + backupInfo.getPhase());
+      if (backupInfo.getPhase().equals(BackupPhase.SNAPSHOTCOPY)
+          || backupInfo.getPhase().equals(BackupPhase.INCREMENTAL_COPY)
+          || backupInfo.getPhase().equals(BackupPhase.STORE_MANIFEST)) {
+        FileSystem outputFs =
+            FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf);
+
+        // now treat one backup as a transaction, clean up data that has been partially copied at
+        // table level
+        for (TableName table : backupInfo.getTables()) {
+          Path targetDirPath =
+              new Path(HBackupFileSystem.getTableBackupDir(backupInfo.getBackupRootDir(),
+                backupInfo.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 " + backupInfo.getBackupId() + " at "
+          + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + ".");
+    }
+  }
+
+  /**
+   * Fail the overall backup.
+   * @param backupInfo backup info
+   * @param e exception
+   * @throws Exception exception
+   */
+  protected void failBackup(Connection conn, BackupInfo backupInfo, BackupManager backupManager,
+      Exception e, String msg, BackupType type, Configuration conf) throws IOException {
+    LOG.error(msg + getMessage(e), e);
+    // If this is a cancel exception, then we've already cleaned.
+
+    // set the failure timestamp of the overall backup
+    backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime());
+
+    // set failure message
+    backupInfo.setFailedMsg(e.getMessage());
+
+    // set overall backup status: failed
+    backupInfo.setState(BackupState.FAILED);
+
+    // compose the backup failed data
+    String backupFailedData =
+        "BackupId=" + backupInfo.getBackupId() + ",startts=" + backupInfo.getStartTs()
+            + ",failedts=" + backupInfo.getCompleteTs() + ",failedphase=" + backupInfo.getPhase()
+            + ",failedmessage=" + backupInfo.getFailedMsg();
+    LOG.error(backupFailedData);
+
+    backupManager.updateBackupInfo(backupInfo);
+
+    // if full backup, then delete HBase snapshots if there already are snapshots taken
+    // and also clean up export snapshot log files if exist
+    if (type == BackupType.FULL) {
+      deleteSnapshot(conn, backupInfo, conf);
+      cleanupExportSnapshotLog(conf);
+    }
+
+    // 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.
+    cleanupTargetDir(backupInfo, conf);
+    LOG.info("Backup " + backupInfo.getBackupId() + " failed.");
+  }
+
+  /**
+   * Add manifest for the current backup. The manifest is stored within the table backup directory.
+   * @param backupInfo The current backup info
+   * @throws IOException exception
+   * @throws BackupException exception
+   */
+  private void addManifest(BackupInfo backupInfo, BackupManager backupManager, BackupType type,
+      Configuration conf) throws IOException, BackupException {
+    // set the overall backup phase : store manifest
+    backupInfo.setPhase(BackupPhase.STORE_MANIFEST);
+
+    BackupManifest manifest;
+
+    // Since we have each table's backup in its own directory structure,
+    // we'll store its manifest with the table directory.
+    for (TableName table : backupInfo.getTables()) {
+      manifest = new BackupManifest(backupInfo, table);
+      ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupInfo, table);
+      for (BackupImage image : ancestors) {
+        manifest.addDependentImage(image);
+      }
+
+      if (type == BackupType.INCREMENTAL) {
+        // We'll store the log timestamps for this table only in its manifest.
+        HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
+            new HashMap<TableName, HashMap<String, Long>>();
+        tableTimestampMap.put(table, backupInfo.getIncrTimestampMap().get(table));
+        manifest.setIncrTimestampMap(tableTimestampMap);
+        ArrayList<BackupImage> ancestorss = backupManager.getAncestors(backupInfo);
+        for (BackupImage image : ancestorss) {
+          manifest.addDependentImage(image);
+        }
+      }
+      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 (type == BackupType.INCREMENTAL) {
+      manifest = new BackupManifest(backupInfo);
+      // set the table region server start and end timestamps for incremental backup
+      manifest.setIncrTimestampMap(backupInfo.getIncrTimestampMap());
+      ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupInfo);
+      for (BackupImage image : ancestors) {
+        manifest.addDependentImage(image);
+      }
+      manifest.store(conf);
+    }
+  }
+
+  /**
+   * Get backup request meta data dir as string.
+   * @param backupInfo backup info
+   * @return meta data dir
+   */
+  private String obtainBackupMetaDataStr(BackupInfo backupInfo) {
+    StringBuffer sb = new StringBuffer();
+    sb.append("type=" + backupInfo.getType() + ",tablelist=");
+    for (TableName table : backupInfo.getTables()) {
+      sb.append(table + ";");
+    }
+    if (sb.lastIndexOf(";") > 0) {
+      sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1);
+    }
+    sb.append(",targetRootDir=" + backupInfo.getBackupRootDir());
+
+    return sb.toString();
+  }
+
+  /**
+   * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying
+   * hlogs.
+   * @throws IOException exception
+   */
+  private void cleanupDistCpLog(BackupInfo backupInfo, Configuration conf) throws IOException {
+    Path rootPath = new Path(backupInfo.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);
+      }
+    }
+  }
+
+  /**
+   * Complete the overall backup.
+   * @param backupInfo backup info
+   * @throws Exception exception
+   */
+  protected void completeBackup(final Connection conn, BackupInfo backupInfo,
+      BackupManager backupManager, BackupType type, Configuration conf) throws IOException {
+    // set the complete timestamp of the overall backup
+    backupInfo.setCompleteTs(EnvironmentEdgeManager.currentTime());
+    // set overall backup status: complete
+    backupInfo.setState(BackupState.COMPLETE);
+    backupInfo.setProgress(100);
+    // add and store the manifest for the backup
+    addManifest(backupInfo, backupManager, type, conf);
+
+    // 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 =
+        obtainBackupMetaDataStr(backupInfo) + ",startts=" + backupInfo.getStartTs()
+            + ",completets=" + backupInfo.getCompleteTs() + ",bytescopied="
+            + backupInfo.getTotalBytesCopied();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup " + backupInfo.getBackupId() + " finished: " + backupCompleteData);
+    }
+    backupManager.updateBackupInfo(backupInfo);
+
+    // when full backup is done:
+    // - delete HBase snapshot
+    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
+    // snapshots
+    if (type == BackupType.FULL) {
+      deleteSnapshot(conn, backupInfo, conf);
+      cleanupExportSnapshotLog(conf);
+    } else if (type == BackupType.INCREMENTAL) {
+      cleanupDistCpLog(backupInfo, conf);
+    }
+    LOG.info("Backup " + backupInfo.getBackupId() + " completed.");
+  }
+
+  /**
+   * Backup request execution
+   * @throws IOException
+   */
+  public abstract void execute() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
new file mode 100644
index 0000000..5641720
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/HFileSplitterJob.java
@@ -0,0 +1,181 @@
+/**
+ * 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.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.HFileInputFormat;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
+import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer;
+import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * A tool to split HFiles into new region boundaries as a MapReduce job. The tool generates HFiles
+ * for later bulk importing.
+ */
+@InterfaceAudience.Private
+public class HFileSplitterJob extends Configured implements Tool {
+  private static final Log LOG = LogFactory.getLog(HFileSplitterJob.class);
+  final static String NAME = "HFileSplitterJob";
+  public final static String BULK_OUTPUT_CONF_KEY = "hfile.bulk.output";
+  public final static String TABLES_KEY = "hfile.input.tables";
+  public final static String TABLE_MAP_KEY = "hfile.input.tablesmap";
+  private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
+
+  public HFileSplitterJob() {
+  }
+
+  protected HFileSplitterJob(final Configuration c) {
+    super(c);
+  }
+
+  /**
+   * A mapper that just writes out cells. This one can be used together with
+   * {@link KeyValueSortReducer}
+   */
+  static class HFileCellMapper extends
+      Mapper<NullWritable, KeyValue, ImmutableBytesWritable, KeyValue> {
+
+    @Override
+    public void map(NullWritable key, KeyValue value, Context context) throws IOException,
+        InterruptedException {
+      // Convert value to KeyValue if subclass
+      if (!value.getClass().equals(KeyValue.class)) {
+        value =
+            new KeyValue(value.getRowArray(), value.getRowOffset(), value.getRowLength(),
+                value.getFamilyArray(), value.getFamilyOffset(), value.getFamilyLength(),
+                value.getQualifierArray(), value.getQualifierOffset(), value.getQualifierLength(),
+                value.getTimestamp(), Type.codeToType(value.getTypeByte()), value.getValueArray(),
+                value.getValueOffset(), value.getValueLength());
+      }
+      context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), value);
+    }
+
+    @Override
+    public void setup(Context context) throws IOException {
+      // do nothing
+    }
+  }
+
+  /**
+   * Sets up the actual job.
+   * @param args The command line parameters.
+   * @return The newly created job.
+   * @throws IOException When setting up the job fails.
+   */
+  public Job createSubmittableJob(String[] args) throws IOException {
+    Configuration conf = getConf();
+    String inputDirs = args[0];
+    String tabName = args[1];
+    conf.setStrings(TABLES_KEY, tabName);
+    Job job =
+        Job.getInstance(conf,
+          conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime()));
+    job.setJarByClass(HFileSplitterJob.class);
+    FileInputFormat.addInputPaths(job, inputDirs);
+    job.setInputFormatClass(HFileInputFormat.class);
+    job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+    String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
+    if (hfileOutPath != null) {
+      LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs);
+      TableName tableName = TableName.valueOf(tabName);
+      job.setMapperClass(HFileCellMapper.class);
+      job.setReducerClass(KeyValueSortReducer.class);
+      Path outputDir = new Path(hfileOutPath);
+      FileOutputFormat.setOutputPath(job, outputDir);
+      job.setMapOutputValueClass(KeyValue.class);
+      try (Connection conn = ConnectionFactory.createConnection(conf);
+          Table table = conn.getTable(tableName);
+          RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
+        HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
+      }
+      LOG.debug("success configuring load incremental job");
+
+      TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+        com.google.common.base.Preconditions.class);
+    } else {
+      throw new IOException("No bulk output directory specified");
+    }
+    return job;
+  }
+
+  /**
+   * Print usage
+   * @param errorMsg Error message. Can be null.
+   */
+  private void usage(final String errorMsg) {
+    if (errorMsg != null && errorMsg.length() > 0) {
+      System.err.println("ERROR: " + errorMsg);
+    }
+    System.err.println("Usage: " + NAME + " [options] <HFile inputdir(s)> <table>");
+    System.err.println("Read all HFile's for <table> and split them to <table> region boundaries.");
+    System.err.println("<table>  table to load.\n");
+    System.err.println("To generate HFiles for a bulk data load, pass the option:");
+    System.err.println("  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
+    System.err.println("Other options:");
+    System.err.println("   -D " + JOB_NAME_CONF_KEY
+        + "=jobName - use the specified mapreduce job name for the HFile splitter");
+    System.err.println("For performance also consider the following options:\n"
+        + "  -Dmapreduce.map.speculative=false\n" + "  -Dmapreduce.reduce.speculative=false");
+  }
+
+  /**
+   * Main entry point.
+   * @param args The command line parameters.
+   * @throws Exception When running the job fails.
+   */
+  public static void main(String[] args) throws Exception {
+    int ret = ToolRunner.run(new HFileSplitterJob(HBaseConfiguration.create()), args);
+    System.exit(ret);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    if (args.length < 2) {
+      usage("Wrong number of arguments: " + args.length);
+      System.exit(-1);
+    }
+    Job job = createSubmittableJob(args);
+    int result = job.waitForCompletion(true) ? 0 : 1;
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java
new file mode 100644
index 0000000..016d1a4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java
@@ -0,0 +1,344 @@
+/**
+ * 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.TableName;
+import org.apache.hadoop.hbase.backup.BackupCopyJob;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.snapshot.ExportSnapshot;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.tools.DistCp;
+import org.apache.hadoop.tools.DistCpConstants;
+import org.apache.hadoop.tools.DistCpOptions;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+
+/**
+ * Map-Reduce implementation of {@link BackupCopyJob}. Basically, there are 2 types of copy
+ * operation: one is copying from snapshot, which bases on extending ExportSnapshot's function, the
+ * other is copying for incremental log files, which bases on extending DistCp's function.
+ */
+@InterfaceAudience.Private
+public class MapReduceBackupCopyJob implements BackupCopyJob {
+  private static final Log LOG = LogFactory.getLog(MapReduceBackupCopyJob.class);
+
+  private Configuration conf;
+
+  // 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 MapReduceBackupCopyJob() {
+  }
+
+  @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(BackupInfo, BackupManager, Configuration, BackupType, String[])}
+   * @param subTaskPercntgInWholeTask The percentage of the copy subtask
+   */
+  public void setSubTaskPercntgInWholeTask(float subTaskPercntgInWholeTask) {
+    this.subTaskPercntgInWholeTask = subTaskPercntgInWholeTask;
+  }
+
+  static class SnapshotCopy extends ExportSnapshot {
+    private BackupInfo backupInfo;
+    private TableName table;
+
+    public SnapshotCopy(BackupInfo backupInfo, TableName table) {
+      super();
+      this.backupInfo = backupInfo;
+      this.table = table;
+    }
+
+    public TableName getTable() {
+      return this.table;
+    }
+
+    public BackupInfo getBackupInfo() {
+      return this.backupInfo;
+    }
+  }
+
+  /**
+   * Update the ongoing backup with new progress.
+   * @param backupInfo backup info
+   * @param newProgress progress
+   * @param bytesCopied bytes copied
+   * @throws NoNodeException exception
+   */
+  static void updateProgress(BackupInfo backupInfo, BackupManager backupManager,
+      int newProgress, long bytesCopied) throws IOException {
+    // compose the new backup progress data, using fake number for now
+    String backupProgressData = newProgress + "%";
+
+    backupInfo.setProgress(newProgress);
+    backupManager.updateBackupInfo(backupInfo);
+    LOG.debug("Backup progress data \"" + backupProgressData
+        + "\" has been updated to backup system table for " + backupInfo.getBackupId());
+  }
+
+  /**
+   * Extends DistCp for progress updating to backup system table
+   * 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 BackupInfo backupInfo;
+    private BackupManager backupManager;
+
+    public BackupDistCp(Configuration conf, DistCpOptions options, BackupInfo backupInfo,
+        BackupManager backupManager) throws Exception {
+      super(conf, options);
+      this.backupInfo = backupInfo;
+      this.backupManager = backupManager;
+    }
+
+    @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;
+
+      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(super.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 +=
+              BackupUtils.getFilesLength(aSrc.getFileSystem(super.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 =
+            MapReduceBackupCopyJob.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);
+            updateProgress(backupInfo, backupManager, progressData.intValue(), bytesCopied);
+            LOG.debug("Backup progress data updated to backup system table: \"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 + " subTask: " + subTaskPercntgInWholeTask
+            + " mapProgress: " + job.mapProgress());
+
+        // accumulate the overall backup progress
+        progressDone = newProgress;
+        bytesCopied += totalSrcLgth;
+
+        updateProgress(backupInfo, backupManager, progressData.intValue(), bytesCopied);
+        LOG.debug("Backup progress data updated to backup system table: \"Progress: "
+            + newProgressStr + " - " + bytesCopied + " bytes copied.\"");
+      } catch (Throwable t) {
+        LOG.error("distcp " + job == null ? "" : job.getJobID() + " encountered error", t);
+        throw t;
+      } 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 + " completed: " + job.isComplete() + " "
+          + job.isSuccessful());
+      Counters ctrs = job.getCounters();
+      LOG.debug(ctrs);
+      if (job.isComplete() && !job.isSuccessful()) {
+        throw new Exception("DistCp job-id: " + jobID + " failed");
+      }
+
+      return job;
+    }
+
+  }
+
+  /**
+   * Do backup copy based on different types.
+   * @param context The backup info
+   * @param conf The hadoop configuration
+   * @param copyType The backup copy type
+   * @param options Options for customized ExportSnapshot or DistCp
+   * @throws Exception exception
+   */
+  @Override
+  public int copy(BackupInfo context, BackupManager backupManager, Configuration conf,
+      BackupType copyType, String[] options) throws IOException {
+    int res = 0;
+
+    try {
+      if (copyType == BackupType.FULL) {
+        SnapshotCopy snapshotCp = new SnapshotCopy(context, context.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 == BackupType.INCREMENTAL) {
+        LOG.debug("Doing COPY_TYPE_DISTCP");
+        setSubTaskPercntgInWholeTask(1f);
+
+        BackupDistCp distcp =
+            new BackupDistCp(new Configuration(conf), null, context, backupManager);
+        // 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));
+        Path dest = new Path(options[options.length - 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);
+    }
+  }
+
+  @Override
+  public void cancel(String jobId) throws IOException {
+    JobID id = JobID.forName(jobId);
+    Cluster cluster = new Cluster(this.getConf());
+    try {
+      Job job = cluster.getJob(id);
+      if (job == null) {
+        LOG.error("No job found for " + id);
+        // should we throw exception
+        return;
+      }
+      if (job.isComplete() || job.isRetired()) {
+        return;
+      }
+
+      job.killJob();
+      LOG.debug("Killed copy job " + id);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
new file mode 100644
index 0000000..ffb61ec
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
@@ -0,0 +1,182 @@
+/**
+ * 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.lang.StringUtils;
+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.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.RestoreJob;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.mapreduce.WALPlayer;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.util.Tool;
+
+/**
+ * MapReduce implementation of {@link RestoreJob}
+ *
+ * For full backup restore, it runs {@link HFileSplitterJob} job and creates
+ * HFiles which are aligned with a region boundaries of a table being
+ * restored, for incremental backup restore it runs {@link WALPlayer} in
+ * bulk load mode (creates HFiles from WAL edits).
+ *
+ * The resulting HFiles then are loaded using HBase bulk load tool
+ * {@link LoadIncrementalHFiles}
+ */
+@InterfaceAudience.Private
+public class MapReduceRestoreJob implements RestoreJob {
+  public static final Log LOG = LogFactory.getLog(MapReduceRestoreJob.class);
+
+  private Tool player;
+  private Configuration conf;
+
+  public MapReduceRestoreJob() {
+  }
+
+  @Override
+  public void run(Path[] dirPaths, TableName[] tableNames, TableName[] newTableNames,
+      boolean fullBackupRestore) throws IOException {
+
+    String bulkOutputConfKey;
+
+    if (fullBackupRestore) {
+      player = new HFileSplitterJob();
+      bulkOutputConfKey = HFileSplitterJob.BULK_OUTPUT_CONF_KEY;
+    } else {
+      player = new WALPlayer();
+      bulkOutputConfKey = WALPlayer.BULK_OUTPUT_CONF_KEY;
+    }
+    // Player reads all files in arbitrary directory structure and creates
+    // a Map task for each file
+    String dirs = StringUtils.join(dirPaths, ",");
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Restore " + (fullBackupRestore ? "full" : "incremental")
+          + " backup from directory " + dirs + " from hbase tables "
+          + StringUtils.join(tableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND) +
+          " to tables "
+          + StringUtils.join(newTableNames, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND));
+    }
+
+    for (int i = 0; i < tableNames.length; i++) {
+
+      LOG.info("Restore " + tableNames[i] + " into " + newTableNames[i]);
+
+      Path bulkOutputPath = getBulkOutputDir(getFileNameCompatibleString(newTableNames[i]));
+      Configuration conf = getConf();
+      conf.set(bulkOutputConfKey, bulkOutputPath.toString());
+      String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
+
+      int result = 0;
+      int loaderResult = 0;
+      try {
+
+        player.setConf(getConf());
+        result = player.run(playerArgs);
+        if (succeeded(result)) {
+          // do bulk load
+          LoadIncrementalHFiles loader = createLoader();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Restoring HFiles from directory " + bulkOutputPath);
+          }
+          String[] args = { bulkOutputPath.toString(), newTableNames[i].getNameAsString() };
+          loaderResult = loader.run(args);
+
+          if (failed(loaderResult)) {
+            throw new IOException("Can not restore from backup directory " + dirs
+                + " (check Hadoop and HBase logs). Bulk loader return code =" + loaderResult);
+          }
+        } else {
+          throw new IOException("Can not restore from backup directory " + dirs
+              + " (check Hadoop/MR and HBase logs). Player return code =" + result);
+        }
+        LOG.debug("Restore Job finished:" + result);
+      } catch (Exception e) {
+        throw new IOException("Can not restore from backup directory " + dirs
+            + " (check Hadoop and HBase logs) ", e);
+      }
+
+    }
+  }
+
+  private String getFileNameCompatibleString(TableName table) {
+    return table.getNamespaceAsString() + "-" + table.getQualifierAsString();
+  }
+
+  private boolean failed(int result) {
+    return result != 0;
+  }
+
+  private boolean succeeded(int result) {
+    return result == 0;
+  }
+
+  private LoadIncrementalHFiles createLoader() throws IOException {
+    // set configuration for restore:
+    // LoadIncrementalHFile needs more time
+    // <name>hbase.rpc.timeout</name> <value>600000</value>
+    // calculates
+    Integer milliSecInHour = 3600000;
+    Configuration conf = new Configuration(getConf());
+    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, milliSecInHour);
+
+    // By default, it is 32 and loader will fail if # of files in any region exceed this
+    // limit. Bad for snapshot restore.
+    conf.setInt(LoadIncrementalHFiles.MAX_FILES_PER_REGION_PER_FAMILY, Integer.MAX_VALUE);
+    conf.set(LoadIncrementalHFiles.IGNORE_UNMATCHED_CF_CONF_KEY, "yes");
+    LoadIncrementalHFiles loader = null;
+    try {
+      loader = new LoadIncrementalHFiles(conf);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    return loader;
+  }
+
+  private Path getBulkOutputDir(String tableName) throws IOException {
+    Configuration conf = getConf();
+    FileSystem fs = FileSystem.get(conf);
+    String tmp =
+        conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY,
+          HConstants.DEFAULT_TEMPORARY_HDFS_DIRECTORY);
+    Path path =
+        new Path(tmp + Path.SEPARATOR + "bulk_output-" + tableName + "-"
+            + EnvironmentEdgeManager.currentTime());
+    fs.deleteOnExit(path);
+    return path;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..b5b887c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
@@ -0,0 +1,142 @@
+/**
+ *
+ * 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.Map;
+
+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.TableNotFoundException;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+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.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class BackupLogCleaner extends BaseLogCleanerDelegate {
+  private static final Log LOG = LogFactory.getLog(BackupLogCleaner.class);
+
+  private boolean stopped = false;
+  private Connection conn;
+
+  public BackupLogCleaner() {
+  }
+
+  @Override
+  public void init(Map<String, Object> params) {
+    if (params != null && params.containsKey(HMaster.MASTER)) {
+      MasterServices master = (MasterServices) params.get(HMaster.MASTER);
+      conn = master.getConnection();
+      if (getConf() == null) {
+        super.setConf(conn.getConfiguration());
+      }
+    }
+    if (conn == null) {
+      try {
+        conn = ConnectionFactory.createConnection(getConf());
+      } catch (IOException ioe) {
+        throw new RuntimeException("Failed to create connection", ioe);
+      }
+    }
+  }
+
+  @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 || !BackupManager.isBackupEnabled(getConf())) {
+      LOG.warn("Backup is not enabled. Check your " + BackupRestoreConstants.BACKUP_ENABLE_KEY
+          + " setting");
+      return files;
+    }
+
+    List<FileStatus> list = new ArrayList<FileStatus>();
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      // If we do not have recorded backup sessions
+      try {
+        if (!table.hasBackupSessions()) {
+          LOG.trace("BackupLogCleaner has no backup sessions");
+          return files;
+        }
+      } catch (TableNotFoundException tnfe) {
+        LOG.warn("backup system table is not available" + tnfe.getMessage());
+        return files;
+      }
+
+      for (FileStatus file : files) {
+        String wal = file.getPath().toString();
+        boolean logInSystemTable = table.isWALFileDeletable(wal);
+        if (LOG.isDebugEnabled()) {
+          if (logInSystemTable) {
+            LOG.debug("Found log file in backup system table, deleting: " + wal);
+            list.add(file);
+          } else {
+            LOG.debug("Didn't find this log in backup system table, keeping: " + wal);
+          }
+        }
+      }
+      return list;
+    } catch (IOException e) {
+      LOG.error("Failed to get backup system table 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(BackupRestoreConstants.BACKUP_ENABLE_KEY,
+      BackupRestoreConstants.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;
+  }
+
+}