You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2017/08/23 16:46:58 UTC

[01/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Repository: hbase
Updated Branches:
  refs/heads/branch-2 af8e6aeab -> 37c659462
  refs/heads/master 6c0e219dd -> 2dda37120


http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
deleted file mode 100644
index 7011ed3..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
+++ /dev/null
@@ -1,336 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
-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.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.impl.BackupAdminImpl;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceHFileSplitterJob;
-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.Table;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Pair;
-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 TestIncrementalBackupMergeWithFailures extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupMergeWithFailures.class);
-
-  static enum FailurePhase {
-    PHASE1, PHASE2, PHASE3, PHASE4
-  }
-  public final static String FAILURE_PHASE_KEY = "failurePhase";
-
-  static class BackupMergeJobWithFailures extends MapReduceBackupMergeJob {
-
-    FailurePhase failurePhase;
-
-    @Override
-    public void setConf(Configuration conf) {
-      super.setConf(conf);
-      String val = conf.get(FAILURE_PHASE_KEY);
-      if (val != null) {
-        failurePhase = FailurePhase.valueOf(val);
-      } else {
-        Assert.fail("Failure phase is not set");
-      }
-    }
-
-
-    /**
-     * This is the exact copy of parent's run() with injections
-     * of different types of failures
-     */
-    @Override
-    public void run(String[] backupIds) throws IOException {
-      String bulkOutputConfKey;
-
-      // TODO : run player on remote cluster
-      player = new MapReduceHFileSplitterJob();
-      bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
-      // Player reads all files in arbitrary directory structure and creates
-      // a Map task for each file
-      String bids = StringUtils.join(backupIds, ",");
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Merge backup images " + bids);
-      }
-
-      List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
-      boolean finishedTables = false;
-      Connection conn = ConnectionFactory.createConnection(getConf());
-      BackupSystemTable table = new BackupSystemTable(conn);
-      FileSystem fs = FileSystem.get(getConf());
-
-      try {
-
-        // Start backup exclusive operation
-        table.startBackupExclusiveOperation();
-        // Start merge operation
-        table.startMergeOperation(backupIds);
-
-        // Select most recent backup id
-        String mergedBackupId = findMostRecentBackupId(backupIds);
-
-        TableName[] tableNames = getTableNamesInBackupImages(backupIds);
-        String backupRoot = null;
-
-        BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
-        backupRoot = bInfo.getBackupRootDir();
-        // PHASE 1
-        checkFailure(FailurePhase.PHASE1);
-
-        for (int i = 0; i < tableNames.length; i++) {
-
-          LOG.info("Merge backup images for " + tableNames[i]);
-
-          // Find input directories for table
-
-          Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
-          String dirs = StringUtils.join(dirPaths, ",");
-          Path bulkOutputPath =
-              BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
-                getConf(), false);
-          // Delete content if exists
-          if (fs.exists(bulkOutputPath)) {
-            if (!fs.delete(bulkOutputPath, true)) {
-              LOG.warn("Can not delete: " + bulkOutputPath);
-            }
-          }
-          Configuration conf = getConf();
-          conf.set(bulkOutputConfKey, bulkOutputPath.toString());
-          String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
-
-          int result = 0;
-          // PHASE 2
-          checkFailure(FailurePhase.PHASE2);
-          player.setConf(getConf());
-          result = player.run(playerArgs);
-          if (succeeded(result)) {
-            // Add to processed table list
-            processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
-          } else {
-            throw new IOException("Can not merge backup images for " + dirs
-                + " (check Hadoop/MR and HBase logs). Player return code =" + result);
-          }
-          LOG.debug("Merge Job finished:" + result);
-        }
-        List<TableName> tableList = toTableNameList(processedTableList);
-        // PHASE 3
-        checkFailure(FailurePhase.PHASE3);
-        table.updateProcessedTablesForMerge(tableList);
-        finishedTables = true;
-
-        // Move data
-        for (Pair<TableName, Path> tn : processedTableList) {
-          moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
-        }
-        // PHASE 4
-        checkFailure(FailurePhase.PHASE4);
-        // Delete old data and update manifest
-        List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
-        deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
-        updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
-        // Finish merge session
-        table.finishMergeOperation();
-      } catch (RuntimeException e) {
-        throw e;
-      } catch (Exception e) {
-        LOG.error(e);
-        if (!finishedTables) {
-          // cleanup bulk directories and finish merge
-          // merge MUST be repeated (no need for repair)
-          cleanupBulkLoadDirs(fs, toPathList(processedTableList));
-          table.finishMergeOperation();
-          table.finishBackupExclusiveOperation();
-          throw new IOException("Backup merge operation failed, you should try it again", e);
-        } else {
-          // backup repair must be run
-          throw new IOException(
-              "Backup merge operation failed, run backup repair tool to restore system's integrity",
-              e);
-        }
-      } finally {
-        table.close();
-        conn.close();
-      }
-
-    }
-
-    private void checkFailure(FailurePhase phase) throws IOException {
-      if ( failurePhase != null && failurePhase == phase) {
-        throw new IOException (phase.toString());
-      }
-    }
-
-  }
-
-
-  @Test
-  public void TestIncBackupMergeRestore() 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);
-    // Set custom Merge Job implementation
-    conf1.setClass(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS,
-      BackupMergeJobWithFailures.class, BackupMergeJob.class);
-
-    Connection conn = ConnectionFactory.createConnection(conf1);
-
-    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 table1
-    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);
-    t1.close();
-    LOG.debug("written " + ADD_ROWS + " rows to " + table1);
-
-    HTable t2 = insertIntoTable(conn, table2, famName, 1, ADD_ROWS);
-
-    Assert.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + ADD_ROWS);
-    t2.close();
-    LOG.debug("written " + ADD_ROWS + " rows to " + table2);
-
-    // #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));
-
-    t1 = insertIntoTable(conn, table1, famName, 2, ADD_ROWS);
-    t1.close();
-
-    t2 = insertIntoTable(conn, table2, famName, 2, ADD_ROWS);
-    t2.close();
-
-    // #3 - incremental backup for multiple tables
-    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
-    String backupIdIncMultiple2 = client.backupTables(request);
-    assertTrue(checkSucceeded(backupIdIncMultiple2));
-
-    // #4 Merge backup images with failures
-
-    for ( FailurePhase phase : FailurePhase.values()) {
-      Configuration conf = conn.getConfiguration();
-
-      conf.set(FAILURE_PHASE_KEY, phase.toString());
-
-      try (BackupAdmin bAdmin = new BackupAdminImpl(conn);)
-      {
-        String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
-        bAdmin.mergeBackups(backups);
-        Assert.fail("Expected IOException");
-      } catch (IOException e) {
-        BackupSystemTable table = new BackupSystemTable(conn);
-        if(phase.ordinal() < FailurePhase.PHASE4.ordinal()) {
-          // No need to repair:
-          // Both Merge and backup exclusive operations are finished
-          assertFalse(table.isMergeInProgress());
-          try {
-            table.finishBackupExclusiveOperation();
-            Assert.fail("IOException is expected");
-          } catch(IOException ee) {
-            // Expected
-          }
-        } else {
-          // Repair is required
-          assertTrue(table.isMergeInProgress());
-          try {
-            table.startBackupExclusiveOperation();
-            Assert.fail("IOException is expected");
-          } catch(IOException ee) {
-            // Expected - clean up before proceeding
-            table.finishMergeOperation();
-            table.finishBackupExclusiveOperation();
-          }
-        }
-        table.close();
-        LOG.debug("Expected :"+ e.getMessage());
-      }
-    }
-
-    // Now merge w/o failures
-    Configuration conf = conn.getConfiguration();
-    conf.unset(FAILURE_PHASE_KEY);
-    conf.unset(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS);
-
-    try (BackupAdmin bAdmin = new BackupAdminImpl(conn);) {
-      String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
-      bAdmin.mergeBackups(backups);
-    }
-
-    // #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));
-
-    Table 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 + 2 * ADD_ROWS);
-
-    hTable.close();
-
-    hTable = conn.getTable(table2_restore);
-    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + 2 * ADD_ROWS);
-    hTable.close();
-
-    admin.close();
-    conn.close();
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
deleted file mode 100644
index 769785f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Collection;
-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.impl.BackupAdminImpl;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-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.mapreduce.TestLoadIncrementalHFiles;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-/**
- * 1. Create table t1
- * 2. Load data to t1
- * 3 Full backup t1
- * 4 Load data to t1
- * 5 bulk load into t1
- * 6 Incremental backup t1
- */
-@Category(LargeTests.class)
-@RunWith(Parameterized.class)
-public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupDeleteTable.class);
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> data() {
-    secure = true;
-    List<Object[]> params = new ArrayList<Object[]>();
-    params.add(new Object[] {Boolean.TRUE});
-    return params;
-  }
-
-  public TestIncrementalBackupWithBulkLoad(Boolean b) {
-  }
-  // implement all test cases in 1 test since incremental backup/restore has dependencies
-  @Test
-  public void TestIncBackupDeleteTable() throws Exception {
-    String testName = "TestIncBackupDeleteTable";
-    // #1 - create full backup for all tables
-    LOG.info("create full backup image for all tables");
-
-    List<TableName> tables = Lists.newArrayList(table1);
-    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();
-
-    int NB_ROWS2 = 20;
-    LOG.debug("bulk loading into " + testName);
-    int actual = TestLoadIncrementalHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
-        qualName, false, null, new byte[][][] {
-      new byte[][]{ Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
-      new byte[][]{ Bytes.toBytes("ddd"), Bytes.toBytes("ooo") },
-    }, true, false, true, NB_ROWS_IN_BATCH*2, NB_ROWS2);
-
-    // #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));
-
-    // #5.1 - check tables for full restore */
-    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
-
-    // #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 = (HTable) conn.getTable(table1_restore);
-    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH * 2+actual);
-    request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
-
-    backupIdFull = client.backupTables(request);
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>> pair
-      = table.readBulkloadRows(tables);
-      assertTrue("map still has " + pair.getSecond().size() + " entries",
-          pair.getSecond().isEmpty());
-    }
-    assertTrue(checkSucceeded(backupIdFull));
-
-    hTable.close();
-    admin.close();
-    conn.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
deleted file mode 100644
index 84a596e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertFalse;
-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.TableName;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
-import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.util.ToolRunner;
-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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-@Category(LargeTests.class)
-@RunWith(Parameterized.class)
-public class TestIncrementalBackupWithFailures extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupWithFailures.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 TestIncrementalBackupWithFailures(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);
-
-    // #3 - incremental backup for multiple tables
-    incrementalBackupWithFailures();
-
-    admin.close();
-    conn.close();
-
-  }
-
-
-  private void incrementalBackupWithFailures() throws Exception {
-    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
-      IncrementalTableBackupClientForTest.class.getName());
-    int maxStage = Stage.values().length -1;
-    // Fail stages between 0 and 4 inclusive
-    for (int stage = 0; stage <= maxStage; stage++) {
-      LOG.info("Running stage " + stage);
-      runBackupAndFailAtStage(stage);
-    }
-  }
-
-  private void runBackupAndFailAtStage(int stage) throws Exception {
-
-    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
-    try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
-      int before = table.getBackupHistory().size();
-      String[] args =
-          new String[] { "create", "incremental", BACKUP_ROOT_DIR, "-t",
-              table1.getNameAsString() + "," + table2.getNameAsString() };
-      // Run backup
-      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
-      assertFalse(ret == 0);
-      List<BackupInfo> backups = table.getBackupHistory();
-      int after = table.getBackupHistory().size();
-
-      assertTrue(after ==  before +1);
-      for (BackupInfo data : backups) {
-        if(data.getType() == BackupType.FULL) {
-          assertTrue(data.getState() == BackupState.COMPLETE);
-        } else {
-          assertTrue(data.getState() == BackupState.FAILED);
-        }
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 36a9ee2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-@Category(LargeTests.class)
-public class TestRemoteBackup extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestRemoteBackup.class);
-
-  @Override
-  public void setUp () throws Exception {
-    useSecondCluster = true;
-    super.setUp();
-  }
-
-  /**
-   * 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/2dda3712/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
deleted file mode 100644
index 0386c27..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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);
-
-  @Override
-  public void setUp () throws Exception {
-    useSecondCluster = true;
-    super.setUp();
-  }
-
-
-  /**
-   * 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/2dda3712/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
deleted file mode 100644
index 556521f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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.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.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-@Category(LargeTests.class)
-public class TestRepairAfterFailedDelete extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestRepairAfterFailedDelete.class);
-
-  @Test
-  public void testRepairBackupDelete() throws Exception {
-    LOG.info("test repair 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));
-
-    // Snapshot backup system table before delete
-    String snapshotName = "snapshot-backup";
-    Connection conn = TEST_UTIL.getConnection();
-    Admin admin = conn.getAdmin();
-    admin.snapshot(snapshotName, BackupSystemTable.getTableName(conf1));
-
-    int deleted = getBackupAdmin().deleteBackups(backupIds);
-
-    assertTrue(!fs.exists(path));
-    assertTrue(fs.exists(new Path(info.getBackupRootDir())));
-    assertTrue(1 == deleted);
-
-    // Emulate delete failure
-    // Restore backup system table
-    admin.disableTable(BackupSystemTable.getTableName(conf1));
-    admin.restoreSnapshot(snapshotName);
-    admin.enableTable(BackupSystemTable.getTableName(conf1));
-    // Start backup session
-    table.startBackupExclusiveOperation();
-    // Start delete operation
-    table.startDeleteOperation(backupIds);
-
-    // Now run repair command to repair "failed" delete operation
-    String[] args = new String[] {"repair"};
-    // Run restore
-    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
-    assertTrue(ret == 0);
-    // Verify that history length == 0
-    assertTrue (table.getBackupHistory().size() == 0);
-    table.close();
-    admin.close();
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index c61b018..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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/2dda3712/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
deleted file mode 100644
index 6443421..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 backup system table snapshot
-   * @throws Exception
-   */
- // @Test
-  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/2dda3712/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
deleted file mode 100644
index 5f72f45..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
-import org.apache.hadoop.hbase.shaded.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/2dda3712/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6f7079e..31ef241 100755
--- a/pom.xml
+++ b/pom.xml
@@ -89,6 +89,7 @@
     <module>hbase-metrics-api</module>
     <module>hbase-metrics</module>
     <module>hbase-spark-it</module>
+    <module>hbase-backup</module>
   </modules>
   <!--Add apache snapshots in case we want to use unreleased versions of plugins:
       e.g. surefire 2.18-SNAPSHOT-->


[22/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
deleted file mode 100644
index 49e8c75..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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 MapReduceHFileSplitterJob extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(MapReduceHFileSplitterJob.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 MapReduceHFileSplitterJob() {
-  }
-
-  protected MapReduceHFileSplitterJob(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);
-    conf.set(FileInputFormat.INPUT_DIR, inputDirs);
-    Job job =
-        Job.getInstance(conf,
-          conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime()));
-    job.setJarByClass(MapReduceHFileSplitterJob.class);
-    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.getDescriptor(), regionLocator);
-      }
-      LOG.debug("success configuring load incremental job");
-
-      TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
-        org.apache.hadoop.hbase.shaded.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 MapReduceHFileSplitterJob(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/37c65946/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
deleted file mode 100644
index 1209e7c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.mapreduce;
-
-import static org.apache.hadoop.hbase.backup.util.BackupUtils.failed;
-import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
-
-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.Path;
-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.backup.util.BackupUtils;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.util.Tool;
-
-
-/**
- * MapReduce implementation of {@link RestoreJob}
- *
- * For backup restore, it runs {@link MapReduceHFileSplitterJob} job and creates
- * HFiles which are aligned with a region boundaries of a table being
- * restored.
- *
- * 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;
-
-    player = new MapReduceHFileSplitterJob();
-    bulkOutputConfKey = MapReduceHFileSplitterJob.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 =
-          BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(newTableNames[i]),
-            getConf());
-      Configuration conf = getConf();
-      conf.set(bulkOutputConfKey, bulkOutputPath.toString());
-      String[] playerArgs =
-          {
-              dirs,
-              fullBackupRestore ? newTableNames[i].getNameAsString() : 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 = BackupUtils.createLoader(getConf());
-          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) {
-        LOG.error(e);
-        throw new IOException("Can not restore from backup directory " + dirs
-            + " (check Hadoop and HBase logs) ", e);
-      }
-    }
-  }
-
-  @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/37c65946/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
deleted file mode 100644
index b5b887c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 47e428c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/37c65946/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
deleted file mode 100644
index 8fc644c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/37c65946/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
deleted file mode 100644
index 65a1fa3..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/37c65946/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
deleted file mode 100644
index 9d5a858..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/37c65946/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
deleted file mode 100644
index 0da6fc4..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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/37c65946/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
deleted file mode 100644
index ce77645..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
+++ /dev/null
@@ -1,747 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-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 = ".";
-  public static final int MILLISEC_IN_HOUR = 3600000;
-
-  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;
-  }
-
-  public static Path getBulkOutputDir(String tableName, Configuration conf, boolean deleteOnExit)
-      throws IOException {
-    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());
-    if (deleteOnExit) {
-      fs.deleteOnExit(path);
-    }
-    return path;
-  }
-
-  public static Path getBulkOutputDir(String tableName, Configuration conf) throws IOException {
-    return getBulkOutputDir(tableName, conf, true);
-  }
-
-  public static String getFileNameCompatibleString(TableName table) {
-    return table.getNamespaceAsString() + "-" + table.getQualifierAsString();
-  }
-
-  public static boolean failed(int result) {
-    return result != 0;
-  }
-
-  public static boolean succeeded(int result) {
-    return result == 0;
-  }
-
-  public static LoadIncrementalHFiles createLoader(Configuration config) throws IOException {
-    // set configuration for restore:
-    // LoadIncrementalHFile needs more time
-    // <name>hbase.rpc.timeout</name> <value>600000</value>
-    // calculates
-    Configuration conf = new Configuration(config);
-    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, MILLISEC_IN_HOUR);
-
-    // 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;
-  }
-}


[31/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
new file mode 100644
index 0000000..49e8c75
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.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 MapReduceHFileSplitterJob extends Configured implements Tool {
+  private static final Log LOG = LogFactory.getLog(MapReduceHFileSplitterJob.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 MapReduceHFileSplitterJob() {
+  }
+
+  protected MapReduceHFileSplitterJob(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);
+    conf.set(FileInputFormat.INPUT_DIR, inputDirs);
+    Job job =
+        Job.getInstance(conf,
+          conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime()));
+    job.setJarByClass(MapReduceHFileSplitterJob.class);
+    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.getDescriptor(), regionLocator);
+      }
+      LOG.debug("success configuring load incremental job");
+
+      TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+        org.apache.hadoop.hbase.shaded.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 MapReduceHFileSplitterJob(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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
new file mode 100644
index 0000000..1209e7c
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
@@ -0,0 +1,136 @@
+/**
+ * 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 static org.apache.hadoop.hbase.backup.util.BackupUtils.failed;
+import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
+
+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.Path;
+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.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.util.Tool;
+
+
+/**
+ * MapReduce implementation of {@link RestoreJob}
+ *
+ * For backup restore, it runs {@link MapReduceHFileSplitterJob} job and creates
+ * HFiles which are aligned with a region boundaries of a table being
+ * restored.
+ *
+ * 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;
+
+    player = new MapReduceHFileSplitterJob();
+    bulkOutputConfKey = MapReduceHFileSplitterJob.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 =
+          BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(newTableNames[i]),
+            getConf());
+      Configuration conf = getConf();
+      conf.set(bulkOutputConfKey, bulkOutputPath.toString());
+      String[] playerArgs =
+          {
+              dirs,
+              fullBackupRestore ? newTableNames[i].getNameAsString() : 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 = BackupUtils.createLoader(getConf());
+          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) {
+        LOG.error(e);
+        throw new IOException("Can not restore from backup directory " + dirs
+            + " (check Hadoop and HBase logs) ", e);
+      }
+    }
+  }
+
+  @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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
new file mode 100644
index 0000000..b5b887c
--- /dev/null
+++ b/hbase-backup/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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
new file mode 100644
index 0000000..47e428c
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
new file mode 100644
index 0000000..8fc644c
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
new file mode 100644
index 0000000..65a1fa3
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
new file mode 100644
index 0000000..9d5a858
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
new file mode 100644
index 0000000..0da6fc4
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
new file mode 100644
index 0000000..ce77645
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
@@ -0,0 +1,747 @@
+/**
+ * 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.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+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 = ".";
+  public static final int MILLISEC_IN_HOUR = 3600000;
+
+  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;
+  }
+
+  public static Path getBulkOutputDir(String tableName, Configuration conf, boolean deleteOnExit)
+      throws IOException {
+    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());
+    if (deleteOnExit) {
+      fs.deleteOnExit(path);
+    }
+    return path;
+  }
+
+  public static Path getBulkOutputDir(String tableName, Configuration conf) throws IOException {
+    return getBulkOutputDir(tableName, conf, true);
+  }
+
+  public static String getFileNameCompatibleString(TableName table) {
+    return table.getNamespaceAsString() + "-" + table.getQualifierAsString();
+  }
+
+  public static boolean failed(int result) {
+    return result != 0;
+  }
+
+  public static boolean succeeded(int result) {
+    return result == 0;
+  }
+
+  public static LoadIncrementalHFiles createLoader(Configuration config) throws IOException {
+    // set configuration for restore:
+    // LoadIncrementalHFile needs more time
+    // <name>hbase.rpc.timeout</name> <value>600000</value>
+    // calculates
+    Configuration conf = new Configuration(config);
+    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, MILLISEC_IN_HOUR);
+
+    // 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;
+  }
+}


[19/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
deleted file mode 100644
index 7011ed3..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
+++ /dev/null
@@ -1,336 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
-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.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.impl.BackupAdminImpl;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceHFileSplitterJob;
-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.Table;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Pair;
-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 TestIncrementalBackupMergeWithFailures extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupMergeWithFailures.class);
-
-  static enum FailurePhase {
-    PHASE1, PHASE2, PHASE3, PHASE4
-  }
-  public final static String FAILURE_PHASE_KEY = "failurePhase";
-
-  static class BackupMergeJobWithFailures extends MapReduceBackupMergeJob {
-
-    FailurePhase failurePhase;
-
-    @Override
-    public void setConf(Configuration conf) {
-      super.setConf(conf);
-      String val = conf.get(FAILURE_PHASE_KEY);
-      if (val != null) {
-        failurePhase = FailurePhase.valueOf(val);
-      } else {
-        Assert.fail("Failure phase is not set");
-      }
-    }
-
-
-    /**
-     * This is the exact copy of parent's run() with injections
-     * of different types of failures
-     */
-    @Override
-    public void run(String[] backupIds) throws IOException {
-      String bulkOutputConfKey;
-
-      // TODO : run player on remote cluster
-      player = new MapReduceHFileSplitterJob();
-      bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
-      // Player reads all files in arbitrary directory structure and creates
-      // a Map task for each file
-      String bids = StringUtils.join(backupIds, ",");
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Merge backup images " + bids);
-      }
-
-      List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
-      boolean finishedTables = false;
-      Connection conn = ConnectionFactory.createConnection(getConf());
-      BackupSystemTable table = new BackupSystemTable(conn);
-      FileSystem fs = FileSystem.get(getConf());
-
-      try {
-
-        // Start backup exclusive operation
-        table.startBackupExclusiveOperation();
-        // Start merge operation
-        table.startMergeOperation(backupIds);
-
-        // Select most recent backup id
-        String mergedBackupId = findMostRecentBackupId(backupIds);
-
-        TableName[] tableNames = getTableNamesInBackupImages(backupIds);
-        String backupRoot = null;
-
-        BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
-        backupRoot = bInfo.getBackupRootDir();
-        // PHASE 1
-        checkFailure(FailurePhase.PHASE1);
-
-        for (int i = 0; i < tableNames.length; i++) {
-
-          LOG.info("Merge backup images for " + tableNames[i]);
-
-          // Find input directories for table
-
-          Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
-          String dirs = StringUtils.join(dirPaths, ",");
-          Path bulkOutputPath =
-              BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
-                getConf(), false);
-          // Delete content if exists
-          if (fs.exists(bulkOutputPath)) {
-            if (!fs.delete(bulkOutputPath, true)) {
-              LOG.warn("Can not delete: " + bulkOutputPath);
-            }
-          }
-          Configuration conf = getConf();
-          conf.set(bulkOutputConfKey, bulkOutputPath.toString());
-          String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
-
-          int result = 0;
-          // PHASE 2
-          checkFailure(FailurePhase.PHASE2);
-          player.setConf(getConf());
-          result = player.run(playerArgs);
-          if (succeeded(result)) {
-            // Add to processed table list
-            processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
-          } else {
-            throw new IOException("Can not merge backup images for " + dirs
-                + " (check Hadoop/MR and HBase logs). Player return code =" + result);
-          }
-          LOG.debug("Merge Job finished:" + result);
-        }
-        List<TableName> tableList = toTableNameList(processedTableList);
-        // PHASE 3
-        checkFailure(FailurePhase.PHASE3);
-        table.updateProcessedTablesForMerge(tableList);
-        finishedTables = true;
-
-        // Move data
-        for (Pair<TableName, Path> tn : processedTableList) {
-          moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
-        }
-        // PHASE 4
-        checkFailure(FailurePhase.PHASE4);
-        // Delete old data and update manifest
-        List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
-        deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
-        updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
-        // Finish merge session
-        table.finishMergeOperation();
-      } catch (RuntimeException e) {
-        throw e;
-      } catch (Exception e) {
-        LOG.error(e);
-        if (!finishedTables) {
-          // cleanup bulk directories and finish merge
-          // merge MUST be repeated (no need for repair)
-          cleanupBulkLoadDirs(fs, toPathList(processedTableList));
-          table.finishMergeOperation();
-          table.finishBackupExclusiveOperation();
-          throw new IOException("Backup merge operation failed, you should try it again", e);
-        } else {
-          // backup repair must be run
-          throw new IOException(
-              "Backup merge operation failed, run backup repair tool to restore system's integrity",
-              e);
-        }
-      } finally {
-        table.close();
-        conn.close();
-      }
-
-    }
-
-    private void checkFailure(FailurePhase phase) throws IOException {
-      if ( failurePhase != null && failurePhase == phase) {
-        throw new IOException (phase.toString());
-      }
-    }
-
-  }
-
-
-  @Test
-  public void TestIncBackupMergeRestore() 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);
-    // Set custom Merge Job implementation
-    conf1.setClass(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS,
-      BackupMergeJobWithFailures.class, BackupMergeJob.class);
-
-    Connection conn = ConnectionFactory.createConnection(conf1);
-
-    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 table1
-    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);
-    t1.close();
-    LOG.debug("written " + ADD_ROWS + " rows to " + table1);
-
-    HTable t2 = insertIntoTable(conn, table2, famName, 1, ADD_ROWS);
-
-    Assert.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + ADD_ROWS);
-    t2.close();
-    LOG.debug("written " + ADD_ROWS + " rows to " + table2);
-
-    // #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));
-
-    t1 = insertIntoTable(conn, table1, famName, 2, ADD_ROWS);
-    t1.close();
-
-    t2 = insertIntoTable(conn, table2, famName, 2, ADD_ROWS);
-    t2.close();
-
-    // #3 - incremental backup for multiple tables
-    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
-    String backupIdIncMultiple2 = client.backupTables(request);
-    assertTrue(checkSucceeded(backupIdIncMultiple2));
-
-    // #4 Merge backup images with failures
-
-    for ( FailurePhase phase : FailurePhase.values()) {
-      Configuration conf = conn.getConfiguration();
-
-      conf.set(FAILURE_PHASE_KEY, phase.toString());
-
-      try (BackupAdmin bAdmin = new BackupAdminImpl(conn);)
-      {
-        String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
-        bAdmin.mergeBackups(backups);
-        Assert.fail("Expected IOException");
-      } catch (IOException e) {
-        BackupSystemTable table = new BackupSystemTable(conn);
-        if(phase.ordinal() < FailurePhase.PHASE4.ordinal()) {
-          // No need to repair:
-          // Both Merge and backup exclusive operations are finished
-          assertFalse(table.isMergeInProgress());
-          try {
-            table.finishBackupExclusiveOperation();
-            Assert.fail("IOException is expected");
-          } catch(IOException ee) {
-            // Expected
-          }
-        } else {
-          // Repair is required
-          assertTrue(table.isMergeInProgress());
-          try {
-            table.startBackupExclusiveOperation();
-            Assert.fail("IOException is expected");
-          } catch(IOException ee) {
-            // Expected - clean up before proceeding
-            table.finishMergeOperation();
-            table.finishBackupExclusiveOperation();
-          }
-        }
-        table.close();
-        LOG.debug("Expected :"+ e.getMessage());
-      }
-    }
-
-    // Now merge w/o failures
-    Configuration conf = conn.getConfiguration();
-    conf.unset(FAILURE_PHASE_KEY);
-    conf.unset(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS);
-
-    try (BackupAdmin bAdmin = new BackupAdminImpl(conn);) {
-      String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
-      bAdmin.mergeBackups(backups);
-    }
-
-    // #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));
-
-    Table 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 + 2 * ADD_ROWS);
-
-    hTable.close();
-
-    hTable = conn.getTable(table2_restore);
-    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + 2 * ADD_ROWS);
-    hTable.close();
-
-    admin.close();
-    conn.close();
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
deleted file mode 100644
index 769785f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.Collection;
-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.impl.BackupAdminImpl;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-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.mapreduce.TestLoadIncrementalHFiles;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-/**
- * 1. Create table t1
- * 2. Load data to t1
- * 3 Full backup t1
- * 4 Load data to t1
- * 5 bulk load into t1
- * 6 Incremental backup t1
- */
-@Category(LargeTests.class)
-@RunWith(Parameterized.class)
-public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupDeleteTable.class);
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> data() {
-    secure = true;
-    List<Object[]> params = new ArrayList<Object[]>();
-    params.add(new Object[] {Boolean.TRUE});
-    return params;
-  }
-
-  public TestIncrementalBackupWithBulkLoad(Boolean b) {
-  }
-  // implement all test cases in 1 test since incremental backup/restore has dependencies
-  @Test
-  public void TestIncBackupDeleteTable() throws Exception {
-    String testName = "TestIncBackupDeleteTable";
-    // #1 - create full backup for all tables
-    LOG.info("create full backup image for all tables");
-
-    List<TableName> tables = Lists.newArrayList(table1);
-    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();
-
-    int NB_ROWS2 = 20;
-    LOG.debug("bulk loading into " + testName);
-    int actual = TestLoadIncrementalHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
-        qualName, false, null, new byte[][][] {
-      new byte[][]{ Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
-      new byte[][]{ Bytes.toBytes("ddd"), Bytes.toBytes("ooo") },
-    }, true, false, true, NB_ROWS_IN_BATCH*2, NB_ROWS2);
-
-    // #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));
-
-    // #5.1 - check tables for full restore */
-    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
-
-    // #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 = (HTable) conn.getTable(table1_restore);
-    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH * 2+actual);
-    request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
-
-    backupIdFull = client.backupTables(request);
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>> pair
-      = table.readBulkloadRows(tables);
-      assertTrue("map still has " + pair.getSecond().size() + " entries",
-          pair.getSecond().isEmpty());
-    }
-    assertTrue(checkSucceeded(backupIdFull));
-
-    hTable.close();
-    admin.close();
-    conn.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
deleted file mode 100644
index 84a596e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertFalse;
-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.TableName;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
-import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.util.ToolRunner;
-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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-@Category(LargeTests.class)
-@RunWith(Parameterized.class)
-public class TestIncrementalBackupWithFailures extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupWithFailures.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 TestIncrementalBackupWithFailures(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);
-
-    // #3 - incremental backup for multiple tables
-    incrementalBackupWithFailures();
-
-    admin.close();
-    conn.close();
-
-  }
-
-
-  private void incrementalBackupWithFailures() throws Exception {
-    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
-      IncrementalTableBackupClientForTest.class.getName());
-    int maxStage = Stage.values().length -1;
-    // Fail stages between 0 and 4 inclusive
-    for (int stage = 0; stage <= maxStage; stage++) {
-      LOG.info("Running stage " + stage);
-      runBackupAndFailAtStage(stage);
-    }
-  }
-
-  private void runBackupAndFailAtStage(int stage) throws Exception {
-
-    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
-    try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
-      int before = table.getBackupHistory().size();
-      String[] args =
-          new String[] { "create", "incremental", BACKUP_ROOT_DIR, "-t",
-              table1.getNameAsString() + "," + table2.getNameAsString() };
-      // Run backup
-      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
-      assertFalse(ret == 0);
-      List<BackupInfo> backups = table.getBackupHistory();
-      int after = table.getBackupHistory().size();
-
-      assertTrue(after ==  before +1);
-      for (BackupInfo data : backups) {
-        if(data.getType() == BackupType.FULL) {
-          assertTrue(data.getState() == BackupState.COMPLETE);
-        } else {
-          assertTrue(data.getState() == BackupState.FAILED);
-        }
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 36a9ee2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-@Category(LargeTests.class)
-public class TestRemoteBackup extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestRemoteBackup.class);
-
-  @Override
-  public void setUp () throws Exception {
-    useSecondCluster = true;
-    super.setUp();
-  }
-
-  /**
-   * 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/37c65946/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
deleted file mode 100644
index 0386c27..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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);
-
-  @Override
-  public void setUp () throws Exception {
-    useSecondCluster = true;
-    super.setUp();
-  }
-
-
-  /**
-   * 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/37c65946/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
deleted file mode 100644
index 556521f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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.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.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-@Category(LargeTests.class)
-public class TestRepairAfterFailedDelete extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestRepairAfterFailedDelete.class);
-
-  @Test
-  public void testRepairBackupDelete() throws Exception {
-    LOG.info("test repair 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));
-
-    // Snapshot backup system table before delete
-    String snapshotName = "snapshot-backup";
-    Connection conn = TEST_UTIL.getConnection();
-    Admin admin = conn.getAdmin();
-    admin.snapshot(snapshotName, BackupSystemTable.getTableName(conf1));
-
-    int deleted = getBackupAdmin().deleteBackups(backupIds);
-
-    assertTrue(!fs.exists(path));
-    assertTrue(fs.exists(new Path(info.getBackupRootDir())));
-    assertTrue(1 == deleted);
-
-    // Emulate delete failure
-    // Restore backup system table
-    admin.disableTable(BackupSystemTable.getTableName(conf1));
-    admin.restoreSnapshot(snapshotName);
-    admin.enableTable(BackupSystemTable.getTableName(conf1));
-    // Start backup session
-    table.startBackupExclusiveOperation();
-    // Start delete operation
-    table.startDeleteOperation(backupIds);
-
-    // Now run repair command to repair "failed" delete operation
-    String[] args = new String[] {"repair"};
-    // Run restore
-    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
-    assertTrue(ret == 0);
-    // Verify that history length == 0
-    assertTrue (table.getBackupHistory().size() == 0);
-    table.close();
-    admin.close();
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index c61b018..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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/37c65946/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
deleted file mode 100644
index 6443421..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 backup system table snapshot
-   * @throws Exception
-   */
- // @Test
-  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/37c65946/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
deleted file mode 100644
index 5f72f45..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
-import org.apache.hadoop.hbase.shaded.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/37c65946/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 484ba19..0ee9ba0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -87,6 +87,7 @@
     <module>hbase-metrics-api</module>
     <module>hbase-metrics</module>
     <module>hbase-spark-it</module>
+    <module>hbase-backup</module>
   </modules>
   <!--Add apache snapshots in case we want to use unreleased versions of plugins:
       e.g. surefire 2.18-SNAPSHOT-->


[35/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
new file mode 100644
index 0000000..1c43e88
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
@@ -0,0 +1,146 @@
+/**
+ *
+ * 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;
+  }
+
+  public static String getTableBackupDataDir(String backupRootDir, String backupId,
+      TableName tableName) {
+    return getTableBackupDir(backupRootDir, backupId, tableName) + Path.SEPARATOR + "data";
+  }
+
+  public static Path getBackupPath(String backupRootDir, String backupId) {
+    return new Path(backupRootDir + Path.SEPARATOR + backupId);
+  }
+
+  /**
+   * Given the backup root dir, backup id and the table name, return the backup image location,
+   * which is also where the backup manifest file is. return value look like:
+   * "hdfs://backup.hbase.org:9000/user/biadmin/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));
+  }
+
+  // TODO we do not keep WAL files anymore
+  // Move manifest file to other place
+  private static Path getManifestPath(Configuration conf, Path backupRootPath, String backupId)
+      throws IOException {
+    Path manifestPath = null;
+
+    FileSystem fs = backupRootPath.getFileSystem(conf);
+    manifestPath =
+        new Path(getBackupPath(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(Configuration conf, Path backupRootPath, String backupId) throws IOException {
+    BackupManifest manifest =
+        new BackupManifest(conf, getManifestPath(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(conf, backupRootPath, backupId);
+      backupManifestMap.put(tableName, manifest);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
new file mode 100644
index 0000000..1becb75
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
new file mode 100644
index 0000000..82a1b56
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
new file mode 100644
index 0000000..86fb963
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
new file mode 100644
index 0000000..de3ad5a
--- /dev/null
+++ b/hbase-backup/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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
new file mode 100644
index 0000000..99fb06c
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
@@ -0,0 +1,743 @@
+/**
+ * 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.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.BackupClientFactory;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupMergeJob;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
+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.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+@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 {
+  }
+
+  @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 {
+
+    int totalDeleted = 0;
+    Map<String, HashSet<TableName>> allTablesMap = new HashMap<String, HashSet<TableName>>();
+
+    boolean deleteSessionStarted = false;
+    boolean snapshotDone = false;
+    try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) {
+
+      // Step 1: Make sure there is no active session
+      // is running by using startBackupSession API
+      // If there is an active session in progress, exception will be thrown
+      try {
+        sysTable.startBackupExclusiveOperation();
+        deleteSessionStarted = true;
+      } catch (IOException e) {
+        LOG.warn("You can not run delete command while active backup session is in progress. \n"
+            + "If there is no active backup session running, run backup repair utility to restore \n"
+            + "backup system integrity.");
+        return -1;
+      }
+
+      // Step 2: Make sure there is no failed session
+      List<BackupInfo> list = sysTable.getBackupInfos(BackupState.RUNNING);
+      if (list.size() != 0) {
+        // ailed sessions found
+        LOG.warn("Failed backup session found. Run backup repair tool first.");
+        return -1;
+      }
+
+      // Step 3: Record delete session
+      sysTable.startDeleteOperation(backupIds);
+      // Step 4: Snapshot backup system table
+      if (!BackupSystemTable.snapshotExists(conn)) {
+        BackupSystemTable.snapshot(conn);
+      } else {
+        LOG.warn("Backup system table snapshot exists");
+      }
+      snapshotDone = true;
+      try {
+        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);
+        // Finish
+        sysTable.finishDeleteOperation();
+        // delete snapshot
+        BackupSystemTable.deleteSnapshot(conn);
+      } catch (IOException e) {
+        // Fail delete operation
+        // Step 1
+        if (snapshotDone) {
+          if (BackupSystemTable.snapshotExists(conn)) {
+            BackupSystemTable.restoreFromSnapshot(conn);
+            // delete snapshot
+            BackupSystemTable.deleteSnapshot(conn);
+            // We still have record with unfinished delete operation
+            LOG.error("Delete operation failed, please run backup repair utility to restore "
+                + "backup system integrity", e);
+            throw e;
+          } else {
+            LOG.warn("Delete operation succeeded, there were some errors: ", e);
+          }
+        }
+
+      } finally {
+        if (deleteSessionStarted) {
+          sysTable.finishBackupExclusiveOperation();
+        }
+      }
+    }
+    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() + " ...");
+      // Step 1: clean up data for backup session (idempotent)
+      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 = getAffectedBackupSessions(backupInfo, tn, sysTable);
+        for (BackupInfo info : affectedBackups) {
+          if (info.equals(backupInfo)) {
+            continue;
+          }
+          removeTableFromBackupImage(info, tn, sysTable);
+        }
+      }
+      Map<byte[], String> map = sysTable.readBulkLoadedFiles(backupId);
+      FileSystem fs = FileSystem.get(conn.getConfiguration());
+      boolean success = true;
+      int numDeleted = 0;
+      for (String f : map.values()) {
+        Path p = new Path(f);
+        try {
+          LOG.debug("Delete backup info " + p + " for " + backupInfo.getBackupId());
+          if (!fs.delete(p)) {
+            if (fs.exists(p)) {
+              LOG.warn(f + " was not deleted");
+              success = false;
+            }
+          } else {
+            numDeleted++;
+          }
+        } catch (IOException ioe) {
+          LOG.warn(f + " was not deleted", ioe);
+          success = false;
+        }
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(numDeleted + " bulk loaded files out of " + map.size() + " were deleted");
+      }
+      if (success) {
+        sysTable.deleteBulkLoadedFiles(map);
+      }
+
+      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());
+        // Idempotent operation
+        BackupUtils.cleanupBackupData(info, conn.getConfiguration());
+      } else {
+        info.setTables(tables);
+        sysTable.updateBackupInfo(info);
+        // Now, clean up directory for table (idempotent)
+        cleanupBackupDir(info, tn, conn.getConfiguration());
+      }
+    }
+  }
+
+  private List<BackupInfo> getAffectedBackupSessions(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();
+
+    TableBackupClient client = null;
+    try {
+      client = BackupClientFactory.create(conn, backupId, request);
+    } catch (IOException e) {
+      LOG.error("There is an active session already running");
+      throw e;
+    }
+
+    client.execute();
+
+    return backupId;
+  }
+
+  private List<TableName> excludeNonExistingTables(List<TableName> tableList,
+      List<TableName> nonExistingTableList) {
+
+    for (TableName table : nonExistingTableList) {
+      tableList.remove(table);
+    }
+    return tableList;
+  }
+
+  @Override
+  public void mergeBackups(String[] backupIds) throws IOException {
+    try (final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+      checkIfValidForMerge(backupIds, sysTable);
+      BackupMergeJob job = BackupRestoreFactory.getBackupMergeJob(conn.getConfiguration());
+      job.run(backupIds);
+    }
+  }
+
+  /**
+   * Verifies that backup images are valid for merge.
+   *
+   * <ul>
+   * <li>All backups MUST be in the same destination
+   * <li>No FULL backups are allowed - only INCREMENTAL
+   * <li>All backups must be in COMPLETE state
+   * <li>No holes in backup list are allowed
+   * </ul>
+   * <p>
+   * @param backupIds list of backup ids
+   * @param table backup system table
+   * @throws IOException
+   */
+  private void checkIfValidForMerge(String[] backupIds, BackupSystemTable table) throws IOException {
+    String backupRoot = null;
+
+    final Set<TableName> allTables = new HashSet<TableName>();
+    final Set<String> allBackups = new HashSet<String>();
+    long minTime = Long.MAX_VALUE, maxTime = Long.MIN_VALUE;
+    for (String backupId : backupIds) {
+      BackupInfo bInfo = table.readBackupInfo(backupId);
+      if (bInfo == null) {
+        String msg = "Backup session " + backupId + " not found";
+        throw new IOException(msg);
+      }
+      if (backupRoot == null) {
+        backupRoot = bInfo.getBackupRootDir();
+      } else if (!bInfo.getBackupRootDir().equals(backupRoot)) {
+        throw new IOException("Found different backup destinations in a list of a backup sessions \n"
+            + "1. " + backupRoot + "\n" + "2. " + bInfo.getBackupRootDir());
+      }
+      if (bInfo.getType() == BackupType.FULL) {
+        throw new IOException("FULL backup image can not be merged for: \n" + bInfo);
+      }
+
+      if (bInfo.getState() != BackupState.COMPLETE) {
+        throw new IOException("Backup image " + backupId
+            + " can not be merged becuase of its state: " + bInfo.getState());
+      }
+      allBackups.add(backupId);
+      allTables.addAll(bInfo.getTableNames());
+      long time = bInfo.getStartTs();
+      if (time < minTime) {
+        minTime = time;
+      }
+      if (time > maxTime) {
+        maxTime = time;
+      }
+    }
+
+
+    final long startRangeTime  = minTime;
+    final long endRangeTime = maxTime;
+    final String backupDest = backupRoot;
+    // Check we have no 'holes' in backup id list
+    // Filter 1 : backupRoot
+    // Filter 2 : time range filter
+    // Filter 3 : table filter
+
+    BackupInfo.Filter destinationFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        return info.getBackupRootDir().equals(backupDest);
+      }
+    };
+
+    BackupInfo.Filter timeRangeFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        long time = info.getStartTs();
+        return time >= startRangeTime && time <= endRangeTime ;
+      }
+    };
+
+    BackupInfo.Filter tableFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        List<TableName> tables = info.getTableNames();
+        return !Collections.disjoint(allTables, tables);
+      }
+    };
+
+    BackupInfo.Filter typeFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        return info.getType() == BackupType.INCREMENTAL;
+      }
+    };
+
+    BackupInfo.Filter stateFilter = new  BackupInfo.Filter() {
+      @Override
+      public boolean apply(BackupInfo info) {
+        return info.getState() == BackupState.COMPLETE;
+      }
+    };
+
+    List<BackupInfo> allInfos =
+        table.getBackupHistory( -1, destinationFilter,
+          timeRangeFilter, tableFilter, typeFilter, stateFilter);
+    if (allInfos.size() != allBackups.size()) {
+      // Yes we have at least one  hole in backup image sequence
+      List<String> missingIds = new ArrayList<String>();
+      for(BackupInfo info: allInfos) {
+        if(allBackups.contains(info.getBackupId())) {
+          continue;
+        }
+        missingIds.add(info.getBackupId());
+      }
+      String errMsg =
+          "Sequence of backup ids has 'holes'. The following backup images must be added:" +
+           org.apache.hadoop.util.StringUtils.join(",", missingIds);
+      throw new IOException(errMsg);
+    }
+  }
+}


[05/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 6330899..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
+++ /dev/null
@@ -1,387 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-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.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.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.
-   * @return The new HashMap of RS log time stamps after the log roll for this incremental backup.
-   * @throws IOException exception
-   */
-  public HashMap<String, Long> getIncrBackupLogFileMap()
-      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());
-    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
-    backupInfo.setIncrBackupFileList(logList);
-
-    return newTimestamps;
-  }
-
-  /**
-   * Get list of WAL files eligible for incremental backup
-   * @return list of WAL files
-   * @throws IOException
-   */
-  public List<String> getIncrBackupLogFileList()
-      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.");
-    }
-
-    newTimestamps = readRegionServerLastLogRollResult();
-
-    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
-    List<WALItem> logFromSystemTable =
-        getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps, getBackupInfo()
-            .getBackupRootDir());
-
-    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
-    backupInfo.setIncrBackupFileList(logList);
-
-    return logList;
-  }
-
-
-  private List<String> excludeAlreadyBackedUpWALs(List<String> logList,
-      List<WALItem> logFromSystemTable) {
-
-    Set<String> walFileNameSet = convertToSet(logFromSystemTable);
-
-    List<String> list = new ArrayList<String>();
-    for (int i=0; i < logList.size(); i++) {
-      Path p = new Path(logList.get(i));
-      String name  = p.getName();
-      if (walFileNameSet.contains(name)) continue;
-      list.add(logList.get(i));
-    }
-    return list;
-  }
-
-  /**
-   * Create Set of WAL file names (not full path names)
-   * @param logFromSystemTable
-   * @return set of WAL file names
-   */
-  private Set<String> convertToSet(List<WALItem> logFromSystemTable) {
-
-    Set<String> set = new HashSet<String>();
-    for (int i=0; i < logFromSystemTable.size(); i++) {
-      WALItem item = logFromSystemTable.get(i);
-      set.add(item.walFile);
-    }
-    return set;
-  }
-
-  /**
-   * 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/2dda3712/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
deleted file mode 100644
index 6d48c32..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
+++ /dev/null
@@ -1,377 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-
-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.FileUtil;
-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.BackupInfo.BackupPhase;
-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.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.mapreduce.WALPlayer;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.HFileArchiveUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.util.Tool;
-
-/**
- * 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);
-
-  protected IncrementalTableBackupClient() {
-  }
-
-  public IncrementalTableBackupClient(final Connection conn, final String backupId,
-      BackupRequest request) throws IOException {
-    super(conn, backupId, request);
-  }
-
-  protected List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
-    FileSystem fs = FileSystem.get(conf);
-    List<String> list = new ArrayList<String>();
-    for (String file : incrBackupFileList) {
-      Path p = new Path(file);
-      if (fs.exists(p) || isActiveWalPath(p)) {
-        list.add(file);
-      } else {
-        LOG.warn("Can't find file: " + file);
-      }
-    }
-    return list;
-  }
-
-  /**
-   * Check if a given path is belongs to active WAL directory
-   * @param p path
-   * @return true, if yes
-   */
-  protected boolean isActiveWalPath(Path p) {
-    return !AbstractFSWALProvider.isArchivedLogFile(p);
-  }
-
-  protected static int getIndex(TableName tbl, List<TableName> sTableList) {
-    if (sTableList == null) return 0;
-    for (int i = 0; i < sTableList.size(); i++) {
-      if (tbl.equals(sTableList.get(i))) {
-        return i;
-      }
-    }
-    return -1;
-  }
-
-  /*
-   * Reads bulk load records from backup table, iterates through the records and forms the paths
-   * for bulk loaded hfiles. Copies the bulk loaded hfiles to backup destination
-   * @param sTableList list of tables to be backed up
-   * @return map of table to List of files
-   */
-  protected Map<byte[], List<Path>>[] handleBulkLoad(List<TableName> sTableList) throws IOException {
-    Map<byte[], List<Path>>[] mapForSrc = new Map[sTableList.size()];
-    Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>> pair =
-    backupManager.readBulkloadRows(sTableList);
-    Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> map = pair.getFirst();
-    FileSystem fs = FileSystem.get(conf);
-    FileSystem tgtFs;
-    try {
-      tgtFs = FileSystem.get(new URI(backupInfo.getBackupRootDir()), conf);
-    } catch (URISyntaxException use) {
-      throw new IOException("Unable to get FileSystem", use);
-    }
-    Path rootdir = FSUtils.getRootDir(conf);
-    Path tgtRoot = new Path(new Path(backupInfo.getBackupRootDir()), backupId);
-    for (Map.Entry<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> tblEntry :
-      map.entrySet()) {
-      TableName srcTable = tblEntry.getKey();
-      int srcIdx = getIndex(srcTable, sTableList);
-      if (srcIdx < 0) {
-        LOG.warn("Couldn't find " + srcTable + " in source table List");
-        continue;
-      }
-      if (mapForSrc[srcIdx] == null) {
-        mapForSrc[srcIdx] = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
-      }
-      Path tblDir = FSUtils.getTableDir(rootdir, srcTable);
-      Path tgtTable = new Path(new Path(tgtRoot, srcTable.getNamespaceAsString()),
-          srcTable.getQualifierAsString());
-      for (Map.Entry<String,Map<String,List<Pair<String, Boolean>>>> regionEntry :
-        tblEntry.getValue().entrySet()){
-        String regionName = regionEntry.getKey();
-        Path regionDir = new Path(tblDir, regionName);
-        // map from family to List of hfiles
-        for (Map.Entry<String,List<Pair<String, Boolean>>> famEntry :
-          regionEntry.getValue().entrySet()) {
-          String fam = famEntry.getKey();
-          Path famDir = new Path(regionDir, fam);
-          List<Path> files;
-          if (!mapForSrc[srcIdx].containsKey(fam.getBytes())) {
-            files = new ArrayList<Path>();
-            mapForSrc[srcIdx].put(fam.getBytes(), files);
-          } else {
-            files = mapForSrc[srcIdx].get(fam.getBytes());
-          }
-          Path archiveDir = HFileArchiveUtil.getStoreArchivePath(conf, srcTable, regionName, fam);
-          String tblName = srcTable.getQualifierAsString();
-          Path tgtFam = new Path(new Path(tgtTable, regionName), fam);
-          if (!tgtFs.mkdirs(tgtFam)) {
-            throw new IOException("couldn't create " + tgtFam);
-          }
-          for (Pair<String, Boolean> fileWithState : famEntry.getValue()) {
-            String file = fileWithState.getFirst();
-            boolean raw = fileWithState.getSecond();
-            int idx = file.lastIndexOf("/");
-            String filename = file;
-            if (idx > 0) {
-              filename = file.substring(idx+1);
-            }
-            Path p = new Path(famDir, filename);
-            Path tgt = new Path(tgtFam, filename);
-            Path archive = new Path(archiveDir, filename);
-            if (fs.exists(p)) {
-              if (LOG.isTraceEnabled()) {
-                LOG.trace("found bulk hfile " + file + " in " + famDir + " for " + tblName);
-              }
-              try {
-                if (LOG.isTraceEnabled()) {
-                  LOG.trace("copying " + p + " to " + tgt);
-                }
-                FileUtil.copy(fs, p, tgtFs, tgt, false,conf);
-              } catch (FileNotFoundException e) {
-                LOG.debug("copying archive " + archive + " to " + tgt);
-                try {
-                  FileUtil.copy(fs, archive, tgtFs, tgt, false, conf);
-                } catch (FileNotFoundException fnfe) {
-                  if (!raw) throw fnfe;
-                }
-              }
-            } else {
-              LOG.debug("copying archive " + archive + " to " + tgt);
-              try {
-                FileUtil.copy(fs, archive, tgtFs, tgt, false, conf);
-              } catch (FileNotFoundException fnfe) {
-                if (!raw) throw fnfe;
-              }
-            }
-            files.add(tgt);
-          }
-        }
-      }
-    }
-    backupManager.writeBulkLoadedFiles(sTableList, mapForSrc);
-    backupManager.removeBulkLoadedRows(sTableList, pair.getSecond());
-    return mapForSrc;
-  }
-
-  @Override
-  public void execute() throws IOException {
-
-    try {
-      // case PREPARE_INCREMENTAL:
-      beginBackup(backupManager, backupInfo);
-      backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
-      LOG.debug("For incremental backup, current table set is "
-          + backupManager.getIncrementalBackupTableSet());
-      newTimestamps =
-          ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
-    } catch (Exception e) {
-      // fail the overall backup and return
-      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
-        BackupType.INCREMENTAL, conf);
-      return;
-    }
-
-    // case INCREMENTAL_COPY:
-    try {
-      // copy out the table and region info files for each table
-      BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
-      // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
-      convertWALsToHFiles(backupInfo);
-      incrementalCopyHFiles(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);
-      return;
-    }
-    // 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 {
-      // 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);
-
-      handleBulkLoad(backupInfo.getTableNames());
-      // backup complete
-      completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
-
-    } catch (IOException e) {
-      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
-        BackupType.INCREMENTAL, conf);
-    }
-  }
-
-  protected void incrementalCopyHFiles(BackupInfo backupInfo) throws Exception {
-
-    try {
-      LOG.debug("Incremental copy HFiles 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 = new ArrayList<String>();
-      // Add Bulk output
-      incrBackupFileList.add(getBulkOutputDir().toString());
-      String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
-      strArr[strArr.length - 1] = backupInfo.getBackupRootDir();
-      BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf);
-      int res = copyService.copy(backupInfo, backupManager, conf, BackupType.INCREMENTAL, strArr);
-      if (res != 0) {
-        LOG.error("Copy incremental HFile files failed with return code: " + res + ".");
-        throw new IOException("Failed copy from " + StringUtils.join(incrBackupFileList, ',')
-            + " to " + backupInfo.getHLogTargetDir());
-      }
-      LOG.debug("Incremental copy HFiles from " + StringUtils.join(incrBackupFileList, ',')
-          + " to " + backupInfo.getBackupRootDir() + " finished.");
-    } finally {
-      deleteBulkLoadDirectory();
-    }
-  }
-
-  protected void deleteBulkLoadDirectory() throws IOException {
-    // delete original bulk load directory on method exit
-    Path path = getBulkOutputDir();
-    FileSystem fs = FileSystem.get(conf);
-    boolean result = fs.delete(path, true);
-    if (!result) {
-      LOG.warn("Could not delete " + path);
-    }
-
-  }
-
-  protected void convertWALsToHFiles(BackupInfo backupInfo) throws IOException {
-    // get incremental backup file list and prepare parameters for DistCp
-    List<String> incrBackupFileList = backupInfo.getIncrBackupFileList();
-    // Get list of tables in incremental backup set
-    Set<TableName> tableSet = backupManager.getIncrementalBackupTableSet();
-    // filter missing files out (they have been copied by previous backups)
-    incrBackupFileList = filterMissingFiles(incrBackupFileList);
-    for (TableName table : tableSet) {
-      // Check if table exists
-      if (tableExists(table, conn)) {
-        walToHFiles(incrBackupFileList, table);
-      } else {
-        LOG.warn("Table " + table + " does not exists. Skipping in WAL converter");
-      }
-    }
-  }
-
-
-  protected boolean tableExists(TableName table, Connection conn) throws IOException {
-    try (Admin admin = conn.getAdmin();) {
-      return admin.tableExists(table);
-    }
-  }
-
-  protected void walToHFiles(List<String> dirPaths, TableName tableName) throws IOException {
-
-    Tool player = new WALPlayer();
-
-    // Player reads all files in arbitrary directory structure and creates
-    // a Map task for each file. We use ';' as separator
-    // because WAL file names contains ','
-    String dirs = StringUtils.join(dirPaths, ';');
-
-    Path bulkOutputPath = getBulkOutputDirForTable(tableName);
-    conf.set(WALPlayer.BULK_OUTPUT_CONF_KEY, bulkOutputPath.toString());
-    conf.set(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ";");
-    String[] playerArgs = { dirs, tableName.getNameAsString() };
-
-    try {
-      player.setConf(conf);
-      int result = player.run(playerArgs);
-      if(result != 0) {
-        throw new IOException("WAL Player failed");
-      }
-      conf.unset(WALPlayer.INPUT_FILES_SEPARATOR_KEY);
-    } catch (IOException e) {
-      throw e;
-    } catch (Exception ee) {
-      throw new IOException("Can not convert from directory " + dirs
-          + " (check Hadoop, HBase and WALPlayer M/R job logs) ", ee);
-    }
-  }
-
-  protected Path getBulkOutputDirForTable(TableName table) {
-    Path tablePath = getBulkOutputDir();
-    tablePath = new Path(tablePath, table.getNamespaceAsString());
-    tablePath = new Path(tablePath, table.getQualifierAsString());
-    return new Path(tablePath, "data");
-  }
-
-  protected Path getBulkOutputDir() {
-    String backupId = backupInfo.getBackupId();
-    Path path = new Path(backupInfo.getBackupRootDir());
-    path = new Path(path, ".tmp");
-    path = new Path(path, backupId);
-    return path;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index ea7a7b8..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-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.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.BackupUtils;
-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;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.LoadQueueItem;
-
-/**
- * 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(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 fileBackupDir =
-          HBackupFileSystem.getTableBackupDataDir(im.getRootDir(), im.getBackupId(), sTable);
-      dirList.add(new Path(fileBackupDir));
-    }
-
-    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;
-    Set<String> backupIdSet = new HashSet<>();
-
-    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));
-          if (image.getType() == BackupType.INCREMENTAL) {
-            backupIdSet.add(image.getBackupId());
-            LOG.debug("adding " + image.getBackupId() + " for bulk load");
-          }
-        }
-      }
-    }
-    try (BackupSystemTable table = new BackupSystemTable(conn)) {
-      List<TableName> sTableList = Arrays.asList(sTableArray);
-      for (String id : backupIdSet) {
-        LOG.debug("restoring bulk load for " + id);
-        Map<byte[], List<Path>>[] mapForSrc = table.readBulkLoadedFiles(id, sTableList);
-        Map<LoadQueueItem, ByteBuffer> loaderResult;
-        conf.setBoolean(LoadIncrementalHFiles.ALWAYS_COPY_FILES, true);
-        LoadIncrementalHFiles loader = BackupUtils.createLoader(conf);
-        for (int i = 0; i < sTableList.size(); i++) {
-          if (mapForSrc[i] != null && !mapForSrc[i].isEmpty()) {
-            loaderResult = loader.run(null, mapForSrc[i], tTableArray[i]);
-            LOG.debug("bulk loading " + sTableList.get(i) + " to " + tTableArray[i]);
-            if (loaderResult.isEmpty()) {
-              String msg = "Couldn't bulk load for " + sTableList.get(i) + " to " + tTableArray[i];
-              LOG.error(msg);
-              throw new IOException(msg);
-            }
-          }
-        }
-      }
-    }
-    LOG.debug("restoreStage finished");
-  }
-
-  static long getTsFromBackupId(String backupId) {
-    if (backupId == null) {
-      return 0;
-    }
-    return Long.parseLong(backupId.substring(backupId.lastIndexOf("_") + 1));
-  }
-
-  static boolean withinRange(long a, long lower, long upper) {
-    if (a < lower || a > upper) {
-      return false;
-    }
-    return true;
-  }
-
-  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/2dda3712/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
deleted file mode 100644
index 6eec460..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
+++ /dev/null
@@ -1,436 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
-/**
- * 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 {
-
-  public static final String BACKUP_CLIENT_IMPL_CLASS = "backup.client.impl.class";
-
-  @VisibleForTesting
-  public static final String BACKUP_TEST_MODE_STAGE = "backup.test.mode.stage";
-
-  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() {
-  }
-
-  public TableBackupClient(final Connection conn, final String backupId, BackupRequest request)
-      throws IOException {
-    init(conn, backupId, request);
-  }
-
-  public void init(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());
-    }
-    // Start new session
-    backupManager.startBackupSession();
-  }
-
-  /**
-   * Begin the overall backup.
-   * @param backupInfo backup info
-   * @throws IOException exception
-   */
-  protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo)
-      throws IOException {
-
-    BackupSystemTable.snapshot(conn);
-    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.");
-    }
-  }
-
-  protected 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
-   */
-  protected static void deleteSnapshots(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);
-      }
-      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
-   */
-  protected static 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.
-   */
-  protected static 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.debug("Cleaning up uncompleted backup data at " + targetDirPath.toString()
-                + " done.");
-          } else {
-            LOG.debug("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 {
-
-    try {
-      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);
-      cleanupAndRestoreBackupSystem(conn, backupInfo, conf);
-      // If backup session is updated to FAILED state - means we
-      // processed recovery already.
-      backupManager.updateBackupInfo(backupInfo);
-      backupManager.finishBackupSession();
-      LOG.error("Backup " + backupInfo.getBackupId() + " failed.");
-    } catch (IOException ee) {
-      LOG.error("Please run backup repair tool manually to restore backup system integrity");
-      throw ee;
-    }
-  }
-
-  public static void cleanupAndRestoreBackupSystem (Connection conn, BackupInfo backupInfo,
-      Configuration conf) throws IOException
-  {
-    BackupType type = backupInfo.getType();
-     // 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) {
-       deleteSnapshots(conn, backupInfo, conf);
-       cleanupExportSnapshotLog(conf);
-     }
-     BackupSystemTable.restoreFromSnapshot(conn);
-     BackupSystemTable.deleteSnapshot(conn);
-     // 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);
-  }
-
-
-
-  /**
-   * 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
-   */
-  protected 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
-   */
-  protected 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
-   */
-  protected 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);
-
-    // 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);
-    }
-
-    // when full backup is done:
-    // - delete HBase snapshot
-    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
-    // snapshots
-    if (type == BackupType.FULL) {
-      deleteSnapshots(conn, backupInfo, conf);
-      cleanupExportSnapshotLog(conf);
-    } else if (type == BackupType.INCREMENTAL) {
-      cleanupDistCpLog(backupInfo, conf);
-    }
-    BackupSystemTable.deleteSnapshot(conn);
-    backupManager.updateBackupInfo(backupInfo);
-
-    // Finish active session
-    backupManager.finishBackupSession();
-
-    LOG.info("Backup " + backupInfo.getBackupId() + " completed.");
-  }
-
-  /**
-   * Backup request execution
-   * @throws IOException
-   */
-  public abstract void execute() throws IOException;
-
-  @VisibleForTesting
-  protected Stage getTestStage() {
-    return Stage.valueOf("stage_"+ conf.getInt(BACKUP_TEST_MODE_STAGE, 0));
-  }
-
-  @VisibleForTesting
-  protected void failStageIf(Stage stage) throws IOException {
-    Stage current = getTestStage();
-    if (current == stage) {
-      throw new IOException("Failed stage " + stage+" in testing");
-    }
-  }
-
-  public static enum Stage {
-    stage_0, stage_1, stage_2, stage_3, stage_4
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 016d1a4..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java
+++ /dev/null
@@ -1,344 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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/2dda3712/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
deleted file mode 100644
index 00c5b83..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
+++ /dev/null
@@ -1,321 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.mapreduce;
-
-import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-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.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.BackupMergeJob;
-import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
-import org.apache.hadoop.hbase.backup.HBackupFileSystem;
-import org.apache.hadoop.hbase.backup.impl.BackupManifest;
-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.Pair;
-import org.apache.hadoop.util.Tool;
-
-/**
- * MapReduce implementation of {@link BackupMergeJob}
- * Must be initialized with configuration of a backup destination cluster
- *
- */
-
-@InterfaceAudience.Private
-public class MapReduceBackupMergeJob implements BackupMergeJob {
-  public static final Log LOG = LogFactory.getLog(MapReduceBackupMergeJob.class);
-
-  protected Tool player;
-  protected Configuration conf;
-
-  public MapReduceBackupMergeJob() {
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  @Override
-  public void run(String[] backupIds) throws IOException {
-    String bulkOutputConfKey;
-
-    // TODO : run player on remote cluster
-    player = new MapReduceHFileSplitterJob();
-    bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
-    // Player reads all files in arbitrary directory structure and creates
-    // a Map task for each file
-    String bids = StringUtils.join(backupIds, ",");
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Merge backup images " + bids);
-    }
-
-    List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
-    boolean finishedTables = false;
-    Connection conn = ConnectionFactory.createConnection(getConf());
-    BackupSystemTable table = new BackupSystemTable(conn);
-    FileSystem fs = FileSystem.get(getConf());
-
-    try {
-
-      // Get exclusive lock on backup system
-      table.startBackupExclusiveOperation();
-      // Start merge operation
-      table.startMergeOperation(backupIds);
-
-      // Select most recent backup id
-      String mergedBackupId = findMostRecentBackupId(backupIds);
-
-      TableName[] tableNames = getTableNamesInBackupImages(backupIds);
-      String backupRoot = null;
-
-      BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
-      backupRoot = bInfo.getBackupRootDir();
-
-      for (int i = 0; i < tableNames.length; i++) {
-
-        LOG.info("Merge backup images for " + tableNames[i]);
-
-        // Find input directories for table
-
-        Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
-        String dirs = StringUtils.join(dirPaths, ",");
-        Path bulkOutputPath =
-            BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
-              getConf(), false);
-        // Delete content if exists
-        if (fs.exists(bulkOutputPath)) {
-          if (!fs.delete(bulkOutputPath, true)) {
-            LOG.warn("Can not delete: " + bulkOutputPath);
-          }
-        }
-        Configuration conf = getConf();
-        conf.set(bulkOutputConfKey, bulkOutputPath.toString());
-        String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
-
-        int result = 0;
-
-        player.setConf(getConf());
-        result = player.run(playerArgs);
-        if (!succeeded(result)) {
-          throw new IOException("Can not merge backup images for " + dirs
-              + " (check Hadoop/MR and HBase logs). Player return code =" + result);
-        }
-        // Add to processed table list
-        processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
-        LOG.debug("Merge Job finished:" + result);
-      }
-      List<TableName> tableList = toTableNameList(processedTableList);
-      table.updateProcessedTablesForMerge(tableList);
-      finishedTables = true;
-
-      // Move data
-      for (Pair<TableName, Path> tn : processedTableList) {
-        moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
-      }
-
-      // Delete old data and update manifest
-      List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
-      deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
-      updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
-      // Finish merge session
-      table.finishMergeOperation();
-      // Release lock
-      table.finishBackupExclusiveOperation();
-    } catch (RuntimeException e) {
-
-      throw e;
-    } catch (Exception e) {
-      LOG.error(e);
-      if (!finishedTables) {
-        // cleanup bulk directories and finish merge
-        // merge MUST be repeated (no need for repair)
-        cleanupBulkLoadDirs(fs, toPathList(processedTableList));
-        table.finishMergeOperation();
-        table.finishBackupExclusiveOperation();
-        throw new IOException("Backup merge operation failed, you should try it again", e);
-      } else {
-        // backup repair must be run
-        throw new IOException(
-            "Backup merge operation failed, run backup repair tool to restore system's integrity",
-            e);
-      }
-    } finally {
-      table.close();
-      conn.close();
-    }
-  }
-
-  protected List<Path> toPathList(List<Pair<TableName, Path>> processedTableList) {
-    ArrayList<Path> list = new ArrayList<Path>();
-    for (Pair<TableName, Path> p : processedTableList) {
-      list.add(p.getSecond());
-    }
-    return list;
-  }
-
-  protected List<TableName> toTableNameList(List<Pair<TableName, Path>> processedTableList) {
-    ArrayList<TableName> list = new ArrayList<TableName>();
-    for (Pair<TableName, Path> p : processedTableList) {
-      list.add(p.getFirst());
-    }
-    return list;
-  }
-
-  protected void cleanupBulkLoadDirs(FileSystem fs, List<Path> pathList) throws IOException {
-    for (Path path : pathList) {
-
-      if (!fs.delete(path, true)) {
-        LOG.warn("Can't delete " + path);
-      }
-    }
-  }
-
-  protected void updateBackupManifest(String backupRoot, String mergedBackupId,
-      List<String> backupsToDelete) throws IllegalArgumentException, IOException {
-
-    BackupManifest manifest =
-        HBackupFileSystem.getManifest(conf, new Path(backupRoot), mergedBackupId);
-    manifest.getBackupImage().removeAncestors(backupsToDelete);
-    // save back
-    manifest.store(conf);
-
-  }
-
-  protected void deleteBackupImages(List<String> backupIds, Connection conn, FileSystem fs,
-      String backupRoot) throws IOException {
-
-    // Delete from backup system table
-    try (BackupSystemTable table = new BackupSystemTable(conn);) {
-      for (String backupId : backupIds) {
-        table.deleteBackupInfo(backupId);
-      }
-    }
-
-    // Delete from file system
-    for (String backupId : backupIds) {
-      Path backupDirPath = HBackupFileSystem.getBackupPath(backupRoot, backupId);
-
-      if (!fs.delete(backupDirPath, true)) {
-        LOG.warn("Could not delete " + backupDirPath);
-      }
-    }
-  }
-
-  protected List<String> getBackupIdsToDelete(String[] backupIds, String mergedBackupId) {
-    List<String> list = new ArrayList<String>();
-    for (String id : backupIds) {
-      if (id.equals(mergedBackupId)) {
-        continue;
-      }
-      list.add(id);
-    }
-    return list;
-  }
-
-  protected void moveData(FileSystem fs, String backupRoot, Path bulkOutputPath, TableName tableName,
-      String mergedBackupId) throws IllegalArgumentException, IOException {
-
-    Path dest =
-        new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, mergedBackupId, tableName));
-
-    // Delete all in dest
-    if (!fs.delete(dest, true)) {
-      throw new IOException("Could not delete " + dest);
-    }
-
-    FileStatus[] fsts = fs.listStatus(bulkOutputPath);
-    for (FileStatus fst : fsts) {
-      if (fst.isDirectory()) {
-        fs.rename(fst.getPath().getParent(), dest);
-      }
-    }
-
-  }
-
-  protected String findMostRecentBackupId(String[] backupIds) {
-    long recentTimestamp = Long.MIN_VALUE;
-    for (String backupId : backupIds) {
-      long ts = Long.parseLong(backupId.split("_")[1]);
-      if (ts > recentTimestamp) {
-        recentTimestamp = ts;
-      }
-    }
-    return BackupRestoreConstants.BACKUPID_PREFIX + recentTimestamp;
-  }
-
-  protected TableName[] getTableNamesInBackupImages(String[] backupIds) throws IOException {
-
-    Set<TableName> allSet = new HashSet<TableName>();
-
-    try (Connection conn = ConnectionFactory.createConnection(conf);
-        BackupSystemTable table = new BackupSystemTable(conn);) {
-      for (String backupId : backupIds) {
-        BackupInfo bInfo = table.readBackupInfo(backupId);
-
-        allSet.addAll(bInfo.getTableNames());
-      }
-    }
-
-    TableName[] ret = new TableName[allSet.size()];
-    return allSet.toArray(ret);
-  }
-
-  protected Path[] findInputDirectories(FileSystem fs, String backupRoot, TableName tableName,
-      String[] backupIds) throws IOException {
-
-    List<Path> dirs = new ArrayList<Path>();
-
-    for (String backupId : backupIds) {
-      Path fileBackupDirPath =
-          new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, backupId, tableName));
-      if (fs.exists(fileBackupDirPath)) {
-        dirs.add(fileBackupDirPath);
-      } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("File: " + fileBackupDirPath + " does not exist.");
-        }
-      }
-    }
-    Path[] ret = new Path[dirs.size()];
-    return dirs.toArray(ret);
-  }
-
-}


[16/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
new file mode 100644
index 0000000..650ba2e
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
@@ -0,0 +1,1022 @@
+/**
+ * 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.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.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.backup.BackupType;
+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.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+/**
+ * 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"
+      + "  repair     repair backup system table\n"
+      + "  merge      merge backup images\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 REPAIR_CMD_USAGE = "Usage: hbase backup repair\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 MERGE_CMD_USAGE = "Usage: hbase backup merge [backup_ids]\n"
+      + "  backup_ids      Comma separated list of backup image ids.\n";
+
+  public static final String USAGE_FOOTER = "";
+
+  public static abstract class Command extends Configured {
+    CommandLine cmdline;
+    Connection conn;
+
+    Command(Configuration conf) {
+      if (conf == null) {
+        conf = HBaseConfiguration.create();
+      }
+      setConf(conf);
+    }
+
+    public void execute() throws IOException {
+      if (cmdline.hasOption("h") || cmdline.hasOption("help")) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      // Create connection
+      conn = ConnectionFactory.createConnection(getConf());
+      if (requiresNoActiveSession()) {
+        // Check active session
+        try (BackupSystemTable table = new BackupSystemTable(conn);) {
+          List<BackupInfo> sessions = table.getBackupInfos(BackupState.RUNNING);
+
+          if (sessions.size() > 0) {
+            System.err.println("Found backup session in a RUNNING state: ");
+            System.err.println(sessions.get(0));
+            System.err.println("This may indicate that a previous session has failed abnormally.");
+            System.err.println("In this case, backup recovery is recommended.");
+            throw new IOException("Active session found, aborted command execution");
+          }
+        }
+      }
+      if (requiresConsistentState()) {
+        // Check failed delete
+        try (BackupSystemTable table = new BackupSystemTable(conn);) {
+          String[] ids = table.getListOfBackupIdsFromDeleteOperation();
+
+          if (ids != null && ids.length > 0) {
+            System.err.println("Found failed backup DELETE coommand. ");
+            System.err.println("Backup system recovery is required.");
+            throw new IOException("Failed backup DELETE found, aborted command execution");
+          }
+
+          ids = table.getListOfBackupIdsFromMergeOperation();
+          if (ids != null && ids.length > 0) {
+            System.err.println("Found failed backup MERGE coommand. ");
+            System.err.println("Backup system recovery is required.");
+            throw new IOException("Failed backup MERGE found, aborted command execution");
+          }
+
+        }
+      }
+    }
+
+    public void finish() throws IOException {
+      if (conn != null) {
+        conn.close();
+      }
+    }
+
+    protected abstract void printUsage();
+
+    /**
+     * The command can't be run if active backup session is in progress
+     * @return true if no active sessions are in progress
+     */
+    protected boolean requiresNoActiveSession() {
+      return false;
+    }
+
+    /**
+     * Command requires consistent state of a backup system Backup system may become inconsistent
+     * because of an abnormal termination of a backup session or delete command
+     * @return true, if yes
+     */
+    protected boolean requiresConsistentState() {
+      return false;
+    }
+  }
+
+  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 REPAIR:
+      cmd = new RepairCommand(conf, cmdline);
+      break;
+    case MERGE:
+      cmd = new MergeCommand(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
+    protected boolean requiresNoActiveSession() {
+      return true;
+    }
+
+    @Override
+    protected boolean requiresConsistentState() {
+      return true;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      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;
+
+      // 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);
+      }
+      // Creates connection
+      super.execute();
+      // Check backup set
+      String setName = null;
+      if (cmdline.hasOption(OPTION_SET)) {
+        setName = cmdline.getOptionValue(OPTION_SET);
+        tables = getTablesForSet(setName, getConf());
+
+        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 (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 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 {
+      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 {
+      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);
+      }
+
+      super.execute();
+
+      String backupId = args[1];
+      try (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 {
+
+      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);
+      }
+
+      super.execute();
+
+      String backupId = (args == null || args.length <= 1) ? null : args[1];
+      try (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
+    protected boolean requiresNoActiveSession() {
+      return true;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      super.execute();
+
+      String[] args = cmdline.getArgs();
+      String[] backupIds = new String[args.length - 1];
+      System.arraycopy(args, 1, backupIds, 0, backupIds.length);
+      try (BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        int deleted = admin.deleteBackups(backupIds);
+        System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
+      } catch (IOException e) {
+        System.err
+            .println("Delete command FAILED. Please run backup repair tool to restore backup system integrity");
+        throw e;
+      }
+
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(DELETE_CMD_USAGE);
+    }
+  }
+
+  private static class RepairCommand extends Command {
+
+    RepairCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+
+      String[] args = cmdline == null ? null : cmdline.getArgs();
+      if (args != null && args.length > 1) {
+        System.err.println("ERROR: wrong number of arguments: " + args.length);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+
+        // Failed backup
+        BackupInfo backupInfo;
+        List<BackupInfo> list = sysTable.getBackupInfos(BackupState.RUNNING);
+        if (list.size() == 0) {
+          // No failed sessions found
+          System.out.println("REPAIR status: no failed sessions found."
+              + " Checking failed delete backup operation ...");
+          repairFailedBackupDeletionIfAny(conn, sysTable);
+          repairFailedBackupMergeIfAny(conn, sysTable);
+          return;
+        }
+        backupInfo = list.get(0);
+        // 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("REPAIR status: repaired after failure:\n" + backupInfo);
+        // 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();
+        System.out.println(backupFailedData);
+        TableBackupClient.cleanupAndRestoreBackupSystem(conn, backupInfo, conf);
+        // If backup session is updated to FAILED state - means we
+        // processed recovery already.
+        sysTable.updateBackupInfo(backupInfo);
+        sysTable.finishBackupExclusiveOperation();
+        System.out.println("REPAIR status: finished repair failed session:\n " + backupInfo);
+
+      }
+    }
+
+    private void repairFailedBackupDeletionIfAny(Connection conn, BackupSystemTable sysTable)
+        throws IOException {
+      String[] backupIds = sysTable.getListOfBackupIdsFromDeleteOperation();
+      if (backupIds == null || backupIds.length == 0) {
+        System.out.println("No failed backup DELETE operation found");
+        // Delete backup table snapshot if exists
+        BackupSystemTable.deleteSnapshot(conn);
+        return;
+      }
+      System.out.println("Found failed DELETE operation for: " + StringUtils.join(backupIds));
+      System.out.println("Running DELETE again ...");
+      // Restore table from snapshot
+      BackupSystemTable.restoreFromSnapshot(conn);
+      // Finish previous failed session
+      sysTable.finishBackupExclusiveOperation();
+      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
+        admin.deleteBackups(backupIds);
+      }
+      System.out.println("DELETE operation finished OK: " + StringUtils.join(backupIds));
+
+    }
+
+    private void repairFailedBackupMergeIfAny(Connection conn, BackupSystemTable sysTable)
+        throws IOException {
+      String[] backupIds = sysTable.getListOfBackupIdsFromMergeOperation();
+      if (backupIds == null || backupIds.length == 0) {
+        System.out.println("No failed backup MERGE operation found");
+        // Delete backup table snapshot if exists
+        BackupSystemTable.deleteSnapshot(conn);
+        return;
+      }
+      System.out.println("Found failed MERGE operation for: " + StringUtils.join(backupIds));
+      System.out.println("Running MERGE again ...");
+      // Restore table from snapshot
+      BackupSystemTable.restoreFromSnapshot(conn);
+      // Unlock backupo system
+      sysTable.finishBackupExclusiveOperation();
+      // Finish previous failed session
+      sysTable.finishMergeOperation();
+      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
+        admin.mergeBackups(backupIds);
+      }
+      System.out.println("MERGE operation finished OK: " + StringUtils.join(backupIds));
+
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(REPAIR_CMD_USAGE);
+    }
+  }
+
+  private static class MergeCommand extends Command {
+
+    MergeCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    protected boolean requiresNoActiveSession() {
+      return true;
+    }
+
+    @Override
+    protected boolean requiresConsistentState() {
+      return true;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+
+      String[] args = cmdline == null ? null : cmdline.getArgs();
+      if (args == null || (args.length != 2)) {
+        System.err.println("ERROR: wrong number of arguments: "
+            + (args == null ? null : args.length));
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String[] backupIds = args[1].split(",");
+      if (backupIds.length < 2) {
+        String msg = "ERROR: can not merge a single backup image. "+
+            "Number of images must be greater than 1.";
+        System.err.println(msg);
+        throw new IOException(msg);
+
+      }
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        admin.mergeBackups(backupIds);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(MERGE_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 {
+
+      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;
+      if (backupRootPath == null) {
+        // Load from backup system table
+        super.execute();
+        try (final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+          history = sysTable.getBackupHistory(n, tableNameFilter, tableSetFilter);
+        }
+      } else {
+        // load from backup FS
+        history =
+            BackupUtils.getHistory(getConf(), 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 {
+      // 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 {
+      super.execute();
+
+      // List all backup set names
+      // does not expect any args
+      try (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);
+      }
+      super.execute();
+
+      String setName = args[2];
+      try (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);
+      }
+      super.execute();
+
+      String setName = args[2];
+      try (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);
+      }
+      super.execute();
+
+      String setName = args[2];
+      String[] tables = args[3].split(",");
+      TableName[] tableNames = toTableNames(tables);
+      try (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);
+      }
+      super.execute();
+
+      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]);
+      }
+      try (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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
new file mode 100644
index 0000000..2c7d35f
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
new file mode 100644
index 0000000..8fe5eaf
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -0,0 +1,502 @@
+/**
+ * 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.Map;
+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 org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.shaded.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.getBackupPath(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);
+  }
+
+  /**
+   * Starts new backup session
+   * @throws IOException if active session already exists
+   */
+  public void startBackupSession() throws IOException {
+    systemTable.startBackupExclusiveOperation();
+  }
+
+  /**
+   * Finishes active backup session
+   * @throws IOException if no active session
+   */
+  public void finishBackupSession() throws IOException {
+    systemTable.finishBackupExclusiveOperation();
+  }
+
+  /**
+   * 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());
+  }
+
+  public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
+      readBulkloadRows(List<TableName> tableList) throws IOException {
+    return systemTable.readBulkloadRows(tableList);
+  }
+
+  public void removeBulkLoadedRows(List<TableName> lst, List<byte[]> rows) throws IOException {
+    systemTable.removeBulkLoadedRows(lst, rows);
+  }
+
+  public void writeBulkLoadedFiles(List<TableName> sTableList, Map<byte[], List<Path>>[] maps)
+      throws IOException {
+    systemTable.writeBulkLoadedFiles(sTableList, maps, backupInfo.getBackupId());
+  }
+
+  /**
+   * 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 plug-in 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
new file mode 100644
index 0000000..7e3201e
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
@@ -0,0 +1,674 @@
+/**
+ * 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.HBackupFileSystem;
+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;
+    }
+
+    public void removeAncestors(List<String> backupIds) {
+      List<BackupImage> toRemove = new ArrayList<BackupImage>();
+      for (BackupImage im : this.ancestors) {
+        if (backupIds.contains(im.getBackupId())) {
+          toRemove.add(im);
+        }
+      }
+      this.ancestors.removeAll(toRemove);
+    }
+
+    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();
+  }
+
+  /**
+   * TODO: fix it. 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
+    Path manifestFilePath =
+        new Path(HBackupFileSystem.getBackupPath(backupImage.getRootDir(),
+          backupImage.getBackupId()), 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;
+  }
+}


[26/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 99fb06c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
+++ /dev/null
@@ -1,743 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-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.BackupClientFactory;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
-import org.apache.hadoop.hbase.backup.BackupMergeJob;
-import org.apache.hadoop.hbase.backup.BackupRequest;
-import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
-import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
-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.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-@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 {
-  }
-
-  @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 {
-
-    int totalDeleted = 0;
-    Map<String, HashSet<TableName>> allTablesMap = new HashMap<String, HashSet<TableName>>();
-
-    boolean deleteSessionStarted = false;
-    boolean snapshotDone = false;
-    try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) {
-
-      // Step 1: Make sure there is no active session
-      // is running by using startBackupSession API
-      // If there is an active session in progress, exception will be thrown
-      try {
-        sysTable.startBackupExclusiveOperation();
-        deleteSessionStarted = true;
-      } catch (IOException e) {
-        LOG.warn("You can not run delete command while active backup session is in progress. \n"
-            + "If there is no active backup session running, run backup repair utility to restore \n"
-            + "backup system integrity.");
-        return -1;
-      }
-
-      // Step 2: Make sure there is no failed session
-      List<BackupInfo> list = sysTable.getBackupInfos(BackupState.RUNNING);
-      if (list.size() != 0) {
-        // ailed sessions found
-        LOG.warn("Failed backup session found. Run backup repair tool first.");
-        return -1;
-      }
-
-      // Step 3: Record delete session
-      sysTable.startDeleteOperation(backupIds);
-      // Step 4: Snapshot backup system table
-      if (!BackupSystemTable.snapshotExists(conn)) {
-        BackupSystemTable.snapshot(conn);
-      } else {
-        LOG.warn("Backup system table snapshot exists");
-      }
-      snapshotDone = true;
-      try {
-        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);
-        // Finish
-        sysTable.finishDeleteOperation();
-        // delete snapshot
-        BackupSystemTable.deleteSnapshot(conn);
-      } catch (IOException e) {
-        // Fail delete operation
-        // Step 1
-        if (snapshotDone) {
-          if (BackupSystemTable.snapshotExists(conn)) {
-            BackupSystemTable.restoreFromSnapshot(conn);
-            // delete snapshot
-            BackupSystemTable.deleteSnapshot(conn);
-            // We still have record with unfinished delete operation
-            LOG.error("Delete operation failed, please run backup repair utility to restore "
-                + "backup system integrity", e);
-            throw e;
-          } else {
-            LOG.warn("Delete operation succeeded, there were some errors: ", e);
-          }
-        }
-
-      } finally {
-        if (deleteSessionStarted) {
-          sysTable.finishBackupExclusiveOperation();
-        }
-      }
-    }
-    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() + " ...");
-      // Step 1: clean up data for backup session (idempotent)
-      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 = getAffectedBackupSessions(backupInfo, tn, sysTable);
-        for (BackupInfo info : affectedBackups) {
-          if (info.equals(backupInfo)) {
-            continue;
-          }
-          removeTableFromBackupImage(info, tn, sysTable);
-        }
-      }
-      Map<byte[], String> map = sysTable.readBulkLoadedFiles(backupId);
-      FileSystem fs = FileSystem.get(conn.getConfiguration());
-      boolean success = true;
-      int numDeleted = 0;
-      for (String f : map.values()) {
-        Path p = new Path(f);
-        try {
-          LOG.debug("Delete backup info " + p + " for " + backupInfo.getBackupId());
-          if (!fs.delete(p)) {
-            if (fs.exists(p)) {
-              LOG.warn(f + " was not deleted");
-              success = false;
-            }
-          } else {
-            numDeleted++;
-          }
-        } catch (IOException ioe) {
-          LOG.warn(f + " was not deleted", ioe);
-          success = false;
-        }
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(numDeleted + " bulk loaded files out of " + map.size() + " were deleted");
-      }
-      if (success) {
-        sysTable.deleteBulkLoadedFiles(map);
-      }
-
-      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());
-        // Idempotent operation
-        BackupUtils.cleanupBackupData(info, conn.getConfiguration());
-      } else {
-        info.setTables(tables);
-        sysTable.updateBackupInfo(info);
-        // Now, clean up directory for table (idempotent)
-        cleanupBackupDir(info, tn, conn.getConfiguration());
-      }
-    }
-  }
-
-  private List<BackupInfo> getAffectedBackupSessions(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();
-
-    TableBackupClient client = null;
-    try {
-      client = BackupClientFactory.create(conn, backupId, request);
-    } catch (IOException e) {
-      LOG.error("There is an active session already running");
-      throw e;
-    }
-
-    client.execute();
-
-    return backupId;
-  }
-
-  private List<TableName> excludeNonExistingTables(List<TableName> tableList,
-      List<TableName> nonExistingTableList) {
-
-    for (TableName table : nonExistingTableList) {
-      tableList.remove(table);
-    }
-    return tableList;
-  }
-
-  @Override
-  public void mergeBackups(String[] backupIds) throws IOException {
-    try (final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
-      checkIfValidForMerge(backupIds, sysTable);
-      BackupMergeJob job = BackupRestoreFactory.getBackupMergeJob(conn.getConfiguration());
-      job.run(backupIds);
-    }
-  }
-
-  /**
-   * Verifies that backup images are valid for merge.
-   *
-   * <ul>
-   * <li>All backups MUST be in the same destination
-   * <li>No FULL backups are allowed - only INCREMENTAL
-   * <li>All backups must be in COMPLETE state
-   * <li>No holes in backup list are allowed
-   * </ul>
-   * <p>
-   * @param backupIds list of backup ids
-   * @param table backup system table
-   * @throws IOException
-   */
-  private void checkIfValidForMerge(String[] backupIds, BackupSystemTable table) throws IOException {
-    String backupRoot = null;
-
-    final Set<TableName> allTables = new HashSet<TableName>();
-    final Set<String> allBackups = new HashSet<String>();
-    long minTime = Long.MAX_VALUE, maxTime = Long.MIN_VALUE;
-    for (String backupId : backupIds) {
-      BackupInfo bInfo = table.readBackupInfo(backupId);
-      if (bInfo == null) {
-        String msg = "Backup session " + backupId + " not found";
-        throw new IOException(msg);
-      }
-      if (backupRoot == null) {
-        backupRoot = bInfo.getBackupRootDir();
-      } else if (!bInfo.getBackupRootDir().equals(backupRoot)) {
-        throw new IOException("Found different backup destinations in a list of a backup sessions \n"
-            + "1. " + backupRoot + "\n" + "2. " + bInfo.getBackupRootDir());
-      }
-      if (bInfo.getType() == BackupType.FULL) {
-        throw new IOException("FULL backup image can not be merged for: \n" + bInfo);
-      }
-
-      if (bInfo.getState() != BackupState.COMPLETE) {
-        throw new IOException("Backup image " + backupId
-            + " can not be merged becuase of its state: " + bInfo.getState());
-      }
-      allBackups.add(backupId);
-      allTables.addAll(bInfo.getTableNames());
-      long time = bInfo.getStartTs();
-      if (time < minTime) {
-        minTime = time;
-      }
-      if (time > maxTime) {
-        maxTime = time;
-      }
-    }
-
-
-    final long startRangeTime  = minTime;
-    final long endRangeTime = maxTime;
-    final String backupDest = backupRoot;
-    // Check we have no 'holes' in backup id list
-    // Filter 1 : backupRoot
-    // Filter 2 : time range filter
-    // Filter 3 : table filter
-
-    BackupInfo.Filter destinationFilter = new  BackupInfo.Filter() {
-
-      @Override
-      public boolean apply(BackupInfo info) {
-        return info.getBackupRootDir().equals(backupDest);
-      }
-    };
-
-    BackupInfo.Filter timeRangeFilter = new  BackupInfo.Filter() {
-
-      @Override
-      public boolean apply(BackupInfo info) {
-        long time = info.getStartTs();
-        return time >= startRangeTime && time <= endRangeTime ;
-      }
-    };
-
-    BackupInfo.Filter tableFilter = new  BackupInfo.Filter() {
-
-      @Override
-      public boolean apply(BackupInfo info) {
-        List<TableName> tables = info.getTableNames();
-        return !Collections.disjoint(allTables, tables);
-      }
-    };
-
-    BackupInfo.Filter typeFilter = new  BackupInfo.Filter() {
-
-      @Override
-      public boolean apply(BackupInfo info) {
-        return info.getType() == BackupType.INCREMENTAL;
-      }
-    };
-
-    BackupInfo.Filter stateFilter = new  BackupInfo.Filter() {
-      @Override
-      public boolean apply(BackupInfo info) {
-        return info.getState() == BackupState.COMPLETE;
-      }
-    };
-
-    List<BackupInfo> allInfos =
-        table.getBackupHistory( -1, destinationFilter,
-          timeRangeFilter, tableFilter, typeFilter, stateFilter);
-    if (allInfos.size() != allBackups.size()) {
-      // Yes we have at least one  hole in backup image sequence
-      List<String> missingIds = new ArrayList<String>();
-      for(BackupInfo info: allInfos) {
-        if(allBackups.contains(info.getBackupId())) {
-          continue;
-        }
-        missingIds.add(info.getBackupId());
-      }
-      String errMsg =
-          "Sequence of backup ids has 'holes'. The following backup images must be added:" +
-           org.apache.hadoop.util.StringUtils.join(",", missingIds);
-      throw new IOException(errMsg);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 650ba2e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
+++ /dev/null
@@ -1,1022 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.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.BackupRestoreConstants.BackupCommand;
-import org.apache.hadoop.hbase.backup.BackupType;
-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.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-/**
- * 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"
-      + "  repair     repair backup system table\n"
-      + "  merge      merge backup images\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 REPAIR_CMD_USAGE = "Usage: hbase backup repair\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 MERGE_CMD_USAGE = "Usage: hbase backup merge [backup_ids]\n"
-      + "  backup_ids      Comma separated list of backup image ids.\n";
-
-  public static final String USAGE_FOOTER = "";
-
-  public static abstract class Command extends Configured {
-    CommandLine cmdline;
-    Connection conn;
-
-    Command(Configuration conf) {
-      if (conf == null) {
-        conf = HBaseConfiguration.create();
-      }
-      setConf(conf);
-    }
-
-    public void execute() throws IOException {
-      if (cmdline.hasOption("h") || cmdline.hasOption("help")) {
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      // Create connection
-      conn = ConnectionFactory.createConnection(getConf());
-      if (requiresNoActiveSession()) {
-        // Check active session
-        try (BackupSystemTable table = new BackupSystemTable(conn);) {
-          List<BackupInfo> sessions = table.getBackupInfos(BackupState.RUNNING);
-
-          if (sessions.size() > 0) {
-            System.err.println("Found backup session in a RUNNING state: ");
-            System.err.println(sessions.get(0));
-            System.err.println("This may indicate that a previous session has failed abnormally.");
-            System.err.println("In this case, backup recovery is recommended.");
-            throw new IOException("Active session found, aborted command execution");
-          }
-        }
-      }
-      if (requiresConsistentState()) {
-        // Check failed delete
-        try (BackupSystemTable table = new BackupSystemTable(conn);) {
-          String[] ids = table.getListOfBackupIdsFromDeleteOperation();
-
-          if (ids != null && ids.length > 0) {
-            System.err.println("Found failed backup DELETE coommand. ");
-            System.err.println("Backup system recovery is required.");
-            throw new IOException("Failed backup DELETE found, aborted command execution");
-          }
-
-          ids = table.getListOfBackupIdsFromMergeOperation();
-          if (ids != null && ids.length > 0) {
-            System.err.println("Found failed backup MERGE coommand. ");
-            System.err.println("Backup system recovery is required.");
-            throw new IOException("Failed backup MERGE found, aborted command execution");
-          }
-
-        }
-      }
-    }
-
-    public void finish() throws IOException {
-      if (conn != null) {
-        conn.close();
-      }
-    }
-
-    protected abstract void printUsage();
-
-    /**
-     * The command can't be run if active backup session is in progress
-     * @return true if no active sessions are in progress
-     */
-    protected boolean requiresNoActiveSession() {
-      return false;
-    }
-
-    /**
-     * Command requires consistent state of a backup system Backup system may become inconsistent
-     * because of an abnormal termination of a backup session or delete command
-     * @return true, if yes
-     */
-    protected boolean requiresConsistentState() {
-      return false;
-    }
-  }
-
-  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 REPAIR:
-      cmd = new RepairCommand(conf, cmdline);
-      break;
-    case MERGE:
-      cmd = new MergeCommand(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
-    protected boolean requiresNoActiveSession() {
-      return true;
-    }
-
-    @Override
-    protected boolean requiresConsistentState() {
-      return true;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      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;
-
-      // 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);
-      }
-      // Creates connection
-      super.execute();
-      // Check backup set
-      String setName = null;
-      if (cmdline.hasOption(OPTION_SET)) {
-        setName = cmdline.getOptionValue(OPTION_SET);
-        tables = getTablesForSet(setName, getConf());
-
-        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 (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 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 {
-      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 {
-      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);
-      }
-
-      super.execute();
-
-      String backupId = args[1];
-      try (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 {
-
-      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);
-      }
-
-      super.execute();
-
-      String backupId = (args == null || args.length <= 1) ? null : args[1];
-      try (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
-    protected boolean requiresNoActiveSession() {
-      return true;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      super.execute();
-
-      String[] args = cmdline.getArgs();
-      String[] backupIds = new String[args.length - 1];
-      System.arraycopy(args, 1, backupIds, 0, backupIds.length);
-      try (BackupAdminImpl admin = new BackupAdminImpl(conn);) {
-        int deleted = admin.deleteBackups(backupIds);
-        System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
-      } catch (IOException e) {
-        System.err
-            .println("Delete command FAILED. Please run backup repair tool to restore backup system integrity");
-        throw e;
-      }
-
-    }
-
-    @Override
-    protected void printUsage() {
-      System.out.println(DELETE_CMD_USAGE);
-    }
-  }
-
-  private static class RepairCommand extends Command {
-
-    RepairCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-
-      String[] args = cmdline == null ? null : cmdline.getArgs();
-      if (args != null && args.length > 1) {
-        System.err.println("ERROR: wrong number of arguments: " + args.length);
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
-      try (final Connection conn = ConnectionFactory.createConnection(conf);
-          final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
-
-        // Failed backup
-        BackupInfo backupInfo;
-        List<BackupInfo> list = sysTable.getBackupInfos(BackupState.RUNNING);
-        if (list.size() == 0) {
-          // No failed sessions found
-          System.out.println("REPAIR status: no failed sessions found."
-              + " Checking failed delete backup operation ...");
-          repairFailedBackupDeletionIfAny(conn, sysTable);
-          repairFailedBackupMergeIfAny(conn, sysTable);
-          return;
-        }
-        backupInfo = list.get(0);
-        // 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("REPAIR status: repaired after failure:\n" + backupInfo);
-        // 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();
-        System.out.println(backupFailedData);
-        TableBackupClient.cleanupAndRestoreBackupSystem(conn, backupInfo, conf);
-        // If backup session is updated to FAILED state - means we
-        // processed recovery already.
-        sysTable.updateBackupInfo(backupInfo);
-        sysTable.finishBackupExclusiveOperation();
-        System.out.println("REPAIR status: finished repair failed session:\n " + backupInfo);
-
-      }
-    }
-
-    private void repairFailedBackupDeletionIfAny(Connection conn, BackupSystemTable sysTable)
-        throws IOException {
-      String[] backupIds = sysTable.getListOfBackupIdsFromDeleteOperation();
-      if (backupIds == null || backupIds.length == 0) {
-        System.out.println("No failed backup DELETE operation found");
-        // Delete backup table snapshot if exists
-        BackupSystemTable.deleteSnapshot(conn);
-        return;
-      }
-      System.out.println("Found failed DELETE operation for: " + StringUtils.join(backupIds));
-      System.out.println("Running DELETE again ...");
-      // Restore table from snapshot
-      BackupSystemTable.restoreFromSnapshot(conn);
-      // Finish previous failed session
-      sysTable.finishBackupExclusiveOperation();
-      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
-        admin.deleteBackups(backupIds);
-      }
-      System.out.println("DELETE operation finished OK: " + StringUtils.join(backupIds));
-
-    }
-
-    private void repairFailedBackupMergeIfAny(Connection conn, BackupSystemTable sysTable)
-        throws IOException {
-      String[] backupIds = sysTable.getListOfBackupIdsFromMergeOperation();
-      if (backupIds == null || backupIds.length == 0) {
-        System.out.println("No failed backup MERGE operation found");
-        // Delete backup table snapshot if exists
-        BackupSystemTable.deleteSnapshot(conn);
-        return;
-      }
-      System.out.println("Found failed MERGE operation for: " + StringUtils.join(backupIds));
-      System.out.println("Running MERGE again ...");
-      // Restore table from snapshot
-      BackupSystemTable.restoreFromSnapshot(conn);
-      // Unlock backupo system
-      sysTable.finishBackupExclusiveOperation();
-      // Finish previous failed session
-      sysTable.finishMergeOperation();
-      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
-        admin.mergeBackups(backupIds);
-      }
-      System.out.println("MERGE operation finished OK: " + StringUtils.join(backupIds));
-
-    }
-
-    @Override
-    protected void printUsage() {
-      System.out.println(REPAIR_CMD_USAGE);
-    }
-  }
-
-  private static class MergeCommand extends Command {
-
-    MergeCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    protected boolean requiresNoActiveSession() {
-      return true;
-    }
-
-    @Override
-    protected boolean requiresConsistentState() {
-      return true;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-
-      String[] args = cmdline == null ? null : cmdline.getArgs();
-      if (args == null || (args.length != 2)) {
-        System.err.println("ERROR: wrong number of arguments: "
-            + (args == null ? null : args.length));
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      String[] backupIds = args[1].split(",");
-      if (backupIds.length < 2) {
-        String msg = "ERROR: can not merge a single backup image. "+
-            "Number of images must be greater than 1.";
-        System.err.println(msg);
-        throw new IOException(msg);
-
-      }
-      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
-      try (final Connection conn = ConnectionFactory.createConnection(conf);
-          final BackupAdminImpl admin = new BackupAdminImpl(conn);) {
-        admin.mergeBackups(backupIds);
-      }
-    }
-
-    @Override
-    protected void printUsage() {
-      System.out.println(MERGE_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 {
-
-      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;
-      if (backupRootPath == null) {
-        // Load from backup system table
-        super.execute();
-        try (final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
-          history = sysTable.getBackupHistory(n, tableNameFilter, tableSetFilter);
-        }
-      } else {
-        // load from backup FS
-        history =
-            BackupUtils.getHistory(getConf(), 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 {
-      // 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 {
-      super.execute();
-
-      // List all backup set names
-      // does not expect any args
-      try (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);
-      }
-      super.execute();
-
-      String setName = args[2];
-      try (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);
-      }
-      super.execute();
-
-      String setName = args[2];
-      try (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);
-      }
-      super.execute();
-
-      String setName = args[2];
-      String[] tables = args[3].split(",");
-      TableName[] tableNames = toTableNames(tables);
-      try (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);
-      }
-      super.execute();
-
-      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]);
-      }
-      try (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/37c65946/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
deleted file mode 100644
index 2c7d35f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/37c65946/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
deleted file mode 100644
index 8fe5eaf..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
+++ /dev/null
@@ -1,502 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.Map;
-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 org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.hadoop.hbase.shaded.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.getBackupPath(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);
-  }
-
-  /**
-   * Starts new backup session
-   * @throws IOException if active session already exists
-   */
-  public void startBackupSession() throws IOException {
-    systemTable.startBackupExclusiveOperation();
-  }
-
-  /**
-   * Finishes active backup session
-   * @throws IOException if no active session
-   */
-  public void finishBackupSession() throws IOException {
-    systemTable.finishBackupExclusiveOperation();
-  }
-
-  /**
-   * 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());
-  }
-
-  public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
-      readBulkloadRows(List<TableName> tableList) throws IOException {
-    return systemTable.readBulkloadRows(tableList);
-  }
-
-  public void removeBulkLoadedRows(List<TableName> lst, List<byte[]> rows) throws IOException {
-    systemTable.removeBulkLoadedRows(lst, rows);
-  }
-
-  public void writeBulkLoadedFiles(List<TableName> sTableList, Map<byte[], List<Path>>[] maps)
-      throws IOException {
-    systemTable.writeBulkLoadedFiles(sTableList, maps, backupInfo.getBackupId());
-  }
-
-  /**
-   * 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 plug-in 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;
-  }
-}


[13/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
new file mode 100644
index 0000000..49e8c75
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.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 MapReduceHFileSplitterJob extends Configured implements Tool {
+  private static final Log LOG = LogFactory.getLog(MapReduceHFileSplitterJob.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 MapReduceHFileSplitterJob() {
+  }
+
+  protected MapReduceHFileSplitterJob(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);
+    conf.set(FileInputFormat.INPUT_DIR, inputDirs);
+    Job job =
+        Job.getInstance(conf,
+          conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime()));
+    job.setJarByClass(MapReduceHFileSplitterJob.class);
+    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.getDescriptor(), regionLocator);
+      }
+      LOG.debug("success configuring load incremental job");
+
+      TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
+        org.apache.hadoop.hbase.shaded.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 MapReduceHFileSplitterJob(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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
new file mode 100644
index 0000000..1209e7c
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
@@ -0,0 +1,136 @@
+/**
+ * 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 static org.apache.hadoop.hbase.backup.util.BackupUtils.failed;
+import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
+
+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.Path;
+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.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.util.Tool;
+
+
+/**
+ * MapReduce implementation of {@link RestoreJob}
+ *
+ * For backup restore, it runs {@link MapReduceHFileSplitterJob} job and creates
+ * HFiles which are aligned with a region boundaries of a table being
+ * restored.
+ *
+ * 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;
+
+    player = new MapReduceHFileSplitterJob();
+    bulkOutputConfKey = MapReduceHFileSplitterJob.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 =
+          BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(newTableNames[i]),
+            getConf());
+      Configuration conf = getConf();
+      conf.set(bulkOutputConfKey, bulkOutputPath.toString());
+      String[] playerArgs =
+          {
+              dirs,
+              fullBackupRestore ? newTableNames[i].getNameAsString() : 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 = BackupUtils.createLoader(getConf());
+          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) {
+        LOG.error(e);
+        throw new IOException("Can not restore from backup directory " + dirs
+            + " (check Hadoop and HBase logs) ", e);
+      }
+    }
+  }
+
+  @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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
new file mode 100644
index 0000000..b5b887c
--- /dev/null
+++ b/hbase-backup/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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
new file mode 100644
index 0000000..47e428c
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
new file mode 100644
index 0000000..8fc644c
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
new file mode 100644
index 0000000..65a1fa3
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
new file mode 100644
index 0000000..9d5a858
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
new file mode 100644
index 0000000..0da6fc4
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
new file mode 100644
index 0000000..ce77645
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
@@ -0,0 +1,747 @@
+/**
+ * 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.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+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 = ".";
+  public static final int MILLISEC_IN_HOUR = 3600000;
+
+  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;
+  }
+
+  public static Path getBulkOutputDir(String tableName, Configuration conf, boolean deleteOnExit)
+      throws IOException {
+    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());
+    if (deleteOnExit) {
+      fs.deleteOnExit(path);
+    }
+    return path;
+  }
+
+  public static Path getBulkOutputDir(String tableName, Configuration conf) throws IOException {
+    return getBulkOutputDir(tableName, conf, true);
+  }
+
+  public static String getFileNameCompatibleString(TableName table) {
+    return table.getNamespaceAsString() + "-" + table.getQualifierAsString();
+  }
+
+  public static boolean failed(int result) {
+    return result != 0;
+  }
+
+  public static boolean succeeded(int result) {
+    return result == 0;
+  }
+
+  public static LoadIncrementalHFiles createLoader(Configuration config) throws IOException {
+    // set configuration for restore:
+    // LoadIncrementalHFile needs more time
+    // <name>hbase.rpc.timeout</name> <value>600000</value>
+    // calculates
+    Configuration conf = new Configuration(config);
+    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, MILLISEC_IN_HOUR);
+
+    // 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;
+  }
+}


[25/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 7e3201e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
+++ /dev/null
@@ -1,674 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.HBackupFileSystem;
-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;
-    }
-
-    public void removeAncestors(List<String> backupIds) {
-      List<BackupImage> toRemove = new ArrayList<BackupImage>();
-      for (BackupImage im : this.ancestors) {
-        if (backupIds.contains(im.getBackupId())) {
-          toRemove.add(im);
-        }
-      }
-      this.ancestors.removeAll(toRemove);
-    }
-
-    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();
-  }
-
-  /**
-   * TODO: fix it. 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
-    Path manifestFilePath =
-        new Path(HBackupFileSystem.getBackupPath(backupImage.getRootDir(),
-          backupImage.getBackupId()), 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;
-  }
-}


[20/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 0944ea2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.hadoop.hbase.shaded.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/37c65946/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
deleted file mode 100644
index 686d34b..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-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.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
-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 TestBackupRepair extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestBackupRepair.class);
-
-
-  @Test
-  public void testFullBackupWithFailuresAndRestore() throws Exception {
-
-    autoRestoreOnFailure = false;
-
-    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
-      FullTableBackupClientForTest.class.getName());
-    int maxStage = Stage.values().length -1;
-    // Fail stage in loop between 0 and 4 inclusive
-    for (int stage = 0; stage < maxStage; stage++) {
-      LOG.info("Running stage " + stage);
-      runBackupAndFailAtStageWithRestore(stage);
-    }
-  }
-
-  public void runBackupAndFailAtStageWithRestore(int stage) throws Exception {
-
-    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
-    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);
-      assertFalse(ret == 0);
-
-      // Now run restore
-      args = new String[] {"repair"};
-
-      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();
-        assertFalse(checkSucceeded(backupId));
-      }
-      Set<TableName> tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR);
-      assertTrue(tables.size() == 0);
-    }
-  }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 4e922a2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/37c65946/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
deleted file mode 100644
index 73d8d9f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/37c65946/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
deleted file mode 100644
index 5814d87..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
+++ /dev/null
@@ -1,511 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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/37c65946/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
deleted file mode 100644
index 124d19f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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/37c65946/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
deleted file mode 100644
index 4dc894b..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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/37c65946/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
deleted file mode 100644
index 6b007f9..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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/37c65946/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
deleted file mode 100644
index d18de88..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-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.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
-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 TestFullBackupWithFailures extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestFullBackupWithFailures.class);
-
-  @Test
-  public void testFullBackupWithFailures() throws Exception {
-    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
-      FullTableBackupClientForTest.class.getName());
-    int maxStage = Stage.values().length -1;
-    // Fail stages between 0 and 4 inclusive
-    for (int stage = 0; stage <= maxStage; stage++) {
-      LOG.info("Running stage " + stage);
-      runBackupAndFailAtStage(stage);
-    }
-  }
-
-  public void runBackupAndFailAtStage(int stage) throws Exception {
-
-    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
-    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);
-      assertFalse(ret == 0);
-      List<BackupInfo> backups = table.getBackupHistory();
-      int after = table.getBackupHistory().size();
-
-      assertTrue(after ==  before +1);
-      for (BackupInfo data : backups) {
-        String backupId = data.getBackupId();
-        assertFalse(checkSucceeded(backupId));
-      }
-      Set<TableName> tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR);
-      assertTrue(tables.size() == 0);
-    }
-  }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 48a553f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/37c65946/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
deleted file mode 100644
index 73598f3..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-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 org.apache.hadoop.hbase.shaded.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();
-    try {
-      admin.splitRegion(name);
-    } catch (IOException e) {
-      //although split fail, this may not affect following check
-      //In old split without AM2, if region's best split key is not found,
-      //there are not exception thrown. But in current API, exception
-      //will be thrown.
-      LOG.debug("region is not splittable, because " + e);
-    }
-
-    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.getDescriptor());
-    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/37c65946/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
deleted file mode 100644
index 747c1dd..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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();
-  }
-
-}


[10/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
new file mode 100644
index 0000000..7011ed3
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
@@ -0,0 +1,336 @@
+/**
+ * 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.util.BackupUtils.succeeded;
+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.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.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceHFileSplitterJob;
+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.Table;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Pair;
+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 TestIncrementalBackupMergeWithFailures extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupMergeWithFailures.class);
+
+  static enum FailurePhase {
+    PHASE1, PHASE2, PHASE3, PHASE4
+  }
+  public final static String FAILURE_PHASE_KEY = "failurePhase";
+
+  static class BackupMergeJobWithFailures extends MapReduceBackupMergeJob {
+
+    FailurePhase failurePhase;
+
+    @Override
+    public void setConf(Configuration conf) {
+      super.setConf(conf);
+      String val = conf.get(FAILURE_PHASE_KEY);
+      if (val != null) {
+        failurePhase = FailurePhase.valueOf(val);
+      } else {
+        Assert.fail("Failure phase is not set");
+      }
+    }
+
+
+    /**
+     * This is the exact copy of parent's run() with injections
+     * of different types of failures
+     */
+    @Override
+    public void run(String[] backupIds) throws IOException {
+      String bulkOutputConfKey;
+
+      // TODO : run player on remote cluster
+      player = new MapReduceHFileSplitterJob();
+      bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
+      // Player reads all files in arbitrary directory structure and creates
+      // a Map task for each file
+      String bids = StringUtils.join(backupIds, ",");
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Merge backup images " + bids);
+      }
+
+      List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
+      boolean finishedTables = false;
+      Connection conn = ConnectionFactory.createConnection(getConf());
+      BackupSystemTable table = new BackupSystemTable(conn);
+      FileSystem fs = FileSystem.get(getConf());
+
+      try {
+
+        // Start backup exclusive operation
+        table.startBackupExclusiveOperation();
+        // Start merge operation
+        table.startMergeOperation(backupIds);
+
+        // Select most recent backup id
+        String mergedBackupId = findMostRecentBackupId(backupIds);
+
+        TableName[] tableNames = getTableNamesInBackupImages(backupIds);
+        String backupRoot = null;
+
+        BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
+        backupRoot = bInfo.getBackupRootDir();
+        // PHASE 1
+        checkFailure(FailurePhase.PHASE1);
+
+        for (int i = 0; i < tableNames.length; i++) {
+
+          LOG.info("Merge backup images for " + tableNames[i]);
+
+          // Find input directories for table
+
+          Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
+          String dirs = StringUtils.join(dirPaths, ",");
+          Path bulkOutputPath =
+              BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
+                getConf(), false);
+          // Delete content if exists
+          if (fs.exists(bulkOutputPath)) {
+            if (!fs.delete(bulkOutputPath, true)) {
+              LOG.warn("Can not delete: " + bulkOutputPath);
+            }
+          }
+          Configuration conf = getConf();
+          conf.set(bulkOutputConfKey, bulkOutputPath.toString());
+          String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
+
+          int result = 0;
+          // PHASE 2
+          checkFailure(FailurePhase.PHASE2);
+          player.setConf(getConf());
+          result = player.run(playerArgs);
+          if (succeeded(result)) {
+            // Add to processed table list
+            processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
+          } else {
+            throw new IOException("Can not merge backup images for " + dirs
+                + " (check Hadoop/MR and HBase logs). Player return code =" + result);
+          }
+          LOG.debug("Merge Job finished:" + result);
+        }
+        List<TableName> tableList = toTableNameList(processedTableList);
+        // PHASE 3
+        checkFailure(FailurePhase.PHASE3);
+        table.updateProcessedTablesForMerge(tableList);
+        finishedTables = true;
+
+        // Move data
+        for (Pair<TableName, Path> tn : processedTableList) {
+          moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
+        }
+        // PHASE 4
+        checkFailure(FailurePhase.PHASE4);
+        // Delete old data and update manifest
+        List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
+        deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
+        updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
+        // Finish merge session
+        table.finishMergeOperation();
+      } catch (RuntimeException e) {
+        throw e;
+      } catch (Exception e) {
+        LOG.error(e);
+        if (!finishedTables) {
+          // cleanup bulk directories and finish merge
+          // merge MUST be repeated (no need for repair)
+          cleanupBulkLoadDirs(fs, toPathList(processedTableList));
+          table.finishMergeOperation();
+          table.finishBackupExclusiveOperation();
+          throw new IOException("Backup merge operation failed, you should try it again", e);
+        } else {
+          // backup repair must be run
+          throw new IOException(
+              "Backup merge operation failed, run backup repair tool to restore system's integrity",
+              e);
+        }
+      } finally {
+        table.close();
+        conn.close();
+      }
+
+    }
+
+    private void checkFailure(FailurePhase phase) throws IOException {
+      if ( failurePhase != null && failurePhase == phase) {
+        throw new IOException (phase.toString());
+      }
+    }
+
+  }
+
+
+  @Test
+  public void TestIncBackupMergeRestore() 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);
+    // Set custom Merge Job implementation
+    conf1.setClass(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS,
+      BackupMergeJobWithFailures.class, BackupMergeJob.class);
+
+    Connection conn = ConnectionFactory.createConnection(conf1);
+
+    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 table1
+    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);
+    t1.close();
+    LOG.debug("written " + ADD_ROWS + " rows to " + table1);
+
+    HTable t2 = insertIntoTable(conn, table2, famName, 1, ADD_ROWS);
+
+    Assert.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + ADD_ROWS);
+    t2.close();
+    LOG.debug("written " + ADD_ROWS + " rows to " + table2);
+
+    // #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));
+
+    t1 = insertIntoTable(conn, table1, famName, 2, ADD_ROWS);
+    t1.close();
+
+    t2 = insertIntoTable(conn, table2, famName, 2, ADD_ROWS);
+    t2.close();
+
+    // #3 - incremental backup for multiple tables
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdIncMultiple2 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdIncMultiple2));
+
+    // #4 Merge backup images with failures
+
+    for ( FailurePhase phase : FailurePhase.values()) {
+      Configuration conf = conn.getConfiguration();
+
+      conf.set(FAILURE_PHASE_KEY, phase.toString());
+
+      try (BackupAdmin bAdmin = new BackupAdminImpl(conn);)
+      {
+        String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
+        bAdmin.mergeBackups(backups);
+        Assert.fail("Expected IOException");
+      } catch (IOException e) {
+        BackupSystemTable table = new BackupSystemTable(conn);
+        if(phase.ordinal() < FailurePhase.PHASE4.ordinal()) {
+          // No need to repair:
+          // Both Merge and backup exclusive operations are finished
+          assertFalse(table.isMergeInProgress());
+          try {
+            table.finishBackupExclusiveOperation();
+            Assert.fail("IOException is expected");
+          } catch(IOException ee) {
+            // Expected
+          }
+        } else {
+          // Repair is required
+          assertTrue(table.isMergeInProgress());
+          try {
+            table.startBackupExclusiveOperation();
+            Assert.fail("IOException is expected");
+          } catch(IOException ee) {
+            // Expected - clean up before proceeding
+            table.finishMergeOperation();
+            table.finishBackupExclusiveOperation();
+          }
+        }
+        table.close();
+        LOG.debug("Expected :"+ e.getMessage());
+      }
+    }
+
+    // Now merge w/o failures
+    Configuration conf = conn.getConfiguration();
+    conf.unset(FAILURE_PHASE_KEY);
+    conf.unset(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS);
+
+    try (BackupAdmin bAdmin = new BackupAdminImpl(conn);) {
+      String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
+      bAdmin.mergeBackups(backups);
+    }
+
+    // #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));
+
+    Table 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 + 2 * ADD_ROWS);
+
+    hTable.close();
+
+    hTable = conn.getTable(table2_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + 2 * ADD_ROWS);
+    hTable.close();
+
+    admin.close();
+    conn.close();
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
new file mode 100644
index 0000000..769785f
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
@@ -0,0 +1,145 @@
+/**
+ * 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 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.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+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.mapreduce.TestLoadIncrementalHFiles;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+/**
+ * 1. Create table t1
+ * 2. Load data to t1
+ * 3 Full backup t1
+ * 4 Load data to t1
+ * 5 bulk load into t1
+ * 6 Incremental backup t1
+ */
+@Category(LargeTests.class)
+@RunWith(Parameterized.class)
+public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupDeleteTable.class);
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    secure = true;
+    List<Object[]> params = new ArrayList<Object[]>();
+    params.add(new Object[] {Boolean.TRUE});
+    return params;
+  }
+
+  public TestIncrementalBackupWithBulkLoad(Boolean b) {
+  }
+  // implement all test cases in 1 test since incremental backup/restore has dependencies
+  @Test
+  public void TestIncBackupDeleteTable() throws Exception {
+    String testName = "TestIncBackupDeleteTable";
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+
+    List<TableName> tables = Lists.newArrayList(table1);
+    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();
+
+    int NB_ROWS2 = 20;
+    LOG.debug("bulk loading into " + testName);
+    int actual = TestLoadIncrementalHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
+        qualName, false, null, new byte[][][] {
+      new byte[][]{ Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
+      new byte[][]{ Bytes.toBytes("ddd"), Bytes.toBytes("ooo") },
+    }, true, false, true, NB_ROWS_IN_BATCH*2, NB_ROWS2);
+
+    // #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));
+
+    // #5.1 - check tables for full restore */
+    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+
+    // #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 = (HTable) conn.getTable(table1_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH * 2+actual);
+    request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+
+    backupIdFull = client.backupTables(request);
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>> pair
+      = table.readBulkloadRows(tables);
+      assertTrue("map still has " + pair.getSecond().size() + " entries",
+          pair.getSecond().isEmpty());
+    }
+    assertTrue(checkSucceeded(backupIdFull));
+
+    hTable.close();
+    admin.close();
+    conn.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
new file mode 100644
index 0000000..84a596e
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.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.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.util.ToolRunner;
+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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+@RunWith(Parameterized.class)
+public class TestIncrementalBackupWithFailures extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupWithFailures.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 TestIncrementalBackupWithFailures(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);
+
+    // #3 - incremental backup for multiple tables
+    incrementalBackupWithFailures();
+
+    admin.close();
+    conn.close();
+
+  }
+
+
+  private void incrementalBackupWithFailures() throws Exception {
+    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
+      IncrementalTableBackupClientForTest.class.getName());
+    int maxStage = Stage.values().length -1;
+    // Fail stages between 0 and 4 inclusive
+    for (int stage = 0; stage <= maxStage; stage++) {
+      LOG.info("Running stage " + stage);
+      runBackupAndFailAtStage(stage);
+    }
+  }
+
+  private void runBackupAndFailAtStage(int stage) throws Exception {
+
+    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
+    try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
+      int before = table.getBackupHistory().size();
+      String[] args =
+          new String[] { "create", "incremental", BACKUP_ROOT_DIR, "-t",
+              table1.getNameAsString() + "," + table2.getNameAsString() };
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertFalse(ret == 0);
+      List<BackupInfo> backups = table.getBackupHistory();
+      int after = table.getBackupHistory().size();
+
+      assertTrue(after ==  before +1);
+      for (BackupInfo data : backups) {
+        if(data.getType() == BackupType.FULL) {
+          assertTrue(data.getState() == BackupState.COMPLETE);
+        } else {
+          assertTrue(data.getState() == BackupState.FAILED);
+        }
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
new file mode 100644
index 0000000..36a9ee2
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.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 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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestRemoteBackup extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRemoteBackup.class);
+
+  @Override
+  public void setUp () throws Exception {
+    useSecondCluster = true;
+    super.setUp();
+  }
+
+  /**
+   * 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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
new file mode 100644
index 0000000..0386c27
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.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 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);
+
+  @Override
+  public void setUp () throws Exception {
+    useSecondCluster = true;
+    super.setUp();
+  }
+
+
+  /**
+   * 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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
new file mode 100644
index 0000000..556521f
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
@@ -0,0 +1,93 @@
+/**
+ * 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.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.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestRepairAfterFailedDelete extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRepairAfterFailedDelete.class);
+
+  @Test
+  public void testRepairBackupDelete() throws Exception {
+    LOG.info("test repair 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));
+
+    // Snapshot backup system table before delete
+    String snapshotName = "snapshot-backup";
+    Connection conn = TEST_UTIL.getConnection();
+    Admin admin = conn.getAdmin();
+    admin.snapshot(snapshotName, BackupSystemTable.getTableName(conf1));
+
+    int deleted = getBackupAdmin().deleteBackups(backupIds);
+
+    assertTrue(!fs.exists(path));
+    assertTrue(fs.exists(new Path(info.getBackupRootDir())));
+    assertTrue(1 == deleted);
+
+    // Emulate delete failure
+    // Restore backup system table
+    admin.disableTable(BackupSystemTable.getTableName(conf1));
+    admin.restoreSnapshot(snapshotName);
+    admin.enableTable(BackupSystemTable.getTableName(conf1));
+    // Start backup session
+    table.startBackupExclusiveOperation();
+    // Start delete operation
+    table.startDeleteOperation(backupIds);
+
+    // Now run repair command to repair "failed" delete operation
+    String[] args = new String[] {"repair"};
+    // Run restore
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    // Verify that history length == 0
+    assertTrue (table.getBackupHistory().size() == 0);
+    table.close();
+    admin.close();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
new file mode 100644
index 0000000..c61b018
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
new file mode 100644
index 0000000..6443421
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.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 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 backup system table snapshot
+   * @throws Exception
+   */
+ // @Test
+  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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
new file mode 100644
index 0000000..5f72f45
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
+import org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-backup/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/resources/log4j.properties b/hbase-backup/src/test/resources/log4j.properties
new file mode 100644
index 0000000..c322699
--- /dev/null
+++ b/hbase-backup/src/test/resources/log4j.properties
@@ -0,0 +1,68 @@
+# 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.
+
+# Define some default values that can be overridden by system properties
+hbase.root.logger=INFO,console
+hbase.log.dir=.
+hbase.log.file=hbase.log
+
+# Define the root logger to the system property "hbase.root.logger".
+log4j.rootLogger=${hbase.root.logger}
+
+# Logging Threshold
+log4j.threshold=ALL
+
+#
+# Daily Rolling File Appender
+#
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hbase.log.dir}/${hbase.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+# Debugging Pattern format
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L): %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L): %m%n
+
+# Custom Logging levels
+
+#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.zookeeper=ERROR
+log4j.logger.org.apache.hadoop.hbase=DEBUG
+
+#These settings are workarounds against spurious logs from the minicluster.
+#See HBASE-4709
+log4j.logger.org.apache.hadoop.metrics2.impl.MetricsConfig=WARN
+log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSinkAdapter=WARN
+log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=WARN
+log4j.logger.org.apache.hadoop.metrics2.util.MBeans=WARN
+# Enable this to get detailed connection error/retry logging.
+# log4j.logger.org.apache.hadoop.hbase.client.ConnectionImplementation=TRACE

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-it/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index 783d9a2..b9b12a4 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -232,6 +232,11 @@
       <artifactId>hbase-testing-util</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-backup</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hbase.thirdparty</groupId>
       <artifactId>hbase-shaded-miscellaneous</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 9dc6382..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.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 array of backup ids
-   * @return total number of deleted sessions
-   * @throws IOException exception
-   */
-  int deleteBackups(String[] backupIds) throws IOException;
-
-  /**
-   * Merge backup images command
-   * @param backupIds array of backup ids of images to be merged
-   *        The resulting backup image will have the same backup id as the most
-   *        recent image from a list of images to be merged
-   * @throws IOException exception
-   */
-  void mergeBackups(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 array 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 array 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/2dda3712/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
deleted file mode 100644
index 21d73cc..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.backup.impl.FullTableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
-import org.apache.hadoop.hbase.client.Connection;
-
-public class BackupClientFactory {
-
-  public static TableBackupClient create (Connection conn, String backupId, BackupRequest request)
-    throws IOException
-  {
-    Configuration conf = conn.getConfiguration();
-    try {
-      String clsName = conf.get(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS);
-      if (clsName != null) {
-        Class<?> clientImpl = Class.forName(clsName);
-        TableBackupClient client = (TableBackupClient) clientImpl.newInstance();
-        client.init(conn, backupId, request);
-        return client;
-      }
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-
-    BackupType type = request.getBackupType();
-    if (type == BackupType.FULL) {
-      return new FullTableBackupClient(conn, backupId, request);
-    } else {
-      return new IncrementalTableBackupClient(conn, backupId, request);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 007e4c1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import 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/2dda3712/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
deleted file mode 100644
index 9dd8531..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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 if (BackupCommand.REPAIR.name().equalsIgnoreCase(cmd)) {
-      type = BackupCommand.REPAIR;
-    } else if (BackupCommand.MERGE.name().equalsIgnoreCase(cmd)) {
-      type = BackupCommand.MERGE;
-    } 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;
-    } finally {
-      command.finish();
-    }
-    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/2dda3712/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
deleted file mode 100644
index ed554ad..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-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.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.TableName;
-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.cleaner.BaseHFileCleanerDelegate;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
-
-/**
- * Implementation of a file cleaner that checks if an hfile is still referenced by backup before
- * deleting it from hfile archive directory.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class BackupHFileCleaner extends BaseHFileCleanerDelegate implements Abortable {
-  private static final Log LOG = LogFactory.getLog(BackupHFileCleaner.class);
-  private boolean stopped = false;
-  private boolean aborted;
-  private Configuration conf;
-  private Connection connection;
-  private long prevReadFromBackupTbl = 0, // timestamp of most recent read from backup:system table
-      secondPrevReadFromBackupTbl = 0; // timestamp of 2nd most recent read from backup:system table
-  //used by unit test to skip reading backup:system
-  private boolean checkForFullyBackedUpTables = true;
-  private List<TableName> fullyBackedUpTables = null;
-
-  private Set<String> getFilenameFromBulkLoad(Map<byte[], List<Path>>[] maps) {
-    Set<String> filenames = new HashSet<String>();
-    for (Map<byte[], List<Path>> map : maps) {
-      if (map == null) continue;
-      for (List<Path> paths : map.values()) {
-        for (Path p : paths) {
-          filenames.add(p.getName());
-        }
-      }
-    }
-    return filenames;
-  }
-
-  private Set<String> loadHFileRefs(List<TableName> tableList) throws IOException {
-    if (connection == null) {
-      connection = ConnectionFactory.createConnection(conf);
-    }
-    try (BackupSystemTable tbl = new BackupSystemTable(connection)) {
-      Map<byte[], List<Path>>[] res =
-          tbl.readBulkLoadedFiles(null, tableList);
-      secondPrevReadFromBackupTbl = prevReadFromBackupTbl;
-      prevReadFromBackupTbl = EnvironmentEdgeManager.currentTime();
-      return getFilenameFromBulkLoad(res);
-    }
-  }
-
-  @VisibleForTesting
-  void setCheckForFullyBackedUpTables(boolean b) {
-    checkForFullyBackedUpTables = b;
-  }
-  @Override
-  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
-    if (conf == null) {
-      return files;
-    }
-    // obtain the Set of TableName's which have been fully backed up
-    // so that we filter BulkLoad to be returned from server
-    if (checkForFullyBackedUpTables) {
-      if (connection == null) return files;
-      try (BackupSystemTable tbl = new BackupSystemTable(connection)) {
-        fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      } catch (IOException ioe) {
-        LOG.error("Failed to get tables which have been fully backed up, skipping checking", ioe);
-        return Collections.emptyList();
-      }
-      Collections.sort(fullyBackedUpTables);
-    }
-    final Set<String> hfileRefs;
-    try {
-      hfileRefs = loadHFileRefs(fullyBackedUpTables);
-    } catch (IOException ioe) {
-      LOG.error("Failed to read hfile references, skipping checking deletable files", ioe);
-      return Collections.emptyList();
-    }
-    Iterable<FileStatus> deletables = Iterables.filter(files, new Predicate<FileStatus>() {
-      @Override
-      public boolean apply(FileStatus file) {
-        // If the file is recent, be conservative and wait for one more scan of backup:system table
-        if (file.getModificationTime() > secondPrevReadFromBackupTbl) {
-          return false;
-        }
-        String hfile = file.getPath().getName();
-        boolean foundHFileRef = hfileRefs.contains(hfile);
-        return !foundHFileRef;
-      }
-    });
-    return deletables;
-  }
-
-  @Override
-  public boolean isFileDeletable(FileStatus fStat) {
-    // work is done in getDeletableFiles()
-    return true;
-  }
-
-  @Override
-  public void setConf(Configuration config) {
-    this.conf = config;
-    this.connection = null;
-    try {
-      this.connection = ConnectionFactory.createConnection(conf);
-    } catch (IOException ioe) {
-      LOG.error("Couldn't establish connection", ioe);
-    }
-  }
-
-  @Override
-  public void stop(String why) {
-    if (this.stopped) {
-      return;
-    }
-    if (this.connection != null) {
-      try {
-        this.connection.close();
-      } catch (IOException ioe) {
-        LOG.debug("Got " + ioe + " when closing connection");
-      }
-    }
-    this.stopped = true;
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped;
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    LOG.warn("Aborting ReplicationHFileCleaner because " + why, e);
-    this.aborted = true;
-    stop(why);
-  }
-
-  @Override
-  public boolean isAborted() {
-    return this.aborted;
-  }
-}


[04/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
deleted file mode 100644
index 49e8c75..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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 MapReduceHFileSplitterJob extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(MapReduceHFileSplitterJob.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 MapReduceHFileSplitterJob() {
-  }
-
-  protected MapReduceHFileSplitterJob(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);
-    conf.set(FileInputFormat.INPUT_DIR, inputDirs);
-    Job job =
-        Job.getInstance(conf,
-          conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime()));
-    job.setJarByClass(MapReduceHFileSplitterJob.class);
-    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.getDescriptor(), regionLocator);
-      }
-      LOG.debug("success configuring load incremental job");
-
-      TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
-        org.apache.hadoop.hbase.shaded.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 MapReduceHFileSplitterJob(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/2dda3712/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
deleted file mode 100644
index 1209e7c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreJob.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.mapreduce;
-
-import static org.apache.hadoop.hbase.backup.util.BackupUtils.failed;
-import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
-
-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.Path;
-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.backup.util.BackupUtils;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.util.Tool;
-
-
-/**
- * MapReduce implementation of {@link RestoreJob}
- *
- * For backup restore, it runs {@link MapReduceHFileSplitterJob} job and creates
- * HFiles which are aligned with a region boundaries of a table being
- * restored.
- *
- * 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;
-
-    player = new MapReduceHFileSplitterJob();
-    bulkOutputConfKey = MapReduceHFileSplitterJob.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 =
-          BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(newTableNames[i]),
-            getConf());
-      Configuration conf = getConf();
-      conf.set(bulkOutputConfKey, bulkOutputPath.toString());
-      String[] playerArgs =
-          {
-              dirs,
-              fullBackupRestore ? newTableNames[i].getNameAsString() : 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 = BackupUtils.createLoader(getConf());
-          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) {
-        LOG.error(e);
-        throw new IOException("Can not restore from backup directory " + dirs
-            + " (check Hadoop and HBase logs) ", e);
-      }
-    }
-  }
-
-  @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/2dda3712/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
deleted file mode 100644
index b5b887c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 47e428c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/2dda3712/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
deleted file mode 100644
index 8fc644c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/2dda3712/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
deleted file mode 100644
index 65a1fa3..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/2dda3712/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
deleted file mode 100644
index 9d5a858..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/2dda3712/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
deleted file mode 100644
index 0da6fc4..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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/2dda3712/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
deleted file mode 100644
index ce77645..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
+++ /dev/null
@@ -1,747 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-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 = ".";
-  public static final int MILLISEC_IN_HOUR = 3600000;
-
-  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;
-  }
-
-  public static Path getBulkOutputDir(String tableName, Configuration conf, boolean deleteOnExit)
-      throws IOException {
-    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());
-    if (deleteOnExit) {
-      fs.deleteOnExit(path);
-    }
-    return path;
-  }
-
-  public static Path getBulkOutputDir(String tableName, Configuration conf) throws IOException {
-    return getBulkOutputDir(tableName, conf, true);
-  }
-
-  public static String getFileNameCompatibleString(TableName table) {
-    return table.getNamespaceAsString() + "-" + table.getQualifierAsString();
-  }
-
-  public static boolean failed(int result) {
-    return result != 0;
-  }
-
-  public static boolean succeeded(int result) {
-    return result == 0;
-  }
-
-  public static LoadIncrementalHFiles createLoader(Configuration config) throws IOException {
-    // set configuration for restore:
-    // LoadIncrementalHFile needs more time
-    // <name>hbase.rpc.timeout</name> <value>600000</value>
-    // calculates
-    Configuration conf = new Configuration(config);
-    conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, MILLISEC_IN_HOUR);
-
-    // 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;
-  }
-}


[24/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 4dab046..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
+++ /dev/null
@@ -1,2051 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-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.CellComparator;
-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.NamespaceDescriptor;
-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.BackupType;
-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.SnapshotDescription;
-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;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * 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 {
-  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
-
-  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 TableName tableName;
-  /**
-   * Stores backup sessions (contexts)
-   */
-  final static byte[] SESSIONS_FAMILY = "session".getBytes();
-  /**
-   * Stores other meta
-   */
-  final static byte[] META_FAMILY = "meta".getBytes();
-  final static byte[] BULK_LOAD_FAMILY = "bulk".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 byte[] ACTIVE_SESSION_ROW = "activesession:".getBytes();
-  private final static byte[] ACTIVE_SESSION_COL = "c".getBytes();
-
-  private final static byte[] ACTIVE_SESSION_YES = "yes".getBytes();
-  private final static byte[] ACTIVE_SESSION_NO = "no".getBytes();
-
-  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 BULK_LOAD_PREFIX = "bulk:";
-  private final static byte[] BULK_LOAD_PREFIX_BYTES = BULK_LOAD_PREFIX.getBytes();
-  private final static byte[] DELETE_OP_ROW = "delete_op_row".getBytes();
-  private final static byte[] MERGE_OP_ROW = "merge_op_row".getBytes();
-
-  final static byte[] TBL_COL = Bytes.toBytes("tbl");
-  final static byte[] FAM_COL = Bytes.toBytes("fam");
-  final static byte[] PATH_COL = Bytes.toBytes("path");
-  final static byte[] STATE_COL = Bytes.toBytes("state");
-  // the two states a bulk loaded file can be
-  final static byte[] BL_PREPARE = Bytes.toBytes("R");
-  final static byte[] BL_COMMIT = Bytes.toBytes("D");
-
-  private final static String WALS_PREFIX = "wals:";
-  private final static String SET_KEY_PREFIX = "backupset:";
-
-  // separator between BULK_LOAD_PREFIX and ordinals
-  protected final static String BLK_LD_DELIM = ":";
-  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();) {
-
-      verifyNamespaceExists(admin);
-
-      if (!admin.tableExists(tableName)) {
-        HTableDescriptor backupHTD =
-            BackupSystemTable.getSystemTableDescriptor(connection.getConfiguration());
-        admin.createTable(backupHTD);
-      }
-      waitForSystemTable(admin);
-    }
-  }
-
-  private void verifyNamespaceExists(Admin admin) throws IOException {
-    String namespaceName = tableName.getNamespaceAsString();
-    NamespaceDescriptor ns = NamespaceDescriptor.create(namespaceName).build();
-    NamespaceDescriptor[] list = admin.listNamespaceDescriptors();
-    boolean exists = false;
-    for (NamespaceDescriptor nsd : list) {
-      if (nsd.getName().equals(ns.getName())) {
-        exists = true;
-        break;
-      }
-    }
-    if (!exists) {
-      admin.createNamespace(ns);
-    }
-  }
-
-  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);
-    }
-  }
-
-  /*
-   * @param backupId the backup Id
-   * @return Map of rows to path of bulk loaded hfile
-   */
-  Map<byte[], String> readBulkLoadedFiles(String backupId) throws IOException {
-    Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId);
-    try (Table table = connection.getTable(tableName);
-        ResultScanner scanner = table.getScanner(scan)) {
-      Result res = null;
-      Map<byte[], String> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        byte[] row = CellUtil.cloneRow(res.listCells().get(0));
-        for (Cell cell : res.listCells()) {
-          if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
-            BackupSystemTable.PATH_COL.length) == 0) {
-            map.put(row, Bytes.toString(CellUtil.cloneValue(cell)));
-          }
-        }
-      }
-      return map;
-    }
-  }
-
-  /*
-   * Used during restore
-   * @param backupId the backup Id
-   * @param sTableList List of tables
-   * @return array of Map of family to List of Paths
-   */
-  public Map<byte[], List<Path>>[] readBulkLoadedFiles(String backupId, List<TableName> sTableList)
-      throws IOException {
-    Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId);
-    Map<byte[], List<Path>>[] mapForSrc = new Map[sTableList == null ? 1 : sTableList.size()];
-    try (Table table = connection.getTable(tableName);
-        ResultScanner scanner = table.getScanner(scan)) {
-      Result res = null;
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        TableName tbl = null;
-        byte[] fam = null;
-        String path = null;
-        for (Cell cell : res.listCells()) {
-          if (CellComparator.compareQualifiers(cell, BackupSystemTable.TBL_COL, 0,
-            BackupSystemTable.TBL_COL.length) == 0) {
-            tbl = TableName.valueOf(CellUtil.cloneValue(cell));
-          } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
-            BackupSystemTable.FAM_COL.length) == 0) {
-            fam = CellUtil.cloneValue(cell);
-          } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
-            BackupSystemTable.PATH_COL.length) == 0) {
-            path = Bytes.toString(CellUtil.cloneValue(cell));
-          }
-        }
-        int srcIdx = IncrementalTableBackupClient.getIndex(tbl, sTableList);
-        if (srcIdx == -1) {
-          // the table is not among the query
-          continue;
-        }
-        if (mapForSrc[srcIdx] == null) {
-          mapForSrc[srcIdx] = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-        }
-        List<Path> files;
-        if (!mapForSrc[srcIdx].containsKey(fam)) {
-          files = new ArrayList<Path>();
-          mapForSrc[srcIdx].put(fam, files);
-        } else {
-          files = mapForSrc[srcIdx].get(fam);
-        }
-        files.add(new Path(path));
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("found bulk loaded file : " + tbl + " " + Bytes.toString(fam) + " " + path);
-        }
-      }
-      ;
-      return mapForSrc;
-    }
-  }
-
-  /*
-   * @param map Map of row keys to path of bulk loaded hfile
-   */
-  void deleteBulkLoadedFiles(Map<byte[], String> map) throws IOException {
-    try (Table table = connection.getTable(tableName)) {
-      List<Delete> dels = new ArrayList<>();
-      for (byte[] row : map.keySet()) {
-        dels.add(new Delete(row).addFamily(BackupSystemTable.META_FAMILY));
-      }
-      table.delete(dels);
-    }
-  }
-
-  /**
-   * 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);
-    }
-  }
-
-  /*
-   * For postBulkLoadHFile() hook.
-   * @param tabName table name
-   * @param region the region receiving hfile
-   * @param finalPaths family and associated hfiles
-   */
-  public void writePathsPostBulkLoad(TableName tabName, byte[] region,
-      Map<byte[], List<Path>> finalPaths) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + finalPaths.size()
-          + " entries");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      List<Put> puts = BackupSystemTable.createPutForCommittedBulkload(tabName, region, finalPaths);
-      table.put(puts);
-      LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
-    }
-  }
-
-  /*
-   * For preCommitStoreFile() hook
-   * @param tabName table name
-   * @param region the region receiving hfile
-   * @param family column family
-   * @param pairs list of paths for hfiles
-   */
-  public void writeFilesForBulkLoadPreCommit(TableName tabName, byte[] region, final byte[] family,
-      final List<Pair<Path, Path>> pairs) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + pairs.size()
-          + " entries");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      List<Put> puts =
-          BackupSystemTable.createPutForPreparedBulkload(tabName, region, family, pairs);
-      table.put(puts);
-      LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
-    }
-  }
-
-  /*
-   * Removes rows recording bulk loaded hfiles from backup table
-   * @param lst list of table names
-   * @param rows the rows to be deleted
-   */
-  public void removeBulkLoadedRows(List<TableName> lst, List<byte[]> rows) throws IOException {
-    try (Table table = connection.getTable(tableName)) {
-      List<Delete> lstDels = new ArrayList<>();
-      for (byte[] row : rows) {
-        Delete del = new Delete(row);
-        lstDels.add(del);
-        LOG.debug("orig deleting the row: " + Bytes.toString(row));
-      }
-      table.delete(lstDels);
-      LOG.debug("deleted " + rows.size() + " original bulkload rows for " + lst.size() + " tables");
-    }
-  }
-
-  /*
-   * Reads the rows from backup table recording bulk loaded hfiles
-   * @param tableList list of table names
-   * @return The keys of the Map are table, region and column family. Value of the map reflects
-   * whether the hfile was recorded by preCommitStoreFile hook (true)
-   */
-  public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
-      readBulkloadRows(List<TableName> tableList) throws IOException {
-    Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> map = new HashMap<>();
-    List<byte[]> rows = new ArrayList<>();
-    for (TableName tTable : tableList) {
-      Scan scan = BackupSystemTable.createScanForOrigBulkLoadedFiles(tTable);
-      Map<String, Map<String, List<Pair<String, Boolean>>>> tblMap = map.get(tTable);
-      try (Table table = connection.getTable(tableName);
-          ResultScanner scanner = table.getScanner(scan)) {
-        Result res = null;
-        while ((res = scanner.next()) != null) {
-          res.advance();
-          String fam = null;
-          String path = null;
-          boolean raw = false;
-          byte[] row = null;
-          String region = null;
-          for (Cell cell : res.listCells()) {
-            row = CellUtil.cloneRow(cell);
-            rows.add(row);
-            String rowStr = Bytes.toString(row);
-            region = BackupSystemTable.getRegionNameFromOrigBulkLoadRow(rowStr);
-            if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
-              BackupSystemTable.FAM_COL.length) == 0) {
-              fam = Bytes.toString(CellUtil.cloneValue(cell));
-            } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
-              BackupSystemTable.PATH_COL.length) == 0) {
-              path = Bytes.toString(CellUtil.cloneValue(cell));
-            } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.STATE_COL, 0,
-              BackupSystemTable.STATE_COL.length) == 0) {
-              byte[] state = CellUtil.cloneValue(cell);
-              if (Bytes.equals(BackupSystemTable.BL_PREPARE, state)) {
-                raw = true;
-              } else raw = false;
-            }
-          }
-          if (map.get(tTable) == null) {
-            map.put(tTable, new HashMap<String, Map<String, List<Pair<String, Boolean>>>>());
-            tblMap = map.get(tTable);
-          }
-          if (tblMap.get(region) == null) {
-            tblMap.put(region, new HashMap<String, List<Pair<String, Boolean>>>());
-          }
-          Map<String, List<Pair<String, Boolean>>> famMap = tblMap.get(region);
-          if (famMap.get(fam) == null) {
-            famMap.put(fam, new ArrayList<Pair<String, Boolean>>());
-          }
-          famMap.get(fam).add(new Pair<>(path, raw));
-          LOG.debug("found orig " + path + " for " + fam + " of table " + region);
-        }
-      }
-    }
-    return new Pair<>(map, rows);
-  }
-
-  /*
-   * @param sTableList List of tables
-   * @param maps array of Map of family to List of Paths
-   * @param backupId the backup Id
-   */
-  public void writeBulkLoadedFiles(List<TableName> sTableList, Map<byte[], List<Path>>[] maps,
-      String backupId) throws IOException {
-    try (Table table = connection.getTable(tableName)) {
-      long ts = EnvironmentEdgeManager.currentTime();
-      int cnt = 0;
-      List<Put> puts = new ArrayList<>();
-      for (int idx = 0; idx < maps.length; idx++) {
-        Map<byte[], List<Path>> map = maps[idx];
-        TableName tn = sTableList.get(idx);
-        if (map == null) continue;
-        for (Map.Entry<byte[], List<Path>> entry : map.entrySet()) {
-          byte[] fam = entry.getKey();
-          List<Path> paths = entry.getValue();
-          for (Path p : paths) {
-            Put put =
-                BackupSystemTable.createPutForBulkLoadedFile(tn, fam, p.toString(), backupId, ts,
-                  cnt++);
-            puts.add(put);
-          }
-        }
-      }
-      if (!puts.isEmpty()) {
-        table.put(puts);
-      }
-    }
-  }
-
-  /**
-   * 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);
-    }
-  }
-
-  /**
-   * Exclusive operations are:
-   * create, delete, merge
-   * @throws IOException
-   */
-  public void startBackupExclusiveOperation() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Start new backup exclusive operation");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Put put = createPutForStartBackupSession();
-      // First try to put if row does not exist
-      if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL, null, put)) {
-        // Row exists, try to put if value == ACTIVE_SESSION_NO
-        if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
-          ACTIVE_SESSION_NO, put)) {
-          throw new IOException("There is an active backup exclusive operation");
-        }
-      }
-    }
-  }
-
-  private Put createPutForStartBackupSession() {
-    Put put = new Put(ACTIVE_SESSION_ROW);
-    put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_YES);
-    return put;
-  }
-
-  public void finishBackupExclusiveOperation() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Finish backup exclusive operation");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Put put = createPutForStopBackupSession();
-      if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
-        ACTIVE_SESSION_YES, put)) {
-        throw new IOException("There is no active backup exclusive operation");
-      }
-    }
-  }
-
-  private Put createPutForStopBackupSession() {
-    Put put = new Put(ACTIVE_SESSION_ROW);
-    put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_NO);
-    return 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, if n== -1 - max number
-   *        is ignored
-   * @return list of records
-   * @throws IOException
-   */
-  public List<BackupInfo> getHistory(int n) throws IOException {
-
-    List<BackupInfo> history = getBackupHistory();
-    if (n == -1 || 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, if n == -1 , then max number
-   *        is ignored
-   * @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 (n >= 0 && 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;
-
-  }
-
-  /*
-   * Retrieve TableName's for completed backup of given type
-   * @param type backup type
-   * @return List of table names
-   */
-  public List<TableName> getTablesForBackupType(BackupType type) throws IOException {
-    Set<TableName> names = new HashSet<>();
-    List<BackupInfo> infos = getBackupHistory(true);
-    for (BackupInfo info : infos) {
-      if (info.getType() != type) continue;
-      names.addAll(info.getTableNames());
-    }
-    return new ArrayList(names);
-  }
-
-  /**
-   * 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
-   * TODO: multiple backup destination support
-   */
-  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();
-  }
-
-  public static String getSnapshotName(Configuration conf) {
-    return "snapshot_" + getTableNameAsString(conf).replace(":", "_");
-  }
-
-  /**
-   * 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's for bulk load resulting from running LoadIncrementalHFiles
-   */
-  static List<Put> createPutForCommittedBulkload(TableName table, byte[] region,
-      Map<byte[], List<Path>> finalPaths) {
-    List<Put> puts = new ArrayList<>();
-    for (Map.Entry<byte[], List<Path>> entry : finalPaths.entrySet()) {
-      for (Path path : entry.getValue()) {
-        String file = path.toString();
-        int lastSlash = file.lastIndexOf("/");
-        String filename = file.substring(lastSlash + 1);
-        Put put =
-            new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM,
-              Bytes.toString(region), BLK_LD_DELIM, filename));
-        put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
-        put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, entry.getKey());
-        put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
-        put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_COMMIT);
-        puts.add(put);
-        LOG.debug("writing done bulk path " + file + " for " + table + " " + Bytes.toString(region));
-      }
-    }
-    return puts;
-  }
-
-  public static void snapshot(Connection conn) throws IOException {
-
-    try (Admin admin = conn.getAdmin();) {
-      Configuration conf = conn.getConfiguration();
-      admin.snapshot(BackupSystemTable.getSnapshotName(conf), BackupSystemTable.getTableName(conf));
-    }
-  }
-
-  public static void restoreFromSnapshot(Connection conn) throws IOException {
-
-    Configuration conf = conn.getConfiguration();
-    LOG.debug("Restoring " + BackupSystemTable.getTableNameAsString(conf) + " from snapshot");
-    try (Admin admin = conn.getAdmin();) {
-      String snapshotName = BackupSystemTable.getSnapshotName(conf);
-      if (snapshotExists(admin, snapshotName)) {
-        admin.disableTable(BackupSystemTable.getTableName(conf));
-        admin.restoreSnapshot(snapshotName);
-        admin.enableTable(BackupSystemTable.getTableName(conf));
-        LOG.debug("Done restoring backup system table");
-      } else {
-        // Snapshot does not exists, i.e completeBackup failed after
-        // deleting backup system table snapshot
-        // In this case we log WARN and proceed
-        LOG.warn("Could not restore backup system table. Snapshot " + snapshotName
-            + " does not exists.");
-      }
-    }
-  }
-
-  protected static boolean snapshotExists(Admin admin, String snapshotName) throws IOException {
-
-    List<SnapshotDescription> list = admin.listSnapshots();
-    for (SnapshotDescription desc : list) {
-      if (desc.getName().equals(snapshotName)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public static boolean snapshotExists(Connection conn) throws IOException {
-    return snapshotExists(conn.getAdmin(), getSnapshotName(conn.getConfiguration()));
-  }
-
-  public static void deleteSnapshot(Connection conn) throws IOException {
-
-    Configuration conf = conn.getConfiguration();
-    LOG.debug("Deleting " + BackupSystemTable.getSnapshotName(conf) + " from the system");
-    try (Admin admin = conn.getAdmin();) {
-      String snapshotName = BackupSystemTable.getSnapshotName(conf);
-      if (snapshotExists(admin, snapshotName)) {
-        admin.deleteSnapshot(snapshotName);
-        LOG.debug("Done deleting backup system table snapshot");
-      } else {
-        LOG.error("Snapshot " + snapshotName + " does not exists");
-      }
-    }
-  }
-
-  /*
-   * Creates Put's for bulk load resulting from running LoadIncrementalHFiles
-   */
-  static List<Put> createPutForPreparedBulkload(TableName table, byte[] region,
-      final byte[] family, final List<Pair<Path, Path>> pairs) {
-    List<Put> puts = new ArrayList<>();
-    for (Pair<Path, Path> pair : pairs) {
-      Path path = pair.getSecond();
-      String file = path.toString();
-      int lastSlash = file.lastIndexOf("/");
-      String filename = file.substring(lastSlash + 1);
-      Put put =
-          new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM, Bytes.toString(region),
-            BLK_LD_DELIM, filename));
-      put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
-      put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, family);
-      put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
-      put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_PREPARE);
-      puts.add(put);
-      LOG.debug("writing raw bulk path " + file + " for " + table + " " + Bytes.toString(region));
-    }
-    return puts;
-  }
-
-  public static List<Delete> createDeleteForOrigBulkLoad(List<TableName> lst) {
-    List<Delete> lstDels = new ArrayList<>();
-    for (TableName table : lst) {
-      Delete del = new Delete(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM));
-      del.addFamily(BackupSystemTable.META_FAMILY);
-      lstDels.add(del);
-    }
-    return lstDels;
-  }
-
-  private Put createPutForDeleteOperation(String[] backupIdList) {
-
-    byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ","));
-    Put put = new Put(DELETE_OP_ROW);
-    put.addColumn(META_FAMILY, FAM_COL, value);
-    return put;
-  }
-
-  private Delete createDeleteForBackupDeleteOperation() {
-
-    Delete delete = new Delete(DELETE_OP_ROW);
-    delete.addFamily(META_FAMILY);
-    return delete;
-  }
-
-  private Get createGetForDeleteOperation() {
-
-    Get get = new Get(DELETE_OP_ROW);
-    get.addFamily(META_FAMILY);
-    return get;
-  }
-
-  public void startDeleteOperation(String[] backupIdList) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Start delete operation for backups: " + StringUtils.join(backupIdList));
-    }
-    Put put = createPutForDeleteOperation(backupIdList);
-    try (Table table = connection.getTable(tableName)) {
-      table.put(put);
-    }
-  }
-
-  public void finishDeleteOperation() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Finsih delete operation for backup ids ");
-    }
-    Delete delete = createDeleteForBackupDeleteOperation();
-    try (Table table = connection.getTable(tableName)) {
-      table.delete(delete);
-    }
-  }
-
-  public String[] getListOfBackupIdsFromDeleteOperation() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Get delete operation for backup ids ");
-    }
-    Get get = createGetForDeleteOperation();
-    try (Table table = connection.getTable(tableName)) {
-      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).split(",");
-    }
-  }
-
-  private Put createPutForMergeOperation(String[] backupIdList) {
-
-    byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ","));
-    Put put = new Put(MERGE_OP_ROW);
-    put.addColumn(META_FAMILY, FAM_COL, value);
-    return put;
-  }
-
-  public boolean isMergeInProgress() throws IOException {
-    Get get = new Get(MERGE_OP_ROW);
-    try (Table table = connection.getTable(tableName)) {
-      Result res = table.get(get);
-      if (res.isEmpty()) {
-        return false;
-      }
-      return true;
-    }
-  }
-
-  private Put createPutForUpdateTablesForMerge(List<TableName> tables) {
-
-    byte[] value = Bytes.toBytes(StringUtils.join(tables, ","));
-    Put put = new Put(MERGE_OP_ROW);
-    put.addColumn(META_FAMILY, PATH_COL, value);
-    return put;
-  }
-
-  private Delete createDeleteForBackupMergeOperation() {
-
-    Delete delete = new Delete(MERGE_OP_ROW);
-    delete.addFamily(META_FAMILY);
-    return delete;
-  }
-
-  private Get createGetForMergeOperation() {
-
-    Get get = new Get(MERGE_OP_ROW);
-    get.addFamily(META_FAMILY);
-    return get;
-  }
-
-  public void startMergeOperation(String[] backupIdList) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Start merge operation for backups: " + StringUtils.join(backupIdList));
-    }
-    Put put = createPutForMergeOperation(backupIdList);
-    try (Table table = connection.getTable(tableName)) {
-      table.put(put);
-    }
-  }
-
-  public void updateProcessedTablesForMerge(List<TableName> tables) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Update tables for merge : " + StringUtils.join(tables, ","));
-    }
-    Put put = createPutForUpdateTablesForMerge(tables);
-    try (Table table = connection.getTable(tableName)) {
-      table.put(put);
-    }
-  }
-
-  public void finishMergeOperation() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Finsih merge operation for backup ids ");
-    }
-    Delete delete = createDeleteForBackupMergeOperation();
-    try (Table table = connection.getTable(tableName)) {
-      table.delete(delete);
-    }
-  }
-
-  public String[] getListOfBackupIdsFromMergeOperation() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Get backup ids for merge operation");
-    }
-    Get get = createGetForMergeOperation();
-    try (Table table = connection.getTable(tableName)) {
-      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).split(",");
-    }
-  }
-
-  static Scan createScanForOrigBulkLoadedFiles(TableName table) throws IOException {
-    Scan scan = new Scan();
-    byte[] startRow = rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM);
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.withStartRow(startRow);
-    scan.withStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.META_FAMILY);
-    scan.setMaxVersions(1);
-    return scan;
-  }
-
-  static String getTableNameFromOrigBulkLoadRow(String rowStr) {
-    String[] parts = rowStr.split(BLK_LD_DELIM);
-    return parts[1];
-  }
-
-  static String getRegionNameFromOrigBulkLoadRow(String rowStr) {
-    // format is bulk : namespace : table : region : file
-    String[] parts = rowStr.split(BLK_LD_DELIM);
-    int idx = 3;
-    if (parts.length == 4) {
-      // the table is in default namespace
-      idx = 2;
-    }
-    LOG.debug("bulk row string " + rowStr + " region " + parts[idx]);
-    return parts[idx];
-  }
-
-  /*
-   * Used to query bulk loaded hfiles which have been copied by incremental backup
-   * @param backupId the backup Id. It can be null when querying for all tables
-   * @return the Scan object
-   */
-  static Scan createScanForBulkLoadedFiles(String backupId) throws IOException {
-    Scan scan = new Scan();
-    byte[] startRow =
-        backupId == null ? BULK_LOAD_PREFIX_BYTES : rowkey(BULK_LOAD_PREFIX, backupId
-            + BLK_LD_DELIM);
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    // scan.setTimeRange(lower, Long.MAX_VALUE);
-    scan.addFamily(BackupSystemTable.META_FAMILY);
-    scan.setMaxVersions(1);
-    return scan;
-  }
-
-  static Put createPutForBulkLoadedFile(TableName tn, byte[] fam, String p, String backupId,
-      long ts, int idx) {
-    Put put = new Put(rowkey(BULK_LOAD_PREFIX, backupId + BLK_LD_DELIM + ts + BLK_LD_DELIM + idx));
-    put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, tn.getName());
-    put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, fam);
-    put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, p.getBytes());
-    return put;
-  }
-
-  /**
-   * 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 static 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/37c65946/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
deleted file mode 100644
index e323e96..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
+++ /dev/null
@@ -1,224 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_ATTEMPTS_PAUSE_MS_KEY;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_MAX_ATTEMPTS_KEY;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_MAX_ATTEMPTS;
-
-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() {
-  }
-
-  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
-   */
-  protected 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();
-
-        snapshotTable(admin, tableName, snapshotName);
-        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);
-    }
-
-  }
-
-
-  protected void snapshotTable(Admin admin, TableName tableName, String snapshotName)
-      throws IOException {
-
-    int maxAttempts =
-        conf.getInt(BACKUP_MAX_ATTEMPTS_KEY, DEFAULT_BACKUP_MAX_ATTEMPTS);
-    int pause =
-        conf.getInt(BACKUP_ATTEMPTS_PAUSE_MS_KEY, DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS);
-    int attempts = 0;
-
-    while (attempts++ < maxAttempts) {
-      try {
-        admin.snapshot(snapshotName, tableName);
-        return;
-      } catch (IOException ee) {
-        LOG.warn("Snapshot attempt " + attempts + " failed for table " + tableName
-            + ", sleeping for " + pause + "ms", ee);
-        if (attempts < maxAttempts) {
-          try {
-            Thread.sleep(pause);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            break;
-          }
-        }
-      }
-    }
-    throw new IOException("Failed to snapshot table "+ tableName);
-  }
-}


[21/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 0cfe099..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ /dev/null
@@ -1,516 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.StoreFileInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.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 static long TABLE_AVAILABILITY_WAIT_TIME = 180000;
-
-  private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR };
-  protected Configuration conf = null;
-  protected Path backupRootPath;
-  protected String backupId;
-  protected FileSystem fs;
-
-  // 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);
-  }
-
-  /**
-   * return value represent path for:
-   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn"
-   * @param tableName 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 = new HTableDescriptor(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 {
-    createAndRestoreTable(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;
-  }
-
-  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 createAndRestoreTable(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 = new HTableDescriptor(newTableName, tableDescriptor);
-        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 = new HTableDescriptor(newTableName, tableDescriptor);
-    }
-
-    // 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);
-      RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf);
-      Path[] paths = new Path[regionPathList.size()];
-      regionPathList.toArray(paths);
-      restoreService.run(paths, new TableName[]{tableName}, new TableName[] {newTableName}, true);
-
-    } catch (Exception e) {
-      LOG.error(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;
-  }
-
-  /**
-   * 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(TableName, String, Admin)}
-   * @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)) {
-        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/37c65946/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
deleted file mode 100644
index 7fe9a61..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
+++ /dev/null
@@ -1,503 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-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.BackupPhase;
-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.backup.impl.FullTableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager;
-import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
-import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
-import org.apache.hadoop.hbase.backup.util.BackupUtils;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.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.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.mapreduce.HadoopSecurityEnabledUserProviderForTesting;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.security.access.SecureTestUtil;
-import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.junit.AfterClass;
-import org.junit.Before;
-
-/**
- * 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  protected static HBaseTestingUtility TEST_UTIL2;
-  protected static Configuration conf1 = TEST_UTIL.getConfiguration();
-  protected static Configuration conf2;
-
-  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";
-  protected static boolean secure = false;
-
-  protected static boolean autoRestoreOnFailure = true;
-  protected static boolean setupIsDone = false;
-  protected static boolean useSecondCluster = false;
-
-
-  static class IncrementalTableBackupClientForTest extends IncrementalTableBackupClient
-  {
-
-    public IncrementalTableBackupClientForTest() {
-    }
-
-    public IncrementalTableBackupClientForTest(Connection conn,
-        String backupId, BackupRequest request) throws IOException {
-      super(conn, backupId, request);
-    }
-
-    @Override
-    public void execute() throws IOException
-    {
-      // case INCREMENTAL_COPY:
-      try {
-        // case PREPARE_INCREMENTAL:
-        failStageIf(Stage.stage_0);
-        beginBackup(backupManager, backupInfo);
-
-        failStageIf(Stage.stage_1);
-        backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
-        LOG.debug("For incremental backup, current table set is "
-            + backupManager.getIncrementalBackupTableSet());
-        newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
-        // copy out the table and region info files for each table
-        BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
-        // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
-        convertWALsToHFiles(backupInfo);
-        incrementalCopyHFiles(backupInfo);
-        failStageIf(Stage.stage_2);
-        // Save list of WAL files copied
-        backupManager.recordWALFiles(backupInfo.getIncrBackupFileList());
-
-        // 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
-        // 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);
-        failStageIf(Stage.stage_3);
-
-        HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
-            backupManager.readLogTimestampMap();
-
-        Long newStartCode =
-            BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap));
-        backupManager.writeBackupStartCode(newStartCode);
-
-        handleBulkLoad(backupInfo.getTableNames());
-        failStageIf(Stage.stage_4);
-
-        // backup complete
-        completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
-
-      } catch (Exception e) {
-        failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
-          BackupType.INCREMENTAL, conf);
-        throw new IOException(e);
-      }
-
-    }
-  }
-
-  static class FullTableBackupClientForTest extends FullTableBackupClient
-  {
-
-
-    public FullTableBackupClientForTest() {
-    }
-
-    public FullTableBackupClientForTest(Connection conn, String backupId, BackupRequest request)
-        throws IOException {
-      super(conn, backupId, request);
-    }
-
-    @Override
-    public void execute() throws IOException
-    {
-      // Get the stage ID to fail on
-      try (Admin admin = conn.getAdmin();) {
-        // Begin BACKUP
-        beginBackup(backupManager, backupInfo);
-        failStageIf(Stage.stage_0);
-        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);
-        }
-        failStageIf(Stage.stage_1);
-        // 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);
-        failStageIf(Stage.stage_2);
-        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();
-
-          snapshotTable(admin, tableName, snapshotName);
-          backupInfo.setSnapshotName(tableName, snapshotName);
-        }
-        failStageIf(Stage.stage_3);
-        // 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);
-        failStageIf(Stage.stage_4);
-        // backup complete
-        completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf);
-
-      } catch (Exception e) {
-
-        if(autoRestoreOnFailure) {
-          failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ",
-            BackupType.FULL, conf);
-        }
-        throw new IOException(e);
-      }
-    }
-
-  }
-
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Before
-  public void setUp() throws Exception {
-    if (setupIsDone) {
-      return;
-    }
-    if (secure) {
-      // set the always on security provider
-      UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(),
-          HadoopSecurityEnabledUserProviderForTesting.class);
-      // setup configuration
-      SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
-    }
-    String coproc = conf1.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY);
-    conf1.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, (coproc == null ? "" : coproc + ",") +
-        BackupObserver.class.getName());
-    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.startMiniCluster();
-
-    if (useSecondCluster) {
-      conf2 = HBaseConfiguration.create(conf1);
-      conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
-      TEST_UTIL2 = new HBaseTestingUtility(conf2);
-      TEST_UTIL2.setZkCluster(TEST_UTIL.getZkCluster());
-      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);
-    if (useSecondCluster) {
-      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);
-    setupIsDone = true;
-  }
-
-  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 tearDown() throws Exception {
-    try{
-      SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
-    } catch (Exception e) {
-    }
-    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
-    if (useSecondCluster) {
-      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/37c65946/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
deleted file mode 100644
index 40a01b9..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.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 org.apache.hadoop.hbase.shaded.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/37c65946/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
deleted file mode 100644
index 08002fb..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java
+++ /dev/null
@@ -1,431 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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);
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index ebfc735..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/37c65946/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
deleted file mode 100644
index 208e081..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/37c65946/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
deleted file mode 100644
index 966f519..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertNull;
-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.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.BackupSystemTable;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-/**
- * 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
- */
-@Category(LargeTests.class)
-public class TestBackupDeleteWithFailures extends TestBackupBase{
-
-  private static final Log LOG = LogFactory.getLog(TestBackupDeleteWithFailures.class);
-
-
-
-  public static enum Failure {
-    NO_FAILURES,
-    PRE_SNAPSHOT_FAILURE,
-    PRE_DELETE_SNAPSHOT_FAILURE,
-    POST_DELETE_SNAPSHOT_FAILURE
-  }
-
-  public static class MasterSnapshotObserver implements MasterObserver {
-
-
-    List<Failure> failures = new ArrayList<Failure>();
-
-    public void setFailures(Failure ... f) {
-      failures.clear();
-      for (int i = 0; i < f.length; i++) {
-        failures.add(f[i]);
-      }
-    }
-
-    @Override
-    public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
-        throws IOException
-    {
-       if (failures.contains(Failure.PRE_SNAPSHOT_FAILURE)) {
-         throw new IOException ("preSnapshot");
-       }
-    }
-
-    @Override
-    public void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        SnapshotDescription snapshot) throws IOException {
-      if (failures.contains(Failure.PRE_DELETE_SNAPSHOT_FAILURE)) {
-        throw new IOException ("preDeleteSnapshot");
-      }
-    }
-
-    @Override
-    public void postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        SnapshotDescription snapshot) throws IOException {
-      if (failures.contains(Failure.POST_DELETE_SNAPSHOT_FAILURE)) {
-        throw new IOException ("postDeleteSnapshot");
-      }
-    }
-
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Override
-  @Before
-  public void setUp() throws Exception {
-    conf1.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
-      MasterSnapshotObserver.class.getName());
-    conf1.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
-    super.setUp();
-  }
-
-
-  private MasterSnapshotObserver getMasterSnapshotObserver() {
-    return (MasterSnapshotObserver)TEST_UTIL.getHBaseCluster().getMaster()
-      .getMasterCoprocessorHost().findCoprocessor(MasterSnapshotObserver.class.getName());
-  }
-
-  @Test
-  public void testBackupDeleteWithFailures() throws Exception
-  {
-     testBackupDeleteWithFailuresAfter(1, Failure.PRE_DELETE_SNAPSHOT_FAILURE);
-     testBackupDeleteWithFailuresAfter(0, Failure.POST_DELETE_SNAPSHOT_FAILURE);
-     testBackupDeleteWithFailuresAfter(1, Failure.PRE_SNAPSHOT_FAILURE);
-  }
-
-  private void testBackupDeleteWithFailuresAfter(int expected, Failure ...failures) throws Exception {
-    LOG.info("test repair backup delete on a single table with data and failures "+ failures[0]);
-    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));
-
-    Connection conn = TEST_UTIL.getConnection();
-    Admin admin = conn.getAdmin();
-    MasterSnapshotObserver observer = getMasterSnapshotObserver();
-
-    observer.setFailures(failures);
-    try {
-      getBackupAdmin().deleteBackups(backupIds);
-    } catch(IOException e) {
-      if(expected != 1) assertTrue(false);
-    }
-
-    // Verify that history length == expected after delete failure
-    assertTrue (table.getBackupHistory().size() == expected);
-
-    String[] ids = table.getListOfBackupIdsFromDeleteOperation();
-
-    // Verify that we still have delete record in backup system table
-    if(expected == 1) {
-      assertTrue(ids.length == 1);
-      assertTrue(ids[0].equals(backupId));
-    } else {
-      assertNull(ids);
-    }
-
-    // Now run repair command to repair "failed" delete operation
-    String[] args = new String[] {"repair"};
-
-    observer.setFailures(Failure.NO_FAILURES);
-
-    // Run repair
-    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
-    assertTrue(ret == 0);
-    // Verify that history length == 0
-    assertTrue (table.getBackupHistory().size() == 0);
-    ids = table.getListOfBackupIdsFromDeleteOperation();
-
-    // Verify that we do not have delete record in backup system table
-    assertNull(ids);
-
-    table.close();
-    admin.close();
-  }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 0672325..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/37c65946/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
deleted file mode 100644
index dfbe106..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-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.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.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-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.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-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;
-
-@Category({ MasterTests.class, SmallTests.class })
-public class TestBackupHFileCleaner {
-  private static final Log LOG = LogFactory.getLog(TestBackupHFileCleaner.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static Configuration conf = TEST_UTIL.getConfiguration();
-  private static TableName tableName = TableName.valueOf("backup.hfile.cleaner");
-  private static String famName = "fam";
-  static FileSystem fs = null;
-  Path root;
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true);
-    TEST_UTIL.startMiniZKCluster();
-    TEST_UTIL.startMiniCluster(1);
-    fs = FileSystem.get(conf);
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    if (fs != null) {
-      fs.close();
-    }
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Before
-  public void setup() throws IOException {
-    root = TEST_UTIL.getDataTestDirOnTestFS();
-  }
-
-  @After
-  public void cleanup() {
-    try {
-      fs.delete(root, true);
-    } catch (IOException e) {
-      LOG.warn("Failed to delete files recursively from path " + root);
-    }
-  }
-
-  @Test
-  public void testGetDeletableFiles() throws IOException {
-    // 1. Create a file
-    Path file = new Path(root, "testIsFileDeletableWithNoHFileRefs");
-    fs.createNewFile(file);
-    // 2. Assert file is successfully created
-    assertTrue("Test file not created!", fs.exists(file));
-    BackupHFileCleaner cleaner = new BackupHFileCleaner();
-    cleaner.setConf(conf);
-    cleaner.setCheckForFullyBackedUpTables(false);
-    // 3. Assert that file as is should be deletable
-    List<FileStatus> stats = new ArrayList<>();
-    FileStatus stat = fs.getFileStatus(file);
-    stats.add(stat);
-    Iterable<FileStatus> deletable = cleaner.getDeletableFiles(stats);
-    deletable = cleaner.getDeletableFiles(stats);
-    boolean found = false;
-    for (FileStatus stat1 : deletable) {
-      if (stat.equals(stat1)) found = true;
-    }
-    assertTrue("Cleaner should allow to delete this file as there is no hfile reference "
-        + "for it.", found);
-
-    // 4. Add the file as bulk load
-    List<Path> list = new ArrayList<>(1);
-    list.add(file);
-    try (Connection conn = ConnectionFactory.createConnection(conf);
-        BackupSystemTable sysTbl = new BackupSystemTable(conn)) {
-      List<TableName> sTableList = new ArrayList<>();
-      sTableList.add(tableName);
-      Map<byte[], List<Path>>[] maps = new Map[1];
-      maps[0] = new HashMap<>();
-      maps[0].put(famName.getBytes(), list);
-      sysTbl.writeBulkLoadedFiles(sTableList, maps, "1");
-    }
-
-    // 5. Assert file should not be deletable
-    deletable = cleaner.getDeletableFiles(stats);
-    deletable = cleaner.getDeletableFiles(stats);
-    found = false;
-    for (FileStatus stat1 : deletable) {
-      if (stat.equals(stat1)) found = true;
-    }
-    assertFalse("Cleaner should not allow to delete this file as there is a hfile reference "
-        + "for it.", found);
-  }
-}
\ No newline at end of file


[15/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
new file mode 100644
index 0000000..4dab046
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -0,0 +1,2051 @@
+/**
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+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.CellComparator;
+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.NamespaceDescriptor;
+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.BackupType;
+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.SnapshotDescription;
+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;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * 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 {
+  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
+
+  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 TableName tableName;
+  /**
+   * Stores backup sessions (contexts)
+   */
+  final static byte[] SESSIONS_FAMILY = "session".getBytes();
+  /**
+   * Stores other meta
+   */
+  final static byte[] META_FAMILY = "meta".getBytes();
+  final static byte[] BULK_LOAD_FAMILY = "bulk".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 byte[] ACTIVE_SESSION_ROW = "activesession:".getBytes();
+  private final static byte[] ACTIVE_SESSION_COL = "c".getBytes();
+
+  private final static byte[] ACTIVE_SESSION_YES = "yes".getBytes();
+  private final static byte[] ACTIVE_SESSION_NO = "no".getBytes();
+
+  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 BULK_LOAD_PREFIX = "bulk:";
+  private final static byte[] BULK_LOAD_PREFIX_BYTES = BULK_LOAD_PREFIX.getBytes();
+  private final static byte[] DELETE_OP_ROW = "delete_op_row".getBytes();
+  private final static byte[] MERGE_OP_ROW = "merge_op_row".getBytes();
+
+  final static byte[] TBL_COL = Bytes.toBytes("tbl");
+  final static byte[] FAM_COL = Bytes.toBytes("fam");
+  final static byte[] PATH_COL = Bytes.toBytes("path");
+  final static byte[] STATE_COL = Bytes.toBytes("state");
+  // the two states a bulk loaded file can be
+  final static byte[] BL_PREPARE = Bytes.toBytes("R");
+  final static byte[] BL_COMMIT = Bytes.toBytes("D");
+
+  private final static String WALS_PREFIX = "wals:";
+  private final static String SET_KEY_PREFIX = "backupset:";
+
+  // separator between BULK_LOAD_PREFIX and ordinals
+  protected final static String BLK_LD_DELIM = ":";
+  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();) {
+
+      verifyNamespaceExists(admin);
+
+      if (!admin.tableExists(tableName)) {
+        HTableDescriptor backupHTD =
+            BackupSystemTable.getSystemTableDescriptor(connection.getConfiguration());
+        admin.createTable(backupHTD);
+      }
+      waitForSystemTable(admin);
+    }
+  }
+
+  private void verifyNamespaceExists(Admin admin) throws IOException {
+    String namespaceName = tableName.getNamespaceAsString();
+    NamespaceDescriptor ns = NamespaceDescriptor.create(namespaceName).build();
+    NamespaceDescriptor[] list = admin.listNamespaceDescriptors();
+    boolean exists = false;
+    for (NamespaceDescriptor nsd : list) {
+      if (nsd.getName().equals(ns.getName())) {
+        exists = true;
+        break;
+      }
+    }
+    if (!exists) {
+      admin.createNamespace(ns);
+    }
+  }
+
+  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);
+    }
+  }
+
+  /*
+   * @param backupId the backup Id
+   * @return Map of rows to path of bulk loaded hfile
+   */
+  Map<byte[], String> readBulkLoadedFiles(String backupId) throws IOException {
+    Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId);
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      Map<byte[], String> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        byte[] row = CellUtil.cloneRow(res.listCells().get(0));
+        for (Cell cell : res.listCells()) {
+          if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
+            BackupSystemTable.PATH_COL.length) == 0) {
+            map.put(row, Bytes.toString(CellUtil.cloneValue(cell)));
+          }
+        }
+      }
+      return map;
+    }
+  }
+
+  /*
+   * Used during restore
+   * @param backupId the backup Id
+   * @param sTableList List of tables
+   * @return array of Map of family to List of Paths
+   */
+  public Map<byte[], List<Path>>[] readBulkLoadedFiles(String backupId, List<TableName> sTableList)
+      throws IOException {
+    Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId);
+    Map<byte[], List<Path>>[] mapForSrc = new Map[sTableList == null ? 1 : sTableList.size()];
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        TableName tbl = null;
+        byte[] fam = null;
+        String path = null;
+        for (Cell cell : res.listCells()) {
+          if (CellComparator.compareQualifiers(cell, BackupSystemTable.TBL_COL, 0,
+            BackupSystemTable.TBL_COL.length) == 0) {
+            tbl = TableName.valueOf(CellUtil.cloneValue(cell));
+          } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
+            BackupSystemTable.FAM_COL.length) == 0) {
+            fam = CellUtil.cloneValue(cell);
+          } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
+            BackupSystemTable.PATH_COL.length) == 0) {
+            path = Bytes.toString(CellUtil.cloneValue(cell));
+          }
+        }
+        int srcIdx = IncrementalTableBackupClient.getIndex(tbl, sTableList);
+        if (srcIdx == -1) {
+          // the table is not among the query
+          continue;
+        }
+        if (mapForSrc[srcIdx] == null) {
+          mapForSrc[srcIdx] = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+        }
+        List<Path> files;
+        if (!mapForSrc[srcIdx].containsKey(fam)) {
+          files = new ArrayList<Path>();
+          mapForSrc[srcIdx].put(fam, files);
+        } else {
+          files = mapForSrc[srcIdx].get(fam);
+        }
+        files.add(new Path(path));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("found bulk loaded file : " + tbl + " " + Bytes.toString(fam) + " " + path);
+        }
+      }
+      ;
+      return mapForSrc;
+    }
+  }
+
+  /*
+   * @param map Map of row keys to path of bulk loaded hfile
+   */
+  void deleteBulkLoadedFiles(Map<byte[], String> map) throws IOException {
+    try (Table table = connection.getTable(tableName)) {
+      List<Delete> dels = new ArrayList<>();
+      for (byte[] row : map.keySet()) {
+        dels.add(new Delete(row).addFamily(BackupSystemTable.META_FAMILY));
+      }
+      table.delete(dels);
+    }
+  }
+
+  /**
+   * 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);
+    }
+  }
+
+  /*
+   * For postBulkLoadHFile() hook.
+   * @param tabName table name
+   * @param region the region receiving hfile
+   * @param finalPaths family and associated hfiles
+   */
+  public void writePathsPostBulkLoad(TableName tabName, byte[] region,
+      Map<byte[], List<Path>> finalPaths) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + finalPaths.size()
+          + " entries");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      List<Put> puts = BackupSystemTable.createPutForCommittedBulkload(tabName, region, finalPaths);
+      table.put(puts);
+      LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
+    }
+  }
+
+  /*
+   * For preCommitStoreFile() hook
+   * @param tabName table name
+   * @param region the region receiving hfile
+   * @param family column family
+   * @param pairs list of paths for hfiles
+   */
+  public void writeFilesForBulkLoadPreCommit(TableName tabName, byte[] region, final byte[] family,
+      final List<Pair<Path, Path>> pairs) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + pairs.size()
+          + " entries");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      List<Put> puts =
+          BackupSystemTable.createPutForPreparedBulkload(tabName, region, family, pairs);
+      table.put(puts);
+      LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
+    }
+  }
+
+  /*
+   * Removes rows recording bulk loaded hfiles from backup table
+   * @param lst list of table names
+   * @param rows the rows to be deleted
+   */
+  public void removeBulkLoadedRows(List<TableName> lst, List<byte[]> rows) throws IOException {
+    try (Table table = connection.getTable(tableName)) {
+      List<Delete> lstDels = new ArrayList<>();
+      for (byte[] row : rows) {
+        Delete del = new Delete(row);
+        lstDels.add(del);
+        LOG.debug("orig deleting the row: " + Bytes.toString(row));
+      }
+      table.delete(lstDels);
+      LOG.debug("deleted " + rows.size() + " original bulkload rows for " + lst.size() + " tables");
+    }
+  }
+
+  /*
+   * Reads the rows from backup table recording bulk loaded hfiles
+   * @param tableList list of table names
+   * @return The keys of the Map are table, region and column family. Value of the map reflects
+   * whether the hfile was recorded by preCommitStoreFile hook (true)
+   */
+  public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
+      readBulkloadRows(List<TableName> tableList) throws IOException {
+    Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> map = new HashMap<>();
+    List<byte[]> rows = new ArrayList<>();
+    for (TableName tTable : tableList) {
+      Scan scan = BackupSystemTable.createScanForOrigBulkLoadedFiles(tTable);
+      Map<String, Map<String, List<Pair<String, Boolean>>>> tblMap = map.get(tTable);
+      try (Table table = connection.getTable(tableName);
+          ResultScanner scanner = table.getScanner(scan)) {
+        Result res = null;
+        while ((res = scanner.next()) != null) {
+          res.advance();
+          String fam = null;
+          String path = null;
+          boolean raw = false;
+          byte[] row = null;
+          String region = null;
+          for (Cell cell : res.listCells()) {
+            row = CellUtil.cloneRow(cell);
+            rows.add(row);
+            String rowStr = Bytes.toString(row);
+            region = BackupSystemTable.getRegionNameFromOrigBulkLoadRow(rowStr);
+            if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
+              BackupSystemTable.FAM_COL.length) == 0) {
+              fam = Bytes.toString(CellUtil.cloneValue(cell));
+            } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
+              BackupSystemTable.PATH_COL.length) == 0) {
+              path = Bytes.toString(CellUtil.cloneValue(cell));
+            } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.STATE_COL, 0,
+              BackupSystemTable.STATE_COL.length) == 0) {
+              byte[] state = CellUtil.cloneValue(cell);
+              if (Bytes.equals(BackupSystemTable.BL_PREPARE, state)) {
+                raw = true;
+              } else raw = false;
+            }
+          }
+          if (map.get(tTable) == null) {
+            map.put(tTable, new HashMap<String, Map<String, List<Pair<String, Boolean>>>>());
+            tblMap = map.get(tTable);
+          }
+          if (tblMap.get(region) == null) {
+            tblMap.put(region, new HashMap<String, List<Pair<String, Boolean>>>());
+          }
+          Map<String, List<Pair<String, Boolean>>> famMap = tblMap.get(region);
+          if (famMap.get(fam) == null) {
+            famMap.put(fam, new ArrayList<Pair<String, Boolean>>());
+          }
+          famMap.get(fam).add(new Pair<>(path, raw));
+          LOG.debug("found orig " + path + " for " + fam + " of table " + region);
+        }
+      }
+    }
+    return new Pair<>(map, rows);
+  }
+
+  /*
+   * @param sTableList List of tables
+   * @param maps array of Map of family to List of Paths
+   * @param backupId the backup Id
+   */
+  public void writeBulkLoadedFiles(List<TableName> sTableList, Map<byte[], List<Path>>[] maps,
+      String backupId) throws IOException {
+    try (Table table = connection.getTable(tableName)) {
+      long ts = EnvironmentEdgeManager.currentTime();
+      int cnt = 0;
+      List<Put> puts = new ArrayList<>();
+      for (int idx = 0; idx < maps.length; idx++) {
+        Map<byte[], List<Path>> map = maps[idx];
+        TableName tn = sTableList.get(idx);
+        if (map == null) continue;
+        for (Map.Entry<byte[], List<Path>> entry : map.entrySet()) {
+          byte[] fam = entry.getKey();
+          List<Path> paths = entry.getValue();
+          for (Path p : paths) {
+            Put put =
+                BackupSystemTable.createPutForBulkLoadedFile(tn, fam, p.toString(), backupId, ts,
+                  cnt++);
+            puts.add(put);
+          }
+        }
+      }
+      if (!puts.isEmpty()) {
+        table.put(puts);
+      }
+    }
+  }
+
+  /**
+   * 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);
+    }
+  }
+
+  /**
+   * Exclusive operations are:
+   * create, delete, merge
+   * @throws IOException
+   */
+  public void startBackupExclusiveOperation() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Start new backup exclusive operation");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = createPutForStartBackupSession();
+      // First try to put if row does not exist
+      if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL, null, put)) {
+        // Row exists, try to put if value == ACTIVE_SESSION_NO
+        if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
+          ACTIVE_SESSION_NO, put)) {
+          throw new IOException("There is an active backup exclusive operation");
+        }
+      }
+    }
+  }
+
+  private Put createPutForStartBackupSession() {
+    Put put = new Put(ACTIVE_SESSION_ROW);
+    put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_YES);
+    return put;
+  }
+
+  public void finishBackupExclusiveOperation() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Finish backup exclusive operation");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = createPutForStopBackupSession();
+      if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
+        ACTIVE_SESSION_YES, put)) {
+        throw new IOException("There is no active backup exclusive operation");
+      }
+    }
+  }
+
+  private Put createPutForStopBackupSession() {
+    Put put = new Put(ACTIVE_SESSION_ROW);
+    put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_NO);
+    return 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, if n== -1 - max number
+   *        is ignored
+   * @return list of records
+   * @throws IOException
+   */
+  public List<BackupInfo> getHistory(int n) throws IOException {
+
+    List<BackupInfo> history = getBackupHistory();
+    if (n == -1 || 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, if n == -1 , then max number
+   *        is ignored
+   * @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 (n >= 0 && 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;
+
+  }
+
+  /*
+   * Retrieve TableName's for completed backup of given type
+   * @param type backup type
+   * @return List of table names
+   */
+  public List<TableName> getTablesForBackupType(BackupType type) throws IOException {
+    Set<TableName> names = new HashSet<>();
+    List<BackupInfo> infos = getBackupHistory(true);
+    for (BackupInfo info : infos) {
+      if (info.getType() != type) continue;
+      names.addAll(info.getTableNames());
+    }
+    return new ArrayList(names);
+  }
+
+  /**
+   * 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
+   * TODO: multiple backup destination support
+   */
+  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();
+  }
+
+  public static String getSnapshotName(Configuration conf) {
+    return "snapshot_" + getTableNameAsString(conf).replace(":", "_");
+  }
+
+  /**
+   * 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's for bulk load resulting from running LoadIncrementalHFiles
+   */
+  static List<Put> createPutForCommittedBulkload(TableName table, byte[] region,
+      Map<byte[], List<Path>> finalPaths) {
+    List<Put> puts = new ArrayList<>();
+    for (Map.Entry<byte[], List<Path>> entry : finalPaths.entrySet()) {
+      for (Path path : entry.getValue()) {
+        String file = path.toString();
+        int lastSlash = file.lastIndexOf("/");
+        String filename = file.substring(lastSlash + 1);
+        Put put =
+            new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM,
+              Bytes.toString(region), BLK_LD_DELIM, filename));
+        put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
+        put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, entry.getKey());
+        put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
+        put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_COMMIT);
+        puts.add(put);
+        LOG.debug("writing done bulk path " + file + " for " + table + " " + Bytes.toString(region));
+      }
+    }
+    return puts;
+  }
+
+  public static void snapshot(Connection conn) throws IOException {
+
+    try (Admin admin = conn.getAdmin();) {
+      Configuration conf = conn.getConfiguration();
+      admin.snapshot(BackupSystemTable.getSnapshotName(conf), BackupSystemTable.getTableName(conf));
+    }
+  }
+
+  public static void restoreFromSnapshot(Connection conn) throws IOException {
+
+    Configuration conf = conn.getConfiguration();
+    LOG.debug("Restoring " + BackupSystemTable.getTableNameAsString(conf) + " from snapshot");
+    try (Admin admin = conn.getAdmin();) {
+      String snapshotName = BackupSystemTable.getSnapshotName(conf);
+      if (snapshotExists(admin, snapshotName)) {
+        admin.disableTable(BackupSystemTable.getTableName(conf));
+        admin.restoreSnapshot(snapshotName);
+        admin.enableTable(BackupSystemTable.getTableName(conf));
+        LOG.debug("Done restoring backup system table");
+      } else {
+        // Snapshot does not exists, i.e completeBackup failed after
+        // deleting backup system table snapshot
+        // In this case we log WARN and proceed
+        LOG.warn("Could not restore backup system table. Snapshot " + snapshotName
+            + " does not exists.");
+      }
+    }
+  }
+
+  protected static boolean snapshotExists(Admin admin, String snapshotName) throws IOException {
+
+    List<SnapshotDescription> list = admin.listSnapshots();
+    for (SnapshotDescription desc : list) {
+      if (desc.getName().equals(snapshotName)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public static boolean snapshotExists(Connection conn) throws IOException {
+    return snapshotExists(conn.getAdmin(), getSnapshotName(conn.getConfiguration()));
+  }
+
+  public static void deleteSnapshot(Connection conn) throws IOException {
+
+    Configuration conf = conn.getConfiguration();
+    LOG.debug("Deleting " + BackupSystemTable.getSnapshotName(conf) + " from the system");
+    try (Admin admin = conn.getAdmin();) {
+      String snapshotName = BackupSystemTable.getSnapshotName(conf);
+      if (snapshotExists(admin, snapshotName)) {
+        admin.deleteSnapshot(snapshotName);
+        LOG.debug("Done deleting backup system table snapshot");
+      } else {
+        LOG.error("Snapshot " + snapshotName + " does not exists");
+      }
+    }
+  }
+
+  /*
+   * Creates Put's for bulk load resulting from running LoadIncrementalHFiles
+   */
+  static List<Put> createPutForPreparedBulkload(TableName table, byte[] region,
+      final byte[] family, final List<Pair<Path, Path>> pairs) {
+    List<Put> puts = new ArrayList<>();
+    for (Pair<Path, Path> pair : pairs) {
+      Path path = pair.getSecond();
+      String file = path.toString();
+      int lastSlash = file.lastIndexOf("/");
+      String filename = file.substring(lastSlash + 1);
+      Put put =
+          new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM, Bytes.toString(region),
+            BLK_LD_DELIM, filename));
+      put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
+      put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, family);
+      put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
+      put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_PREPARE);
+      puts.add(put);
+      LOG.debug("writing raw bulk path " + file + " for " + table + " " + Bytes.toString(region));
+    }
+    return puts;
+  }
+
+  public static List<Delete> createDeleteForOrigBulkLoad(List<TableName> lst) {
+    List<Delete> lstDels = new ArrayList<>();
+    for (TableName table : lst) {
+      Delete del = new Delete(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM));
+      del.addFamily(BackupSystemTable.META_FAMILY);
+      lstDels.add(del);
+    }
+    return lstDels;
+  }
+
+  private Put createPutForDeleteOperation(String[] backupIdList) {
+
+    byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ","));
+    Put put = new Put(DELETE_OP_ROW);
+    put.addColumn(META_FAMILY, FAM_COL, value);
+    return put;
+  }
+
+  private Delete createDeleteForBackupDeleteOperation() {
+
+    Delete delete = new Delete(DELETE_OP_ROW);
+    delete.addFamily(META_FAMILY);
+    return delete;
+  }
+
+  private Get createGetForDeleteOperation() {
+
+    Get get = new Get(DELETE_OP_ROW);
+    get.addFamily(META_FAMILY);
+    return get;
+  }
+
+  public void startDeleteOperation(String[] backupIdList) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Start delete operation for backups: " + StringUtils.join(backupIdList));
+    }
+    Put put = createPutForDeleteOperation(backupIdList);
+    try (Table table = connection.getTable(tableName)) {
+      table.put(put);
+    }
+  }
+
+  public void finishDeleteOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Finsih delete operation for backup ids ");
+    }
+    Delete delete = createDeleteForBackupDeleteOperation();
+    try (Table table = connection.getTable(tableName)) {
+      table.delete(delete);
+    }
+  }
+
+  public String[] getListOfBackupIdsFromDeleteOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Get delete operation for backup ids ");
+    }
+    Get get = createGetForDeleteOperation();
+    try (Table table = connection.getTable(tableName)) {
+      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).split(",");
+    }
+  }
+
+  private Put createPutForMergeOperation(String[] backupIdList) {
+
+    byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ","));
+    Put put = new Put(MERGE_OP_ROW);
+    put.addColumn(META_FAMILY, FAM_COL, value);
+    return put;
+  }
+
+  public boolean isMergeInProgress() throws IOException {
+    Get get = new Get(MERGE_OP_ROW);
+    try (Table table = connection.getTable(tableName)) {
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  private Put createPutForUpdateTablesForMerge(List<TableName> tables) {
+
+    byte[] value = Bytes.toBytes(StringUtils.join(tables, ","));
+    Put put = new Put(MERGE_OP_ROW);
+    put.addColumn(META_FAMILY, PATH_COL, value);
+    return put;
+  }
+
+  private Delete createDeleteForBackupMergeOperation() {
+
+    Delete delete = new Delete(MERGE_OP_ROW);
+    delete.addFamily(META_FAMILY);
+    return delete;
+  }
+
+  private Get createGetForMergeOperation() {
+
+    Get get = new Get(MERGE_OP_ROW);
+    get.addFamily(META_FAMILY);
+    return get;
+  }
+
+  public void startMergeOperation(String[] backupIdList) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Start merge operation for backups: " + StringUtils.join(backupIdList));
+    }
+    Put put = createPutForMergeOperation(backupIdList);
+    try (Table table = connection.getTable(tableName)) {
+      table.put(put);
+    }
+  }
+
+  public void updateProcessedTablesForMerge(List<TableName> tables) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Update tables for merge : " + StringUtils.join(tables, ","));
+    }
+    Put put = createPutForUpdateTablesForMerge(tables);
+    try (Table table = connection.getTable(tableName)) {
+      table.put(put);
+    }
+  }
+
+  public void finishMergeOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Finsih merge operation for backup ids ");
+    }
+    Delete delete = createDeleteForBackupMergeOperation();
+    try (Table table = connection.getTable(tableName)) {
+      table.delete(delete);
+    }
+  }
+
+  public String[] getListOfBackupIdsFromMergeOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Get backup ids for merge operation");
+    }
+    Get get = createGetForMergeOperation();
+    try (Table table = connection.getTable(tableName)) {
+      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).split(",");
+    }
+  }
+
+  static Scan createScanForOrigBulkLoadedFiles(TableName table) throws IOException {
+    Scan scan = new Scan();
+    byte[] startRow = rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM);
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.withStartRow(startRow);
+    scan.withStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.META_FAMILY);
+    scan.setMaxVersions(1);
+    return scan;
+  }
+
+  static String getTableNameFromOrigBulkLoadRow(String rowStr) {
+    String[] parts = rowStr.split(BLK_LD_DELIM);
+    return parts[1];
+  }
+
+  static String getRegionNameFromOrigBulkLoadRow(String rowStr) {
+    // format is bulk : namespace : table : region : file
+    String[] parts = rowStr.split(BLK_LD_DELIM);
+    int idx = 3;
+    if (parts.length == 4) {
+      // the table is in default namespace
+      idx = 2;
+    }
+    LOG.debug("bulk row string " + rowStr + " region " + parts[idx]);
+    return parts[idx];
+  }
+
+  /*
+   * Used to query bulk loaded hfiles which have been copied by incremental backup
+   * @param backupId the backup Id. It can be null when querying for all tables
+   * @return the Scan object
+   */
+  static Scan createScanForBulkLoadedFiles(String backupId) throws IOException {
+    Scan scan = new Scan();
+    byte[] startRow =
+        backupId == null ? BULK_LOAD_PREFIX_BYTES : rowkey(BULK_LOAD_PREFIX, backupId
+            + BLK_LD_DELIM);
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    // scan.setTimeRange(lower, Long.MAX_VALUE);
+    scan.addFamily(BackupSystemTable.META_FAMILY);
+    scan.setMaxVersions(1);
+    return scan;
+  }
+
+  static Put createPutForBulkLoadedFile(TableName tn, byte[] fam, String p, String backupId,
+      long ts, int idx) {
+    Put put = new Put(rowkey(BULK_LOAD_PREFIX, backupId + BLK_LD_DELIM + ts + BLK_LD_DELIM + idx));
+    put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, tn.getName());
+    put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, fam);
+    put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, p.getBytes());
+    return put;
+  }
+
+  /**
+   * 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 static 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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
new file mode 100644
index 0000000..e323e96
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
@@ -0,0 +1,224 @@
+/**
+ * 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.BACKUP_ATTEMPTS_PAUSE_MS_KEY;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_MAX_ATTEMPTS_KEY;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_MAX_ATTEMPTS;
+
+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() {
+  }
+
+  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
+   */
+  protected 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();
+
+        snapshotTable(admin, tableName, snapshotName);
+        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);
+    }
+
+  }
+
+
+  protected void snapshotTable(Admin admin, TableName tableName, String snapshotName)
+      throws IOException {
+
+    int maxAttempts =
+        conf.getInt(BACKUP_MAX_ATTEMPTS_KEY, DEFAULT_BACKUP_MAX_ATTEMPTS);
+    int pause =
+        conf.getInt(BACKUP_ATTEMPTS_PAUSE_MS_KEY, DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS);
+    int attempts = 0;
+
+    while (attempts++ < maxAttempts) {
+      try {
+        admin.snapshot(snapshotName, tableName);
+        return;
+      } catch (IOException ee) {
+        LOG.warn("Snapshot attempt " + attempts + " failed for table " + tableName
+            + ", sleeping for " + pause + "ms", ee);
+        if (attempts < maxAttempts) {
+          try {
+            Thread.sleep(pause);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            break;
+          }
+        }
+      }
+    }
+    throw new IOException("Failed to snapshot table "+ tableName);
+  }
+}


[36/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/branch-2
Commit: 37c65946274c6acdb8650fc91f889b61f95b3a64
Parents: af8e6ae
Author: Vladimir Rodionov <vr...@hortonworks.com>
Authored: Tue Aug 22 17:14:48 2017 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Wed Aug 23 12:46:04 2017 -0400

----------------------------------------------------------------------
 hbase-assembly/pom.xml                          |    6 +-
 .../src/main/assembly/hadoop-two-compat.xml     |    1 +
 hbase-assembly/src/main/assembly/src.xml        |    1 +
 hbase-backup/.DS_Store                          |  Bin 0 -> 6148 bytes
 hbase-backup/pom.xml                            |  265 +++
 .../apache/hadoop/hbase/backup/BackupAdmin.java |  136 ++
 .../hbase/backup/BackupClientFactory.java       |   53 +
 .../hadoop/hbase/backup/BackupCopyJob.java      |   55 +
 .../hadoop/hbase/backup/BackupDriver.java       |  210 ++
 .../hadoop/hbase/backup/BackupHFileCleaner.java |  180 ++
 .../apache/hadoop/hbase/backup/BackupInfo.java  |  550 +++++
 .../hadoop/hbase/backup/BackupMergeJob.java     |   40 +
 .../hadoop/hbase/backup/BackupObserver.java     |  102 +
 .../hadoop/hbase/backup/BackupRequest.java      |  139 ++
 .../hbase/backup/BackupRestoreConstants.java    |  123 ++
 .../hbase/backup/BackupRestoreFactory.java      |   82 +
 .../hadoop/hbase/backup/BackupTableInfo.java    |   82 +
 .../hadoop/hbase/backup/HBackupFileSystem.java  |  146 ++
 .../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      |  743 +++++++
 .../hbase/backup/impl/BackupCommands.java       | 1022 +++++++++
 .../hbase/backup/impl/BackupException.java      |   84 +
 .../hadoop/hbase/backup/impl/BackupManager.java |  502 +++++
 .../hbase/backup/impl/BackupManifest.java       |  674 ++++++
 .../hbase/backup/impl/BackupSystemTable.java    | 2051 ++++++++++++++++++
 .../backup/impl/FullTableBackupClient.java      |  224 ++
 .../backup/impl/IncrementalBackupManager.java   |  387 ++++
 .../impl/IncrementalTableBackupClient.java      |  377 ++++
 .../hbase/backup/impl/RestoreTablesClient.java  |  278 +++
 .../hbase/backup/impl/TableBackupClient.java    |  436 ++++
 .../mapreduce/MapReduceBackupCopyJob.java       |  344 +++
 .../mapreduce/MapReduceBackupMergeJob.java      |  321 +++
 .../mapreduce/MapReduceHFileSplitterJob.java    |  181 ++
 .../backup/mapreduce/MapReduceRestoreJob.java   |  136 ++
 .../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   |  747 +++++++
 .../hadoop/hbase/backup/util/RestoreTool.java   |  516 +++++
 .../hadoop/hbase/backup/TestBackupBase.java     |  503 +++++
 .../hbase/backup/TestBackupBoundaryTests.java   |   97 +
 .../hbase/backup/TestBackupCommandLineTool.java |  431 ++++
 .../hadoop/hbase/backup/TestBackupDelete.java   |  102 +
 .../hbase/backup/TestBackupDeleteRestore.java   |   70 +
 .../backup/TestBackupDeleteWithFailures.java    |  194 ++
 .../hadoop/hbase/backup/TestBackupDescribe.java |  110 +
 .../hbase/backup/TestBackupHFileCleaner.java    |  141 ++
 .../hbase/backup/TestBackupMultipleDeletes.java |  158 ++
 .../hadoop/hbase/backup/TestBackupRepair.java   |   91 +
 .../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 ++
 .../backup/TestFullBackupWithFailures.java      |   79 +
 .../hadoop/hbase/backup/TestFullRestore.java    |  345 +++
 .../hbase/backup/TestIncrementalBackup.java     |  209 ++
 .../TestIncrementalBackupDeleteTable.java       |  129 ++
 .../TestIncrementalBackupMergeWithFailures.java |  336 +++
 .../TestIncrementalBackupWithBulkLoad.java      |  145 ++
 .../TestIncrementalBackupWithFailures.java      |  161 ++
 .../hadoop/hbase/backup/TestRemoteBackup.java   |  135 ++
 .../hadoop/hbase/backup/TestRemoteRestore.java  |   59 +
 .../backup/TestRepairAfterFailedDelete.java     |   93 +
 .../hbase/backup/TestRestoreBoundaryTests.java  |   80 +
 .../hbase/backup/TestSystemTableSnapshot.java   |   55 +
 .../backup/master/TestBackupLogCleaner.java     |  162 ++
 .../src/test/resources/log4j.properties         |   68 +
 hbase-it/pom.xml                                |    5 +
 .../apache/hadoop/hbase/backup/BackupAdmin.java |  136 --
 .../hbase/backup/BackupClientFactory.java       |   53 -
 .../hadoop/hbase/backup/BackupCopyJob.java      |   55 -
 .../hadoop/hbase/backup/BackupDriver.java       |  210 --
 .../hadoop/hbase/backup/BackupHFileCleaner.java |  180 --
 .../apache/hadoop/hbase/backup/BackupInfo.java  |  550 -----
 .../hadoop/hbase/backup/BackupMergeJob.java     |   40 -
 .../hadoop/hbase/backup/BackupObserver.java     |  102 -
 .../hadoop/hbase/backup/BackupRequest.java      |  139 --
 .../hbase/backup/BackupRestoreConstants.java    |  123 --
 .../hbase/backup/BackupRestoreFactory.java      |   82 -
 .../hadoop/hbase/backup/BackupTableInfo.java    |   82 -
 .../hadoop/hbase/backup/HBackupFileSystem.java  |  146 --
 .../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      |  743 -------
 .../hbase/backup/impl/BackupCommands.java       | 1022 ---------
 .../hbase/backup/impl/BackupException.java      |   84 -
 .../hadoop/hbase/backup/impl/BackupManager.java |  502 -----
 .../hbase/backup/impl/BackupManifest.java       |  674 ------
 .../hbase/backup/impl/BackupSystemTable.java    | 2051 ------------------
 .../backup/impl/FullTableBackupClient.java      |  224 --
 .../backup/impl/IncrementalBackupManager.java   |  387 ----
 .../impl/IncrementalTableBackupClient.java      |  377 ----
 .../hbase/backup/impl/RestoreTablesClient.java  |  278 ---
 .../hbase/backup/impl/TableBackupClient.java    |  436 ----
 .../mapreduce/MapReduceBackupCopyJob.java       |  344 ---
 .../mapreduce/MapReduceBackupMergeJob.java      |  321 ---
 .../mapreduce/MapReduceHFileSplitterJob.java    |  181 --
 .../backup/mapreduce/MapReduceRestoreJob.java   |  136 --
 .../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   |  747 -------
 .../hadoop/hbase/backup/util/RestoreTool.java   |  516 -----
 .../hadoop/hbase/backup/TestBackupBase.java     |  503 -----
 .../hbase/backup/TestBackupBoundaryTests.java   |   97 -
 .../hbase/backup/TestBackupCommandLineTool.java |  431 ----
 .../hadoop/hbase/backup/TestBackupDelete.java   |  102 -
 .../hbase/backup/TestBackupDeleteRestore.java   |   70 -
 .../backup/TestBackupDeleteWithFailures.java    |  194 --
 .../hadoop/hbase/backup/TestBackupDescribe.java |  110 -
 .../hbase/backup/TestBackupHFileCleaner.java    |  141 --
 .../hbase/backup/TestBackupMultipleDeletes.java |  158 --
 .../hadoop/hbase/backup/TestBackupRepair.java   |   91 -
 .../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 --
 .../backup/TestFullBackupWithFailures.java      |   79 -
 .../hadoop/hbase/backup/TestFullRestore.java    |  345 ---
 .../hbase/backup/TestIncrementalBackup.java     |  209 --
 .../TestIncrementalBackupDeleteTable.java       |  129 --
 .../TestIncrementalBackupMergeWithFailures.java |  336 ---
 .../TestIncrementalBackupWithBulkLoad.java      |  145 --
 .../TestIncrementalBackupWithFailures.java      |  161 --
 .../hadoop/hbase/backup/TestRemoteBackup.java   |  135 --
 .../hadoop/hbase/backup/TestRemoteRestore.java  |   59 -
 .../backup/TestRepairAfterFailedDelete.java     |   93 -
 .../hbase/backup/TestRestoreBoundaryTests.java  |   80 -
 .../hbase/backup/TestSystemTableSnapshot.java   |   55 -
 .../backup/master/TestBackupLogCleaner.java     |  162 --
 pom.xml                                         |    1 +
 146 files changed, 17540 insertions(+), 17195 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 8ea9020..b750f17 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -281,8 +281,12 @@
       <version>${project.version}</version>
       <type>test-jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-backup</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
-
   <profiles>
     <profile>
       <id>rsgroup</id>

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
index 8039c4d..1592a3b 100644
--- a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
+++ b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
@@ -49,6 +49,7 @@
         <include>org.apache.hbase:hbase-spark</include>
         <include>org.apache.hbase:hbase-thrift</include>
         <include>org.apache.hbase:hbase-external-blockcache</include>
+        <include>org.apache.hbase:hbase-backup</include>
       </includes>
       <!-- Binaries for the dependencies also go in the hbase-jars directory -->
       <binaries>

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-assembly/src/main/assembly/src.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/src.xml b/hbase-assembly/src/main/assembly/src.xml
index e5d3faf..3fd7562 100644
--- a/hbase-assembly/src/main/assembly/src.xml
+++ b/hbase-assembly/src/main/assembly/src.xml
@@ -59,6 +59,7 @@
         <include>org.apache.hbase:hbase-spark-it</include>
         <include>org.apache.hbase:hbase-testing-util</include>
         <include>org.apache.hbase:hbase-thrift</include>
+        <include>org.apache.hbase:hbase-backup</include>
       </includes>
       <!-- Include all the sources in the top directory -->
       <sources>

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/.DS_Store
----------------------------------------------------------------------
diff --git a/hbase-backup/.DS_Store b/hbase-backup/.DS_Store
new file mode 100644
index 0000000..ab57a77
Binary files /dev/null and b/hbase-backup/.DS_Store differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-backup/pom.xml b/hbase-backup/pom.xml
new file mode 100644
index 0000000..1b45c93
--- /dev/null
+++ b/hbase-backup/pom.xml
@@ -0,0 +1,265 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <!--
+  /**
+   * 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.
+   */
+  -->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>2.0.0-alpha2</version>
+    <relativePath>..</relativePath>
+  </parent>
+  <artifactId>hbase-backup</artifactId>
+  <name>Apache HBase - Backup</name>
+  <description>Backup for HBase</description>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+    </plugins>
+    <pluginManagement>
+      <plugins>
+        <!--This plugin's configuration is used to store Eclipse m2e settings
+             only. It has no influence on the Maven build itself.-->
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-compiler-plugin</artifactId>
+                    <versionRange>[3.2,)</versionRange>
+                    <goals>
+                      <goal>compile</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore/>
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
+  <dependencies>
+    <!-- Intra-project dependencies -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>jdk.tools</groupId>
+          <artifactId>jdk.tools</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-protocol-shaded</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <!-- General dependencies -->
+    <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-miscellaneous</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-netty</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <profiles>
+    <!-- profile against Hadoop 2.x: This is the default. -->
+    <profile>
+      <id>hadoop-2.0</id>
+      <activation>
+        <property>
+          <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+          <!--h2-->
+          <name>!hadoop.profile</name>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>com.github.stephenc.findbugs</groupId>
+          <artifactId>findbugs-annotations</artifactId>
+          <optional>true</optional>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </dependency>
+        <!--dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+          <exclusions>
+            <exclusion>
+              <groupId>net.java.dev.jets3t</groupId>
+              <artifactId>jets3t</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>javax.servlet.jsp</groupId>
+              <artifactId>jsp-api</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.mortbay.jetty</groupId>
+              <artifactId>jetty</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-server</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-core</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-json</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>javax.servlet</groupId>
+              <artifactId>servlet-api</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>tomcat</groupId>
+              <artifactId>jasper-compiler</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>tomcat</groupId>
+              <artifactId>jasper-runtime</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.google.code.findbugs</groupId>
+              <artifactId>jsr305</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency-->
+<!-- Hadoop needs Netty 3.x at test scope for the minicluster -->
+        <dependency>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+          <version>${netty.hadoop.version}</version>
+          <scope>test</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+    <!--
+      profile for building against Hadoop 3.0.x. Activate using:
+       mvn -Dhadoop.profile=3.0
+    -->
+    <profile>
+      <id>hadoop-3.0</id>
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>3.0</value>
+        </property>
+      </activation>
+      <properties>
+        <hadoop.version>3.0-SNAPSHOT</hadoop.version>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
new file mode 100644
index 0000000..9dc6382
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
@@ -0,0 +1,136 @@
+/**
+ * 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 array of backup ids
+   * @return total number of deleted sessions
+   * @throws IOException exception
+   */
+  int deleteBackups(String[] backupIds) throws IOException;
+
+  /**
+   * Merge backup images command
+   * @param backupIds array of backup ids of images to be merged
+   *        The resulting backup image will have the same backup id as the most
+   *        recent image from a list of images to be merged
+   * @throws IOException exception
+   */
+  void mergeBackups(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 array 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 array 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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
new file mode 100644
index 0000000..21d73cc
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
@@ -0,0 +1,53 @@
+/**
+ * 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.Configuration;
+import org.apache.hadoop.hbase.backup.impl.FullTableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
+import org.apache.hadoop.hbase.client.Connection;
+
+public class BackupClientFactory {
+
+  public static TableBackupClient create (Connection conn, String backupId, BackupRequest request)
+    throws IOException
+  {
+    Configuration conf = conn.getConfiguration();
+    try {
+      String clsName = conf.get(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS);
+      if (clsName != null) {
+        Class<?> clientImpl = Class.forName(clsName);
+        TableBackupClient client = (TableBackupClient) clientImpl.newInstance();
+        client.init(conn, backupId, request);
+        return client;
+      }
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+
+    BackupType type = request.getBackupType();
+    if (type == BackupType.FULL) {
+      return new FullTableBackupClient(conn, backupId, request);
+    } else {
+      return new IncrementalTableBackupClient(conn, backupId, request);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
new file mode 100644
index 0000000..007e4c1
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
new file mode 100644
index 0000000..9dd8531
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
@@ -0,0 +1,210 @@
+/**
+ * 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 if (BackupCommand.REPAIR.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.REPAIR;
+    } else if (BackupCommand.MERGE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.MERGE;
+    } 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;
+    } finally {
+      command.finish();
+    }
+    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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
new file mode 100644
index 0000000..ed554ad
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
@@ -0,0 +1,180 @@
+/*
+ * 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.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+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.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
+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.cleaner.BaseHFileCleanerDelegate;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
+
+/**
+ * Implementation of a file cleaner that checks if an hfile is still referenced by backup before
+ * deleting it from hfile archive directory.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class BackupHFileCleaner extends BaseHFileCleanerDelegate implements Abortable {
+  private static final Log LOG = LogFactory.getLog(BackupHFileCleaner.class);
+  private boolean stopped = false;
+  private boolean aborted;
+  private Configuration conf;
+  private Connection connection;
+  private long prevReadFromBackupTbl = 0, // timestamp of most recent read from backup:system table
+      secondPrevReadFromBackupTbl = 0; // timestamp of 2nd most recent read from backup:system table
+  //used by unit test to skip reading backup:system
+  private boolean checkForFullyBackedUpTables = true;
+  private List<TableName> fullyBackedUpTables = null;
+
+  private Set<String> getFilenameFromBulkLoad(Map<byte[], List<Path>>[] maps) {
+    Set<String> filenames = new HashSet<String>();
+    for (Map<byte[], List<Path>> map : maps) {
+      if (map == null) continue;
+      for (List<Path> paths : map.values()) {
+        for (Path p : paths) {
+          filenames.add(p.getName());
+        }
+      }
+    }
+    return filenames;
+  }
+
+  private Set<String> loadHFileRefs(List<TableName> tableList) throws IOException {
+    if (connection == null) {
+      connection = ConnectionFactory.createConnection(conf);
+    }
+    try (BackupSystemTable tbl = new BackupSystemTable(connection)) {
+      Map<byte[], List<Path>>[] res =
+          tbl.readBulkLoadedFiles(null, tableList);
+      secondPrevReadFromBackupTbl = prevReadFromBackupTbl;
+      prevReadFromBackupTbl = EnvironmentEdgeManager.currentTime();
+      return getFilenameFromBulkLoad(res);
+    }
+  }
+
+  @VisibleForTesting
+  void setCheckForFullyBackedUpTables(boolean b) {
+    checkForFullyBackedUpTables = b;
+  }
+  @Override
+  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
+    if (conf == null) {
+      return files;
+    }
+    // obtain the Set of TableName's which have been fully backed up
+    // so that we filter BulkLoad to be returned from server
+    if (checkForFullyBackedUpTables) {
+      if (connection == null) return files;
+      try (BackupSystemTable tbl = new BackupSystemTable(connection)) {
+        fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
+      } catch (IOException ioe) {
+        LOG.error("Failed to get tables which have been fully backed up, skipping checking", ioe);
+        return Collections.emptyList();
+      }
+      Collections.sort(fullyBackedUpTables);
+    }
+    final Set<String> hfileRefs;
+    try {
+      hfileRefs = loadHFileRefs(fullyBackedUpTables);
+    } catch (IOException ioe) {
+      LOG.error("Failed to read hfile references, skipping checking deletable files", ioe);
+      return Collections.emptyList();
+    }
+    Iterable<FileStatus> deletables = Iterables.filter(files, new Predicate<FileStatus>() {
+      @Override
+      public boolean apply(FileStatus file) {
+        // If the file is recent, be conservative and wait for one more scan of backup:system table
+        if (file.getModificationTime() > secondPrevReadFromBackupTbl) {
+          return false;
+        }
+        String hfile = file.getPath().getName();
+        boolean foundHFileRef = hfileRefs.contains(hfile);
+        return !foundHFileRef;
+      }
+    });
+    return deletables;
+  }
+
+  @Override
+  public boolean isFileDeletable(FileStatus fStat) {
+    // work is done in getDeletableFiles()
+    return true;
+  }
+
+  @Override
+  public void setConf(Configuration config) {
+    this.conf = config;
+    this.connection = null;
+    try {
+      this.connection = ConnectionFactory.createConnection(conf);
+    } catch (IOException ioe) {
+      LOG.error("Couldn't establish connection", ioe);
+    }
+  }
+
+  @Override
+  public void stop(String why) {
+    if (this.stopped) {
+      return;
+    }
+    if (this.connection != null) {
+      try {
+        this.connection.close();
+      } catch (IOException ioe) {
+        LOG.debug("Got " + ioe + " when closing connection");
+      }
+    }
+    this.stopped = true;
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    LOG.warn("Aborting ReplicationHFileCleaner because " + why, e);
+    this.aborted = true;
+    stop(why);
+  }
+
+  @Override
+  public boolean isAborted() {
+    return this.aborted;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
new file mode 100644
index 0000000..1765bf3
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
@@ -0,0 +1,550 @@
+/**
+ * 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 = BackupPhase.REQUEST;
+
+  /**
+   * 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;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return backupId;
+  }
+
+  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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
new file mode 100644
index 0000000..136782f
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Backup merge operation job interface. Concrete implementation is provided by backup provider, see
+ * {@link BackupRestoreFactory}
+ */
+
+@InterfaceAudience.Private
+public interface BackupMergeJob extends Configurable {
+
+  /**
+   * Run backup merge operation
+   * @param backupIds backup image ids
+   * @throws IOException
+   */
+  void run(String[] backupIds) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
new file mode 100644
index 0000000..595e862
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.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 java.io.IOException;
+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.Path;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+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.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * An Observer to facilitate backup operations
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class BackupObserver implements RegionObserver {
+  private static final Log LOG = LogFactory.getLog(BackupObserver.class);
+  @Override
+  public boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
+    List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths,
+    boolean hasLoaded) throws IOException {
+    Configuration cfg = ctx.getEnvironment().getConfiguration();
+    if (!hasLoaded) {
+      // there is no need to record state
+      return hasLoaded;
+    }
+    if (finalPaths == null || !BackupManager.isBackupEnabled(cfg)) {
+      LOG.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled");
+      return hasLoaded;
+    }
+    try (Connection connection = ConnectionFactory.createConnection(cfg);
+        BackupSystemTable tbl = new BackupSystemTable(connection)) {
+      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
+      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
+      TableName tableName = info.getTable();
+      if (!fullyBackedUpTables.contains(tableName)) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(tableName + " has not gone thru full backup");
+        }
+        return hasLoaded;
+      }
+      tbl.writePathsPostBulkLoad(tableName, info.getEncodedNameAsBytes(), finalPaths);
+      return hasLoaded;
+    } catch (IOException ioe) {
+      LOG.error("Failed to get tables which have been fully backed up", ioe);
+      return false;
+    }
+  }
+  @Override
+  public void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
+      final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
+    Configuration cfg = ctx.getEnvironment().getConfiguration();
+    if (pairs == null || pairs.isEmpty() || !BackupManager.isBackupEnabled(cfg)) {
+      LOG.debug("skipping recording bulk load in preCommitStoreFile since backup is disabled");
+      return;
+    }
+    try (Connection connection = ConnectionFactory.createConnection(cfg);
+        BackupSystemTable tbl = new BackupSystemTable(connection)) {
+      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
+      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
+      TableName tableName = info.getTable();
+      if (!fullyBackedUpTables.contains(tableName)) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(tableName + " has not gone thru full backup");
+        }
+        return;
+      }
+      tbl.writeFilesForBulkLoadPreCommit(tableName, info.getEncodedNameAsBytes(), family, pairs);
+      return;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
new file mode 100644
index 0000000..dadd861
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
new file mode 100644
index 0000000..48e70a1
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
@@ -0,0 +1,123 @@
+/**
+ * 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 = "backup:system";
+
+  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;
+
+
+  public static final String BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.attempts.max";
+  public static final int DEFAULT_BACKUP_MAX_ATTEMPTS = 10;
+
+  public static final String BACKUP_ATTEMPTS_PAUSE_MS_KEY = "hbase.backup.attempts.pause.ms";
+  public static final int DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS = 10000;
+
+  /*
+   *  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, REPAIR
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
new file mode 100644
index 0000000..d72c884
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.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.conf.Configuration;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
+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";
+  public final static String HBASE_BACKUP_MERGE_IMPL_CLASS = "hbase.backup.merge.class";
+
+  private BackupRestoreFactory() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Gets backup restore job
+   * @param conf configuration
+   * @return backup restore job 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 job instance
+   */
+  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;
+  }
+
+  /**
+   * Gets backup merge job
+   * @param conf configuration
+   * @return backup merge job instance
+   */
+  public static BackupMergeJob getBackupMergeJob(Configuration conf) {
+    Class<? extends BackupMergeJob> cls =
+        conf.getClass(HBASE_BACKUP_MERGE_IMPL_CLASS, MapReduceBackupMergeJob.class,
+          BackupMergeJob.class);
+    BackupMergeJob service = ReflectionUtils.newInstance(cls, conf);
+    service.setConf(conf);
+    return service;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
new file mode 100644
index 0000000..a5390a6
--- /dev/null
+++ b/hbase-backup/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();
+  }
+}


[23/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 6330899..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
+++ /dev/null
@@ -1,387 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-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.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.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.
-   * @return The new HashMap of RS log time stamps after the log roll for this incremental backup.
-   * @throws IOException exception
-   */
-  public HashMap<String, Long> getIncrBackupLogFileMap()
-      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());
-    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
-    backupInfo.setIncrBackupFileList(logList);
-
-    return newTimestamps;
-  }
-
-  /**
-   * Get list of WAL files eligible for incremental backup
-   * @return list of WAL files
-   * @throws IOException
-   */
-  public List<String> getIncrBackupLogFileList()
-      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.");
-    }
-
-    newTimestamps = readRegionServerLastLogRollResult();
-
-    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
-    List<WALItem> logFromSystemTable =
-        getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps, getBackupInfo()
-            .getBackupRootDir());
-
-    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
-    backupInfo.setIncrBackupFileList(logList);
-
-    return logList;
-  }
-
-
-  private List<String> excludeAlreadyBackedUpWALs(List<String> logList,
-      List<WALItem> logFromSystemTable) {
-
-    Set<String> walFileNameSet = convertToSet(logFromSystemTable);
-
-    List<String> list = new ArrayList<String>();
-    for (int i=0; i < logList.size(); i++) {
-      Path p = new Path(logList.get(i));
-      String name  = p.getName();
-      if (walFileNameSet.contains(name)) continue;
-      list.add(logList.get(i));
-    }
-    return list;
-  }
-
-  /**
-   * Create Set of WAL file names (not full path names)
-   * @param logFromSystemTable
-   * @return set of WAL file names
-   */
-  private Set<String> convertToSet(List<WALItem> logFromSystemTable) {
-
-    Set<String> set = new HashSet<String>();
-    for (int i=0; i < logFromSystemTable.size(); i++) {
-      WALItem item = logFromSystemTable.get(i);
-      set.add(item.walFile);
-    }
-    return set;
-  }
-
-  /**
-   * 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/37c65946/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
deleted file mode 100644
index 6d48c32..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
+++ /dev/null
@@ -1,377 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-
-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.FileUtil;
-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.BackupInfo.BackupPhase;
-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.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.mapreduce.WALPlayer;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.HFileArchiveUtil;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.util.Tool;
-
-/**
- * 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);
-
-  protected IncrementalTableBackupClient() {
-  }
-
-  public IncrementalTableBackupClient(final Connection conn, final String backupId,
-      BackupRequest request) throws IOException {
-    super(conn, backupId, request);
-  }
-
-  protected List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
-    FileSystem fs = FileSystem.get(conf);
-    List<String> list = new ArrayList<String>();
-    for (String file : incrBackupFileList) {
-      Path p = new Path(file);
-      if (fs.exists(p) || isActiveWalPath(p)) {
-        list.add(file);
-      } else {
-        LOG.warn("Can't find file: " + file);
-      }
-    }
-    return list;
-  }
-
-  /**
-   * Check if a given path is belongs to active WAL directory
-   * @param p path
-   * @return true, if yes
-   */
-  protected boolean isActiveWalPath(Path p) {
-    return !AbstractFSWALProvider.isArchivedLogFile(p);
-  }
-
-  protected static int getIndex(TableName tbl, List<TableName> sTableList) {
-    if (sTableList == null) return 0;
-    for (int i = 0; i < sTableList.size(); i++) {
-      if (tbl.equals(sTableList.get(i))) {
-        return i;
-      }
-    }
-    return -1;
-  }
-
-  /*
-   * Reads bulk load records from backup table, iterates through the records and forms the paths
-   * for bulk loaded hfiles. Copies the bulk loaded hfiles to backup destination
-   * @param sTableList list of tables to be backed up
-   * @return map of table to List of files
-   */
-  protected Map<byte[], List<Path>>[] handleBulkLoad(List<TableName> sTableList) throws IOException {
-    Map<byte[], List<Path>>[] mapForSrc = new Map[sTableList.size()];
-    Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>> pair =
-    backupManager.readBulkloadRows(sTableList);
-    Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> map = pair.getFirst();
-    FileSystem fs = FileSystem.get(conf);
-    FileSystem tgtFs;
-    try {
-      tgtFs = FileSystem.get(new URI(backupInfo.getBackupRootDir()), conf);
-    } catch (URISyntaxException use) {
-      throw new IOException("Unable to get FileSystem", use);
-    }
-    Path rootdir = FSUtils.getRootDir(conf);
-    Path tgtRoot = new Path(new Path(backupInfo.getBackupRootDir()), backupId);
-    for (Map.Entry<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> tblEntry :
-      map.entrySet()) {
-      TableName srcTable = tblEntry.getKey();
-      int srcIdx = getIndex(srcTable, sTableList);
-      if (srcIdx < 0) {
-        LOG.warn("Couldn't find " + srcTable + " in source table List");
-        continue;
-      }
-      if (mapForSrc[srcIdx] == null) {
-        mapForSrc[srcIdx] = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
-      }
-      Path tblDir = FSUtils.getTableDir(rootdir, srcTable);
-      Path tgtTable = new Path(new Path(tgtRoot, srcTable.getNamespaceAsString()),
-          srcTable.getQualifierAsString());
-      for (Map.Entry<String,Map<String,List<Pair<String, Boolean>>>> regionEntry :
-        tblEntry.getValue().entrySet()){
-        String regionName = regionEntry.getKey();
-        Path regionDir = new Path(tblDir, regionName);
-        // map from family to List of hfiles
-        for (Map.Entry<String,List<Pair<String, Boolean>>> famEntry :
-          regionEntry.getValue().entrySet()) {
-          String fam = famEntry.getKey();
-          Path famDir = new Path(regionDir, fam);
-          List<Path> files;
-          if (!mapForSrc[srcIdx].containsKey(fam.getBytes())) {
-            files = new ArrayList<Path>();
-            mapForSrc[srcIdx].put(fam.getBytes(), files);
-          } else {
-            files = mapForSrc[srcIdx].get(fam.getBytes());
-          }
-          Path archiveDir = HFileArchiveUtil.getStoreArchivePath(conf, srcTable, regionName, fam);
-          String tblName = srcTable.getQualifierAsString();
-          Path tgtFam = new Path(new Path(tgtTable, regionName), fam);
-          if (!tgtFs.mkdirs(tgtFam)) {
-            throw new IOException("couldn't create " + tgtFam);
-          }
-          for (Pair<String, Boolean> fileWithState : famEntry.getValue()) {
-            String file = fileWithState.getFirst();
-            boolean raw = fileWithState.getSecond();
-            int idx = file.lastIndexOf("/");
-            String filename = file;
-            if (idx > 0) {
-              filename = file.substring(idx+1);
-            }
-            Path p = new Path(famDir, filename);
-            Path tgt = new Path(tgtFam, filename);
-            Path archive = new Path(archiveDir, filename);
-            if (fs.exists(p)) {
-              if (LOG.isTraceEnabled()) {
-                LOG.trace("found bulk hfile " + file + " in " + famDir + " for " + tblName);
-              }
-              try {
-                if (LOG.isTraceEnabled()) {
-                  LOG.trace("copying " + p + " to " + tgt);
-                }
-                FileUtil.copy(fs, p, tgtFs, tgt, false,conf);
-              } catch (FileNotFoundException e) {
-                LOG.debug("copying archive " + archive + " to " + tgt);
-                try {
-                  FileUtil.copy(fs, archive, tgtFs, tgt, false, conf);
-                } catch (FileNotFoundException fnfe) {
-                  if (!raw) throw fnfe;
-                }
-              }
-            } else {
-              LOG.debug("copying archive " + archive + " to " + tgt);
-              try {
-                FileUtil.copy(fs, archive, tgtFs, tgt, false, conf);
-              } catch (FileNotFoundException fnfe) {
-                if (!raw) throw fnfe;
-              }
-            }
-            files.add(tgt);
-          }
-        }
-      }
-    }
-    backupManager.writeBulkLoadedFiles(sTableList, mapForSrc);
-    backupManager.removeBulkLoadedRows(sTableList, pair.getSecond());
-    return mapForSrc;
-  }
-
-  @Override
-  public void execute() throws IOException {
-
-    try {
-      // case PREPARE_INCREMENTAL:
-      beginBackup(backupManager, backupInfo);
-      backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
-      LOG.debug("For incremental backup, current table set is "
-          + backupManager.getIncrementalBackupTableSet());
-      newTimestamps =
-          ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
-    } catch (Exception e) {
-      // fail the overall backup and return
-      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
-        BackupType.INCREMENTAL, conf);
-      return;
-    }
-
-    // case INCREMENTAL_COPY:
-    try {
-      // copy out the table and region info files for each table
-      BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
-      // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
-      convertWALsToHFiles(backupInfo);
-      incrementalCopyHFiles(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);
-      return;
-    }
-    // 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 {
-      // 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);
-
-      handleBulkLoad(backupInfo.getTableNames());
-      // backup complete
-      completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
-
-    } catch (IOException e) {
-      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
-        BackupType.INCREMENTAL, conf);
-    }
-  }
-
-  protected void incrementalCopyHFiles(BackupInfo backupInfo) throws Exception {
-
-    try {
-      LOG.debug("Incremental copy HFiles 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 = new ArrayList<String>();
-      // Add Bulk output
-      incrBackupFileList.add(getBulkOutputDir().toString());
-      String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
-      strArr[strArr.length - 1] = backupInfo.getBackupRootDir();
-      BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf);
-      int res = copyService.copy(backupInfo, backupManager, conf, BackupType.INCREMENTAL, strArr);
-      if (res != 0) {
-        LOG.error("Copy incremental HFile files failed with return code: " + res + ".");
-        throw new IOException("Failed copy from " + StringUtils.join(incrBackupFileList, ',')
-            + " to " + backupInfo.getHLogTargetDir());
-      }
-      LOG.debug("Incremental copy HFiles from " + StringUtils.join(incrBackupFileList, ',')
-          + " to " + backupInfo.getBackupRootDir() + " finished.");
-    } finally {
-      deleteBulkLoadDirectory();
-    }
-  }
-
-  protected void deleteBulkLoadDirectory() throws IOException {
-    // delete original bulk load directory on method exit
-    Path path = getBulkOutputDir();
-    FileSystem fs = FileSystem.get(conf);
-    boolean result = fs.delete(path, true);
-    if (!result) {
-      LOG.warn("Could not delete " + path);
-    }
-
-  }
-
-  protected void convertWALsToHFiles(BackupInfo backupInfo) throws IOException {
-    // get incremental backup file list and prepare parameters for DistCp
-    List<String> incrBackupFileList = backupInfo.getIncrBackupFileList();
-    // Get list of tables in incremental backup set
-    Set<TableName> tableSet = backupManager.getIncrementalBackupTableSet();
-    // filter missing files out (they have been copied by previous backups)
-    incrBackupFileList = filterMissingFiles(incrBackupFileList);
-    for (TableName table : tableSet) {
-      // Check if table exists
-      if (tableExists(table, conn)) {
-        walToHFiles(incrBackupFileList, table);
-      } else {
-        LOG.warn("Table " + table + " does not exists. Skipping in WAL converter");
-      }
-    }
-  }
-
-
-  protected boolean tableExists(TableName table, Connection conn) throws IOException {
-    try (Admin admin = conn.getAdmin();) {
-      return admin.tableExists(table);
-    }
-  }
-
-  protected void walToHFiles(List<String> dirPaths, TableName tableName) throws IOException {
-
-    Tool player = new WALPlayer();
-
-    // Player reads all files in arbitrary directory structure and creates
-    // a Map task for each file. We use ';' as separator
-    // because WAL file names contains ','
-    String dirs = StringUtils.join(dirPaths, ';');
-
-    Path bulkOutputPath = getBulkOutputDirForTable(tableName);
-    conf.set(WALPlayer.BULK_OUTPUT_CONF_KEY, bulkOutputPath.toString());
-    conf.set(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ";");
-    String[] playerArgs = { dirs, tableName.getNameAsString() };
-
-    try {
-      player.setConf(conf);
-      int result = player.run(playerArgs);
-      if(result != 0) {
-        throw new IOException("WAL Player failed");
-      }
-      conf.unset(WALPlayer.INPUT_FILES_SEPARATOR_KEY);
-    } catch (IOException e) {
-      throw e;
-    } catch (Exception ee) {
-      throw new IOException("Can not convert from directory " + dirs
-          + " (check Hadoop, HBase and WALPlayer M/R job logs) ", ee);
-    }
-  }
-
-  protected Path getBulkOutputDirForTable(TableName table) {
-    Path tablePath = getBulkOutputDir();
-    tablePath = new Path(tablePath, table.getNamespaceAsString());
-    tablePath = new Path(tablePath, table.getQualifierAsString());
-    return new Path(tablePath, "data");
-  }
-
-  protected Path getBulkOutputDir() {
-    String backupId = backupInfo.getBackupId();
-    Path path = new Path(backupInfo.getBackupRootDir());
-    path = new Path(path, ".tmp");
-    path = new Path(path, backupId);
-    return path;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index ea7a7b8..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-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.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.BackupUtils;
-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;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.LoadQueueItem;
-
-/**
- * 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(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 fileBackupDir =
-          HBackupFileSystem.getTableBackupDataDir(im.getRootDir(), im.getBackupId(), sTable);
-      dirList.add(new Path(fileBackupDir));
-    }
-
-    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;
-    Set<String> backupIdSet = new HashSet<>();
-
-    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));
-          if (image.getType() == BackupType.INCREMENTAL) {
-            backupIdSet.add(image.getBackupId());
-            LOG.debug("adding " + image.getBackupId() + " for bulk load");
-          }
-        }
-      }
-    }
-    try (BackupSystemTable table = new BackupSystemTable(conn)) {
-      List<TableName> sTableList = Arrays.asList(sTableArray);
-      for (String id : backupIdSet) {
-        LOG.debug("restoring bulk load for " + id);
-        Map<byte[], List<Path>>[] mapForSrc = table.readBulkLoadedFiles(id, sTableList);
-        Map<LoadQueueItem, ByteBuffer> loaderResult;
-        conf.setBoolean(LoadIncrementalHFiles.ALWAYS_COPY_FILES, true);
-        LoadIncrementalHFiles loader = BackupUtils.createLoader(conf);
-        for (int i = 0; i < sTableList.size(); i++) {
-          if (mapForSrc[i] != null && !mapForSrc[i].isEmpty()) {
-            loaderResult = loader.run(null, mapForSrc[i], tTableArray[i]);
-            LOG.debug("bulk loading " + sTableList.get(i) + " to " + tTableArray[i]);
-            if (loaderResult.isEmpty()) {
-              String msg = "Couldn't bulk load for " + sTableList.get(i) + " to " + tTableArray[i];
-              LOG.error(msg);
-              throw new IOException(msg);
-            }
-          }
-        }
-      }
-    }
-    LOG.debug("restoreStage finished");
-  }
-
-  static long getTsFromBackupId(String backupId) {
-    if (backupId == null) {
-      return 0;
-    }
-    return Long.parseLong(backupId.substring(backupId.lastIndexOf("_") + 1));
-  }
-
-  static boolean withinRange(long a, long lower, long upper) {
-    if (a < lower || a > upper) {
-      return false;
-    }
-    return true;
-  }
-
-  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/37c65946/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
deleted file mode 100644
index 6eec460..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
+++ /dev/null
@@ -1,436 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
-/**
- * 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 {
-
-  public static final String BACKUP_CLIENT_IMPL_CLASS = "backup.client.impl.class";
-
-  @VisibleForTesting
-  public static final String BACKUP_TEST_MODE_STAGE = "backup.test.mode.stage";
-
-  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() {
-  }
-
-  public TableBackupClient(final Connection conn, final String backupId, BackupRequest request)
-      throws IOException {
-    init(conn, backupId, request);
-  }
-
-  public void init(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());
-    }
-    // Start new session
-    backupManager.startBackupSession();
-  }
-
-  /**
-   * Begin the overall backup.
-   * @param backupInfo backup info
-   * @throws IOException exception
-   */
-  protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo)
-      throws IOException {
-
-    BackupSystemTable.snapshot(conn);
-    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.");
-    }
-  }
-
-  protected 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
-   */
-  protected static void deleteSnapshots(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);
-      }
-      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
-   */
-  protected static 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.
-   */
-  protected static 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.debug("Cleaning up uncompleted backup data at " + targetDirPath.toString()
-                + " done.");
-          } else {
-            LOG.debug("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 {
-
-    try {
-      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);
-      cleanupAndRestoreBackupSystem(conn, backupInfo, conf);
-      // If backup session is updated to FAILED state - means we
-      // processed recovery already.
-      backupManager.updateBackupInfo(backupInfo);
-      backupManager.finishBackupSession();
-      LOG.error("Backup " + backupInfo.getBackupId() + " failed.");
-    } catch (IOException ee) {
-      LOG.error("Please run backup repair tool manually to restore backup system integrity");
-      throw ee;
-    }
-  }
-
-  public static void cleanupAndRestoreBackupSystem (Connection conn, BackupInfo backupInfo,
-      Configuration conf) throws IOException
-  {
-    BackupType type = backupInfo.getType();
-     // 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) {
-       deleteSnapshots(conn, backupInfo, conf);
-       cleanupExportSnapshotLog(conf);
-     }
-     BackupSystemTable.restoreFromSnapshot(conn);
-     BackupSystemTable.deleteSnapshot(conn);
-     // 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);
-  }
-
-
-
-  /**
-   * 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
-   */
-  protected 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
-   */
-  protected 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
-   */
-  protected 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);
-
-    // 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);
-    }
-
-    // when full backup is done:
-    // - delete HBase snapshot
-    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
-    // snapshots
-    if (type == BackupType.FULL) {
-      deleteSnapshots(conn, backupInfo, conf);
-      cleanupExportSnapshotLog(conf);
-    } else if (type == BackupType.INCREMENTAL) {
-      cleanupDistCpLog(backupInfo, conf);
-    }
-    BackupSystemTable.deleteSnapshot(conn);
-    backupManager.updateBackupInfo(backupInfo);
-
-    // Finish active session
-    backupManager.finishBackupSession();
-
-    LOG.info("Backup " + backupInfo.getBackupId() + " completed.");
-  }
-
-  /**
-   * Backup request execution
-   * @throws IOException
-   */
-  public abstract void execute() throws IOException;
-
-  @VisibleForTesting
-  protected Stage getTestStage() {
-    return Stage.valueOf("stage_"+ conf.getInt(BACKUP_TEST_MODE_STAGE, 0));
-  }
-
-  @VisibleForTesting
-  protected void failStageIf(Stage stage) throws IOException {
-    Stage current = getTestStage();
-    if (current == stage) {
-      throw new IOException("Failed stage " + stage+" in testing");
-    }
-  }
-
-  public static enum Stage {
-    stage_0, stage_1, stage_2, stage_3, stage_4
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 016d1a4..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java
+++ /dev/null
@@ -1,344 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.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/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
deleted file mode 100644
index 00c5b83..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
+++ /dev/null
@@ -1,321 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.mapreduce;
-
-import static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-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.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.BackupMergeJob;
-import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
-import org.apache.hadoop.hbase.backup.HBackupFileSystem;
-import org.apache.hadoop.hbase.backup.impl.BackupManifest;
-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.Pair;
-import org.apache.hadoop.util.Tool;
-
-/**
- * MapReduce implementation of {@link BackupMergeJob}
- * Must be initialized with configuration of a backup destination cluster
- *
- */
-
-@InterfaceAudience.Private
-public class MapReduceBackupMergeJob implements BackupMergeJob {
-  public static final Log LOG = LogFactory.getLog(MapReduceBackupMergeJob.class);
-
-  protected Tool player;
-  protected Configuration conf;
-
-  public MapReduceBackupMergeJob() {
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  @Override
-  public void run(String[] backupIds) throws IOException {
-    String bulkOutputConfKey;
-
-    // TODO : run player on remote cluster
-    player = new MapReduceHFileSplitterJob();
-    bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
-    // Player reads all files in arbitrary directory structure and creates
-    // a Map task for each file
-    String bids = StringUtils.join(backupIds, ",");
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Merge backup images " + bids);
-    }
-
-    List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
-    boolean finishedTables = false;
-    Connection conn = ConnectionFactory.createConnection(getConf());
-    BackupSystemTable table = new BackupSystemTable(conn);
-    FileSystem fs = FileSystem.get(getConf());
-
-    try {
-
-      // Get exclusive lock on backup system
-      table.startBackupExclusiveOperation();
-      // Start merge operation
-      table.startMergeOperation(backupIds);
-
-      // Select most recent backup id
-      String mergedBackupId = findMostRecentBackupId(backupIds);
-
-      TableName[] tableNames = getTableNamesInBackupImages(backupIds);
-      String backupRoot = null;
-
-      BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
-      backupRoot = bInfo.getBackupRootDir();
-
-      for (int i = 0; i < tableNames.length; i++) {
-
-        LOG.info("Merge backup images for " + tableNames[i]);
-
-        // Find input directories for table
-
-        Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
-        String dirs = StringUtils.join(dirPaths, ",");
-        Path bulkOutputPath =
-            BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
-              getConf(), false);
-        // Delete content if exists
-        if (fs.exists(bulkOutputPath)) {
-          if (!fs.delete(bulkOutputPath, true)) {
-            LOG.warn("Can not delete: " + bulkOutputPath);
-          }
-        }
-        Configuration conf = getConf();
-        conf.set(bulkOutputConfKey, bulkOutputPath.toString());
-        String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
-
-        int result = 0;
-
-        player.setConf(getConf());
-        result = player.run(playerArgs);
-        if (!succeeded(result)) {
-          throw new IOException("Can not merge backup images for " + dirs
-              + " (check Hadoop/MR and HBase logs). Player return code =" + result);
-        }
-        // Add to processed table list
-        processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
-        LOG.debug("Merge Job finished:" + result);
-      }
-      List<TableName> tableList = toTableNameList(processedTableList);
-      table.updateProcessedTablesForMerge(tableList);
-      finishedTables = true;
-
-      // Move data
-      for (Pair<TableName, Path> tn : processedTableList) {
-        moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
-      }
-
-      // Delete old data and update manifest
-      List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
-      deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
-      updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
-      // Finish merge session
-      table.finishMergeOperation();
-      // Release lock
-      table.finishBackupExclusiveOperation();
-    } catch (RuntimeException e) {
-
-      throw e;
-    } catch (Exception e) {
-      LOG.error(e);
-      if (!finishedTables) {
-        // cleanup bulk directories and finish merge
-        // merge MUST be repeated (no need for repair)
-        cleanupBulkLoadDirs(fs, toPathList(processedTableList));
-        table.finishMergeOperation();
-        table.finishBackupExclusiveOperation();
-        throw new IOException("Backup merge operation failed, you should try it again", e);
-      } else {
-        // backup repair must be run
-        throw new IOException(
-            "Backup merge operation failed, run backup repair tool to restore system's integrity",
-            e);
-      }
-    } finally {
-      table.close();
-      conn.close();
-    }
-  }
-
-  protected List<Path> toPathList(List<Pair<TableName, Path>> processedTableList) {
-    ArrayList<Path> list = new ArrayList<Path>();
-    for (Pair<TableName, Path> p : processedTableList) {
-      list.add(p.getSecond());
-    }
-    return list;
-  }
-
-  protected List<TableName> toTableNameList(List<Pair<TableName, Path>> processedTableList) {
-    ArrayList<TableName> list = new ArrayList<TableName>();
-    for (Pair<TableName, Path> p : processedTableList) {
-      list.add(p.getFirst());
-    }
-    return list;
-  }
-
-  protected void cleanupBulkLoadDirs(FileSystem fs, List<Path> pathList) throws IOException {
-    for (Path path : pathList) {
-
-      if (!fs.delete(path, true)) {
-        LOG.warn("Can't delete " + path);
-      }
-    }
-  }
-
-  protected void updateBackupManifest(String backupRoot, String mergedBackupId,
-      List<String> backupsToDelete) throws IllegalArgumentException, IOException {
-
-    BackupManifest manifest =
-        HBackupFileSystem.getManifest(conf, new Path(backupRoot), mergedBackupId);
-    manifest.getBackupImage().removeAncestors(backupsToDelete);
-    // save back
-    manifest.store(conf);
-
-  }
-
-  protected void deleteBackupImages(List<String> backupIds, Connection conn, FileSystem fs,
-      String backupRoot) throws IOException {
-
-    // Delete from backup system table
-    try (BackupSystemTable table = new BackupSystemTable(conn);) {
-      for (String backupId : backupIds) {
-        table.deleteBackupInfo(backupId);
-      }
-    }
-
-    // Delete from file system
-    for (String backupId : backupIds) {
-      Path backupDirPath = HBackupFileSystem.getBackupPath(backupRoot, backupId);
-
-      if (!fs.delete(backupDirPath, true)) {
-        LOG.warn("Could not delete " + backupDirPath);
-      }
-    }
-  }
-
-  protected List<String> getBackupIdsToDelete(String[] backupIds, String mergedBackupId) {
-    List<String> list = new ArrayList<String>();
-    for (String id : backupIds) {
-      if (id.equals(mergedBackupId)) {
-        continue;
-      }
-      list.add(id);
-    }
-    return list;
-  }
-
-  protected void moveData(FileSystem fs, String backupRoot, Path bulkOutputPath, TableName tableName,
-      String mergedBackupId) throws IllegalArgumentException, IOException {
-
-    Path dest =
-        new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, mergedBackupId, tableName));
-
-    // Delete all in dest
-    if (!fs.delete(dest, true)) {
-      throw new IOException("Could not delete " + dest);
-    }
-
-    FileStatus[] fsts = fs.listStatus(bulkOutputPath);
-    for (FileStatus fst : fsts) {
-      if (fst.isDirectory()) {
-        fs.rename(fst.getPath().getParent(), dest);
-      }
-    }
-
-  }
-
-  protected String findMostRecentBackupId(String[] backupIds) {
-    long recentTimestamp = Long.MIN_VALUE;
-    for (String backupId : backupIds) {
-      long ts = Long.parseLong(backupId.split("_")[1]);
-      if (ts > recentTimestamp) {
-        recentTimestamp = ts;
-      }
-    }
-    return BackupRestoreConstants.BACKUPID_PREFIX + recentTimestamp;
-  }
-
-  protected TableName[] getTableNamesInBackupImages(String[] backupIds) throws IOException {
-
-    Set<TableName> allSet = new HashSet<TableName>();
-
-    try (Connection conn = ConnectionFactory.createConnection(conf);
-        BackupSystemTable table = new BackupSystemTable(conn);) {
-      for (String backupId : backupIds) {
-        BackupInfo bInfo = table.readBackupInfo(backupId);
-
-        allSet.addAll(bInfo.getTableNames());
-      }
-    }
-
-    TableName[] ret = new TableName[allSet.size()];
-    return allSet.toArray(ret);
-  }
-
-  protected Path[] findInputDirectories(FileSystem fs, String backupRoot, TableName tableName,
-      String[] backupIds) throws IOException {
-
-    List<Path> dirs = new ArrayList<Path>();
-
-    for (String backupId : backupIds) {
-      Path fileBackupDirPath =
-          new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, backupId, tableName));
-      if (fs.exists(fileBackupDirPath)) {
-        dirs.add(fileBackupDirPath);
-      } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("File: " + fileBackupDirPath + " does not exist.");
-        }
-      }
-    }
-    Path[] ret = new Path[dirs.size()];
-    return dirs.toArray(ret);
-  }
-
-}


[17/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
new file mode 100644
index 0000000..1c43e88
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
@@ -0,0 +1,146 @@
+/**
+ *
+ * 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;
+  }
+
+  public static String getTableBackupDataDir(String backupRootDir, String backupId,
+      TableName tableName) {
+    return getTableBackupDir(backupRootDir, backupId, tableName) + Path.SEPARATOR + "data";
+  }
+
+  public static Path getBackupPath(String backupRootDir, String backupId) {
+    return new Path(backupRootDir + Path.SEPARATOR + backupId);
+  }
+
+  /**
+   * Given the backup root dir, backup id and the table name, return the backup image location,
+   * which is also where the backup manifest file is. return value look like:
+   * "hdfs://backup.hbase.org:9000/user/biadmin/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));
+  }
+
+  // TODO we do not keep WAL files anymore
+  // Move manifest file to other place
+  private static Path getManifestPath(Configuration conf, Path backupRootPath, String backupId)
+      throws IOException {
+    Path manifestPath = null;
+
+    FileSystem fs = backupRootPath.getFileSystem(conf);
+    manifestPath =
+        new Path(getBackupPath(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(Configuration conf, Path backupRootPath, String backupId) throws IOException {
+    BackupManifest manifest =
+        new BackupManifest(conf, getManifestPath(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(conf, backupRootPath, backupId);
+      backupManifestMap.put(tableName, manifest);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
new file mode 100644
index 0000000..1becb75
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
new file mode 100644
index 0000000..82a1b56
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
new file mode 100644
index 0000000..86fb963
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
new file mode 100644
index 0000000..de3ad5a
--- /dev/null
+++ b/hbase-backup/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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
new file mode 100644
index 0000000..99fb06c
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
@@ -0,0 +1,743 @@
+/**
+ * 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.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.BackupClientFactory;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupMergeJob;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
+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.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+@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 {
+  }
+
+  @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 {
+
+    int totalDeleted = 0;
+    Map<String, HashSet<TableName>> allTablesMap = new HashMap<String, HashSet<TableName>>();
+
+    boolean deleteSessionStarted = false;
+    boolean snapshotDone = false;
+    try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) {
+
+      // Step 1: Make sure there is no active session
+      // is running by using startBackupSession API
+      // If there is an active session in progress, exception will be thrown
+      try {
+        sysTable.startBackupExclusiveOperation();
+        deleteSessionStarted = true;
+      } catch (IOException e) {
+        LOG.warn("You can not run delete command while active backup session is in progress. \n"
+            + "If there is no active backup session running, run backup repair utility to restore \n"
+            + "backup system integrity.");
+        return -1;
+      }
+
+      // Step 2: Make sure there is no failed session
+      List<BackupInfo> list = sysTable.getBackupInfos(BackupState.RUNNING);
+      if (list.size() != 0) {
+        // ailed sessions found
+        LOG.warn("Failed backup session found. Run backup repair tool first.");
+        return -1;
+      }
+
+      // Step 3: Record delete session
+      sysTable.startDeleteOperation(backupIds);
+      // Step 4: Snapshot backup system table
+      if (!BackupSystemTable.snapshotExists(conn)) {
+        BackupSystemTable.snapshot(conn);
+      } else {
+        LOG.warn("Backup system table snapshot exists");
+      }
+      snapshotDone = true;
+      try {
+        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);
+        // Finish
+        sysTable.finishDeleteOperation();
+        // delete snapshot
+        BackupSystemTable.deleteSnapshot(conn);
+      } catch (IOException e) {
+        // Fail delete operation
+        // Step 1
+        if (snapshotDone) {
+          if (BackupSystemTable.snapshotExists(conn)) {
+            BackupSystemTable.restoreFromSnapshot(conn);
+            // delete snapshot
+            BackupSystemTable.deleteSnapshot(conn);
+            // We still have record with unfinished delete operation
+            LOG.error("Delete operation failed, please run backup repair utility to restore "
+                + "backup system integrity", e);
+            throw e;
+          } else {
+            LOG.warn("Delete operation succeeded, there were some errors: ", e);
+          }
+        }
+
+      } finally {
+        if (deleteSessionStarted) {
+          sysTable.finishBackupExclusiveOperation();
+        }
+      }
+    }
+    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() + " ...");
+      // Step 1: clean up data for backup session (idempotent)
+      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 = getAffectedBackupSessions(backupInfo, tn, sysTable);
+        for (BackupInfo info : affectedBackups) {
+          if (info.equals(backupInfo)) {
+            continue;
+          }
+          removeTableFromBackupImage(info, tn, sysTable);
+        }
+      }
+      Map<byte[], String> map = sysTable.readBulkLoadedFiles(backupId);
+      FileSystem fs = FileSystem.get(conn.getConfiguration());
+      boolean success = true;
+      int numDeleted = 0;
+      for (String f : map.values()) {
+        Path p = new Path(f);
+        try {
+          LOG.debug("Delete backup info " + p + " for " + backupInfo.getBackupId());
+          if (!fs.delete(p)) {
+            if (fs.exists(p)) {
+              LOG.warn(f + " was not deleted");
+              success = false;
+            }
+          } else {
+            numDeleted++;
+          }
+        } catch (IOException ioe) {
+          LOG.warn(f + " was not deleted", ioe);
+          success = false;
+        }
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(numDeleted + " bulk loaded files out of " + map.size() + " were deleted");
+      }
+      if (success) {
+        sysTable.deleteBulkLoadedFiles(map);
+      }
+
+      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());
+        // Idempotent operation
+        BackupUtils.cleanupBackupData(info, conn.getConfiguration());
+      } else {
+        info.setTables(tables);
+        sysTable.updateBackupInfo(info);
+        // Now, clean up directory for table (idempotent)
+        cleanupBackupDir(info, tn, conn.getConfiguration());
+      }
+    }
+  }
+
+  private List<BackupInfo> getAffectedBackupSessions(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();
+
+    TableBackupClient client = null;
+    try {
+      client = BackupClientFactory.create(conn, backupId, request);
+    } catch (IOException e) {
+      LOG.error("There is an active session already running");
+      throw e;
+    }
+
+    client.execute();
+
+    return backupId;
+  }
+
+  private List<TableName> excludeNonExistingTables(List<TableName> tableList,
+      List<TableName> nonExistingTableList) {
+
+    for (TableName table : nonExistingTableList) {
+      tableList.remove(table);
+    }
+    return tableList;
+  }
+
+  @Override
+  public void mergeBackups(String[] backupIds) throws IOException {
+    try (final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+      checkIfValidForMerge(backupIds, sysTable);
+      BackupMergeJob job = BackupRestoreFactory.getBackupMergeJob(conn.getConfiguration());
+      job.run(backupIds);
+    }
+  }
+
+  /**
+   * Verifies that backup images are valid for merge.
+   *
+   * <ul>
+   * <li>All backups MUST be in the same destination
+   * <li>No FULL backups are allowed - only INCREMENTAL
+   * <li>All backups must be in COMPLETE state
+   * <li>No holes in backup list are allowed
+   * </ul>
+   * <p>
+   * @param backupIds list of backup ids
+   * @param table backup system table
+   * @throws IOException
+   */
+  private void checkIfValidForMerge(String[] backupIds, BackupSystemTable table) throws IOException {
+    String backupRoot = null;
+
+    final Set<TableName> allTables = new HashSet<TableName>();
+    final Set<String> allBackups = new HashSet<String>();
+    long minTime = Long.MAX_VALUE, maxTime = Long.MIN_VALUE;
+    for (String backupId : backupIds) {
+      BackupInfo bInfo = table.readBackupInfo(backupId);
+      if (bInfo == null) {
+        String msg = "Backup session " + backupId + " not found";
+        throw new IOException(msg);
+      }
+      if (backupRoot == null) {
+        backupRoot = bInfo.getBackupRootDir();
+      } else if (!bInfo.getBackupRootDir().equals(backupRoot)) {
+        throw new IOException("Found different backup destinations in a list of a backup sessions \n"
+            + "1. " + backupRoot + "\n" + "2. " + bInfo.getBackupRootDir());
+      }
+      if (bInfo.getType() == BackupType.FULL) {
+        throw new IOException("FULL backup image can not be merged for: \n" + bInfo);
+      }
+
+      if (bInfo.getState() != BackupState.COMPLETE) {
+        throw new IOException("Backup image " + backupId
+            + " can not be merged becuase of its state: " + bInfo.getState());
+      }
+      allBackups.add(backupId);
+      allTables.addAll(bInfo.getTableNames());
+      long time = bInfo.getStartTs();
+      if (time < minTime) {
+        minTime = time;
+      }
+      if (time > maxTime) {
+        maxTime = time;
+      }
+    }
+
+
+    final long startRangeTime  = minTime;
+    final long endRangeTime = maxTime;
+    final String backupDest = backupRoot;
+    // Check we have no 'holes' in backup id list
+    // Filter 1 : backupRoot
+    // Filter 2 : time range filter
+    // Filter 3 : table filter
+
+    BackupInfo.Filter destinationFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        return info.getBackupRootDir().equals(backupDest);
+      }
+    };
+
+    BackupInfo.Filter timeRangeFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        long time = info.getStartTs();
+        return time >= startRangeTime && time <= endRangeTime ;
+      }
+    };
+
+    BackupInfo.Filter tableFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        List<TableName> tables = info.getTableNames();
+        return !Collections.disjoint(allTables, tables);
+      }
+    };
+
+    BackupInfo.Filter typeFilter = new  BackupInfo.Filter() {
+
+      @Override
+      public boolean apply(BackupInfo info) {
+        return info.getType() == BackupType.INCREMENTAL;
+      }
+    };
+
+    BackupInfo.Filter stateFilter = new  BackupInfo.Filter() {
+      @Override
+      public boolean apply(BackupInfo info) {
+        return info.getState() == BackupState.COMPLETE;
+      }
+    };
+
+    List<BackupInfo> allInfos =
+        table.getBackupHistory( -1, destinationFilter,
+          timeRangeFilter, tableFilter, typeFilter, stateFilter);
+    if (allInfos.size() != allBackups.size()) {
+      // Yes we have at least one  hole in backup image sequence
+      List<String> missingIds = new ArrayList<String>();
+      for(BackupInfo info: allInfos) {
+        if(allBackups.contains(info.getBackupId())) {
+          continue;
+        }
+        missingIds.add(info.getBackupId());
+      }
+      String errMsg =
+          "Sequence of backup ids has 'holes'. The following backup images must be added:" +
+           org.apache.hadoop.util.StringUtils.join(",", missingIds);
+      throw new IOException(errMsg);
+    }
+  }
+}


[18/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/master
Commit: 2dda371209b2e810fa76034b8fa8dcf47140e265
Parents: 6c0e219
Author: Vladimir Rodionov <vr...@hortonworks.com>
Authored: Tue Aug 22 17:14:48 2017 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Wed Aug 23 12:40:57 2017 -0400

----------------------------------------------------------------------
 hbase-assembly/pom.xml                          |    6 +-
 .../src/main/assembly/hadoop-two-compat.xml     |    1 +
 hbase-assembly/src/main/assembly/src.xml        |    1 +
 hbase-backup/.DS_Store                          |  Bin 0 -> 6148 bytes
 hbase-backup/pom.xml                            |  265 +++
 .../apache/hadoop/hbase/backup/BackupAdmin.java |  136 ++
 .../hbase/backup/BackupClientFactory.java       |   53 +
 .../hadoop/hbase/backup/BackupCopyJob.java      |   55 +
 .../hadoop/hbase/backup/BackupDriver.java       |  210 ++
 .../hadoop/hbase/backup/BackupHFileCleaner.java |  180 ++
 .../apache/hadoop/hbase/backup/BackupInfo.java  |  550 +++++
 .../hadoop/hbase/backup/BackupMergeJob.java     |   40 +
 .../hadoop/hbase/backup/BackupObserver.java     |  102 +
 .../hadoop/hbase/backup/BackupRequest.java      |  139 ++
 .../hbase/backup/BackupRestoreConstants.java    |  123 ++
 .../hbase/backup/BackupRestoreFactory.java      |   82 +
 .../hadoop/hbase/backup/BackupTableInfo.java    |   82 +
 .../hadoop/hbase/backup/HBackupFileSystem.java  |  146 ++
 .../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      |  743 +++++++
 .../hbase/backup/impl/BackupCommands.java       | 1022 +++++++++
 .../hbase/backup/impl/BackupException.java      |   84 +
 .../hadoop/hbase/backup/impl/BackupManager.java |  502 +++++
 .../hbase/backup/impl/BackupManifest.java       |  674 ++++++
 .../hbase/backup/impl/BackupSystemTable.java    | 2051 ++++++++++++++++++
 .../backup/impl/FullTableBackupClient.java      |  224 ++
 .../backup/impl/IncrementalBackupManager.java   |  387 ++++
 .../impl/IncrementalTableBackupClient.java      |  377 ++++
 .../hbase/backup/impl/RestoreTablesClient.java  |  278 +++
 .../hbase/backup/impl/TableBackupClient.java    |  436 ++++
 .../mapreduce/MapReduceBackupCopyJob.java       |  344 +++
 .../mapreduce/MapReduceBackupMergeJob.java      |  321 +++
 .../mapreduce/MapReduceHFileSplitterJob.java    |  181 ++
 .../backup/mapreduce/MapReduceRestoreJob.java   |  136 ++
 .../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   |  747 +++++++
 .../hadoop/hbase/backup/util/RestoreTool.java   |  516 +++++
 .../hadoop/hbase/backup/TestBackupBase.java     |  503 +++++
 .../hbase/backup/TestBackupBoundaryTests.java   |   97 +
 .../hbase/backup/TestBackupCommandLineTool.java |  431 ++++
 .../hadoop/hbase/backup/TestBackupDelete.java   |  102 +
 .../hbase/backup/TestBackupDeleteRestore.java   |   70 +
 .../backup/TestBackupDeleteWithFailures.java    |  194 ++
 .../hadoop/hbase/backup/TestBackupDescribe.java |  110 +
 .../hbase/backup/TestBackupHFileCleaner.java    |  141 ++
 .../hbase/backup/TestBackupMultipleDeletes.java |  158 ++
 .../hadoop/hbase/backup/TestBackupRepair.java   |   91 +
 .../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 ++
 .../backup/TestFullBackupWithFailures.java      |   79 +
 .../hadoop/hbase/backup/TestFullRestore.java    |  345 +++
 .../hbase/backup/TestIncrementalBackup.java     |  209 ++
 .../TestIncrementalBackupDeleteTable.java       |  129 ++
 .../TestIncrementalBackupMergeWithFailures.java |  336 +++
 .../TestIncrementalBackupWithBulkLoad.java      |  145 ++
 .../TestIncrementalBackupWithFailures.java      |  161 ++
 .../hadoop/hbase/backup/TestRemoteBackup.java   |  135 ++
 .../hadoop/hbase/backup/TestRemoteRestore.java  |   59 +
 .../backup/TestRepairAfterFailedDelete.java     |   93 +
 .../hbase/backup/TestRestoreBoundaryTests.java  |   80 +
 .../hbase/backup/TestSystemTableSnapshot.java   |   55 +
 .../backup/master/TestBackupLogCleaner.java     |  162 ++
 .../src/test/resources/log4j.properties         |   68 +
 hbase-it/pom.xml                                |    5 +
 .../apache/hadoop/hbase/backup/BackupAdmin.java |  136 --
 .../hbase/backup/BackupClientFactory.java       |   53 -
 .../hadoop/hbase/backup/BackupCopyJob.java      |   55 -
 .../hadoop/hbase/backup/BackupDriver.java       |  210 --
 .../hadoop/hbase/backup/BackupHFileCleaner.java |  180 --
 .../apache/hadoop/hbase/backup/BackupInfo.java  |  550 -----
 .../hadoop/hbase/backup/BackupMergeJob.java     |   40 -
 .../hadoop/hbase/backup/BackupObserver.java     |  102 -
 .../hadoop/hbase/backup/BackupRequest.java      |  139 --
 .../hbase/backup/BackupRestoreConstants.java    |  123 --
 .../hbase/backup/BackupRestoreFactory.java      |   82 -
 .../hadoop/hbase/backup/BackupTableInfo.java    |   82 -
 .../hadoop/hbase/backup/HBackupFileSystem.java  |  146 --
 .../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      |  743 -------
 .../hbase/backup/impl/BackupCommands.java       | 1022 ---------
 .../hbase/backup/impl/BackupException.java      |   84 -
 .../hadoop/hbase/backup/impl/BackupManager.java |  502 -----
 .../hbase/backup/impl/BackupManifest.java       |  674 ------
 .../hbase/backup/impl/BackupSystemTable.java    | 2051 ------------------
 .../backup/impl/FullTableBackupClient.java      |  224 --
 .../backup/impl/IncrementalBackupManager.java   |  387 ----
 .../impl/IncrementalTableBackupClient.java      |  377 ----
 .../hbase/backup/impl/RestoreTablesClient.java  |  278 ---
 .../hbase/backup/impl/TableBackupClient.java    |  436 ----
 .../mapreduce/MapReduceBackupCopyJob.java       |  344 ---
 .../mapreduce/MapReduceBackupMergeJob.java      |  321 ---
 .../mapreduce/MapReduceHFileSplitterJob.java    |  181 --
 .../backup/mapreduce/MapReduceRestoreJob.java   |  136 --
 .../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   |  747 -------
 .../hadoop/hbase/backup/util/RestoreTool.java   |  516 -----
 .../hadoop/hbase/backup/TestBackupBase.java     |  503 -----
 .../hbase/backup/TestBackupBoundaryTests.java   |   97 -
 .../hbase/backup/TestBackupCommandLineTool.java |  431 ----
 .../hadoop/hbase/backup/TestBackupDelete.java   |  102 -
 .../hbase/backup/TestBackupDeleteRestore.java   |   70 -
 .../backup/TestBackupDeleteWithFailures.java    |  194 --
 .../hadoop/hbase/backup/TestBackupDescribe.java |  110 -
 .../hbase/backup/TestBackupHFileCleaner.java    |  141 --
 .../hbase/backup/TestBackupMultipleDeletes.java |  158 --
 .../hadoop/hbase/backup/TestBackupRepair.java   |   91 -
 .../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 --
 .../backup/TestFullBackupWithFailures.java      |   79 -
 .../hadoop/hbase/backup/TestFullRestore.java    |  345 ---
 .../hbase/backup/TestIncrementalBackup.java     |  209 --
 .../TestIncrementalBackupDeleteTable.java       |  129 --
 .../TestIncrementalBackupMergeWithFailures.java |  336 ---
 .../TestIncrementalBackupWithBulkLoad.java      |  145 --
 .../TestIncrementalBackupWithFailures.java      |  161 --
 .../hadoop/hbase/backup/TestRemoteBackup.java   |  135 --
 .../hadoop/hbase/backup/TestRemoteRestore.java  |   59 -
 .../backup/TestRepairAfterFailedDelete.java     |   93 -
 .../hbase/backup/TestRestoreBoundaryTests.java  |   80 -
 .../hbase/backup/TestSystemTableSnapshot.java   |   55 -
 .../backup/master/TestBackupLogCleaner.java     |  162 --
 pom.xml                                         |    1 +
 146 files changed, 17540 insertions(+), 17195 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 14074ad..c9488ca 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -281,8 +281,12 @@
       <version>${project.version}</version>
       <type>test-jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-backup</artifactId>
+      <version>${project.version}</version>
+    </dependency>
   </dependencies>
-
   <profiles>
     <profile>
       <id>rsgroup</id>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
index 8039c4d..1592a3b 100644
--- a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
+++ b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
@@ -49,6 +49,7 @@
         <include>org.apache.hbase:hbase-spark</include>
         <include>org.apache.hbase:hbase-thrift</include>
         <include>org.apache.hbase:hbase-external-blockcache</include>
+        <include>org.apache.hbase:hbase-backup</include>
       </includes>
       <!-- Binaries for the dependencies also go in the hbase-jars directory -->
       <binaries>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-assembly/src/main/assembly/src.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/src.xml b/hbase-assembly/src/main/assembly/src.xml
index 436b1ff..b00f05f 100644
--- a/hbase-assembly/src/main/assembly/src.xml
+++ b/hbase-assembly/src/main/assembly/src.xml
@@ -61,6 +61,7 @@
         <include>org.apache.hbase:hbase-spark-it</include>
         <include>org.apache.hbase:hbase-testing-util</include>
         <include>org.apache.hbase:hbase-thrift</include>
+        <include>org.apache.hbase:hbase-backup</include>
       </includes>
       <!-- Include all the sources in the top directory -->
       <sources>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/.DS_Store
----------------------------------------------------------------------
diff --git a/hbase-backup/.DS_Store b/hbase-backup/.DS_Store
new file mode 100644
index 0000000..ab57a77
Binary files /dev/null and b/hbase-backup/.DS_Store differ

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-backup/pom.xml b/hbase-backup/pom.xml
new file mode 100644
index 0000000..7c7d8b5
--- /dev/null
+++ b/hbase-backup/pom.xml
@@ -0,0 +1,265 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <!--
+  /**
+   * 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.
+   */
+  -->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase-build-configuration</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>3.0.0-SNAPSHOT</version>
+    <relativePath>../hbase-build-configuration</relativePath>
+  </parent>
+  <artifactId>hbase-backup</artifactId>
+  <name>Apache HBase - Backup</name>
+  <description>Backup for HBase</description>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+    </plugins>
+    <pluginManagement>
+      <plugins>
+        <!--This plugin's configuration is used to store Eclipse m2e settings
+             only. It has no influence on the Maven build itself.-->
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-compiler-plugin</artifactId>
+                    <versionRange>[3.2,)</versionRange>
+                    <goals>
+                      <goal>compile</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore/>
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
+  <dependencies>
+    <!-- Intra-project dependencies -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>jdk.tools</groupId>
+          <artifactId>jdk.tools</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-protocol-shaded</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <!-- General dependencies -->
+    <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-miscellaneous</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-netty</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <profiles>
+    <!-- profile against Hadoop 2.x: This is the default. -->
+    <profile>
+      <id>hadoop-2.0</id>
+      <activation>
+        <property>
+          <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+          <!--h2-->
+          <name>!hadoop.profile</name>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>com.github.stephenc.findbugs</groupId>
+          <artifactId>findbugs-annotations</artifactId>
+          <optional>true</optional>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </dependency>
+        <!--dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+          <exclusions>
+            <exclusion>
+              <groupId>net.java.dev.jets3t</groupId>
+              <artifactId>jets3t</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>javax.servlet.jsp</groupId>
+              <artifactId>jsp-api</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>org.mortbay.jetty</groupId>
+              <artifactId>jetty</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-server</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-core</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-json</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>javax.servlet</groupId>
+              <artifactId>servlet-api</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>tomcat</groupId>
+              <artifactId>jasper-compiler</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>tomcat</groupId>
+              <artifactId>jasper-runtime</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.google.code.findbugs</groupId>
+              <artifactId>jsr305</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency-->
+<!-- Hadoop needs Netty 3.x at test scope for the minicluster -->
+        <dependency>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+          <version>${netty.hadoop.version}</version>
+          <scope>test</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+    <!--
+      profile for building against Hadoop 3.0.x. Activate using:
+       mvn -Dhadoop.profile=3.0
+    -->
+    <profile>
+      <id>hadoop-3.0</id>
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>3.0</value>
+        </property>
+      </activation>
+      <properties>
+        <hadoop.version>3.0-SNAPSHOT</hadoop.version>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
new file mode 100644
index 0000000..9dc6382
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
@@ -0,0 +1,136 @@
+/**
+ * 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 array of backup ids
+   * @return total number of deleted sessions
+   * @throws IOException exception
+   */
+  int deleteBackups(String[] backupIds) throws IOException;
+
+  /**
+   * Merge backup images command
+   * @param backupIds array of backup ids of images to be merged
+   *        The resulting backup image will have the same backup id as the most
+   *        recent image from a list of images to be merged
+   * @throws IOException exception
+   */
+  void mergeBackups(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 array 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 array 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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
new file mode 100644
index 0000000..21d73cc
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
@@ -0,0 +1,53 @@
+/**
+ * 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.Configuration;
+import org.apache.hadoop.hbase.backup.impl.FullTableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
+import org.apache.hadoop.hbase.client.Connection;
+
+public class BackupClientFactory {
+
+  public static TableBackupClient create (Connection conn, String backupId, BackupRequest request)
+    throws IOException
+  {
+    Configuration conf = conn.getConfiguration();
+    try {
+      String clsName = conf.get(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS);
+      if (clsName != null) {
+        Class<?> clientImpl = Class.forName(clsName);
+        TableBackupClient client = (TableBackupClient) clientImpl.newInstance();
+        client.init(conn, backupId, request);
+        return client;
+      }
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+
+    BackupType type = request.getBackupType();
+    if (type == BackupType.FULL) {
+      return new FullTableBackupClient(conn, backupId, request);
+    } else {
+      return new IncrementalTableBackupClient(conn, backupId, request);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
new file mode 100644
index 0000000..007e4c1
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
new file mode 100644
index 0000000..9dd8531
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
@@ -0,0 +1,210 @@
+/**
+ * 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 if (BackupCommand.REPAIR.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.REPAIR;
+    } else if (BackupCommand.MERGE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.MERGE;
+    } 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;
+    } finally {
+      command.finish();
+    }
+    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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
new file mode 100644
index 0000000..ed554ad
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
@@ -0,0 +1,180 @@
+/*
+ * 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.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+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.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
+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.cleaner.BaseHFileCleanerDelegate;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
+
+/**
+ * Implementation of a file cleaner that checks if an hfile is still referenced by backup before
+ * deleting it from hfile archive directory.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class BackupHFileCleaner extends BaseHFileCleanerDelegate implements Abortable {
+  private static final Log LOG = LogFactory.getLog(BackupHFileCleaner.class);
+  private boolean stopped = false;
+  private boolean aborted;
+  private Configuration conf;
+  private Connection connection;
+  private long prevReadFromBackupTbl = 0, // timestamp of most recent read from backup:system table
+      secondPrevReadFromBackupTbl = 0; // timestamp of 2nd most recent read from backup:system table
+  //used by unit test to skip reading backup:system
+  private boolean checkForFullyBackedUpTables = true;
+  private List<TableName> fullyBackedUpTables = null;
+
+  private Set<String> getFilenameFromBulkLoad(Map<byte[], List<Path>>[] maps) {
+    Set<String> filenames = new HashSet<String>();
+    for (Map<byte[], List<Path>> map : maps) {
+      if (map == null) continue;
+      for (List<Path> paths : map.values()) {
+        for (Path p : paths) {
+          filenames.add(p.getName());
+        }
+      }
+    }
+    return filenames;
+  }
+
+  private Set<String> loadHFileRefs(List<TableName> tableList) throws IOException {
+    if (connection == null) {
+      connection = ConnectionFactory.createConnection(conf);
+    }
+    try (BackupSystemTable tbl = new BackupSystemTable(connection)) {
+      Map<byte[], List<Path>>[] res =
+          tbl.readBulkLoadedFiles(null, tableList);
+      secondPrevReadFromBackupTbl = prevReadFromBackupTbl;
+      prevReadFromBackupTbl = EnvironmentEdgeManager.currentTime();
+      return getFilenameFromBulkLoad(res);
+    }
+  }
+
+  @VisibleForTesting
+  void setCheckForFullyBackedUpTables(boolean b) {
+    checkForFullyBackedUpTables = b;
+  }
+  @Override
+  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
+    if (conf == null) {
+      return files;
+    }
+    // obtain the Set of TableName's which have been fully backed up
+    // so that we filter BulkLoad to be returned from server
+    if (checkForFullyBackedUpTables) {
+      if (connection == null) return files;
+      try (BackupSystemTable tbl = new BackupSystemTable(connection)) {
+        fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
+      } catch (IOException ioe) {
+        LOG.error("Failed to get tables which have been fully backed up, skipping checking", ioe);
+        return Collections.emptyList();
+      }
+      Collections.sort(fullyBackedUpTables);
+    }
+    final Set<String> hfileRefs;
+    try {
+      hfileRefs = loadHFileRefs(fullyBackedUpTables);
+    } catch (IOException ioe) {
+      LOG.error("Failed to read hfile references, skipping checking deletable files", ioe);
+      return Collections.emptyList();
+    }
+    Iterable<FileStatus> deletables = Iterables.filter(files, new Predicate<FileStatus>() {
+      @Override
+      public boolean apply(FileStatus file) {
+        // If the file is recent, be conservative and wait for one more scan of backup:system table
+        if (file.getModificationTime() > secondPrevReadFromBackupTbl) {
+          return false;
+        }
+        String hfile = file.getPath().getName();
+        boolean foundHFileRef = hfileRefs.contains(hfile);
+        return !foundHFileRef;
+      }
+    });
+    return deletables;
+  }
+
+  @Override
+  public boolean isFileDeletable(FileStatus fStat) {
+    // work is done in getDeletableFiles()
+    return true;
+  }
+
+  @Override
+  public void setConf(Configuration config) {
+    this.conf = config;
+    this.connection = null;
+    try {
+      this.connection = ConnectionFactory.createConnection(conf);
+    } catch (IOException ioe) {
+      LOG.error("Couldn't establish connection", ioe);
+    }
+  }
+
+  @Override
+  public void stop(String why) {
+    if (this.stopped) {
+      return;
+    }
+    if (this.connection != null) {
+      try {
+        this.connection.close();
+      } catch (IOException ioe) {
+        LOG.debug("Got " + ioe + " when closing connection");
+      }
+    }
+    this.stopped = true;
+  }
+
+  @Override
+  public boolean isStopped() {
+    return this.stopped;
+  }
+
+  @Override
+  public void abort(String why, Throwable e) {
+    LOG.warn("Aborting ReplicationHFileCleaner because " + why, e);
+    this.aborted = true;
+    stop(why);
+  }
+
+  @Override
+  public boolean isAborted() {
+    return this.aborted;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
new file mode 100644
index 0000000..1765bf3
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
@@ -0,0 +1,550 @@
+/**
+ * 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 = BackupPhase.REQUEST;
+
+  /**
+   * 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;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return backupId;
+  }
+
+  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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
new file mode 100644
index 0000000..136782f
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Backup merge operation job interface. Concrete implementation is provided by backup provider, see
+ * {@link BackupRestoreFactory}
+ */
+
+@InterfaceAudience.Private
+public interface BackupMergeJob extends Configurable {
+
+  /**
+   * Run backup merge operation
+   * @param backupIds backup image ids
+   * @throws IOException
+   */
+  void run(String[] backupIds) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
new file mode 100644
index 0000000..595e862
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.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 java.io.IOException;
+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.Path;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+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.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * An Observer to facilitate backup operations
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class BackupObserver implements RegionObserver {
+  private static final Log LOG = LogFactory.getLog(BackupObserver.class);
+  @Override
+  public boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
+    List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths,
+    boolean hasLoaded) throws IOException {
+    Configuration cfg = ctx.getEnvironment().getConfiguration();
+    if (!hasLoaded) {
+      // there is no need to record state
+      return hasLoaded;
+    }
+    if (finalPaths == null || !BackupManager.isBackupEnabled(cfg)) {
+      LOG.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled");
+      return hasLoaded;
+    }
+    try (Connection connection = ConnectionFactory.createConnection(cfg);
+        BackupSystemTable tbl = new BackupSystemTable(connection)) {
+      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
+      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
+      TableName tableName = info.getTable();
+      if (!fullyBackedUpTables.contains(tableName)) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(tableName + " has not gone thru full backup");
+        }
+        return hasLoaded;
+      }
+      tbl.writePathsPostBulkLoad(tableName, info.getEncodedNameAsBytes(), finalPaths);
+      return hasLoaded;
+    } catch (IOException ioe) {
+      LOG.error("Failed to get tables which have been fully backed up", ioe);
+      return false;
+    }
+  }
+  @Override
+  public void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
+      final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
+    Configuration cfg = ctx.getEnvironment().getConfiguration();
+    if (pairs == null || pairs.isEmpty() || !BackupManager.isBackupEnabled(cfg)) {
+      LOG.debug("skipping recording bulk load in preCommitStoreFile since backup is disabled");
+      return;
+    }
+    try (Connection connection = ConnectionFactory.createConnection(cfg);
+        BackupSystemTable tbl = new BackupSystemTable(connection)) {
+      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
+      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
+      TableName tableName = info.getTable();
+      if (!fullyBackedUpTables.contains(tableName)) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(tableName + " has not gone thru full backup");
+        }
+        return;
+      }
+      tbl.writeFilesForBulkLoadPreCommit(tableName, info.getEncodedNameAsBytes(), family, pairs);
+      return;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
new file mode 100644
index 0000000..dadd861
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
new file mode 100644
index 0000000..48e70a1
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
@@ -0,0 +1,123 @@
+/**
+ * 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 = "backup:system";
+
+  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;
+
+
+  public static final String BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.attempts.max";
+  public static final int DEFAULT_BACKUP_MAX_ATTEMPTS = 10;
+
+  public static final String BACKUP_ATTEMPTS_PAUSE_MS_KEY = "hbase.backup.attempts.pause.ms";
+  public static final int DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS = 10000;
+
+  /*
+   *  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, REPAIR
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
new file mode 100644
index 0000000..d72c884
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.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.conf.Configuration;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
+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";
+  public final static String HBASE_BACKUP_MERGE_IMPL_CLASS = "hbase.backup.merge.class";
+
+  private BackupRestoreFactory() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Gets backup restore job
+   * @param conf configuration
+   * @return backup restore job 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 job instance
+   */
+  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;
+  }
+
+  /**
+   * Gets backup merge job
+   * @param conf configuration
+   * @return backup merge job instance
+   */
+  public static BackupMergeJob getBackupMergeJob(Configuration conf) {
+    Class<? extends BackupMergeJob> cls =
+        conf.getClass(HBASE_BACKUP_MERGE_IMPL_CLASS, MapReduceBackupMergeJob.class,
+          BackupMergeJob.class);
+    BackupMergeJob service = ReflectionUtils.newInstance(cls, conf);
+    service.setConf(conf);
+    return service;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
new file mode 100644
index 0000000..a5390a6
--- /dev/null
+++ b/hbase-backup/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();
+  }
+}


[14/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
new file mode 100644
index 0000000..6330899
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.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.HashSet;
+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.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.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.
+   * @return The new HashMap of RS log time stamps after the log roll for this incremental backup.
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> getIncrBackupLogFileMap()
+      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());
+    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
+    backupInfo.setIncrBackupFileList(logList);
+
+    return newTimestamps;
+  }
+
+  /**
+   * Get list of WAL files eligible for incremental backup
+   * @return list of WAL files
+   * @throws IOException
+   */
+  public List<String> getIncrBackupLogFileList()
+      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.");
+    }
+
+    newTimestamps = readRegionServerLastLogRollResult();
+
+    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
+    List<WALItem> logFromSystemTable =
+        getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps, getBackupInfo()
+            .getBackupRootDir());
+
+    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
+    backupInfo.setIncrBackupFileList(logList);
+
+    return logList;
+  }
+
+
+  private List<String> excludeAlreadyBackedUpWALs(List<String> logList,
+      List<WALItem> logFromSystemTable) {
+
+    Set<String> walFileNameSet = convertToSet(logFromSystemTable);
+
+    List<String> list = new ArrayList<String>();
+    for (int i=0; i < logList.size(); i++) {
+      Path p = new Path(logList.get(i));
+      String name  = p.getName();
+      if (walFileNameSet.contains(name)) continue;
+      list.add(logList.get(i));
+    }
+    return list;
+  }
+
+  /**
+   * Create Set of WAL file names (not full path names)
+   * @param logFromSystemTable
+   * @return set of WAL file names
+   */
+  private Set<String> convertToSet(List<WALItem> logFromSystemTable) {
+
+    Set<String> set = new HashSet<String>();
+    for (int i=0; i < logFromSystemTable.size(); i++) {
+      WALItem item = logFromSystemTable.get(i);
+      set.add(item.walFile);
+    }
+    return set;
+  }
+
+  /**
+   * 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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
new file mode 100644
index 0000000..6d48c32
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
@@ -0,0 +1,377 @@
+/**
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+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.FileUtil;
+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.BackupInfo.BackupPhase;
+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.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.mapreduce.WALPlayer;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.util.Tool;
+
+/**
+ * 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);
+
+  protected IncrementalTableBackupClient() {
+  }
+
+  public IncrementalTableBackupClient(final Connection conn, final String backupId,
+      BackupRequest request) throws IOException {
+    super(conn, backupId, request);
+  }
+
+  protected List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    List<String> list = new ArrayList<String>();
+    for (String file : incrBackupFileList) {
+      Path p = new Path(file);
+      if (fs.exists(p) || isActiveWalPath(p)) {
+        list.add(file);
+      } else {
+        LOG.warn("Can't find file: " + file);
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Check if a given path is belongs to active WAL directory
+   * @param p path
+   * @return true, if yes
+   */
+  protected boolean isActiveWalPath(Path p) {
+    return !AbstractFSWALProvider.isArchivedLogFile(p);
+  }
+
+  protected static int getIndex(TableName tbl, List<TableName> sTableList) {
+    if (sTableList == null) return 0;
+    for (int i = 0; i < sTableList.size(); i++) {
+      if (tbl.equals(sTableList.get(i))) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  /*
+   * Reads bulk load records from backup table, iterates through the records and forms the paths
+   * for bulk loaded hfiles. Copies the bulk loaded hfiles to backup destination
+   * @param sTableList list of tables to be backed up
+   * @return map of table to List of files
+   */
+  protected Map<byte[], List<Path>>[] handleBulkLoad(List<TableName> sTableList) throws IOException {
+    Map<byte[], List<Path>>[] mapForSrc = new Map[sTableList.size()];
+    Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>> pair =
+    backupManager.readBulkloadRows(sTableList);
+    Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> map = pair.getFirst();
+    FileSystem fs = FileSystem.get(conf);
+    FileSystem tgtFs;
+    try {
+      tgtFs = FileSystem.get(new URI(backupInfo.getBackupRootDir()), conf);
+    } catch (URISyntaxException use) {
+      throw new IOException("Unable to get FileSystem", use);
+    }
+    Path rootdir = FSUtils.getRootDir(conf);
+    Path tgtRoot = new Path(new Path(backupInfo.getBackupRootDir()), backupId);
+    for (Map.Entry<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> tblEntry :
+      map.entrySet()) {
+      TableName srcTable = tblEntry.getKey();
+      int srcIdx = getIndex(srcTable, sTableList);
+      if (srcIdx < 0) {
+        LOG.warn("Couldn't find " + srcTable + " in source table List");
+        continue;
+      }
+      if (mapForSrc[srcIdx] == null) {
+        mapForSrc[srcIdx] = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
+      }
+      Path tblDir = FSUtils.getTableDir(rootdir, srcTable);
+      Path tgtTable = new Path(new Path(tgtRoot, srcTable.getNamespaceAsString()),
+          srcTable.getQualifierAsString());
+      for (Map.Entry<String,Map<String,List<Pair<String, Boolean>>>> regionEntry :
+        tblEntry.getValue().entrySet()){
+        String regionName = regionEntry.getKey();
+        Path regionDir = new Path(tblDir, regionName);
+        // map from family to List of hfiles
+        for (Map.Entry<String,List<Pair<String, Boolean>>> famEntry :
+          regionEntry.getValue().entrySet()) {
+          String fam = famEntry.getKey();
+          Path famDir = new Path(regionDir, fam);
+          List<Path> files;
+          if (!mapForSrc[srcIdx].containsKey(fam.getBytes())) {
+            files = new ArrayList<Path>();
+            mapForSrc[srcIdx].put(fam.getBytes(), files);
+          } else {
+            files = mapForSrc[srcIdx].get(fam.getBytes());
+          }
+          Path archiveDir = HFileArchiveUtil.getStoreArchivePath(conf, srcTable, regionName, fam);
+          String tblName = srcTable.getQualifierAsString();
+          Path tgtFam = new Path(new Path(tgtTable, regionName), fam);
+          if (!tgtFs.mkdirs(tgtFam)) {
+            throw new IOException("couldn't create " + tgtFam);
+          }
+          for (Pair<String, Boolean> fileWithState : famEntry.getValue()) {
+            String file = fileWithState.getFirst();
+            boolean raw = fileWithState.getSecond();
+            int idx = file.lastIndexOf("/");
+            String filename = file;
+            if (idx > 0) {
+              filename = file.substring(idx+1);
+            }
+            Path p = new Path(famDir, filename);
+            Path tgt = new Path(tgtFam, filename);
+            Path archive = new Path(archiveDir, filename);
+            if (fs.exists(p)) {
+              if (LOG.isTraceEnabled()) {
+                LOG.trace("found bulk hfile " + file + " in " + famDir + " for " + tblName);
+              }
+              try {
+                if (LOG.isTraceEnabled()) {
+                  LOG.trace("copying " + p + " to " + tgt);
+                }
+                FileUtil.copy(fs, p, tgtFs, tgt, false,conf);
+              } catch (FileNotFoundException e) {
+                LOG.debug("copying archive " + archive + " to " + tgt);
+                try {
+                  FileUtil.copy(fs, archive, tgtFs, tgt, false, conf);
+                } catch (FileNotFoundException fnfe) {
+                  if (!raw) throw fnfe;
+                }
+              }
+            } else {
+              LOG.debug("copying archive " + archive + " to " + tgt);
+              try {
+                FileUtil.copy(fs, archive, tgtFs, tgt, false, conf);
+              } catch (FileNotFoundException fnfe) {
+                if (!raw) throw fnfe;
+              }
+            }
+            files.add(tgt);
+          }
+        }
+      }
+    }
+    backupManager.writeBulkLoadedFiles(sTableList, mapForSrc);
+    backupManager.removeBulkLoadedRows(sTableList, pair.getSecond());
+    return mapForSrc;
+  }
+
+  @Override
+  public void execute() throws IOException {
+
+    try {
+      // case PREPARE_INCREMENTAL:
+      beginBackup(backupManager, backupInfo);
+      backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
+      LOG.debug("For incremental backup, current table set is "
+          + backupManager.getIncrementalBackupTableSet());
+      newTimestamps =
+          ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
+    } catch (Exception e) {
+      // fail the overall backup and return
+      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
+        BackupType.INCREMENTAL, conf);
+      return;
+    }
+
+    // case INCREMENTAL_COPY:
+    try {
+      // copy out the table and region info files for each table
+      BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
+      // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
+      convertWALsToHFiles(backupInfo);
+      incrementalCopyHFiles(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);
+      return;
+    }
+    // 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 {
+      // 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);
+
+      handleBulkLoad(backupInfo.getTableNames());
+      // backup complete
+      completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
+
+    } catch (IOException e) {
+      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
+        BackupType.INCREMENTAL, conf);
+    }
+  }
+
+  protected void incrementalCopyHFiles(BackupInfo backupInfo) throws Exception {
+
+    try {
+      LOG.debug("Incremental copy HFiles 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 = new ArrayList<String>();
+      // Add Bulk output
+      incrBackupFileList.add(getBulkOutputDir().toString());
+      String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
+      strArr[strArr.length - 1] = backupInfo.getBackupRootDir();
+      BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf);
+      int res = copyService.copy(backupInfo, backupManager, conf, BackupType.INCREMENTAL, strArr);
+      if (res != 0) {
+        LOG.error("Copy incremental HFile files failed with return code: " + res + ".");
+        throw new IOException("Failed copy from " + StringUtils.join(incrBackupFileList, ',')
+            + " to " + backupInfo.getHLogTargetDir());
+      }
+      LOG.debug("Incremental copy HFiles from " + StringUtils.join(incrBackupFileList, ',')
+          + " to " + backupInfo.getBackupRootDir() + " finished.");
+    } finally {
+      deleteBulkLoadDirectory();
+    }
+  }
+
+  protected void deleteBulkLoadDirectory() throws IOException {
+    // delete original bulk load directory on method exit
+    Path path = getBulkOutputDir();
+    FileSystem fs = FileSystem.get(conf);
+    boolean result = fs.delete(path, true);
+    if (!result) {
+      LOG.warn("Could not delete " + path);
+    }
+
+  }
+
+  protected void convertWALsToHFiles(BackupInfo backupInfo) throws IOException {
+    // get incremental backup file list and prepare parameters for DistCp
+    List<String> incrBackupFileList = backupInfo.getIncrBackupFileList();
+    // Get list of tables in incremental backup set
+    Set<TableName> tableSet = backupManager.getIncrementalBackupTableSet();
+    // filter missing files out (they have been copied by previous backups)
+    incrBackupFileList = filterMissingFiles(incrBackupFileList);
+    for (TableName table : tableSet) {
+      // Check if table exists
+      if (tableExists(table, conn)) {
+        walToHFiles(incrBackupFileList, table);
+      } else {
+        LOG.warn("Table " + table + " does not exists. Skipping in WAL converter");
+      }
+    }
+  }
+
+
+  protected boolean tableExists(TableName table, Connection conn) throws IOException {
+    try (Admin admin = conn.getAdmin();) {
+      return admin.tableExists(table);
+    }
+  }
+
+  protected void walToHFiles(List<String> dirPaths, TableName tableName) throws IOException {
+
+    Tool player = new WALPlayer();
+
+    // Player reads all files in arbitrary directory structure and creates
+    // a Map task for each file. We use ';' as separator
+    // because WAL file names contains ','
+    String dirs = StringUtils.join(dirPaths, ';');
+
+    Path bulkOutputPath = getBulkOutputDirForTable(tableName);
+    conf.set(WALPlayer.BULK_OUTPUT_CONF_KEY, bulkOutputPath.toString());
+    conf.set(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ";");
+    String[] playerArgs = { dirs, tableName.getNameAsString() };
+
+    try {
+      player.setConf(conf);
+      int result = player.run(playerArgs);
+      if(result != 0) {
+        throw new IOException("WAL Player failed");
+      }
+      conf.unset(WALPlayer.INPUT_FILES_SEPARATOR_KEY);
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception ee) {
+      throw new IOException("Can not convert from directory " + dirs
+          + " (check Hadoop, HBase and WALPlayer M/R job logs) ", ee);
+    }
+  }
+
+  protected Path getBulkOutputDirForTable(TableName table) {
+    Path tablePath = getBulkOutputDir();
+    tablePath = new Path(tablePath, table.getNamespaceAsString());
+    tablePath = new Path(tablePath, table.getQualifierAsString());
+    return new Path(tablePath, "data");
+  }
+
+  protected Path getBulkOutputDir() {
+    String backupId = backupInfo.getBackupId();
+    Path path = new Path(backupInfo.getBackupRootDir());
+    path = new Path(path, ".tmp");
+    path = new Path(path, backupId);
+    return path;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
new file mode 100644
index 0000000..ea7a7b8
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
@@ -0,0 +1,278 @@
+/**
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+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.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.BackupUtils;
+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;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.LoadQueueItem;
+
+/**
+ * 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(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 fileBackupDir =
+          HBackupFileSystem.getTableBackupDataDir(im.getRootDir(), im.getBackupId(), sTable);
+      dirList.add(new Path(fileBackupDir));
+    }
+
+    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;
+    Set<String> backupIdSet = new HashSet<>();
+
+    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));
+          if (image.getType() == BackupType.INCREMENTAL) {
+            backupIdSet.add(image.getBackupId());
+            LOG.debug("adding " + image.getBackupId() + " for bulk load");
+          }
+        }
+      }
+    }
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<TableName> sTableList = Arrays.asList(sTableArray);
+      for (String id : backupIdSet) {
+        LOG.debug("restoring bulk load for " + id);
+        Map<byte[], List<Path>>[] mapForSrc = table.readBulkLoadedFiles(id, sTableList);
+        Map<LoadQueueItem, ByteBuffer> loaderResult;
+        conf.setBoolean(LoadIncrementalHFiles.ALWAYS_COPY_FILES, true);
+        LoadIncrementalHFiles loader = BackupUtils.createLoader(conf);
+        for (int i = 0; i < sTableList.size(); i++) {
+          if (mapForSrc[i] != null && !mapForSrc[i].isEmpty()) {
+            loaderResult = loader.run(null, mapForSrc[i], tTableArray[i]);
+            LOG.debug("bulk loading " + sTableList.get(i) + " to " + tTableArray[i]);
+            if (loaderResult.isEmpty()) {
+              String msg = "Couldn't bulk load for " + sTableList.get(i) + " to " + tTableArray[i];
+              LOG.error(msg);
+              throw new IOException(msg);
+            }
+          }
+        }
+      }
+    }
+    LOG.debug("restoreStage finished");
+  }
+
+  static long getTsFromBackupId(String backupId) {
+    if (backupId == null) {
+      return 0;
+    }
+    return Long.parseLong(backupId.substring(backupId.lastIndexOf("_") + 1));
+  }
+
+  static boolean withinRange(long a, long lower, long upper) {
+    if (a < lower || a > upper) {
+      return false;
+    }
+    return true;
+  }
+
+  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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
new file mode 100644
index 0000000..6eec460
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
@@ -0,0 +1,436 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * 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 {
+
+  public static final String BACKUP_CLIENT_IMPL_CLASS = "backup.client.impl.class";
+
+  @VisibleForTesting
+  public static final String BACKUP_TEST_MODE_STAGE = "backup.test.mode.stage";
+
+  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() {
+  }
+
+  public TableBackupClient(final Connection conn, final String backupId, BackupRequest request)
+      throws IOException {
+    init(conn, backupId, request);
+  }
+
+  public void init(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());
+    }
+    // Start new session
+    backupManager.startBackupSession();
+  }
+
+  /**
+   * Begin the overall backup.
+   * @param backupInfo backup info
+   * @throws IOException exception
+   */
+  protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo)
+      throws IOException {
+
+    BackupSystemTable.snapshot(conn);
+    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.");
+    }
+  }
+
+  protected 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
+   */
+  protected static void deleteSnapshots(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);
+      }
+      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
+   */
+  protected static 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.
+   */
+  protected static 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.debug("Cleaning up uncompleted backup data at " + targetDirPath.toString()
+                + " done.");
+          } else {
+            LOG.debug("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 {
+
+    try {
+      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);
+      cleanupAndRestoreBackupSystem(conn, backupInfo, conf);
+      // If backup session is updated to FAILED state - means we
+      // processed recovery already.
+      backupManager.updateBackupInfo(backupInfo);
+      backupManager.finishBackupSession();
+      LOG.error("Backup " + backupInfo.getBackupId() + " failed.");
+    } catch (IOException ee) {
+      LOG.error("Please run backup repair tool manually to restore backup system integrity");
+      throw ee;
+    }
+  }
+
+  public static void cleanupAndRestoreBackupSystem (Connection conn, BackupInfo backupInfo,
+      Configuration conf) throws IOException
+  {
+    BackupType type = backupInfo.getType();
+     // 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) {
+       deleteSnapshots(conn, backupInfo, conf);
+       cleanupExportSnapshotLog(conf);
+     }
+     BackupSystemTable.restoreFromSnapshot(conn);
+     BackupSystemTable.deleteSnapshot(conn);
+     // 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);
+  }
+
+
+
+  /**
+   * 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
+   */
+  protected 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
+   */
+  protected 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
+   */
+  protected 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);
+
+    // 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);
+    }
+
+    // when full backup is done:
+    // - delete HBase snapshot
+    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
+    // snapshots
+    if (type == BackupType.FULL) {
+      deleteSnapshots(conn, backupInfo, conf);
+      cleanupExportSnapshotLog(conf);
+    } else if (type == BackupType.INCREMENTAL) {
+      cleanupDistCpLog(backupInfo, conf);
+    }
+    BackupSystemTable.deleteSnapshot(conn);
+    backupManager.updateBackupInfo(backupInfo);
+
+    // Finish active session
+    backupManager.finishBackupSession();
+
+    LOG.info("Backup " + backupInfo.getBackupId() + " completed.");
+  }
+
+  /**
+   * Backup request execution
+   * @throws IOException
+   */
+  public abstract void execute() throws IOException;
+
+  @VisibleForTesting
+  protected Stage getTestStage() {
+    return Stage.valueOf("stage_"+ conf.getInt(BACKUP_TEST_MODE_STAGE, 0));
+  }
+
+  @VisibleForTesting
+  protected void failStageIf(Stage stage) throws IOException {
+    Stage current = getTestStage();
+    if (current == stage) {
+      throw new IOException("Failed stage " + stage+" in testing");
+    }
+  }
+
+  public static enum Stage {
+    stage_0, stage_1, stage_2, stage_3, stage_4
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java
new file mode 100644
index 0000000..016d1a4
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
new file mode 100644
index 0000000..00c5b83
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
@@ -0,0 +1,321 @@
+/**
+ * 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 static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+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.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.BackupMergeJob;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+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.Pair;
+import org.apache.hadoop.util.Tool;
+
+/**
+ * MapReduce implementation of {@link BackupMergeJob}
+ * Must be initialized with configuration of a backup destination cluster
+ *
+ */
+
+@InterfaceAudience.Private
+public class MapReduceBackupMergeJob implements BackupMergeJob {
+  public static final Log LOG = LogFactory.getLog(MapReduceBackupMergeJob.class);
+
+  protected Tool player;
+  protected Configuration conf;
+
+  public MapReduceBackupMergeJob() {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void run(String[] backupIds) throws IOException {
+    String bulkOutputConfKey;
+
+    // TODO : run player on remote cluster
+    player = new MapReduceHFileSplitterJob();
+    bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
+    // Player reads all files in arbitrary directory structure and creates
+    // a Map task for each file
+    String bids = StringUtils.join(backupIds, ",");
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Merge backup images " + bids);
+    }
+
+    List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
+    boolean finishedTables = false;
+    Connection conn = ConnectionFactory.createConnection(getConf());
+    BackupSystemTable table = new BackupSystemTable(conn);
+    FileSystem fs = FileSystem.get(getConf());
+
+    try {
+
+      // Get exclusive lock on backup system
+      table.startBackupExclusiveOperation();
+      // Start merge operation
+      table.startMergeOperation(backupIds);
+
+      // Select most recent backup id
+      String mergedBackupId = findMostRecentBackupId(backupIds);
+
+      TableName[] tableNames = getTableNamesInBackupImages(backupIds);
+      String backupRoot = null;
+
+      BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
+      backupRoot = bInfo.getBackupRootDir();
+
+      for (int i = 0; i < tableNames.length; i++) {
+
+        LOG.info("Merge backup images for " + tableNames[i]);
+
+        // Find input directories for table
+
+        Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
+        String dirs = StringUtils.join(dirPaths, ",");
+        Path bulkOutputPath =
+            BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
+              getConf(), false);
+        // Delete content if exists
+        if (fs.exists(bulkOutputPath)) {
+          if (!fs.delete(bulkOutputPath, true)) {
+            LOG.warn("Can not delete: " + bulkOutputPath);
+          }
+        }
+        Configuration conf = getConf();
+        conf.set(bulkOutputConfKey, bulkOutputPath.toString());
+        String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
+
+        int result = 0;
+
+        player.setConf(getConf());
+        result = player.run(playerArgs);
+        if (!succeeded(result)) {
+          throw new IOException("Can not merge backup images for " + dirs
+              + " (check Hadoop/MR and HBase logs). Player return code =" + result);
+        }
+        // Add to processed table list
+        processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
+        LOG.debug("Merge Job finished:" + result);
+      }
+      List<TableName> tableList = toTableNameList(processedTableList);
+      table.updateProcessedTablesForMerge(tableList);
+      finishedTables = true;
+
+      // Move data
+      for (Pair<TableName, Path> tn : processedTableList) {
+        moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
+      }
+
+      // Delete old data and update manifest
+      List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
+      deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
+      updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
+      // Finish merge session
+      table.finishMergeOperation();
+      // Release lock
+      table.finishBackupExclusiveOperation();
+    } catch (RuntimeException e) {
+
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e);
+      if (!finishedTables) {
+        // cleanup bulk directories and finish merge
+        // merge MUST be repeated (no need for repair)
+        cleanupBulkLoadDirs(fs, toPathList(processedTableList));
+        table.finishMergeOperation();
+        table.finishBackupExclusiveOperation();
+        throw new IOException("Backup merge operation failed, you should try it again", e);
+      } else {
+        // backup repair must be run
+        throw new IOException(
+            "Backup merge operation failed, run backup repair tool to restore system's integrity",
+            e);
+      }
+    } finally {
+      table.close();
+      conn.close();
+    }
+  }
+
+  protected List<Path> toPathList(List<Pair<TableName, Path>> processedTableList) {
+    ArrayList<Path> list = new ArrayList<Path>();
+    for (Pair<TableName, Path> p : processedTableList) {
+      list.add(p.getSecond());
+    }
+    return list;
+  }
+
+  protected List<TableName> toTableNameList(List<Pair<TableName, Path>> processedTableList) {
+    ArrayList<TableName> list = new ArrayList<TableName>();
+    for (Pair<TableName, Path> p : processedTableList) {
+      list.add(p.getFirst());
+    }
+    return list;
+  }
+
+  protected void cleanupBulkLoadDirs(FileSystem fs, List<Path> pathList) throws IOException {
+    for (Path path : pathList) {
+
+      if (!fs.delete(path, true)) {
+        LOG.warn("Can't delete " + path);
+      }
+    }
+  }
+
+  protected void updateBackupManifest(String backupRoot, String mergedBackupId,
+      List<String> backupsToDelete) throws IllegalArgumentException, IOException {
+
+    BackupManifest manifest =
+        HBackupFileSystem.getManifest(conf, new Path(backupRoot), mergedBackupId);
+    manifest.getBackupImage().removeAncestors(backupsToDelete);
+    // save back
+    manifest.store(conf);
+
+  }
+
+  protected void deleteBackupImages(List<String> backupIds, Connection conn, FileSystem fs,
+      String backupRoot) throws IOException {
+
+    // Delete from backup system table
+    try (BackupSystemTable table = new BackupSystemTable(conn);) {
+      for (String backupId : backupIds) {
+        table.deleteBackupInfo(backupId);
+      }
+    }
+
+    // Delete from file system
+    for (String backupId : backupIds) {
+      Path backupDirPath = HBackupFileSystem.getBackupPath(backupRoot, backupId);
+
+      if (!fs.delete(backupDirPath, true)) {
+        LOG.warn("Could not delete " + backupDirPath);
+      }
+    }
+  }
+
+  protected List<String> getBackupIdsToDelete(String[] backupIds, String mergedBackupId) {
+    List<String> list = new ArrayList<String>();
+    for (String id : backupIds) {
+      if (id.equals(mergedBackupId)) {
+        continue;
+      }
+      list.add(id);
+    }
+    return list;
+  }
+
+  protected void moveData(FileSystem fs, String backupRoot, Path bulkOutputPath, TableName tableName,
+      String mergedBackupId) throws IllegalArgumentException, IOException {
+
+    Path dest =
+        new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, mergedBackupId, tableName));
+
+    // Delete all in dest
+    if (!fs.delete(dest, true)) {
+      throw new IOException("Could not delete " + dest);
+    }
+
+    FileStatus[] fsts = fs.listStatus(bulkOutputPath);
+    for (FileStatus fst : fsts) {
+      if (fst.isDirectory()) {
+        fs.rename(fst.getPath().getParent(), dest);
+      }
+    }
+
+  }
+
+  protected String findMostRecentBackupId(String[] backupIds) {
+    long recentTimestamp = Long.MIN_VALUE;
+    for (String backupId : backupIds) {
+      long ts = Long.parseLong(backupId.split("_")[1]);
+      if (ts > recentTimestamp) {
+        recentTimestamp = ts;
+      }
+    }
+    return BackupRestoreConstants.BACKUPID_PREFIX + recentTimestamp;
+  }
+
+  protected TableName[] getTableNamesInBackupImages(String[] backupIds) throws IOException {
+
+    Set<TableName> allSet = new HashSet<TableName>();
+
+    try (Connection conn = ConnectionFactory.createConnection(conf);
+        BackupSystemTable table = new BackupSystemTable(conn);) {
+      for (String backupId : backupIds) {
+        BackupInfo bInfo = table.readBackupInfo(backupId);
+
+        allSet.addAll(bInfo.getTableNames());
+      }
+    }
+
+    TableName[] ret = new TableName[allSet.size()];
+    return allSet.toArray(ret);
+  }
+
+  protected Path[] findInputDirectories(FileSystem fs, String backupRoot, TableName tableName,
+      String[] backupIds) throws IOException {
+
+    List<Path> dirs = new ArrayList<Path>();
+
+    for (String backupId : backupIds) {
+      Path fileBackupDirPath =
+          new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, backupId, tableName));
+      if (fs.exists(fileBackupDirPath)) {
+        dirs.add(fileBackupDirPath);
+      } else {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("File: " + fileBackupDirPath + " does not exist.");
+        }
+      }
+    }
+    Path[] ret = new Path[dirs.size()];
+    return dirs.toArray(ret);
+  }
+
+}


[07/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 7e3201e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
+++ /dev/null
@@ -1,674 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.HBackupFileSystem;
-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;
-    }
-
-    public void removeAncestors(List<String> backupIds) {
-      List<BackupImage> toRemove = new ArrayList<BackupImage>();
-      for (BackupImage im : this.ancestors) {
-        if (backupIds.contains(im.getBackupId())) {
-          toRemove.add(im);
-        }
-      }
-      this.ancestors.removeAll(toRemove);
-    }
-
-    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();
-  }
-
-  /**
-   * TODO: fix it. 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
-    Path manifestFilePath =
-        new Path(HBackupFileSystem.getBackupPath(backupImage.getRootDir(),
-          backupImage.getBackupId()), 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;
-  }
-}


[34/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
new file mode 100644
index 0000000..650ba2e
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
@@ -0,0 +1,1022 @@
+/**
+ * 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.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.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.backup.BackupType;
+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.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+/**
+ * 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"
+      + "  repair     repair backup system table\n"
+      + "  merge      merge backup images\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 REPAIR_CMD_USAGE = "Usage: hbase backup repair\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 MERGE_CMD_USAGE = "Usage: hbase backup merge [backup_ids]\n"
+      + "  backup_ids      Comma separated list of backup image ids.\n";
+
+  public static final String USAGE_FOOTER = "";
+
+  public static abstract class Command extends Configured {
+    CommandLine cmdline;
+    Connection conn;
+
+    Command(Configuration conf) {
+      if (conf == null) {
+        conf = HBaseConfiguration.create();
+      }
+      setConf(conf);
+    }
+
+    public void execute() throws IOException {
+      if (cmdline.hasOption("h") || cmdline.hasOption("help")) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      // Create connection
+      conn = ConnectionFactory.createConnection(getConf());
+      if (requiresNoActiveSession()) {
+        // Check active session
+        try (BackupSystemTable table = new BackupSystemTable(conn);) {
+          List<BackupInfo> sessions = table.getBackupInfos(BackupState.RUNNING);
+
+          if (sessions.size() > 0) {
+            System.err.println("Found backup session in a RUNNING state: ");
+            System.err.println(sessions.get(0));
+            System.err.println("This may indicate that a previous session has failed abnormally.");
+            System.err.println("In this case, backup recovery is recommended.");
+            throw new IOException("Active session found, aborted command execution");
+          }
+        }
+      }
+      if (requiresConsistentState()) {
+        // Check failed delete
+        try (BackupSystemTable table = new BackupSystemTable(conn);) {
+          String[] ids = table.getListOfBackupIdsFromDeleteOperation();
+
+          if (ids != null && ids.length > 0) {
+            System.err.println("Found failed backup DELETE coommand. ");
+            System.err.println("Backup system recovery is required.");
+            throw new IOException("Failed backup DELETE found, aborted command execution");
+          }
+
+          ids = table.getListOfBackupIdsFromMergeOperation();
+          if (ids != null && ids.length > 0) {
+            System.err.println("Found failed backup MERGE coommand. ");
+            System.err.println("Backup system recovery is required.");
+            throw new IOException("Failed backup MERGE found, aborted command execution");
+          }
+
+        }
+      }
+    }
+
+    public void finish() throws IOException {
+      if (conn != null) {
+        conn.close();
+      }
+    }
+
+    protected abstract void printUsage();
+
+    /**
+     * The command can't be run if active backup session is in progress
+     * @return true if no active sessions are in progress
+     */
+    protected boolean requiresNoActiveSession() {
+      return false;
+    }
+
+    /**
+     * Command requires consistent state of a backup system Backup system may become inconsistent
+     * because of an abnormal termination of a backup session or delete command
+     * @return true, if yes
+     */
+    protected boolean requiresConsistentState() {
+      return false;
+    }
+  }
+
+  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 REPAIR:
+      cmd = new RepairCommand(conf, cmdline);
+      break;
+    case MERGE:
+      cmd = new MergeCommand(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
+    protected boolean requiresNoActiveSession() {
+      return true;
+    }
+
+    @Override
+    protected boolean requiresConsistentState() {
+      return true;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      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;
+
+      // 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);
+      }
+      // Creates connection
+      super.execute();
+      // Check backup set
+      String setName = null;
+      if (cmdline.hasOption(OPTION_SET)) {
+        setName = cmdline.getOptionValue(OPTION_SET);
+        tables = getTablesForSet(setName, getConf());
+
+        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 (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 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 {
+      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 {
+      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);
+      }
+
+      super.execute();
+
+      String backupId = args[1];
+      try (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 {
+
+      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);
+      }
+
+      super.execute();
+
+      String backupId = (args == null || args.length <= 1) ? null : args[1];
+      try (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
+    protected boolean requiresNoActiveSession() {
+      return true;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      super.execute();
+
+      String[] args = cmdline.getArgs();
+      String[] backupIds = new String[args.length - 1];
+      System.arraycopy(args, 1, backupIds, 0, backupIds.length);
+      try (BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        int deleted = admin.deleteBackups(backupIds);
+        System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
+      } catch (IOException e) {
+        System.err
+            .println("Delete command FAILED. Please run backup repair tool to restore backup system integrity");
+        throw e;
+      }
+
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(DELETE_CMD_USAGE);
+    }
+  }
+
+  private static class RepairCommand extends Command {
+
+    RepairCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+
+      String[] args = cmdline == null ? null : cmdline.getArgs();
+      if (args != null && args.length > 1) {
+        System.err.println("ERROR: wrong number of arguments: " + args.length);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+
+        // Failed backup
+        BackupInfo backupInfo;
+        List<BackupInfo> list = sysTable.getBackupInfos(BackupState.RUNNING);
+        if (list.size() == 0) {
+          // No failed sessions found
+          System.out.println("REPAIR status: no failed sessions found."
+              + " Checking failed delete backup operation ...");
+          repairFailedBackupDeletionIfAny(conn, sysTable);
+          repairFailedBackupMergeIfAny(conn, sysTable);
+          return;
+        }
+        backupInfo = list.get(0);
+        // 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("REPAIR status: repaired after failure:\n" + backupInfo);
+        // 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();
+        System.out.println(backupFailedData);
+        TableBackupClient.cleanupAndRestoreBackupSystem(conn, backupInfo, conf);
+        // If backup session is updated to FAILED state - means we
+        // processed recovery already.
+        sysTable.updateBackupInfo(backupInfo);
+        sysTable.finishBackupExclusiveOperation();
+        System.out.println("REPAIR status: finished repair failed session:\n " + backupInfo);
+
+      }
+    }
+
+    private void repairFailedBackupDeletionIfAny(Connection conn, BackupSystemTable sysTable)
+        throws IOException {
+      String[] backupIds = sysTable.getListOfBackupIdsFromDeleteOperation();
+      if (backupIds == null || backupIds.length == 0) {
+        System.out.println("No failed backup DELETE operation found");
+        // Delete backup table snapshot if exists
+        BackupSystemTable.deleteSnapshot(conn);
+        return;
+      }
+      System.out.println("Found failed DELETE operation for: " + StringUtils.join(backupIds));
+      System.out.println("Running DELETE again ...");
+      // Restore table from snapshot
+      BackupSystemTable.restoreFromSnapshot(conn);
+      // Finish previous failed session
+      sysTable.finishBackupExclusiveOperation();
+      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
+        admin.deleteBackups(backupIds);
+      }
+      System.out.println("DELETE operation finished OK: " + StringUtils.join(backupIds));
+
+    }
+
+    private void repairFailedBackupMergeIfAny(Connection conn, BackupSystemTable sysTable)
+        throws IOException {
+      String[] backupIds = sysTable.getListOfBackupIdsFromMergeOperation();
+      if (backupIds == null || backupIds.length == 0) {
+        System.out.println("No failed backup MERGE operation found");
+        // Delete backup table snapshot if exists
+        BackupSystemTable.deleteSnapshot(conn);
+        return;
+      }
+      System.out.println("Found failed MERGE operation for: " + StringUtils.join(backupIds));
+      System.out.println("Running MERGE again ...");
+      // Restore table from snapshot
+      BackupSystemTable.restoreFromSnapshot(conn);
+      // Unlock backupo system
+      sysTable.finishBackupExclusiveOperation();
+      // Finish previous failed session
+      sysTable.finishMergeOperation();
+      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
+        admin.mergeBackups(backupIds);
+      }
+      System.out.println("MERGE operation finished OK: " + StringUtils.join(backupIds));
+
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(REPAIR_CMD_USAGE);
+    }
+  }
+
+  private static class MergeCommand extends Command {
+
+    MergeCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    protected boolean requiresNoActiveSession() {
+      return true;
+    }
+
+    @Override
+    protected boolean requiresConsistentState() {
+      return true;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+
+      String[] args = cmdline == null ? null : cmdline.getArgs();
+      if (args == null || (args.length != 2)) {
+        System.err.println("ERROR: wrong number of arguments: "
+            + (args == null ? null : args.length));
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String[] backupIds = args[1].split(",");
+      if (backupIds.length < 2) {
+        String msg = "ERROR: can not merge a single backup image. "+
+            "Number of images must be greater than 1.";
+        System.err.println(msg);
+        throw new IOException(msg);
+
+      }
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        admin.mergeBackups(backupIds);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(MERGE_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 {
+
+      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;
+      if (backupRootPath == null) {
+        // Load from backup system table
+        super.execute();
+        try (final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+          history = sysTable.getBackupHistory(n, tableNameFilter, tableSetFilter);
+        }
+      } else {
+        // load from backup FS
+        history =
+            BackupUtils.getHistory(getConf(), 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 {
+      // 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 {
+      super.execute();
+
+      // List all backup set names
+      // does not expect any args
+      try (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);
+      }
+      super.execute();
+
+      String setName = args[2];
+      try (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);
+      }
+      super.execute();
+
+      String setName = args[2];
+      try (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);
+      }
+      super.execute();
+
+      String setName = args[2];
+      String[] tables = args[3].split(",");
+      TableName[] tableNames = toTableNames(tables);
+      try (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);
+      }
+      super.execute();
+
+      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]);
+      }
+      try (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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
new file mode 100644
index 0000000..2c7d35f
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
new file mode 100644
index 0000000..8fe5eaf
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -0,0 +1,502 @@
+/**
+ * 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.Map;
+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 org.apache.hadoop.hbase.util.Pair;
+
+import org.apache.hadoop.hbase.shaded.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.getBackupPath(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);
+  }
+
+  /**
+   * Starts new backup session
+   * @throws IOException if active session already exists
+   */
+  public void startBackupSession() throws IOException {
+    systemTable.startBackupExclusiveOperation();
+  }
+
+  /**
+   * Finishes active backup session
+   * @throws IOException if no active session
+   */
+  public void finishBackupSession() throws IOException {
+    systemTable.finishBackupExclusiveOperation();
+  }
+
+  /**
+   * 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());
+  }
+
+  public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
+      readBulkloadRows(List<TableName> tableList) throws IOException {
+    return systemTable.readBulkloadRows(tableList);
+  }
+
+  public void removeBulkLoadedRows(List<TableName> lst, List<byte[]> rows) throws IOException {
+    systemTable.removeBulkLoadedRows(lst, rows);
+  }
+
+  public void writeBulkLoadedFiles(List<TableName> sTableList, Map<byte[], List<Path>>[] maps)
+      throws IOException {
+    systemTable.writeBulkLoadedFiles(sTableList, maps, backupInfo.getBackupId());
+  }
+
+  /**
+   * 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 plug-in 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
new file mode 100644
index 0000000..7e3201e
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
@@ -0,0 +1,674 @@
+/**
+ * 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.HBackupFileSystem;
+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;
+    }
+
+    public void removeAncestors(List<String> backupIds) {
+      List<BackupImage> toRemove = new ArrayList<BackupImage>();
+      for (BackupImage im : this.ancestors) {
+        if (backupIds.contains(im.getBackupId())) {
+          toRemove.add(im);
+        }
+      }
+      this.ancestors.removeAll(toRemove);
+    }
+
+    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();
+  }
+
+  /**
+   * TODO: fix it. 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
+    Path manifestFilePath =
+        new Path(HBackupFileSystem.getBackupPath(backupImage.getRootDir(),
+          backupImage.getBackupId()), 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;
+  }
+}


[08/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 99fb06c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
+++ /dev/null
@@ -1,743 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-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.BackupClientFactory;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
-import org.apache.hadoop.hbase.backup.BackupMergeJob;
-import org.apache.hadoop.hbase.backup.BackupRequest;
-import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
-import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
-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.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-@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 {
-  }
-
-  @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 {
-
-    int totalDeleted = 0;
-    Map<String, HashSet<TableName>> allTablesMap = new HashMap<String, HashSet<TableName>>();
-
-    boolean deleteSessionStarted = false;
-    boolean snapshotDone = false;
-    try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) {
-
-      // Step 1: Make sure there is no active session
-      // is running by using startBackupSession API
-      // If there is an active session in progress, exception will be thrown
-      try {
-        sysTable.startBackupExclusiveOperation();
-        deleteSessionStarted = true;
-      } catch (IOException e) {
-        LOG.warn("You can not run delete command while active backup session is in progress. \n"
-            + "If there is no active backup session running, run backup repair utility to restore \n"
-            + "backup system integrity.");
-        return -1;
-      }
-
-      // Step 2: Make sure there is no failed session
-      List<BackupInfo> list = sysTable.getBackupInfos(BackupState.RUNNING);
-      if (list.size() != 0) {
-        // ailed sessions found
-        LOG.warn("Failed backup session found. Run backup repair tool first.");
-        return -1;
-      }
-
-      // Step 3: Record delete session
-      sysTable.startDeleteOperation(backupIds);
-      // Step 4: Snapshot backup system table
-      if (!BackupSystemTable.snapshotExists(conn)) {
-        BackupSystemTable.snapshot(conn);
-      } else {
-        LOG.warn("Backup system table snapshot exists");
-      }
-      snapshotDone = true;
-      try {
-        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);
-        // Finish
-        sysTable.finishDeleteOperation();
-        // delete snapshot
-        BackupSystemTable.deleteSnapshot(conn);
-      } catch (IOException e) {
-        // Fail delete operation
-        // Step 1
-        if (snapshotDone) {
-          if (BackupSystemTable.snapshotExists(conn)) {
-            BackupSystemTable.restoreFromSnapshot(conn);
-            // delete snapshot
-            BackupSystemTable.deleteSnapshot(conn);
-            // We still have record with unfinished delete operation
-            LOG.error("Delete operation failed, please run backup repair utility to restore "
-                + "backup system integrity", e);
-            throw e;
-          } else {
-            LOG.warn("Delete operation succeeded, there were some errors: ", e);
-          }
-        }
-
-      } finally {
-        if (deleteSessionStarted) {
-          sysTable.finishBackupExclusiveOperation();
-        }
-      }
-    }
-    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() + " ...");
-      // Step 1: clean up data for backup session (idempotent)
-      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 = getAffectedBackupSessions(backupInfo, tn, sysTable);
-        for (BackupInfo info : affectedBackups) {
-          if (info.equals(backupInfo)) {
-            continue;
-          }
-          removeTableFromBackupImage(info, tn, sysTable);
-        }
-      }
-      Map<byte[], String> map = sysTable.readBulkLoadedFiles(backupId);
-      FileSystem fs = FileSystem.get(conn.getConfiguration());
-      boolean success = true;
-      int numDeleted = 0;
-      for (String f : map.values()) {
-        Path p = new Path(f);
-        try {
-          LOG.debug("Delete backup info " + p + " for " + backupInfo.getBackupId());
-          if (!fs.delete(p)) {
-            if (fs.exists(p)) {
-              LOG.warn(f + " was not deleted");
-              success = false;
-            }
-          } else {
-            numDeleted++;
-          }
-        } catch (IOException ioe) {
-          LOG.warn(f + " was not deleted", ioe);
-          success = false;
-        }
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(numDeleted + " bulk loaded files out of " + map.size() + " were deleted");
-      }
-      if (success) {
-        sysTable.deleteBulkLoadedFiles(map);
-      }
-
-      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());
-        // Idempotent operation
-        BackupUtils.cleanupBackupData(info, conn.getConfiguration());
-      } else {
-        info.setTables(tables);
-        sysTable.updateBackupInfo(info);
-        // Now, clean up directory for table (idempotent)
-        cleanupBackupDir(info, tn, conn.getConfiguration());
-      }
-    }
-  }
-
-  private List<BackupInfo> getAffectedBackupSessions(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();
-
-    TableBackupClient client = null;
-    try {
-      client = BackupClientFactory.create(conn, backupId, request);
-    } catch (IOException e) {
-      LOG.error("There is an active session already running");
-      throw e;
-    }
-
-    client.execute();
-
-    return backupId;
-  }
-
-  private List<TableName> excludeNonExistingTables(List<TableName> tableList,
-      List<TableName> nonExistingTableList) {
-
-    for (TableName table : nonExistingTableList) {
-      tableList.remove(table);
-    }
-    return tableList;
-  }
-
-  @Override
-  public void mergeBackups(String[] backupIds) throws IOException {
-    try (final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
-      checkIfValidForMerge(backupIds, sysTable);
-      BackupMergeJob job = BackupRestoreFactory.getBackupMergeJob(conn.getConfiguration());
-      job.run(backupIds);
-    }
-  }
-
-  /**
-   * Verifies that backup images are valid for merge.
-   *
-   * <ul>
-   * <li>All backups MUST be in the same destination
-   * <li>No FULL backups are allowed - only INCREMENTAL
-   * <li>All backups must be in COMPLETE state
-   * <li>No holes in backup list are allowed
-   * </ul>
-   * <p>
-   * @param backupIds list of backup ids
-   * @param table backup system table
-   * @throws IOException
-   */
-  private void checkIfValidForMerge(String[] backupIds, BackupSystemTable table) throws IOException {
-    String backupRoot = null;
-
-    final Set<TableName> allTables = new HashSet<TableName>();
-    final Set<String> allBackups = new HashSet<String>();
-    long minTime = Long.MAX_VALUE, maxTime = Long.MIN_VALUE;
-    for (String backupId : backupIds) {
-      BackupInfo bInfo = table.readBackupInfo(backupId);
-      if (bInfo == null) {
-        String msg = "Backup session " + backupId + " not found";
-        throw new IOException(msg);
-      }
-      if (backupRoot == null) {
-        backupRoot = bInfo.getBackupRootDir();
-      } else if (!bInfo.getBackupRootDir().equals(backupRoot)) {
-        throw new IOException("Found different backup destinations in a list of a backup sessions \n"
-            + "1. " + backupRoot + "\n" + "2. " + bInfo.getBackupRootDir());
-      }
-      if (bInfo.getType() == BackupType.FULL) {
-        throw new IOException("FULL backup image can not be merged for: \n" + bInfo);
-      }
-
-      if (bInfo.getState() != BackupState.COMPLETE) {
-        throw new IOException("Backup image " + backupId
-            + " can not be merged becuase of its state: " + bInfo.getState());
-      }
-      allBackups.add(backupId);
-      allTables.addAll(bInfo.getTableNames());
-      long time = bInfo.getStartTs();
-      if (time < minTime) {
-        minTime = time;
-      }
-      if (time > maxTime) {
-        maxTime = time;
-      }
-    }
-
-
-    final long startRangeTime  = minTime;
-    final long endRangeTime = maxTime;
-    final String backupDest = backupRoot;
-    // Check we have no 'holes' in backup id list
-    // Filter 1 : backupRoot
-    // Filter 2 : time range filter
-    // Filter 3 : table filter
-
-    BackupInfo.Filter destinationFilter = new  BackupInfo.Filter() {
-
-      @Override
-      public boolean apply(BackupInfo info) {
-        return info.getBackupRootDir().equals(backupDest);
-      }
-    };
-
-    BackupInfo.Filter timeRangeFilter = new  BackupInfo.Filter() {
-
-      @Override
-      public boolean apply(BackupInfo info) {
-        long time = info.getStartTs();
-        return time >= startRangeTime && time <= endRangeTime ;
-      }
-    };
-
-    BackupInfo.Filter tableFilter = new  BackupInfo.Filter() {
-
-      @Override
-      public boolean apply(BackupInfo info) {
-        List<TableName> tables = info.getTableNames();
-        return !Collections.disjoint(allTables, tables);
-      }
-    };
-
-    BackupInfo.Filter typeFilter = new  BackupInfo.Filter() {
-
-      @Override
-      public boolean apply(BackupInfo info) {
-        return info.getType() == BackupType.INCREMENTAL;
-      }
-    };
-
-    BackupInfo.Filter stateFilter = new  BackupInfo.Filter() {
-      @Override
-      public boolean apply(BackupInfo info) {
-        return info.getState() == BackupState.COMPLETE;
-      }
-    };
-
-    List<BackupInfo> allInfos =
-        table.getBackupHistory( -1, destinationFilter,
-          timeRangeFilter, tableFilter, typeFilter, stateFilter);
-    if (allInfos.size() != allBackups.size()) {
-      // Yes we have at least one  hole in backup image sequence
-      List<String> missingIds = new ArrayList<String>();
-      for(BackupInfo info: allInfos) {
-        if(allBackups.contains(info.getBackupId())) {
-          continue;
-        }
-        missingIds.add(info.getBackupId());
-      }
-      String errMsg =
-          "Sequence of backup ids has 'holes'. The following backup images must be added:" +
-           org.apache.hadoop.util.StringUtils.join(",", missingIds);
-      throw new IOException(errMsg);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 650ba2e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
+++ /dev/null
@@ -1,1022 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.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.BackupRestoreConstants.BackupCommand;
-import org.apache.hadoop.hbase.backup.BackupType;
-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.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-/**
- * 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"
-      + "  repair     repair backup system table\n"
-      + "  merge      merge backup images\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 REPAIR_CMD_USAGE = "Usage: hbase backup repair\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 MERGE_CMD_USAGE = "Usage: hbase backup merge [backup_ids]\n"
-      + "  backup_ids      Comma separated list of backup image ids.\n";
-
-  public static final String USAGE_FOOTER = "";
-
-  public static abstract class Command extends Configured {
-    CommandLine cmdline;
-    Connection conn;
-
-    Command(Configuration conf) {
-      if (conf == null) {
-        conf = HBaseConfiguration.create();
-      }
-      setConf(conf);
-    }
-
-    public void execute() throws IOException {
-      if (cmdline.hasOption("h") || cmdline.hasOption("help")) {
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      // Create connection
-      conn = ConnectionFactory.createConnection(getConf());
-      if (requiresNoActiveSession()) {
-        // Check active session
-        try (BackupSystemTable table = new BackupSystemTable(conn);) {
-          List<BackupInfo> sessions = table.getBackupInfos(BackupState.RUNNING);
-
-          if (sessions.size() > 0) {
-            System.err.println("Found backup session in a RUNNING state: ");
-            System.err.println(sessions.get(0));
-            System.err.println("This may indicate that a previous session has failed abnormally.");
-            System.err.println("In this case, backup recovery is recommended.");
-            throw new IOException("Active session found, aborted command execution");
-          }
-        }
-      }
-      if (requiresConsistentState()) {
-        // Check failed delete
-        try (BackupSystemTable table = new BackupSystemTable(conn);) {
-          String[] ids = table.getListOfBackupIdsFromDeleteOperation();
-
-          if (ids != null && ids.length > 0) {
-            System.err.println("Found failed backup DELETE coommand. ");
-            System.err.println("Backup system recovery is required.");
-            throw new IOException("Failed backup DELETE found, aborted command execution");
-          }
-
-          ids = table.getListOfBackupIdsFromMergeOperation();
-          if (ids != null && ids.length > 0) {
-            System.err.println("Found failed backup MERGE coommand. ");
-            System.err.println("Backup system recovery is required.");
-            throw new IOException("Failed backup MERGE found, aborted command execution");
-          }
-
-        }
-      }
-    }
-
-    public void finish() throws IOException {
-      if (conn != null) {
-        conn.close();
-      }
-    }
-
-    protected abstract void printUsage();
-
-    /**
-     * The command can't be run if active backup session is in progress
-     * @return true if no active sessions are in progress
-     */
-    protected boolean requiresNoActiveSession() {
-      return false;
-    }
-
-    /**
-     * Command requires consistent state of a backup system Backup system may become inconsistent
-     * because of an abnormal termination of a backup session or delete command
-     * @return true, if yes
-     */
-    protected boolean requiresConsistentState() {
-      return false;
-    }
-  }
-
-  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 REPAIR:
-      cmd = new RepairCommand(conf, cmdline);
-      break;
-    case MERGE:
-      cmd = new MergeCommand(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
-    protected boolean requiresNoActiveSession() {
-      return true;
-    }
-
-    @Override
-    protected boolean requiresConsistentState() {
-      return true;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      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;
-
-      // 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);
-      }
-      // Creates connection
-      super.execute();
-      // Check backup set
-      String setName = null;
-      if (cmdline.hasOption(OPTION_SET)) {
-        setName = cmdline.getOptionValue(OPTION_SET);
-        tables = getTablesForSet(setName, getConf());
-
-        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 (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 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 {
-      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 {
-      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);
-      }
-
-      super.execute();
-
-      String backupId = args[1];
-      try (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 {
-
-      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);
-      }
-
-      super.execute();
-
-      String backupId = (args == null || args.length <= 1) ? null : args[1];
-      try (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
-    protected boolean requiresNoActiveSession() {
-      return true;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      super.execute();
-
-      String[] args = cmdline.getArgs();
-      String[] backupIds = new String[args.length - 1];
-      System.arraycopy(args, 1, backupIds, 0, backupIds.length);
-      try (BackupAdminImpl admin = new BackupAdminImpl(conn);) {
-        int deleted = admin.deleteBackups(backupIds);
-        System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
-      } catch (IOException e) {
-        System.err
-            .println("Delete command FAILED. Please run backup repair tool to restore backup system integrity");
-        throw e;
-      }
-
-    }
-
-    @Override
-    protected void printUsage() {
-      System.out.println(DELETE_CMD_USAGE);
-    }
-  }
-
-  private static class RepairCommand extends Command {
-
-    RepairCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-
-      String[] args = cmdline == null ? null : cmdline.getArgs();
-      if (args != null && args.length > 1) {
-        System.err.println("ERROR: wrong number of arguments: " + args.length);
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
-      try (final Connection conn = ConnectionFactory.createConnection(conf);
-          final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
-
-        // Failed backup
-        BackupInfo backupInfo;
-        List<BackupInfo> list = sysTable.getBackupInfos(BackupState.RUNNING);
-        if (list.size() == 0) {
-          // No failed sessions found
-          System.out.println("REPAIR status: no failed sessions found."
-              + " Checking failed delete backup operation ...");
-          repairFailedBackupDeletionIfAny(conn, sysTable);
-          repairFailedBackupMergeIfAny(conn, sysTable);
-          return;
-        }
-        backupInfo = list.get(0);
-        // 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("REPAIR status: repaired after failure:\n" + backupInfo);
-        // 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();
-        System.out.println(backupFailedData);
-        TableBackupClient.cleanupAndRestoreBackupSystem(conn, backupInfo, conf);
-        // If backup session is updated to FAILED state - means we
-        // processed recovery already.
-        sysTable.updateBackupInfo(backupInfo);
-        sysTable.finishBackupExclusiveOperation();
-        System.out.println("REPAIR status: finished repair failed session:\n " + backupInfo);
-
-      }
-    }
-
-    private void repairFailedBackupDeletionIfAny(Connection conn, BackupSystemTable sysTable)
-        throws IOException {
-      String[] backupIds = sysTable.getListOfBackupIdsFromDeleteOperation();
-      if (backupIds == null || backupIds.length == 0) {
-        System.out.println("No failed backup DELETE operation found");
-        // Delete backup table snapshot if exists
-        BackupSystemTable.deleteSnapshot(conn);
-        return;
-      }
-      System.out.println("Found failed DELETE operation for: " + StringUtils.join(backupIds));
-      System.out.println("Running DELETE again ...");
-      // Restore table from snapshot
-      BackupSystemTable.restoreFromSnapshot(conn);
-      // Finish previous failed session
-      sysTable.finishBackupExclusiveOperation();
-      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
-        admin.deleteBackups(backupIds);
-      }
-      System.out.println("DELETE operation finished OK: " + StringUtils.join(backupIds));
-
-    }
-
-    private void repairFailedBackupMergeIfAny(Connection conn, BackupSystemTable sysTable)
-        throws IOException {
-      String[] backupIds = sysTable.getListOfBackupIdsFromMergeOperation();
-      if (backupIds == null || backupIds.length == 0) {
-        System.out.println("No failed backup MERGE operation found");
-        // Delete backup table snapshot if exists
-        BackupSystemTable.deleteSnapshot(conn);
-        return;
-      }
-      System.out.println("Found failed MERGE operation for: " + StringUtils.join(backupIds));
-      System.out.println("Running MERGE again ...");
-      // Restore table from snapshot
-      BackupSystemTable.restoreFromSnapshot(conn);
-      // Unlock backupo system
-      sysTable.finishBackupExclusiveOperation();
-      // Finish previous failed session
-      sysTable.finishMergeOperation();
-      try (BackupAdmin admin = new BackupAdminImpl(conn);) {
-        admin.mergeBackups(backupIds);
-      }
-      System.out.println("MERGE operation finished OK: " + StringUtils.join(backupIds));
-
-    }
-
-    @Override
-    protected void printUsage() {
-      System.out.println(REPAIR_CMD_USAGE);
-    }
-  }
-
-  private static class MergeCommand extends Command {
-
-    MergeCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    protected boolean requiresNoActiveSession() {
-      return true;
-    }
-
-    @Override
-    protected boolean requiresConsistentState() {
-      return true;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-
-      String[] args = cmdline == null ? null : cmdline.getArgs();
-      if (args == null || (args.length != 2)) {
-        System.err.println("ERROR: wrong number of arguments: "
-            + (args == null ? null : args.length));
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      String[] backupIds = args[1].split(",");
-      if (backupIds.length < 2) {
-        String msg = "ERROR: can not merge a single backup image. "+
-            "Number of images must be greater than 1.";
-        System.err.println(msg);
-        throw new IOException(msg);
-
-      }
-      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
-      try (final Connection conn = ConnectionFactory.createConnection(conf);
-          final BackupAdminImpl admin = new BackupAdminImpl(conn);) {
-        admin.mergeBackups(backupIds);
-      }
-    }
-
-    @Override
-    protected void printUsage() {
-      System.out.println(MERGE_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 {
-
-      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;
-      if (backupRootPath == null) {
-        // Load from backup system table
-        super.execute();
-        try (final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
-          history = sysTable.getBackupHistory(n, tableNameFilter, tableSetFilter);
-        }
-      } else {
-        // load from backup FS
-        history =
-            BackupUtils.getHistory(getConf(), 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 {
-      // 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 {
-      super.execute();
-
-      // List all backup set names
-      // does not expect any args
-      try (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);
-      }
-      super.execute();
-
-      String setName = args[2];
-      try (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);
-      }
-      super.execute();
-
-      String setName = args[2];
-      try (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);
-      }
-      super.execute();
-
-      String setName = args[2];
-      String[] tables = args[3].split(",");
-      TableName[] tableNames = toTableNames(tables);
-      try (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);
-      }
-      super.execute();
-
-      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]);
-      }
-      try (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/2dda3712/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
deleted file mode 100644
index 2c7d35f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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/2dda3712/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
deleted file mode 100644
index 8fe5eaf..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
+++ /dev/null
@@ -1,502 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.Map;
-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 org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.hadoop.hbase.shaded.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.getBackupPath(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);
-  }
-
-  /**
-   * Starts new backup session
-   * @throws IOException if active session already exists
-   */
-  public void startBackupSession() throws IOException {
-    systemTable.startBackupExclusiveOperation();
-  }
-
-  /**
-   * Finishes active backup session
-   * @throws IOException if no active session
-   */
-  public void finishBackupSession() throws IOException {
-    systemTable.finishBackupExclusiveOperation();
-  }
-
-  /**
-   * 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());
-  }
-
-  public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
-      readBulkloadRows(List<TableName> tableList) throws IOException {
-    return systemTable.readBulkloadRows(tableList);
-  }
-
-  public void removeBulkLoadedRows(List<TableName> lst, List<byte[]> rows) throws IOException {
-    systemTable.removeBulkLoadedRows(lst, rows);
-  }
-
-  public void writeBulkLoadedFiles(List<TableName> sTableList, Map<byte[], List<Path>>[] maps)
-      throws IOException {
-    systemTable.writeBulkLoadedFiles(sTableList, maps, backupInfo.getBackupId());
-  }
-
-  /**
-   * 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 plug-in 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;
-  }
-}


[11/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
new file mode 100644
index 0000000..0944ea2
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
@@ -0,0 +1,158 @@
+/**
+ * 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.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
new file mode 100644
index 0000000..686d34b
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
@@ -0,0 +1,91 @@
+/**
+ * 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.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.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
+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 TestBackupRepair extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupRepair.class);
+
+
+  @Test
+  public void testFullBackupWithFailuresAndRestore() throws Exception {
+
+    autoRestoreOnFailure = false;
+
+    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
+      FullTableBackupClientForTest.class.getName());
+    int maxStage = Stage.values().length -1;
+    // Fail stage in loop between 0 and 4 inclusive
+    for (int stage = 0; stage < maxStage; stage++) {
+      LOG.info("Running stage " + stage);
+      runBackupAndFailAtStageWithRestore(stage);
+    }
+  }
+
+  public void runBackupAndFailAtStageWithRestore(int stage) throws Exception {
+
+    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
+    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);
+      assertFalse(ret == 0);
+
+      // Now run restore
+      args = new String[] {"repair"};
+
+      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();
+        assertFalse(checkSucceeded(backupId));
+      }
+      Set<TableName> tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR);
+      assertTrue(tables.size() == 0);
+    }
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
new file mode 100644
index 0000000..4e922a2
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
new file mode 100644
index 0000000..73d8d9f
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
new file mode 100644
index 0000000..5814d87
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
new file mode 100644
index 0000000..124d19f
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
new file mode 100644
index 0000000..4dc894b
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
new file mode 100644
index 0000000..6b007f9
--- /dev/null
+++ b/hbase-backup/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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
new file mode 100644
index 0000000..d18de88
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
@@ -0,0 +1,79 @@
+/**
+ * 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.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.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
+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 TestFullBackupWithFailures extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestFullBackupWithFailures.class);
+
+  @Test
+  public void testFullBackupWithFailures() throws Exception {
+    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
+      FullTableBackupClientForTest.class.getName());
+    int maxStage = Stage.values().length -1;
+    // Fail stages between 0 and 4 inclusive
+    for (int stage = 0; stage <= maxStage; stage++) {
+      LOG.info("Running stage " + stage);
+      runBackupAndFailAtStage(stage);
+    }
+  }
+
+  public void runBackupAndFailAtStage(int stage) throws Exception {
+
+    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
+    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);
+      assertFalse(ret == 0);
+      List<BackupInfo> backups = table.getBackupHistory();
+      int after = table.getBackupHistory().size();
+
+      assertTrue(after ==  before +1);
+      for (BackupInfo data : backups) {
+        String backupId = data.getBackupId();
+        assertFalse(checkSucceeded(backupId));
+      }
+      Set<TableName> tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR);
+      assertTrue(tables.size() == 0);
+    }
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
new file mode 100644
index 0000000..48a553f
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
new file mode 100644
index 0000000..73598f3
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
@@ -0,0 +1,209 @@
+/**
+ * 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.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 org.apache.hadoop.hbase.shaded.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();
+    try {
+      admin.splitRegion(name);
+    } catch (IOException e) {
+      //although split fail, this may not affect following check
+      //In old split without AM2, if region's best split key is not found,
+      //there are not exception thrown. But in current API, exception
+      //will be thrown.
+      LOG.debug("region is not splittable, because " + e);
+    }
+
+    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.getDescriptor());
+    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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
new file mode 100644
index 0000000..747c1dd
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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();
+  }
+
+}


[28/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
new file mode 100644
index 0000000..7011ed3
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
@@ -0,0 +1,336 @@
+/**
+ * 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.util.BackupUtils.succeeded;
+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.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.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceHFileSplitterJob;
+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.Table;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Pair;
+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 TestIncrementalBackupMergeWithFailures extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupMergeWithFailures.class);
+
+  static enum FailurePhase {
+    PHASE1, PHASE2, PHASE3, PHASE4
+  }
+  public final static String FAILURE_PHASE_KEY = "failurePhase";
+
+  static class BackupMergeJobWithFailures extends MapReduceBackupMergeJob {
+
+    FailurePhase failurePhase;
+
+    @Override
+    public void setConf(Configuration conf) {
+      super.setConf(conf);
+      String val = conf.get(FAILURE_PHASE_KEY);
+      if (val != null) {
+        failurePhase = FailurePhase.valueOf(val);
+      } else {
+        Assert.fail("Failure phase is not set");
+      }
+    }
+
+
+    /**
+     * This is the exact copy of parent's run() with injections
+     * of different types of failures
+     */
+    @Override
+    public void run(String[] backupIds) throws IOException {
+      String bulkOutputConfKey;
+
+      // TODO : run player on remote cluster
+      player = new MapReduceHFileSplitterJob();
+      bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
+      // Player reads all files in arbitrary directory structure and creates
+      // a Map task for each file
+      String bids = StringUtils.join(backupIds, ",");
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Merge backup images " + bids);
+      }
+
+      List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
+      boolean finishedTables = false;
+      Connection conn = ConnectionFactory.createConnection(getConf());
+      BackupSystemTable table = new BackupSystemTable(conn);
+      FileSystem fs = FileSystem.get(getConf());
+
+      try {
+
+        // Start backup exclusive operation
+        table.startBackupExclusiveOperation();
+        // Start merge operation
+        table.startMergeOperation(backupIds);
+
+        // Select most recent backup id
+        String mergedBackupId = findMostRecentBackupId(backupIds);
+
+        TableName[] tableNames = getTableNamesInBackupImages(backupIds);
+        String backupRoot = null;
+
+        BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
+        backupRoot = bInfo.getBackupRootDir();
+        // PHASE 1
+        checkFailure(FailurePhase.PHASE1);
+
+        for (int i = 0; i < tableNames.length; i++) {
+
+          LOG.info("Merge backup images for " + tableNames[i]);
+
+          // Find input directories for table
+
+          Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
+          String dirs = StringUtils.join(dirPaths, ",");
+          Path bulkOutputPath =
+              BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
+                getConf(), false);
+          // Delete content if exists
+          if (fs.exists(bulkOutputPath)) {
+            if (!fs.delete(bulkOutputPath, true)) {
+              LOG.warn("Can not delete: " + bulkOutputPath);
+            }
+          }
+          Configuration conf = getConf();
+          conf.set(bulkOutputConfKey, bulkOutputPath.toString());
+          String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
+
+          int result = 0;
+          // PHASE 2
+          checkFailure(FailurePhase.PHASE2);
+          player.setConf(getConf());
+          result = player.run(playerArgs);
+          if (succeeded(result)) {
+            // Add to processed table list
+            processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
+          } else {
+            throw new IOException("Can not merge backup images for " + dirs
+                + " (check Hadoop/MR and HBase logs). Player return code =" + result);
+          }
+          LOG.debug("Merge Job finished:" + result);
+        }
+        List<TableName> tableList = toTableNameList(processedTableList);
+        // PHASE 3
+        checkFailure(FailurePhase.PHASE3);
+        table.updateProcessedTablesForMerge(tableList);
+        finishedTables = true;
+
+        // Move data
+        for (Pair<TableName, Path> tn : processedTableList) {
+          moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
+        }
+        // PHASE 4
+        checkFailure(FailurePhase.PHASE4);
+        // Delete old data and update manifest
+        List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
+        deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
+        updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
+        // Finish merge session
+        table.finishMergeOperation();
+      } catch (RuntimeException e) {
+        throw e;
+      } catch (Exception e) {
+        LOG.error(e);
+        if (!finishedTables) {
+          // cleanup bulk directories and finish merge
+          // merge MUST be repeated (no need for repair)
+          cleanupBulkLoadDirs(fs, toPathList(processedTableList));
+          table.finishMergeOperation();
+          table.finishBackupExclusiveOperation();
+          throw new IOException("Backup merge operation failed, you should try it again", e);
+        } else {
+          // backup repair must be run
+          throw new IOException(
+              "Backup merge operation failed, run backup repair tool to restore system's integrity",
+              e);
+        }
+      } finally {
+        table.close();
+        conn.close();
+      }
+
+    }
+
+    private void checkFailure(FailurePhase phase) throws IOException {
+      if ( failurePhase != null && failurePhase == phase) {
+        throw new IOException (phase.toString());
+      }
+    }
+
+  }
+
+
+  @Test
+  public void TestIncBackupMergeRestore() 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);
+    // Set custom Merge Job implementation
+    conf1.setClass(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS,
+      BackupMergeJobWithFailures.class, BackupMergeJob.class);
+
+    Connection conn = ConnectionFactory.createConnection(conf1);
+
+    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 table1
+    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);
+    t1.close();
+    LOG.debug("written " + ADD_ROWS + " rows to " + table1);
+
+    HTable t2 = insertIntoTable(conn, table2, famName, 1, ADD_ROWS);
+
+    Assert.assertEquals(TEST_UTIL.countRows(t2), NB_ROWS_IN_BATCH + ADD_ROWS);
+    t2.close();
+    LOG.debug("written " + ADD_ROWS + " rows to " + table2);
+
+    // #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));
+
+    t1 = insertIntoTable(conn, table1, famName, 2, ADD_ROWS);
+    t1.close();
+
+    t2 = insertIntoTable(conn, table2, famName, 2, ADD_ROWS);
+    t2.close();
+
+    // #3 - incremental backup for multiple tables
+    request = createBackupRequest(BackupType.INCREMENTAL, tables, BACKUP_ROOT_DIR);
+    String backupIdIncMultiple2 = client.backupTables(request);
+    assertTrue(checkSucceeded(backupIdIncMultiple2));
+
+    // #4 Merge backup images with failures
+
+    for ( FailurePhase phase : FailurePhase.values()) {
+      Configuration conf = conn.getConfiguration();
+
+      conf.set(FAILURE_PHASE_KEY, phase.toString());
+
+      try (BackupAdmin bAdmin = new BackupAdminImpl(conn);)
+      {
+        String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
+        bAdmin.mergeBackups(backups);
+        Assert.fail("Expected IOException");
+      } catch (IOException e) {
+        BackupSystemTable table = new BackupSystemTable(conn);
+        if(phase.ordinal() < FailurePhase.PHASE4.ordinal()) {
+          // No need to repair:
+          // Both Merge and backup exclusive operations are finished
+          assertFalse(table.isMergeInProgress());
+          try {
+            table.finishBackupExclusiveOperation();
+            Assert.fail("IOException is expected");
+          } catch(IOException ee) {
+            // Expected
+          }
+        } else {
+          // Repair is required
+          assertTrue(table.isMergeInProgress());
+          try {
+            table.startBackupExclusiveOperation();
+            Assert.fail("IOException is expected");
+          } catch(IOException ee) {
+            // Expected - clean up before proceeding
+            table.finishMergeOperation();
+            table.finishBackupExclusiveOperation();
+          }
+        }
+        table.close();
+        LOG.debug("Expected :"+ e.getMessage());
+      }
+    }
+
+    // Now merge w/o failures
+    Configuration conf = conn.getConfiguration();
+    conf.unset(FAILURE_PHASE_KEY);
+    conf.unset(BackupRestoreFactory.HBASE_BACKUP_MERGE_IMPL_CLASS);
+
+    try (BackupAdmin bAdmin = new BackupAdminImpl(conn);) {
+      String[] backups = new String[] { backupIdIncMultiple, backupIdIncMultiple2 };
+      bAdmin.mergeBackups(backups);
+    }
+
+    // #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));
+
+    Table 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 + 2 * ADD_ROWS);
+
+    hTable.close();
+
+    hTable = conn.getTable(table2_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH + 2 * ADD_ROWS);
+    hTable.close();
+
+    admin.close();
+    conn.close();
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
new file mode 100644
index 0000000..769785f
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithBulkLoad.java
@@ -0,0 +1,145 @@
+/**
+ * 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 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.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+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.mapreduce.TestLoadIncrementalHFiles;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+/**
+ * 1. Create table t1
+ * 2. Load data to t1
+ * 3 Full backup t1
+ * 4 Load data to t1
+ * 5 bulk load into t1
+ * 6 Incremental backup t1
+ */
+@Category(LargeTests.class)
+@RunWith(Parameterized.class)
+public class TestIncrementalBackupWithBulkLoad extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupDeleteTable.class);
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    secure = true;
+    List<Object[]> params = new ArrayList<Object[]>();
+    params.add(new Object[] {Boolean.TRUE});
+    return params;
+  }
+
+  public TestIncrementalBackupWithBulkLoad(Boolean b) {
+  }
+  // implement all test cases in 1 test since incremental backup/restore has dependencies
+  @Test
+  public void TestIncBackupDeleteTable() throws Exception {
+    String testName = "TestIncBackupDeleteTable";
+    // #1 - create full backup for all tables
+    LOG.info("create full backup image for all tables");
+
+    List<TableName> tables = Lists.newArrayList(table1);
+    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();
+
+    int NB_ROWS2 = 20;
+    LOG.debug("bulk loading into " + testName);
+    int actual = TestLoadIncrementalHFiles.loadHFiles(testName, table1Desc, TEST_UTIL, famName,
+        qualName, false, null, new byte[][][] {
+      new byte[][]{ Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
+      new byte[][]{ Bytes.toBytes("ddd"), Bytes.toBytes("ooo") },
+    }, true, false, true, NB_ROWS_IN_BATCH*2, NB_ROWS2);
+
+    // #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));
+
+    // #5.1 - check tables for full restore */
+    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
+
+    // #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 = (HTable) conn.getTable(table1_restore);
+    Assert.assertEquals(TEST_UTIL.countRows(hTable), NB_ROWS_IN_BATCH * 2+actual);
+    request = createBackupRequest(BackupType.FULL, tables, BACKUP_ROOT_DIR);
+
+    backupIdFull = client.backupTables(request);
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>> pair
+      = table.readBulkloadRows(tables);
+      assertTrue("map still has " + pair.getSecond().size() + " entries",
+          pair.getSecond().isEmpty());
+    }
+    assertTrue(checkSucceeded(backupIdFull));
+
+    hTable.close();
+    admin.close();
+    conn.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
new file mode 100644
index 0000000..84a596e
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.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.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.util.ToolRunner;
+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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+@RunWith(Parameterized.class)
+public class TestIncrementalBackupWithFailures extends TestBackupBase {
+  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupWithFailures.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 TestIncrementalBackupWithFailures(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);
+
+    // #3 - incremental backup for multiple tables
+    incrementalBackupWithFailures();
+
+    admin.close();
+    conn.close();
+
+  }
+
+
+  private void incrementalBackupWithFailures() throws Exception {
+    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
+      IncrementalTableBackupClientForTest.class.getName());
+    int maxStage = Stage.values().length -1;
+    // Fail stages between 0 and 4 inclusive
+    for (int stage = 0; stage <= maxStage; stage++) {
+      LOG.info("Running stage " + stage);
+      runBackupAndFailAtStage(stage);
+    }
+  }
+
+  private void runBackupAndFailAtStage(int stage) throws Exception {
+
+    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
+    try (BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {
+      int before = table.getBackupHistory().size();
+      String[] args =
+          new String[] { "create", "incremental", BACKUP_ROOT_DIR, "-t",
+              table1.getNameAsString() + "," + table2.getNameAsString() };
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertFalse(ret == 0);
+      List<BackupInfo> backups = table.getBackupHistory();
+      int after = table.getBackupHistory().size();
+
+      assertTrue(after ==  before +1);
+      for (BackupInfo data : backups) {
+        if(data.getType() == BackupType.FULL) {
+          assertTrue(data.getState() == BackupState.COMPLETE);
+        } else {
+          assertTrue(data.getState() == BackupState.FAILED);
+        }
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
new file mode 100644
index 0000000..36a9ee2
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.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 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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestRemoteBackup extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRemoteBackup.class);
+
+  @Override
+  public void setUp () throws Exception {
+    useSecondCluster = true;
+    super.setUp();
+  }
+
+  /**
+   * 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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
new file mode 100644
index 0000000..0386c27
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.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 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);
+
+  @Override
+  public void setUp () throws Exception {
+    useSecondCluster = true;
+    super.setUp();
+  }
+
+
+  /**
+   * 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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
new file mode 100644
index 0000000..556521f
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRepairAfterFailedDelete.java
@@ -0,0 +1,93 @@
+/**
+ * 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.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.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestRepairAfterFailedDelete extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestRepairAfterFailedDelete.class);
+
+  @Test
+  public void testRepairBackupDelete() throws Exception {
+    LOG.info("test repair 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));
+
+    // Snapshot backup system table before delete
+    String snapshotName = "snapshot-backup";
+    Connection conn = TEST_UTIL.getConnection();
+    Admin admin = conn.getAdmin();
+    admin.snapshot(snapshotName, BackupSystemTable.getTableName(conf1));
+
+    int deleted = getBackupAdmin().deleteBackups(backupIds);
+
+    assertTrue(!fs.exists(path));
+    assertTrue(fs.exists(new Path(info.getBackupRootDir())));
+    assertTrue(1 == deleted);
+
+    // Emulate delete failure
+    // Restore backup system table
+    admin.disableTable(BackupSystemTable.getTableName(conf1));
+    admin.restoreSnapshot(snapshotName);
+    admin.enableTable(BackupSystemTable.getTableName(conf1));
+    // Start backup session
+    table.startBackupExclusiveOperation();
+    // Start delete operation
+    table.startDeleteOperation(backupIds);
+
+    // Now run repair command to repair "failed" delete operation
+    String[] args = new String[] {"repair"};
+    // Run restore
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    // Verify that history length == 0
+    assertTrue (table.getBackupHistory().size() == 0);
+    table.close();
+    admin.close();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
new file mode 100644
index 0000000..c61b018
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.java
new file mode 100644
index 0000000..6443421
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestSystemTableSnapshot.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 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 backup system table snapshot
+   * @throws Exception
+   */
+ // @Test
+  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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
new file mode 100644
index 0000000..5f72f45
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
+import org.apache.hadoop.hbase.shaded.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/37c65946/hbase-backup/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/resources/log4j.properties b/hbase-backup/src/test/resources/log4j.properties
new file mode 100644
index 0000000..c322699
--- /dev/null
+++ b/hbase-backup/src/test/resources/log4j.properties
@@ -0,0 +1,68 @@
+# 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.
+
+# Define some default values that can be overridden by system properties
+hbase.root.logger=INFO,console
+hbase.log.dir=.
+hbase.log.file=hbase.log
+
+# Define the root logger to the system property "hbase.root.logger".
+log4j.rootLogger=${hbase.root.logger}
+
+# Logging Threshold
+log4j.threshold=ALL
+
+#
+# Daily Rolling File Appender
+#
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hbase.log.dir}/${hbase.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+# Debugging Pattern format
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L): %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L): %m%n
+
+# Custom Logging levels
+
+#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.zookeeper=ERROR
+log4j.logger.org.apache.hadoop.hbase=DEBUG
+
+#These settings are workarounds against spurious logs from the minicluster.
+#See HBASE-4709
+log4j.logger.org.apache.hadoop.metrics2.impl.MetricsConfig=WARN
+log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSinkAdapter=WARN
+log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=WARN
+log4j.logger.org.apache.hadoop.metrics2.util.MBeans=WARN
+# Enable this to get detailed connection error/retry logging.
+# log4j.logger.org.apache.hadoop.hbase.client.ConnectionImplementation=TRACE

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-it/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index 5d8946f..5709fac 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -232,6 +232,11 @@
       <artifactId>hbase-testing-util</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-backup</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hbase.thirdparty</groupId>
       <artifactId>hbase-shaded-miscellaneous</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 9dc6382..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.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 array of backup ids
-   * @return total number of deleted sessions
-   * @throws IOException exception
-   */
-  int deleteBackups(String[] backupIds) throws IOException;
-
-  /**
-   * Merge backup images command
-   * @param backupIds array of backup ids of images to be merged
-   *        The resulting backup image will have the same backup id as the most
-   *        recent image from a list of images to be merged
-   * @throws IOException exception
-   */
-  void mergeBackups(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 array 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 array 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/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
deleted file mode 100644
index 21d73cc..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClientFactory.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.backup.impl.FullTableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
-import org.apache.hadoop.hbase.client.Connection;
-
-public class BackupClientFactory {
-
-  public static TableBackupClient create (Connection conn, String backupId, BackupRequest request)
-    throws IOException
-  {
-    Configuration conf = conn.getConfiguration();
-    try {
-      String clsName = conf.get(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS);
-      if (clsName != null) {
-        Class<?> clientImpl = Class.forName(clsName);
-        TableBackupClient client = (TableBackupClient) clientImpl.newInstance();
-        client.init(conn, backupId, request);
-        return client;
-      }
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-
-    BackupType type = request.getBackupType();
-    if (type == BackupType.FULL) {
-      return new FullTableBackupClient(conn, backupId, request);
-    } else {
-      return new IncrementalTableBackupClient(conn, backupId, request);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 007e4c1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import 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/37c65946/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
deleted file mode 100644
index 9dd8531..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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 if (BackupCommand.REPAIR.name().equalsIgnoreCase(cmd)) {
-      type = BackupCommand.REPAIR;
-    } else if (BackupCommand.MERGE.name().equalsIgnoreCase(cmd)) {
-      type = BackupCommand.MERGE;
-    } 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;
-    } finally {
-      command.finish();
-    }
-    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/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
deleted file mode 100644
index ed554ad..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHFileCleaner.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-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.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.TableName;
-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.cleaner.BaseHFileCleanerDelegate;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
-
-/**
- * Implementation of a file cleaner that checks if an hfile is still referenced by backup before
- * deleting it from hfile archive directory.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class BackupHFileCleaner extends BaseHFileCleanerDelegate implements Abortable {
-  private static final Log LOG = LogFactory.getLog(BackupHFileCleaner.class);
-  private boolean stopped = false;
-  private boolean aborted;
-  private Configuration conf;
-  private Connection connection;
-  private long prevReadFromBackupTbl = 0, // timestamp of most recent read from backup:system table
-      secondPrevReadFromBackupTbl = 0; // timestamp of 2nd most recent read from backup:system table
-  //used by unit test to skip reading backup:system
-  private boolean checkForFullyBackedUpTables = true;
-  private List<TableName> fullyBackedUpTables = null;
-
-  private Set<String> getFilenameFromBulkLoad(Map<byte[], List<Path>>[] maps) {
-    Set<String> filenames = new HashSet<String>();
-    for (Map<byte[], List<Path>> map : maps) {
-      if (map == null) continue;
-      for (List<Path> paths : map.values()) {
-        for (Path p : paths) {
-          filenames.add(p.getName());
-        }
-      }
-    }
-    return filenames;
-  }
-
-  private Set<String> loadHFileRefs(List<TableName> tableList) throws IOException {
-    if (connection == null) {
-      connection = ConnectionFactory.createConnection(conf);
-    }
-    try (BackupSystemTable tbl = new BackupSystemTable(connection)) {
-      Map<byte[], List<Path>>[] res =
-          tbl.readBulkLoadedFiles(null, tableList);
-      secondPrevReadFromBackupTbl = prevReadFromBackupTbl;
-      prevReadFromBackupTbl = EnvironmentEdgeManager.currentTime();
-      return getFilenameFromBulkLoad(res);
-    }
-  }
-
-  @VisibleForTesting
-  void setCheckForFullyBackedUpTables(boolean b) {
-    checkForFullyBackedUpTables = b;
-  }
-  @Override
-  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
-    if (conf == null) {
-      return files;
-    }
-    // obtain the Set of TableName's which have been fully backed up
-    // so that we filter BulkLoad to be returned from server
-    if (checkForFullyBackedUpTables) {
-      if (connection == null) return files;
-      try (BackupSystemTable tbl = new BackupSystemTable(connection)) {
-        fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      } catch (IOException ioe) {
-        LOG.error("Failed to get tables which have been fully backed up, skipping checking", ioe);
-        return Collections.emptyList();
-      }
-      Collections.sort(fullyBackedUpTables);
-    }
-    final Set<String> hfileRefs;
-    try {
-      hfileRefs = loadHFileRefs(fullyBackedUpTables);
-    } catch (IOException ioe) {
-      LOG.error("Failed to read hfile references, skipping checking deletable files", ioe);
-      return Collections.emptyList();
-    }
-    Iterable<FileStatus> deletables = Iterables.filter(files, new Predicate<FileStatus>() {
-      @Override
-      public boolean apply(FileStatus file) {
-        // If the file is recent, be conservative and wait for one more scan of backup:system table
-        if (file.getModificationTime() > secondPrevReadFromBackupTbl) {
-          return false;
-        }
-        String hfile = file.getPath().getName();
-        boolean foundHFileRef = hfileRefs.contains(hfile);
-        return !foundHFileRef;
-      }
-    });
-    return deletables;
-  }
-
-  @Override
-  public boolean isFileDeletable(FileStatus fStat) {
-    // work is done in getDeletableFiles()
-    return true;
-  }
-
-  @Override
-  public void setConf(Configuration config) {
-    this.conf = config;
-    this.connection = null;
-    try {
-      this.connection = ConnectionFactory.createConnection(conf);
-    } catch (IOException ioe) {
-      LOG.error("Couldn't establish connection", ioe);
-    }
-  }
-
-  @Override
-  public void stop(String why) {
-    if (this.stopped) {
-      return;
-    }
-    if (this.connection != null) {
-      try {
-        this.connection.close();
-      } catch (IOException ioe) {
-        LOG.debug("Got " + ioe + " when closing connection");
-      }
-    }
-    this.stopped = true;
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped;
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    LOG.warn("Aborting ReplicationHFileCleaner because " + why, e);
-    this.aborted = true;
-    stop(why);
-  }
-
-  @Override
-  public boolean isAborted() {
-    return this.aborted;
-  }
-}


[03/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 0cfe099..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ /dev/null
@@ -1,516 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.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.StoreFileInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.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 static long TABLE_AVAILABILITY_WAIT_TIME = 180000;
-
-  private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR };
-  protected Configuration conf = null;
-  protected Path backupRootPath;
-  protected String backupId;
-  protected FileSystem fs;
-
-  // 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);
-  }
-
-  /**
-   * return value represent path for:
-   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn"
-   * @param tableName 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 = new HTableDescriptor(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 {
-    createAndRestoreTable(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;
-  }
-
-  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 createAndRestoreTable(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 = new HTableDescriptor(newTableName, tableDescriptor);
-        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 = new HTableDescriptor(newTableName, tableDescriptor);
-    }
-
-    // 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);
-      RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf);
-      Path[] paths = new Path[regionPathList.size()];
-      regionPathList.toArray(paths);
-      restoreService.run(paths, new TableName[]{tableName}, new TableName[] {newTableName}, true);
-
-    } catch (Exception e) {
-      LOG.error(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;
-  }
-
-  /**
-   * 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(TableName, String, Admin)}
-   * @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)) {
-        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/2dda3712/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
deleted file mode 100644
index 7fe9a61..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
+++ /dev/null
@@ -1,503 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-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.BackupPhase;
-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.backup.impl.FullTableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager;
-import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
-import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
-import org.apache.hadoop.hbase.backup.util.BackupUtils;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.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.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.mapreduce.HadoopSecurityEnabledUserProviderForTesting;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.security.access.SecureTestUtil;
-import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.junit.AfterClass;
-import org.junit.Before;
-
-/**
- * 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  protected static HBaseTestingUtility TEST_UTIL2;
-  protected static Configuration conf1 = TEST_UTIL.getConfiguration();
-  protected static Configuration conf2;
-
-  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";
-  protected static boolean secure = false;
-
-  protected static boolean autoRestoreOnFailure = true;
-  protected static boolean setupIsDone = false;
-  protected static boolean useSecondCluster = false;
-
-
-  static class IncrementalTableBackupClientForTest extends IncrementalTableBackupClient
-  {
-
-    public IncrementalTableBackupClientForTest() {
-    }
-
-    public IncrementalTableBackupClientForTest(Connection conn,
-        String backupId, BackupRequest request) throws IOException {
-      super(conn, backupId, request);
-    }
-
-    @Override
-    public void execute() throws IOException
-    {
-      // case INCREMENTAL_COPY:
-      try {
-        // case PREPARE_INCREMENTAL:
-        failStageIf(Stage.stage_0);
-        beginBackup(backupManager, backupInfo);
-
-        failStageIf(Stage.stage_1);
-        backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
-        LOG.debug("For incremental backup, current table set is "
-            + backupManager.getIncrementalBackupTableSet());
-        newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
-        // copy out the table and region info files for each table
-        BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
-        // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
-        convertWALsToHFiles(backupInfo);
-        incrementalCopyHFiles(backupInfo);
-        failStageIf(Stage.stage_2);
-        // Save list of WAL files copied
-        backupManager.recordWALFiles(backupInfo.getIncrBackupFileList());
-
-        // 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
-        // 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);
-        failStageIf(Stage.stage_3);
-
-        HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
-            backupManager.readLogTimestampMap();
-
-        Long newStartCode =
-            BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap));
-        backupManager.writeBackupStartCode(newStartCode);
-
-        handleBulkLoad(backupInfo.getTableNames());
-        failStageIf(Stage.stage_4);
-
-        // backup complete
-        completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
-
-      } catch (Exception e) {
-        failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
-          BackupType.INCREMENTAL, conf);
-        throw new IOException(e);
-      }
-
-    }
-  }
-
-  static class FullTableBackupClientForTest extends FullTableBackupClient
-  {
-
-
-    public FullTableBackupClientForTest() {
-    }
-
-    public FullTableBackupClientForTest(Connection conn, String backupId, BackupRequest request)
-        throws IOException {
-      super(conn, backupId, request);
-    }
-
-    @Override
-    public void execute() throws IOException
-    {
-      // Get the stage ID to fail on
-      try (Admin admin = conn.getAdmin();) {
-        // Begin BACKUP
-        beginBackup(backupManager, backupInfo);
-        failStageIf(Stage.stage_0);
-        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);
-        }
-        failStageIf(Stage.stage_1);
-        // 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);
-        failStageIf(Stage.stage_2);
-        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();
-
-          snapshotTable(admin, tableName, snapshotName);
-          backupInfo.setSnapshotName(tableName, snapshotName);
-        }
-        failStageIf(Stage.stage_3);
-        // 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);
-        failStageIf(Stage.stage_4);
-        // backup complete
-        completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf);
-
-      } catch (Exception e) {
-
-        if(autoRestoreOnFailure) {
-          failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ",
-            BackupType.FULL, conf);
-        }
-        throw new IOException(e);
-      }
-    }
-
-  }
-
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Before
-  public void setUp() throws Exception {
-    if (setupIsDone) {
-      return;
-    }
-    if (secure) {
-      // set the always on security provider
-      UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(),
-          HadoopSecurityEnabledUserProviderForTesting.class);
-      // setup configuration
-      SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
-    }
-    String coproc = conf1.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY);
-    conf1.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, (coproc == null ? "" : coproc + ",") +
-        BackupObserver.class.getName());
-    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.startMiniCluster();
-
-    if (useSecondCluster) {
-      conf2 = HBaseConfiguration.create(conf1);
-      conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
-      TEST_UTIL2 = new HBaseTestingUtility(conf2);
-      TEST_UTIL2.setZkCluster(TEST_UTIL.getZkCluster());
-      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);
-    if (useSecondCluster) {
-      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);
-    setupIsDone = true;
-  }
-
-  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 tearDown() throws Exception {
-    try{
-      SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
-    } catch (Exception e) {
-    }
-    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
-    if (useSecondCluster) {
-      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/2dda3712/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
deleted file mode 100644
index 40a01b9..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.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 org.apache.hadoop.hbase.shaded.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/2dda3712/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
deleted file mode 100644
index 08002fb..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java
+++ /dev/null
@@ -1,431 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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);
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index ebfc735..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/2dda3712/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
deleted file mode 100644
index 208e081..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
deleted file mode 100644
index 966f519..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertNull;
-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.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.BackupSystemTable;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-/**
- * 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
- */
-@Category(LargeTests.class)
-public class TestBackupDeleteWithFailures extends TestBackupBase{
-
-  private static final Log LOG = LogFactory.getLog(TestBackupDeleteWithFailures.class);
-
-
-
-  public static enum Failure {
-    NO_FAILURES,
-    PRE_SNAPSHOT_FAILURE,
-    PRE_DELETE_SNAPSHOT_FAILURE,
-    POST_DELETE_SNAPSHOT_FAILURE
-  }
-
-  public static class MasterSnapshotObserver implements MasterObserver {
-
-
-    List<Failure> failures = new ArrayList<Failure>();
-
-    public void setFailures(Failure ... f) {
-      failures.clear();
-      for (int i = 0; i < f.length; i++) {
-        failures.add(f[i]);
-      }
-    }
-
-    @Override
-    public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
-        throws IOException
-    {
-       if (failures.contains(Failure.PRE_SNAPSHOT_FAILURE)) {
-         throw new IOException ("preSnapshot");
-       }
-    }
-
-    @Override
-    public void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        SnapshotDescription snapshot) throws IOException {
-      if (failures.contains(Failure.PRE_DELETE_SNAPSHOT_FAILURE)) {
-        throw new IOException ("preDeleteSnapshot");
-      }
-    }
-
-    @Override
-    public void postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        SnapshotDescription snapshot) throws IOException {
-      if (failures.contains(Failure.POST_DELETE_SNAPSHOT_FAILURE)) {
-        throw new IOException ("postDeleteSnapshot");
-      }
-    }
-
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Override
-  @Before
-  public void setUp() throws Exception {
-    conf1.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
-      MasterSnapshotObserver.class.getName());
-    conf1.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
-    super.setUp();
-  }
-
-
-  private MasterSnapshotObserver getMasterSnapshotObserver() {
-    return (MasterSnapshotObserver)TEST_UTIL.getHBaseCluster().getMaster()
-      .getMasterCoprocessorHost().findCoprocessor(MasterSnapshotObserver.class.getName());
-  }
-
-  @Test
-  public void testBackupDeleteWithFailures() throws Exception
-  {
-     testBackupDeleteWithFailuresAfter(1, Failure.PRE_DELETE_SNAPSHOT_FAILURE);
-     testBackupDeleteWithFailuresAfter(0, Failure.POST_DELETE_SNAPSHOT_FAILURE);
-     testBackupDeleteWithFailuresAfter(1, Failure.PRE_SNAPSHOT_FAILURE);
-  }
-
-  private void testBackupDeleteWithFailuresAfter(int expected, Failure ...failures) throws Exception {
-    LOG.info("test repair backup delete on a single table with data and failures "+ failures[0]);
-    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));
-
-    Connection conn = TEST_UTIL.getConnection();
-    Admin admin = conn.getAdmin();
-    MasterSnapshotObserver observer = getMasterSnapshotObserver();
-
-    observer.setFailures(failures);
-    try {
-      getBackupAdmin().deleteBackups(backupIds);
-    } catch(IOException e) {
-      if(expected != 1) assertTrue(false);
-    }
-
-    // Verify that history length == expected after delete failure
-    assertTrue (table.getBackupHistory().size() == expected);
-
-    String[] ids = table.getListOfBackupIdsFromDeleteOperation();
-
-    // Verify that we still have delete record in backup system table
-    if(expected == 1) {
-      assertTrue(ids.length == 1);
-      assertTrue(ids[0].equals(backupId));
-    } else {
-      assertNull(ids);
-    }
-
-    // Now run repair command to repair "failed" delete operation
-    String[] args = new String[] {"repair"};
-
-    observer.setFailures(Failure.NO_FAILURES);
-
-    // Run repair
-    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
-    assertTrue(ret == 0);
-    // Verify that history length == 0
-    assertTrue (table.getBackupHistory().size() == 0);
-    ids = table.getListOfBackupIdsFromDeleteOperation();
-
-    // Verify that we do not have delete record in backup system table
-    assertNull(ids);
-
-    table.close();
-    admin.close();
-  }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 0672325..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
deleted file mode 100644
index dfbe106..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-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.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.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-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.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-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;
-
-@Category({ MasterTests.class, SmallTests.class })
-public class TestBackupHFileCleaner {
-  private static final Log LOG = LogFactory.getLog(TestBackupHFileCleaner.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static Configuration conf = TEST_UTIL.getConfiguration();
-  private static TableName tableName = TableName.valueOf("backup.hfile.cleaner");
-  private static String famName = "fam";
-  static FileSystem fs = null;
-  Path root;
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true);
-    TEST_UTIL.startMiniZKCluster();
-    TEST_UTIL.startMiniCluster(1);
-    fs = FileSystem.get(conf);
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    if (fs != null) {
-      fs.close();
-    }
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Before
-  public void setup() throws IOException {
-    root = TEST_UTIL.getDataTestDirOnTestFS();
-  }
-
-  @After
-  public void cleanup() {
-    try {
-      fs.delete(root, true);
-    } catch (IOException e) {
-      LOG.warn("Failed to delete files recursively from path " + root);
-    }
-  }
-
-  @Test
-  public void testGetDeletableFiles() throws IOException {
-    // 1. Create a file
-    Path file = new Path(root, "testIsFileDeletableWithNoHFileRefs");
-    fs.createNewFile(file);
-    // 2. Assert file is successfully created
-    assertTrue("Test file not created!", fs.exists(file));
-    BackupHFileCleaner cleaner = new BackupHFileCleaner();
-    cleaner.setConf(conf);
-    cleaner.setCheckForFullyBackedUpTables(false);
-    // 3. Assert that file as is should be deletable
-    List<FileStatus> stats = new ArrayList<>();
-    FileStatus stat = fs.getFileStatus(file);
-    stats.add(stat);
-    Iterable<FileStatus> deletable = cleaner.getDeletableFiles(stats);
-    deletable = cleaner.getDeletableFiles(stats);
-    boolean found = false;
-    for (FileStatus stat1 : deletable) {
-      if (stat.equals(stat1)) found = true;
-    }
-    assertTrue("Cleaner should allow to delete this file as there is no hfile reference "
-        + "for it.", found);
-
-    // 4. Add the file as bulk load
-    List<Path> list = new ArrayList<>(1);
-    list.add(file);
-    try (Connection conn = ConnectionFactory.createConnection(conf);
-        BackupSystemTable sysTbl = new BackupSystemTable(conn)) {
-      List<TableName> sTableList = new ArrayList<>();
-      sTableList.add(tableName);
-      Map<byte[], List<Path>>[] maps = new Map[1];
-      maps[0] = new HashMap<>();
-      maps[0].put(famName.getBytes(), list);
-      sysTbl.writeBulkLoadedFiles(sTableList, maps, "1");
-    }
-
-    // 5. Assert file should not be deletable
-    deletable = cleaner.getDeletableFiles(stats);
-    deletable = cleaner.getDeletableFiles(stats);
-    found = false;
-    for (FileStatus stat1 : deletable) {
-      if (stat.equals(stat1)) found = true;
-    }
-    assertFalse("Cleaner should not allow to delete this file as there is a hfile reference "
-        + "for it.", found);
-  }
-}
\ No newline at end of file


[02/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 0944ea2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
deleted file mode 100644
index 686d34b..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-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.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
-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 TestBackupRepair extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestBackupRepair.class);
-
-
-  @Test
-  public void testFullBackupWithFailuresAndRestore() throws Exception {
-
-    autoRestoreOnFailure = false;
-
-    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
-      FullTableBackupClientForTest.class.getName());
-    int maxStage = Stage.values().length -1;
-    // Fail stage in loop between 0 and 4 inclusive
-    for (int stage = 0; stage < maxStage; stage++) {
-      LOG.info("Running stage " + stage);
-      runBackupAndFailAtStageWithRestore(stage);
-    }
-  }
-
-  public void runBackupAndFailAtStageWithRestore(int stage) throws Exception {
-
-    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
-    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);
-      assertFalse(ret == 0);
-
-      // Now run restore
-      args = new String[] {"repair"};
-
-      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();
-        assertFalse(checkSucceeded(backupId));
-      }
-      Set<TableName> tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR);
-      assertTrue(tables.size() == 0);
-    }
-  }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 4e922a2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/2dda3712/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
deleted file mode 100644
index 73d8d9f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/2dda3712/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
deleted file mode 100644
index 5814d87..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
+++ /dev/null
@@ -1,511 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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/2dda3712/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
deleted file mode 100644
index 124d19f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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/2dda3712/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
deleted file mode 100644
index 4dc894b..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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/2dda3712/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
deleted file mode 100644
index 6b007f9..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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/2dda3712/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
deleted file mode 100644
index d18de88..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-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.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
-import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
-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 TestFullBackupWithFailures extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestFullBackupWithFailures.class);
-
-  @Test
-  public void testFullBackupWithFailures() throws Exception {
-    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
-      FullTableBackupClientForTest.class.getName());
-    int maxStage = Stage.values().length -1;
-    // Fail stages between 0 and 4 inclusive
-    for (int stage = 0; stage <= maxStage; stage++) {
-      LOG.info("Running stage " + stage);
-      runBackupAndFailAtStage(stage);
-    }
-  }
-
-  public void runBackupAndFailAtStage(int stage) throws Exception {
-
-    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
-    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);
-      assertFalse(ret == 0);
-      List<BackupInfo> backups = table.getBackupHistory();
-      int after = table.getBackupHistory().size();
-
-      assertTrue(after ==  before +1);
-      for (BackupInfo data : backups) {
-        String backupId = data.getBackupId();
-        assertFalse(checkSucceeded(backupId));
-      }
-      Set<TableName> tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR);
-      assertTrue(tables.size() == 0);
-    }
-  }
-
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 48a553f..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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/2dda3712/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
deleted file mode 100644
index 73598f3..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-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 org.apache.hadoop.hbase.shaded.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();
-    try {
-      admin.splitRegion(name);
-    } catch (IOException e) {
-      //although split fail, this may not affect following check
-      //In old split without AM2, if region's best split key is not found,
-      //there are not exception thrown. But in current API, exception
-      //will be thrown.
-      LOG.debug("region is not splittable, because " + e);
-    }
-
-    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.getDescriptor());
-    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/2dda3712/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
deleted file mode 100644
index 747c1dd..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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 org.apache.hadoop.hbase.shaded.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();
-  }
-
-}


[33/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
new file mode 100644
index 0000000..4dab046
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -0,0 +1,2051 @@
+/**
+ * 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.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+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.CellComparator;
+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.NamespaceDescriptor;
+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.BackupType;
+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.SnapshotDescription;
+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;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * 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 {
+  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
+
+  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 TableName tableName;
+  /**
+   * Stores backup sessions (contexts)
+   */
+  final static byte[] SESSIONS_FAMILY = "session".getBytes();
+  /**
+   * Stores other meta
+   */
+  final static byte[] META_FAMILY = "meta".getBytes();
+  final static byte[] BULK_LOAD_FAMILY = "bulk".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 byte[] ACTIVE_SESSION_ROW = "activesession:".getBytes();
+  private final static byte[] ACTIVE_SESSION_COL = "c".getBytes();
+
+  private final static byte[] ACTIVE_SESSION_YES = "yes".getBytes();
+  private final static byte[] ACTIVE_SESSION_NO = "no".getBytes();
+
+  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 BULK_LOAD_PREFIX = "bulk:";
+  private final static byte[] BULK_LOAD_PREFIX_BYTES = BULK_LOAD_PREFIX.getBytes();
+  private final static byte[] DELETE_OP_ROW = "delete_op_row".getBytes();
+  private final static byte[] MERGE_OP_ROW = "merge_op_row".getBytes();
+
+  final static byte[] TBL_COL = Bytes.toBytes("tbl");
+  final static byte[] FAM_COL = Bytes.toBytes("fam");
+  final static byte[] PATH_COL = Bytes.toBytes("path");
+  final static byte[] STATE_COL = Bytes.toBytes("state");
+  // the two states a bulk loaded file can be
+  final static byte[] BL_PREPARE = Bytes.toBytes("R");
+  final static byte[] BL_COMMIT = Bytes.toBytes("D");
+
+  private final static String WALS_PREFIX = "wals:";
+  private final static String SET_KEY_PREFIX = "backupset:";
+
+  // separator between BULK_LOAD_PREFIX and ordinals
+  protected final static String BLK_LD_DELIM = ":";
+  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();) {
+
+      verifyNamespaceExists(admin);
+
+      if (!admin.tableExists(tableName)) {
+        HTableDescriptor backupHTD =
+            BackupSystemTable.getSystemTableDescriptor(connection.getConfiguration());
+        admin.createTable(backupHTD);
+      }
+      waitForSystemTable(admin);
+    }
+  }
+
+  private void verifyNamespaceExists(Admin admin) throws IOException {
+    String namespaceName = tableName.getNamespaceAsString();
+    NamespaceDescriptor ns = NamespaceDescriptor.create(namespaceName).build();
+    NamespaceDescriptor[] list = admin.listNamespaceDescriptors();
+    boolean exists = false;
+    for (NamespaceDescriptor nsd : list) {
+      if (nsd.getName().equals(ns.getName())) {
+        exists = true;
+        break;
+      }
+    }
+    if (!exists) {
+      admin.createNamespace(ns);
+    }
+  }
+
+  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);
+    }
+  }
+
+  /*
+   * @param backupId the backup Id
+   * @return Map of rows to path of bulk loaded hfile
+   */
+  Map<byte[], String> readBulkLoadedFiles(String backupId) throws IOException {
+    Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId);
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      Map<byte[], String> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        byte[] row = CellUtil.cloneRow(res.listCells().get(0));
+        for (Cell cell : res.listCells()) {
+          if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
+            BackupSystemTable.PATH_COL.length) == 0) {
+            map.put(row, Bytes.toString(CellUtil.cloneValue(cell)));
+          }
+        }
+      }
+      return map;
+    }
+  }
+
+  /*
+   * Used during restore
+   * @param backupId the backup Id
+   * @param sTableList List of tables
+   * @return array of Map of family to List of Paths
+   */
+  public Map<byte[], List<Path>>[] readBulkLoadedFiles(String backupId, List<TableName> sTableList)
+      throws IOException {
+    Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId);
+    Map<byte[], List<Path>>[] mapForSrc = new Map[sTableList == null ? 1 : sTableList.size()];
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        TableName tbl = null;
+        byte[] fam = null;
+        String path = null;
+        for (Cell cell : res.listCells()) {
+          if (CellComparator.compareQualifiers(cell, BackupSystemTable.TBL_COL, 0,
+            BackupSystemTable.TBL_COL.length) == 0) {
+            tbl = TableName.valueOf(CellUtil.cloneValue(cell));
+          } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
+            BackupSystemTable.FAM_COL.length) == 0) {
+            fam = CellUtil.cloneValue(cell);
+          } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
+            BackupSystemTable.PATH_COL.length) == 0) {
+            path = Bytes.toString(CellUtil.cloneValue(cell));
+          }
+        }
+        int srcIdx = IncrementalTableBackupClient.getIndex(tbl, sTableList);
+        if (srcIdx == -1) {
+          // the table is not among the query
+          continue;
+        }
+        if (mapForSrc[srcIdx] == null) {
+          mapForSrc[srcIdx] = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+        }
+        List<Path> files;
+        if (!mapForSrc[srcIdx].containsKey(fam)) {
+          files = new ArrayList<Path>();
+          mapForSrc[srcIdx].put(fam, files);
+        } else {
+          files = mapForSrc[srcIdx].get(fam);
+        }
+        files.add(new Path(path));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("found bulk loaded file : " + tbl + " " + Bytes.toString(fam) + " " + path);
+        }
+      }
+      ;
+      return mapForSrc;
+    }
+  }
+
+  /*
+   * @param map Map of row keys to path of bulk loaded hfile
+   */
+  void deleteBulkLoadedFiles(Map<byte[], String> map) throws IOException {
+    try (Table table = connection.getTable(tableName)) {
+      List<Delete> dels = new ArrayList<>();
+      for (byte[] row : map.keySet()) {
+        dels.add(new Delete(row).addFamily(BackupSystemTable.META_FAMILY));
+      }
+      table.delete(dels);
+    }
+  }
+
+  /**
+   * 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);
+    }
+  }
+
+  /*
+   * For postBulkLoadHFile() hook.
+   * @param tabName table name
+   * @param region the region receiving hfile
+   * @param finalPaths family and associated hfiles
+   */
+  public void writePathsPostBulkLoad(TableName tabName, byte[] region,
+      Map<byte[], List<Path>> finalPaths) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + finalPaths.size()
+          + " entries");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      List<Put> puts = BackupSystemTable.createPutForCommittedBulkload(tabName, region, finalPaths);
+      table.put(puts);
+      LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
+    }
+  }
+
+  /*
+   * For preCommitStoreFile() hook
+   * @param tabName table name
+   * @param region the region receiving hfile
+   * @param family column family
+   * @param pairs list of paths for hfiles
+   */
+  public void writeFilesForBulkLoadPreCommit(TableName tabName, byte[] region, final byte[] family,
+      final List<Pair<Path, Path>> pairs) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + pairs.size()
+          + " entries");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      List<Put> puts =
+          BackupSystemTable.createPutForPreparedBulkload(tabName, region, family, pairs);
+      table.put(puts);
+      LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
+    }
+  }
+
+  /*
+   * Removes rows recording bulk loaded hfiles from backup table
+   * @param lst list of table names
+   * @param rows the rows to be deleted
+   */
+  public void removeBulkLoadedRows(List<TableName> lst, List<byte[]> rows) throws IOException {
+    try (Table table = connection.getTable(tableName)) {
+      List<Delete> lstDels = new ArrayList<>();
+      for (byte[] row : rows) {
+        Delete del = new Delete(row);
+        lstDels.add(del);
+        LOG.debug("orig deleting the row: " + Bytes.toString(row));
+      }
+      table.delete(lstDels);
+      LOG.debug("deleted " + rows.size() + " original bulkload rows for " + lst.size() + " tables");
+    }
+  }
+
+  /*
+   * Reads the rows from backup table recording bulk loaded hfiles
+   * @param tableList list of table names
+   * @return The keys of the Map are table, region and column family. Value of the map reflects
+   * whether the hfile was recorded by preCommitStoreFile hook (true)
+   */
+  public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
+      readBulkloadRows(List<TableName> tableList) throws IOException {
+    Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> map = new HashMap<>();
+    List<byte[]> rows = new ArrayList<>();
+    for (TableName tTable : tableList) {
+      Scan scan = BackupSystemTable.createScanForOrigBulkLoadedFiles(tTable);
+      Map<String, Map<String, List<Pair<String, Boolean>>>> tblMap = map.get(tTable);
+      try (Table table = connection.getTable(tableName);
+          ResultScanner scanner = table.getScanner(scan)) {
+        Result res = null;
+        while ((res = scanner.next()) != null) {
+          res.advance();
+          String fam = null;
+          String path = null;
+          boolean raw = false;
+          byte[] row = null;
+          String region = null;
+          for (Cell cell : res.listCells()) {
+            row = CellUtil.cloneRow(cell);
+            rows.add(row);
+            String rowStr = Bytes.toString(row);
+            region = BackupSystemTable.getRegionNameFromOrigBulkLoadRow(rowStr);
+            if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
+              BackupSystemTable.FAM_COL.length) == 0) {
+              fam = Bytes.toString(CellUtil.cloneValue(cell));
+            } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
+              BackupSystemTable.PATH_COL.length) == 0) {
+              path = Bytes.toString(CellUtil.cloneValue(cell));
+            } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.STATE_COL, 0,
+              BackupSystemTable.STATE_COL.length) == 0) {
+              byte[] state = CellUtil.cloneValue(cell);
+              if (Bytes.equals(BackupSystemTable.BL_PREPARE, state)) {
+                raw = true;
+              } else raw = false;
+            }
+          }
+          if (map.get(tTable) == null) {
+            map.put(tTable, new HashMap<String, Map<String, List<Pair<String, Boolean>>>>());
+            tblMap = map.get(tTable);
+          }
+          if (tblMap.get(region) == null) {
+            tblMap.put(region, new HashMap<String, List<Pair<String, Boolean>>>());
+          }
+          Map<String, List<Pair<String, Boolean>>> famMap = tblMap.get(region);
+          if (famMap.get(fam) == null) {
+            famMap.put(fam, new ArrayList<Pair<String, Boolean>>());
+          }
+          famMap.get(fam).add(new Pair<>(path, raw));
+          LOG.debug("found orig " + path + " for " + fam + " of table " + region);
+        }
+      }
+    }
+    return new Pair<>(map, rows);
+  }
+
+  /*
+   * @param sTableList List of tables
+   * @param maps array of Map of family to List of Paths
+   * @param backupId the backup Id
+   */
+  public void writeBulkLoadedFiles(List<TableName> sTableList, Map<byte[], List<Path>>[] maps,
+      String backupId) throws IOException {
+    try (Table table = connection.getTable(tableName)) {
+      long ts = EnvironmentEdgeManager.currentTime();
+      int cnt = 0;
+      List<Put> puts = new ArrayList<>();
+      for (int idx = 0; idx < maps.length; idx++) {
+        Map<byte[], List<Path>> map = maps[idx];
+        TableName tn = sTableList.get(idx);
+        if (map == null) continue;
+        for (Map.Entry<byte[], List<Path>> entry : map.entrySet()) {
+          byte[] fam = entry.getKey();
+          List<Path> paths = entry.getValue();
+          for (Path p : paths) {
+            Put put =
+                BackupSystemTable.createPutForBulkLoadedFile(tn, fam, p.toString(), backupId, ts,
+                  cnt++);
+            puts.add(put);
+          }
+        }
+      }
+      if (!puts.isEmpty()) {
+        table.put(puts);
+      }
+    }
+  }
+
+  /**
+   * 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);
+    }
+  }
+
+  /**
+   * Exclusive operations are:
+   * create, delete, merge
+   * @throws IOException
+   */
+  public void startBackupExclusiveOperation() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Start new backup exclusive operation");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = createPutForStartBackupSession();
+      // First try to put if row does not exist
+      if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL, null, put)) {
+        // Row exists, try to put if value == ACTIVE_SESSION_NO
+        if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
+          ACTIVE_SESSION_NO, put)) {
+          throw new IOException("There is an active backup exclusive operation");
+        }
+      }
+    }
+  }
+
+  private Put createPutForStartBackupSession() {
+    Put put = new Put(ACTIVE_SESSION_ROW);
+    put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_YES);
+    return put;
+  }
+
+  public void finishBackupExclusiveOperation() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Finish backup exclusive operation");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = createPutForStopBackupSession();
+      if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
+        ACTIVE_SESSION_YES, put)) {
+        throw new IOException("There is no active backup exclusive operation");
+      }
+    }
+  }
+
+  private Put createPutForStopBackupSession() {
+    Put put = new Put(ACTIVE_SESSION_ROW);
+    put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_NO);
+    return 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, if n== -1 - max number
+   *        is ignored
+   * @return list of records
+   * @throws IOException
+   */
+  public List<BackupInfo> getHistory(int n) throws IOException {
+
+    List<BackupInfo> history = getBackupHistory();
+    if (n == -1 || 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, if n == -1 , then max number
+   *        is ignored
+   * @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 (n >= 0 && 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;
+
+  }
+
+  /*
+   * Retrieve TableName's for completed backup of given type
+   * @param type backup type
+   * @return List of table names
+   */
+  public List<TableName> getTablesForBackupType(BackupType type) throws IOException {
+    Set<TableName> names = new HashSet<>();
+    List<BackupInfo> infos = getBackupHistory(true);
+    for (BackupInfo info : infos) {
+      if (info.getType() != type) continue;
+      names.addAll(info.getTableNames());
+    }
+    return new ArrayList(names);
+  }
+
+  /**
+   * 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
+   * TODO: multiple backup destination support
+   */
+  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();
+  }
+
+  public static String getSnapshotName(Configuration conf) {
+    return "snapshot_" + getTableNameAsString(conf).replace(":", "_");
+  }
+
+  /**
+   * 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's for bulk load resulting from running LoadIncrementalHFiles
+   */
+  static List<Put> createPutForCommittedBulkload(TableName table, byte[] region,
+      Map<byte[], List<Path>> finalPaths) {
+    List<Put> puts = new ArrayList<>();
+    for (Map.Entry<byte[], List<Path>> entry : finalPaths.entrySet()) {
+      for (Path path : entry.getValue()) {
+        String file = path.toString();
+        int lastSlash = file.lastIndexOf("/");
+        String filename = file.substring(lastSlash + 1);
+        Put put =
+            new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM,
+              Bytes.toString(region), BLK_LD_DELIM, filename));
+        put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
+        put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, entry.getKey());
+        put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
+        put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_COMMIT);
+        puts.add(put);
+        LOG.debug("writing done bulk path " + file + " for " + table + " " + Bytes.toString(region));
+      }
+    }
+    return puts;
+  }
+
+  public static void snapshot(Connection conn) throws IOException {
+
+    try (Admin admin = conn.getAdmin();) {
+      Configuration conf = conn.getConfiguration();
+      admin.snapshot(BackupSystemTable.getSnapshotName(conf), BackupSystemTable.getTableName(conf));
+    }
+  }
+
+  public static void restoreFromSnapshot(Connection conn) throws IOException {
+
+    Configuration conf = conn.getConfiguration();
+    LOG.debug("Restoring " + BackupSystemTable.getTableNameAsString(conf) + " from snapshot");
+    try (Admin admin = conn.getAdmin();) {
+      String snapshotName = BackupSystemTable.getSnapshotName(conf);
+      if (snapshotExists(admin, snapshotName)) {
+        admin.disableTable(BackupSystemTable.getTableName(conf));
+        admin.restoreSnapshot(snapshotName);
+        admin.enableTable(BackupSystemTable.getTableName(conf));
+        LOG.debug("Done restoring backup system table");
+      } else {
+        // Snapshot does not exists, i.e completeBackup failed after
+        // deleting backup system table snapshot
+        // In this case we log WARN and proceed
+        LOG.warn("Could not restore backup system table. Snapshot " + snapshotName
+            + " does not exists.");
+      }
+    }
+  }
+
+  protected static boolean snapshotExists(Admin admin, String snapshotName) throws IOException {
+
+    List<SnapshotDescription> list = admin.listSnapshots();
+    for (SnapshotDescription desc : list) {
+      if (desc.getName().equals(snapshotName)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public static boolean snapshotExists(Connection conn) throws IOException {
+    return snapshotExists(conn.getAdmin(), getSnapshotName(conn.getConfiguration()));
+  }
+
+  public static void deleteSnapshot(Connection conn) throws IOException {
+
+    Configuration conf = conn.getConfiguration();
+    LOG.debug("Deleting " + BackupSystemTable.getSnapshotName(conf) + " from the system");
+    try (Admin admin = conn.getAdmin();) {
+      String snapshotName = BackupSystemTable.getSnapshotName(conf);
+      if (snapshotExists(admin, snapshotName)) {
+        admin.deleteSnapshot(snapshotName);
+        LOG.debug("Done deleting backup system table snapshot");
+      } else {
+        LOG.error("Snapshot " + snapshotName + " does not exists");
+      }
+    }
+  }
+
+  /*
+   * Creates Put's for bulk load resulting from running LoadIncrementalHFiles
+   */
+  static List<Put> createPutForPreparedBulkload(TableName table, byte[] region,
+      final byte[] family, final List<Pair<Path, Path>> pairs) {
+    List<Put> puts = new ArrayList<>();
+    for (Pair<Path, Path> pair : pairs) {
+      Path path = pair.getSecond();
+      String file = path.toString();
+      int lastSlash = file.lastIndexOf("/");
+      String filename = file.substring(lastSlash + 1);
+      Put put =
+          new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM, Bytes.toString(region),
+            BLK_LD_DELIM, filename));
+      put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
+      put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, family);
+      put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
+      put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_PREPARE);
+      puts.add(put);
+      LOG.debug("writing raw bulk path " + file + " for " + table + " " + Bytes.toString(region));
+    }
+    return puts;
+  }
+
+  public static List<Delete> createDeleteForOrigBulkLoad(List<TableName> lst) {
+    List<Delete> lstDels = new ArrayList<>();
+    for (TableName table : lst) {
+      Delete del = new Delete(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM));
+      del.addFamily(BackupSystemTable.META_FAMILY);
+      lstDels.add(del);
+    }
+    return lstDels;
+  }
+
+  private Put createPutForDeleteOperation(String[] backupIdList) {
+
+    byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ","));
+    Put put = new Put(DELETE_OP_ROW);
+    put.addColumn(META_FAMILY, FAM_COL, value);
+    return put;
+  }
+
+  private Delete createDeleteForBackupDeleteOperation() {
+
+    Delete delete = new Delete(DELETE_OP_ROW);
+    delete.addFamily(META_FAMILY);
+    return delete;
+  }
+
+  private Get createGetForDeleteOperation() {
+
+    Get get = new Get(DELETE_OP_ROW);
+    get.addFamily(META_FAMILY);
+    return get;
+  }
+
+  public void startDeleteOperation(String[] backupIdList) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Start delete operation for backups: " + StringUtils.join(backupIdList));
+    }
+    Put put = createPutForDeleteOperation(backupIdList);
+    try (Table table = connection.getTable(tableName)) {
+      table.put(put);
+    }
+  }
+
+  public void finishDeleteOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Finsih delete operation for backup ids ");
+    }
+    Delete delete = createDeleteForBackupDeleteOperation();
+    try (Table table = connection.getTable(tableName)) {
+      table.delete(delete);
+    }
+  }
+
+  public String[] getListOfBackupIdsFromDeleteOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Get delete operation for backup ids ");
+    }
+    Get get = createGetForDeleteOperation();
+    try (Table table = connection.getTable(tableName)) {
+      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).split(",");
+    }
+  }
+
+  private Put createPutForMergeOperation(String[] backupIdList) {
+
+    byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ","));
+    Put put = new Put(MERGE_OP_ROW);
+    put.addColumn(META_FAMILY, FAM_COL, value);
+    return put;
+  }
+
+  public boolean isMergeInProgress() throws IOException {
+    Get get = new Get(MERGE_OP_ROW);
+    try (Table table = connection.getTable(tableName)) {
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  private Put createPutForUpdateTablesForMerge(List<TableName> tables) {
+
+    byte[] value = Bytes.toBytes(StringUtils.join(tables, ","));
+    Put put = new Put(MERGE_OP_ROW);
+    put.addColumn(META_FAMILY, PATH_COL, value);
+    return put;
+  }
+
+  private Delete createDeleteForBackupMergeOperation() {
+
+    Delete delete = new Delete(MERGE_OP_ROW);
+    delete.addFamily(META_FAMILY);
+    return delete;
+  }
+
+  private Get createGetForMergeOperation() {
+
+    Get get = new Get(MERGE_OP_ROW);
+    get.addFamily(META_FAMILY);
+    return get;
+  }
+
+  public void startMergeOperation(String[] backupIdList) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Start merge operation for backups: " + StringUtils.join(backupIdList));
+    }
+    Put put = createPutForMergeOperation(backupIdList);
+    try (Table table = connection.getTable(tableName)) {
+      table.put(put);
+    }
+  }
+
+  public void updateProcessedTablesForMerge(List<TableName> tables) throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Update tables for merge : " + StringUtils.join(tables, ","));
+    }
+    Put put = createPutForUpdateTablesForMerge(tables);
+    try (Table table = connection.getTable(tableName)) {
+      table.put(put);
+    }
+  }
+
+  public void finishMergeOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Finsih merge operation for backup ids ");
+    }
+    Delete delete = createDeleteForBackupMergeOperation();
+    try (Table table = connection.getTable(tableName)) {
+      table.delete(delete);
+    }
+  }
+
+  public String[] getListOfBackupIdsFromMergeOperation() throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Get backup ids for merge operation");
+    }
+    Get get = createGetForMergeOperation();
+    try (Table table = connection.getTable(tableName)) {
+      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).split(",");
+    }
+  }
+
+  static Scan createScanForOrigBulkLoadedFiles(TableName table) throws IOException {
+    Scan scan = new Scan();
+    byte[] startRow = rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM);
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.withStartRow(startRow);
+    scan.withStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.META_FAMILY);
+    scan.setMaxVersions(1);
+    return scan;
+  }
+
+  static String getTableNameFromOrigBulkLoadRow(String rowStr) {
+    String[] parts = rowStr.split(BLK_LD_DELIM);
+    return parts[1];
+  }
+
+  static String getRegionNameFromOrigBulkLoadRow(String rowStr) {
+    // format is bulk : namespace : table : region : file
+    String[] parts = rowStr.split(BLK_LD_DELIM);
+    int idx = 3;
+    if (parts.length == 4) {
+      // the table is in default namespace
+      idx = 2;
+    }
+    LOG.debug("bulk row string " + rowStr + " region " + parts[idx]);
+    return parts[idx];
+  }
+
+  /*
+   * Used to query bulk loaded hfiles which have been copied by incremental backup
+   * @param backupId the backup Id. It can be null when querying for all tables
+   * @return the Scan object
+   */
+  static Scan createScanForBulkLoadedFiles(String backupId) throws IOException {
+    Scan scan = new Scan();
+    byte[] startRow =
+        backupId == null ? BULK_LOAD_PREFIX_BYTES : rowkey(BULK_LOAD_PREFIX, backupId
+            + BLK_LD_DELIM);
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    // scan.setTimeRange(lower, Long.MAX_VALUE);
+    scan.addFamily(BackupSystemTable.META_FAMILY);
+    scan.setMaxVersions(1);
+    return scan;
+  }
+
+  static Put createPutForBulkLoadedFile(TableName tn, byte[] fam, String p, String backupId,
+      long ts, int idx) {
+    Put put = new Put(rowkey(BULK_LOAD_PREFIX, backupId + BLK_LD_DELIM + ts + BLK_LD_DELIM + idx));
+    put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, tn.getName());
+    put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, fam);
+    put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, p.getBytes());
+    return put;
+  }
+
+  /**
+   * 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 static 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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
new file mode 100644
index 0000000..e323e96
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
@@ -0,0 +1,224 @@
+/**
+ * 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.BACKUP_ATTEMPTS_PAUSE_MS_KEY;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_MAX_ATTEMPTS_KEY;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_MAX_ATTEMPTS;
+
+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() {
+  }
+
+  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
+   */
+  protected 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();
+
+        snapshotTable(admin, tableName, snapshotName);
+        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);
+    }
+
+  }
+
+
+  protected void snapshotTable(Admin admin, TableName tableName, String snapshotName)
+      throws IOException {
+
+    int maxAttempts =
+        conf.getInt(BACKUP_MAX_ATTEMPTS_KEY, DEFAULT_BACKUP_MAX_ATTEMPTS);
+    int pause =
+        conf.getInt(BACKUP_ATTEMPTS_PAUSE_MS_KEY, DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS);
+    int attempts = 0;
+
+    while (attempts++ < maxAttempts) {
+      try {
+        admin.snapshot(snapshotName, tableName);
+        return;
+      } catch (IOException ee) {
+        LOG.warn("Snapshot attempt " + attempts + " failed for table " + tableName
+            + ", sleeping for " + pause + "ms", ee);
+        if (attempts < maxAttempts) {
+          try {
+            Thread.sleep(pause);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            break;
+          }
+        }
+      }
+    }
+    throw new IOException("Failed to snapshot table "+ tableName);
+  }
+}


[30/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
new file mode 100644
index 0000000..0cfe099
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -0,0 +1,516 @@
+/**
+ * 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.StoreFileInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.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 static long TABLE_AVAILABILITY_WAIT_TIME = 180000;
+
+  private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR };
+  protected Configuration conf = null;
+  protected Path backupRootPath;
+  protected String backupId;
+  protected FileSystem fs;
+
+  // 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);
+  }
+
+  /**
+   * return value represent path for:
+   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn"
+   * @param tableName 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 = new HTableDescriptor(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 {
+    createAndRestoreTable(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;
+  }
+
+  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 createAndRestoreTable(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 = new HTableDescriptor(newTableName, tableDescriptor);
+        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 = new HTableDescriptor(newTableName, tableDescriptor);
+    }
+
+    // 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);
+      RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf);
+      Path[] paths = new Path[regionPathList.size()];
+      regionPathList.toArray(paths);
+      restoreService.run(paths, new TableName[]{tableName}, new TableName[] {newTableName}, true);
+
+    } catch (Exception e) {
+      LOG.error(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;
+  }
+
+  /**
+   * 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(TableName, String, Admin)}
+   * @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)) {
+        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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
new file mode 100644
index 0000000..7fe9a61
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
@@ -0,0 +1,503 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+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.BackupPhase;
+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.backup.impl.FullTableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager;
+import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.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.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.mapreduce.HadoopSecurityEnabledUserProviderForTesting;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.access.SecureTestUtil;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.AfterClass;
+import org.junit.Before;
+
+/**
+ * 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static HBaseTestingUtility TEST_UTIL2;
+  protected static Configuration conf1 = TEST_UTIL.getConfiguration();
+  protected static Configuration conf2;
+
+  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";
+  protected static boolean secure = false;
+
+  protected static boolean autoRestoreOnFailure = true;
+  protected static boolean setupIsDone = false;
+  protected static boolean useSecondCluster = false;
+
+
+  static class IncrementalTableBackupClientForTest extends IncrementalTableBackupClient
+  {
+
+    public IncrementalTableBackupClientForTest() {
+    }
+
+    public IncrementalTableBackupClientForTest(Connection conn,
+        String backupId, BackupRequest request) throws IOException {
+      super(conn, backupId, request);
+    }
+
+    @Override
+    public void execute() throws IOException
+    {
+      // case INCREMENTAL_COPY:
+      try {
+        // case PREPARE_INCREMENTAL:
+        failStageIf(Stage.stage_0);
+        beginBackup(backupManager, backupInfo);
+
+        failStageIf(Stage.stage_1);
+        backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
+        LOG.debug("For incremental backup, current table set is "
+            + backupManager.getIncrementalBackupTableSet());
+        newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
+        // copy out the table and region info files for each table
+        BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
+        // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
+        convertWALsToHFiles(backupInfo);
+        incrementalCopyHFiles(backupInfo);
+        failStageIf(Stage.stage_2);
+        // Save list of WAL files copied
+        backupManager.recordWALFiles(backupInfo.getIncrBackupFileList());
+
+        // 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
+        // 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);
+        failStageIf(Stage.stage_3);
+
+        HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
+            backupManager.readLogTimestampMap();
+
+        Long newStartCode =
+            BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap));
+        backupManager.writeBackupStartCode(newStartCode);
+
+        handleBulkLoad(backupInfo.getTableNames());
+        failStageIf(Stage.stage_4);
+
+        // backup complete
+        completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
+
+      } catch (Exception e) {
+        failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
+          BackupType.INCREMENTAL, conf);
+        throw new IOException(e);
+      }
+
+    }
+  }
+
+  static class FullTableBackupClientForTest extends FullTableBackupClient
+  {
+
+
+    public FullTableBackupClientForTest() {
+    }
+
+    public FullTableBackupClientForTest(Connection conn, String backupId, BackupRequest request)
+        throws IOException {
+      super(conn, backupId, request);
+    }
+
+    @Override
+    public void execute() throws IOException
+    {
+      // Get the stage ID to fail on
+      try (Admin admin = conn.getAdmin();) {
+        // Begin BACKUP
+        beginBackup(backupManager, backupInfo);
+        failStageIf(Stage.stage_0);
+        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);
+        }
+        failStageIf(Stage.stage_1);
+        // 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);
+        failStageIf(Stage.stage_2);
+        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();
+
+          snapshotTable(admin, tableName, snapshotName);
+          backupInfo.setSnapshotName(tableName, snapshotName);
+        }
+        failStageIf(Stage.stage_3);
+        // 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);
+        failStageIf(Stage.stage_4);
+        // backup complete
+        completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf);
+
+      } catch (Exception e) {
+
+        if(autoRestoreOnFailure) {
+          failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ",
+            BackupType.FULL, conf);
+        }
+        throw new IOException(e);
+      }
+    }
+
+  }
+
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Before
+  public void setUp() throws Exception {
+    if (setupIsDone) {
+      return;
+    }
+    if (secure) {
+      // set the always on security provider
+      UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(),
+          HadoopSecurityEnabledUserProviderForTesting.class);
+      // setup configuration
+      SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
+    }
+    String coproc = conf1.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY);
+    conf1.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, (coproc == null ? "" : coproc + ",") +
+        BackupObserver.class.getName());
+    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.startMiniCluster();
+
+    if (useSecondCluster) {
+      conf2 = HBaseConfiguration.create(conf1);
+      conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
+      TEST_UTIL2 = new HBaseTestingUtility(conf2);
+      TEST_UTIL2.setZkCluster(TEST_UTIL.getZkCluster());
+      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);
+    if (useSecondCluster) {
+      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);
+    setupIsDone = true;
+  }
+
+  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 tearDown() throws Exception {
+    try{
+      SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
+    } catch (Exception e) {
+    }
+    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
+    if (useSecondCluster) {
+      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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
new file mode 100644
index 0000000..40a01b9
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java
new file mode 100644
index 0000000..08002fb
--- /dev/null
+++ b/hbase-backup/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);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
new file mode 100644
index 0000000..ebfc735
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
new file mode 100644
index 0000000..208e081
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
new file mode 100644
index 0000000..966f519
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
@@ -0,0 +1,194 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import static org.junit.Assert.assertNull;
+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.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.BackupSystemTable;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+/**
+ * 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
+ */
+@Category(LargeTests.class)
+public class TestBackupDeleteWithFailures extends TestBackupBase{
+
+  private static final Log LOG = LogFactory.getLog(TestBackupDeleteWithFailures.class);
+
+
+
+  public static enum Failure {
+    NO_FAILURES,
+    PRE_SNAPSHOT_FAILURE,
+    PRE_DELETE_SNAPSHOT_FAILURE,
+    POST_DELETE_SNAPSHOT_FAILURE
+  }
+
+  public static class MasterSnapshotObserver implements MasterObserver {
+
+
+    List<Failure> failures = new ArrayList<Failure>();
+
+    public void setFailures(Failure ... f) {
+      failures.clear();
+      for (int i = 0; i < f.length; i++) {
+        failures.add(f[i]);
+      }
+    }
+
+    @Override
+    public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
+        throws IOException
+    {
+       if (failures.contains(Failure.PRE_SNAPSHOT_FAILURE)) {
+         throw new IOException ("preSnapshot");
+       }
+    }
+
+    @Override
+    public void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        SnapshotDescription snapshot) throws IOException {
+      if (failures.contains(Failure.PRE_DELETE_SNAPSHOT_FAILURE)) {
+        throw new IOException ("preDeleteSnapshot");
+      }
+    }
+
+    @Override
+    public void postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        SnapshotDescription snapshot) throws IOException {
+      if (failures.contains(Failure.POST_DELETE_SNAPSHOT_FAILURE)) {
+        throw new IOException ("postDeleteSnapshot");
+      }
+    }
+
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    conf1.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+      MasterSnapshotObserver.class.getName());
+    conf1.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
+    super.setUp();
+  }
+
+
+  private MasterSnapshotObserver getMasterSnapshotObserver() {
+    return (MasterSnapshotObserver)TEST_UTIL.getHBaseCluster().getMaster()
+      .getMasterCoprocessorHost().findCoprocessor(MasterSnapshotObserver.class.getName());
+  }
+
+  @Test
+  public void testBackupDeleteWithFailures() throws Exception
+  {
+     testBackupDeleteWithFailuresAfter(1, Failure.PRE_DELETE_SNAPSHOT_FAILURE);
+     testBackupDeleteWithFailuresAfter(0, Failure.POST_DELETE_SNAPSHOT_FAILURE);
+     testBackupDeleteWithFailuresAfter(1, Failure.PRE_SNAPSHOT_FAILURE);
+  }
+
+  private void testBackupDeleteWithFailuresAfter(int expected, Failure ...failures) throws Exception {
+    LOG.info("test repair backup delete on a single table with data and failures "+ failures[0]);
+    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));
+
+    Connection conn = TEST_UTIL.getConnection();
+    Admin admin = conn.getAdmin();
+    MasterSnapshotObserver observer = getMasterSnapshotObserver();
+
+    observer.setFailures(failures);
+    try {
+      getBackupAdmin().deleteBackups(backupIds);
+    } catch(IOException e) {
+      if(expected != 1) assertTrue(false);
+    }
+
+    // Verify that history length == expected after delete failure
+    assertTrue (table.getBackupHistory().size() == expected);
+
+    String[] ids = table.getListOfBackupIdsFromDeleteOperation();
+
+    // Verify that we still have delete record in backup system table
+    if(expected == 1) {
+      assertTrue(ids.length == 1);
+      assertTrue(ids[0].equals(backupId));
+    } else {
+      assertNull(ids);
+    }
+
+    // Now run repair command to repair "failed" delete operation
+    String[] args = new String[] {"repair"};
+
+    observer.setFailures(Failure.NO_FAILURES);
+
+    // Run repair
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    // Verify that history length == 0
+    assertTrue (table.getBackupHistory().size() == 0);
+    ids = table.getListOfBackupIdsFromDeleteOperation();
+
+    // Verify that we do not have delete record in backup system table
+    assertNull(ids);
+
+    table.close();
+    admin.close();
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
new file mode 100644
index 0000000..0672325
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
new file mode 100644
index 0000000..dfbe106
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.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 static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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.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.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+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.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+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;
+
+@Category({ MasterTests.class, SmallTests.class })
+public class TestBackupHFileCleaner {
+  private static final Log LOG = LogFactory.getLog(TestBackupHFileCleaner.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf = TEST_UTIL.getConfiguration();
+  private static TableName tableName = TableName.valueOf("backup.hfile.cleaner");
+  private static String famName = "fam";
+  static FileSystem fs = null;
+  Path root;
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true);
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniCluster(1);
+    fs = FileSystem.get(conf);
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setup() throws IOException {
+    root = TEST_UTIL.getDataTestDirOnTestFS();
+  }
+
+  @After
+  public void cleanup() {
+    try {
+      fs.delete(root, true);
+    } catch (IOException e) {
+      LOG.warn("Failed to delete files recursively from path " + root);
+    }
+  }
+
+  @Test
+  public void testGetDeletableFiles() throws IOException {
+    // 1. Create a file
+    Path file = new Path(root, "testIsFileDeletableWithNoHFileRefs");
+    fs.createNewFile(file);
+    // 2. Assert file is successfully created
+    assertTrue("Test file not created!", fs.exists(file));
+    BackupHFileCleaner cleaner = new BackupHFileCleaner();
+    cleaner.setConf(conf);
+    cleaner.setCheckForFullyBackedUpTables(false);
+    // 3. Assert that file as is should be deletable
+    List<FileStatus> stats = new ArrayList<>();
+    FileStatus stat = fs.getFileStatus(file);
+    stats.add(stat);
+    Iterable<FileStatus> deletable = cleaner.getDeletableFiles(stats);
+    deletable = cleaner.getDeletableFiles(stats);
+    boolean found = false;
+    for (FileStatus stat1 : deletable) {
+      if (stat.equals(stat1)) found = true;
+    }
+    assertTrue("Cleaner should allow to delete this file as there is no hfile reference "
+        + "for it.", found);
+
+    // 4. Add the file as bulk load
+    List<Path> list = new ArrayList<>(1);
+    list.add(file);
+    try (Connection conn = ConnectionFactory.createConnection(conf);
+        BackupSystemTable sysTbl = new BackupSystemTable(conn)) {
+      List<TableName> sTableList = new ArrayList<>();
+      sTableList.add(tableName);
+      Map<byte[], List<Path>>[] maps = new Map[1];
+      maps[0] = new HashMap<>();
+      maps[0].put(famName.getBytes(), list);
+      sysTbl.writeBulkLoadedFiles(sTableList, maps, "1");
+    }
+
+    // 5. Assert file should not be deletable
+    deletable = cleaner.getDeletableFiles(stats);
+    deletable = cleaner.getDeletableFiles(stats);
+    found = false;
+    for (FileStatus stat1 : deletable) {
+      if (stat.equals(stat1)) found = true;
+    }
+    assertFalse("Cleaner should not allow to delete this file as there is a hfile reference "
+        + "for it.", found);
+  }
+}
\ No newline at end of file


[32/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
new file mode 100644
index 0000000..6330899
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.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.HashSet;
+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.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.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.
+   * @return The new HashMap of RS log time stamps after the log roll for this incremental backup.
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> getIncrBackupLogFileMap()
+      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());
+    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
+    backupInfo.setIncrBackupFileList(logList);
+
+    return newTimestamps;
+  }
+
+  /**
+   * Get list of WAL files eligible for incremental backup
+   * @return list of WAL files
+   * @throws IOException
+   */
+  public List<String> getIncrBackupLogFileList()
+      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.");
+    }
+
+    newTimestamps = readRegionServerLastLogRollResult();
+
+    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
+    List<WALItem> logFromSystemTable =
+        getLogFilesFromBackupSystem(previousTimestampMins, newTimestamps, getBackupInfo()
+            .getBackupRootDir());
+
+    logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
+    backupInfo.setIncrBackupFileList(logList);
+
+    return logList;
+  }
+
+
+  private List<String> excludeAlreadyBackedUpWALs(List<String> logList,
+      List<WALItem> logFromSystemTable) {
+
+    Set<String> walFileNameSet = convertToSet(logFromSystemTable);
+
+    List<String> list = new ArrayList<String>();
+    for (int i=0; i < logList.size(); i++) {
+      Path p = new Path(logList.get(i));
+      String name  = p.getName();
+      if (walFileNameSet.contains(name)) continue;
+      list.add(logList.get(i));
+    }
+    return list;
+  }
+
+  /**
+   * Create Set of WAL file names (not full path names)
+   * @param logFromSystemTable
+   * @return set of WAL file names
+   */
+  private Set<String> convertToSet(List<WALItem> logFromSystemTable) {
+
+    Set<String> set = new HashSet<String>();
+    for (int i=0; i < logFromSystemTable.size(); i++) {
+      WALItem item = logFromSystemTable.get(i);
+      set.add(item.walFile);
+    }
+    return set;
+  }
+
+  /**
+   * 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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
new file mode 100644
index 0000000..6d48c32
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
@@ -0,0 +1,377 @@
+/**
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+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.FileUtil;
+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.BackupInfo.BackupPhase;
+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.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.mapreduce.WALPlayer;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.util.Tool;
+
+/**
+ * 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);
+
+  protected IncrementalTableBackupClient() {
+  }
+
+  public IncrementalTableBackupClient(final Connection conn, final String backupId,
+      BackupRequest request) throws IOException {
+    super(conn, backupId, request);
+  }
+
+  protected List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    List<String> list = new ArrayList<String>();
+    for (String file : incrBackupFileList) {
+      Path p = new Path(file);
+      if (fs.exists(p) || isActiveWalPath(p)) {
+        list.add(file);
+      } else {
+        LOG.warn("Can't find file: " + file);
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Check if a given path is belongs to active WAL directory
+   * @param p path
+   * @return true, if yes
+   */
+  protected boolean isActiveWalPath(Path p) {
+    return !AbstractFSWALProvider.isArchivedLogFile(p);
+  }
+
+  protected static int getIndex(TableName tbl, List<TableName> sTableList) {
+    if (sTableList == null) return 0;
+    for (int i = 0; i < sTableList.size(); i++) {
+      if (tbl.equals(sTableList.get(i))) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  /*
+   * Reads bulk load records from backup table, iterates through the records and forms the paths
+   * for bulk loaded hfiles. Copies the bulk loaded hfiles to backup destination
+   * @param sTableList list of tables to be backed up
+   * @return map of table to List of files
+   */
+  protected Map<byte[], List<Path>>[] handleBulkLoad(List<TableName> sTableList) throws IOException {
+    Map<byte[], List<Path>>[] mapForSrc = new Map[sTableList.size()];
+    Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>> pair =
+    backupManager.readBulkloadRows(sTableList);
+    Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> map = pair.getFirst();
+    FileSystem fs = FileSystem.get(conf);
+    FileSystem tgtFs;
+    try {
+      tgtFs = FileSystem.get(new URI(backupInfo.getBackupRootDir()), conf);
+    } catch (URISyntaxException use) {
+      throw new IOException("Unable to get FileSystem", use);
+    }
+    Path rootdir = FSUtils.getRootDir(conf);
+    Path tgtRoot = new Path(new Path(backupInfo.getBackupRootDir()), backupId);
+    for (Map.Entry<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> tblEntry :
+      map.entrySet()) {
+      TableName srcTable = tblEntry.getKey();
+      int srcIdx = getIndex(srcTable, sTableList);
+      if (srcIdx < 0) {
+        LOG.warn("Couldn't find " + srcTable + " in source table List");
+        continue;
+      }
+      if (mapForSrc[srcIdx] == null) {
+        mapForSrc[srcIdx] = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
+      }
+      Path tblDir = FSUtils.getTableDir(rootdir, srcTable);
+      Path tgtTable = new Path(new Path(tgtRoot, srcTable.getNamespaceAsString()),
+          srcTable.getQualifierAsString());
+      for (Map.Entry<String,Map<String,List<Pair<String, Boolean>>>> regionEntry :
+        tblEntry.getValue().entrySet()){
+        String regionName = regionEntry.getKey();
+        Path regionDir = new Path(tblDir, regionName);
+        // map from family to List of hfiles
+        for (Map.Entry<String,List<Pair<String, Boolean>>> famEntry :
+          regionEntry.getValue().entrySet()) {
+          String fam = famEntry.getKey();
+          Path famDir = new Path(regionDir, fam);
+          List<Path> files;
+          if (!mapForSrc[srcIdx].containsKey(fam.getBytes())) {
+            files = new ArrayList<Path>();
+            mapForSrc[srcIdx].put(fam.getBytes(), files);
+          } else {
+            files = mapForSrc[srcIdx].get(fam.getBytes());
+          }
+          Path archiveDir = HFileArchiveUtil.getStoreArchivePath(conf, srcTable, regionName, fam);
+          String tblName = srcTable.getQualifierAsString();
+          Path tgtFam = new Path(new Path(tgtTable, regionName), fam);
+          if (!tgtFs.mkdirs(tgtFam)) {
+            throw new IOException("couldn't create " + tgtFam);
+          }
+          for (Pair<String, Boolean> fileWithState : famEntry.getValue()) {
+            String file = fileWithState.getFirst();
+            boolean raw = fileWithState.getSecond();
+            int idx = file.lastIndexOf("/");
+            String filename = file;
+            if (idx > 0) {
+              filename = file.substring(idx+1);
+            }
+            Path p = new Path(famDir, filename);
+            Path tgt = new Path(tgtFam, filename);
+            Path archive = new Path(archiveDir, filename);
+            if (fs.exists(p)) {
+              if (LOG.isTraceEnabled()) {
+                LOG.trace("found bulk hfile " + file + " in " + famDir + " for " + tblName);
+              }
+              try {
+                if (LOG.isTraceEnabled()) {
+                  LOG.trace("copying " + p + " to " + tgt);
+                }
+                FileUtil.copy(fs, p, tgtFs, tgt, false,conf);
+              } catch (FileNotFoundException e) {
+                LOG.debug("copying archive " + archive + " to " + tgt);
+                try {
+                  FileUtil.copy(fs, archive, tgtFs, tgt, false, conf);
+                } catch (FileNotFoundException fnfe) {
+                  if (!raw) throw fnfe;
+                }
+              }
+            } else {
+              LOG.debug("copying archive " + archive + " to " + tgt);
+              try {
+                FileUtil.copy(fs, archive, tgtFs, tgt, false, conf);
+              } catch (FileNotFoundException fnfe) {
+                if (!raw) throw fnfe;
+              }
+            }
+            files.add(tgt);
+          }
+        }
+      }
+    }
+    backupManager.writeBulkLoadedFiles(sTableList, mapForSrc);
+    backupManager.removeBulkLoadedRows(sTableList, pair.getSecond());
+    return mapForSrc;
+  }
+
+  @Override
+  public void execute() throws IOException {
+
+    try {
+      // case PREPARE_INCREMENTAL:
+      beginBackup(backupManager, backupInfo);
+      backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
+      LOG.debug("For incremental backup, current table set is "
+          + backupManager.getIncrementalBackupTableSet());
+      newTimestamps =
+          ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
+    } catch (Exception e) {
+      // fail the overall backup and return
+      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
+        BackupType.INCREMENTAL, conf);
+      return;
+    }
+
+    // case INCREMENTAL_COPY:
+    try {
+      // copy out the table and region info files for each table
+      BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
+      // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
+      convertWALsToHFiles(backupInfo);
+      incrementalCopyHFiles(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);
+      return;
+    }
+    // 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 {
+      // 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);
+
+      handleBulkLoad(backupInfo.getTableNames());
+      // backup complete
+      completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
+
+    } catch (IOException e) {
+      failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
+        BackupType.INCREMENTAL, conf);
+    }
+  }
+
+  protected void incrementalCopyHFiles(BackupInfo backupInfo) throws Exception {
+
+    try {
+      LOG.debug("Incremental copy HFiles 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 = new ArrayList<String>();
+      // Add Bulk output
+      incrBackupFileList.add(getBulkOutputDir().toString());
+      String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
+      strArr[strArr.length - 1] = backupInfo.getBackupRootDir();
+      BackupCopyJob copyService = BackupRestoreFactory.getBackupCopyJob(conf);
+      int res = copyService.copy(backupInfo, backupManager, conf, BackupType.INCREMENTAL, strArr);
+      if (res != 0) {
+        LOG.error("Copy incremental HFile files failed with return code: " + res + ".");
+        throw new IOException("Failed copy from " + StringUtils.join(incrBackupFileList, ',')
+            + " to " + backupInfo.getHLogTargetDir());
+      }
+      LOG.debug("Incremental copy HFiles from " + StringUtils.join(incrBackupFileList, ',')
+          + " to " + backupInfo.getBackupRootDir() + " finished.");
+    } finally {
+      deleteBulkLoadDirectory();
+    }
+  }
+
+  protected void deleteBulkLoadDirectory() throws IOException {
+    // delete original bulk load directory on method exit
+    Path path = getBulkOutputDir();
+    FileSystem fs = FileSystem.get(conf);
+    boolean result = fs.delete(path, true);
+    if (!result) {
+      LOG.warn("Could not delete " + path);
+    }
+
+  }
+
+  protected void convertWALsToHFiles(BackupInfo backupInfo) throws IOException {
+    // get incremental backup file list and prepare parameters for DistCp
+    List<String> incrBackupFileList = backupInfo.getIncrBackupFileList();
+    // Get list of tables in incremental backup set
+    Set<TableName> tableSet = backupManager.getIncrementalBackupTableSet();
+    // filter missing files out (they have been copied by previous backups)
+    incrBackupFileList = filterMissingFiles(incrBackupFileList);
+    for (TableName table : tableSet) {
+      // Check if table exists
+      if (tableExists(table, conn)) {
+        walToHFiles(incrBackupFileList, table);
+      } else {
+        LOG.warn("Table " + table + " does not exists. Skipping in WAL converter");
+      }
+    }
+  }
+
+
+  protected boolean tableExists(TableName table, Connection conn) throws IOException {
+    try (Admin admin = conn.getAdmin();) {
+      return admin.tableExists(table);
+    }
+  }
+
+  protected void walToHFiles(List<String> dirPaths, TableName tableName) throws IOException {
+
+    Tool player = new WALPlayer();
+
+    // Player reads all files in arbitrary directory structure and creates
+    // a Map task for each file. We use ';' as separator
+    // because WAL file names contains ','
+    String dirs = StringUtils.join(dirPaths, ';');
+
+    Path bulkOutputPath = getBulkOutputDirForTable(tableName);
+    conf.set(WALPlayer.BULK_OUTPUT_CONF_KEY, bulkOutputPath.toString());
+    conf.set(WALPlayer.INPUT_FILES_SEPARATOR_KEY, ";");
+    String[] playerArgs = { dirs, tableName.getNameAsString() };
+
+    try {
+      player.setConf(conf);
+      int result = player.run(playerArgs);
+      if(result != 0) {
+        throw new IOException("WAL Player failed");
+      }
+      conf.unset(WALPlayer.INPUT_FILES_SEPARATOR_KEY);
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception ee) {
+      throw new IOException("Can not convert from directory " + dirs
+          + " (check Hadoop, HBase and WALPlayer M/R job logs) ", ee);
+    }
+  }
+
+  protected Path getBulkOutputDirForTable(TableName table) {
+    Path tablePath = getBulkOutputDir();
+    tablePath = new Path(tablePath, table.getNamespaceAsString());
+    tablePath = new Path(tablePath, table.getQualifierAsString());
+    return new Path(tablePath, "data");
+  }
+
+  protected Path getBulkOutputDir() {
+    String backupId = backupInfo.getBackupId();
+    Path path = new Path(backupInfo.getBackupRootDir());
+    path = new Path(path, ".tmp");
+    path = new Path(path, backupId);
+    return path;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
new file mode 100644
index 0000000..ea7a7b8
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
@@ -0,0 +1,278 @@
+/**
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+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.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.BackupUtils;
+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;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.LoadQueueItem;
+
+/**
+ * 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(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 fileBackupDir =
+          HBackupFileSystem.getTableBackupDataDir(im.getRootDir(), im.getBackupId(), sTable);
+      dirList.add(new Path(fileBackupDir));
+    }
+
+    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;
+    Set<String> backupIdSet = new HashSet<>();
+
+    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));
+          if (image.getType() == BackupType.INCREMENTAL) {
+            backupIdSet.add(image.getBackupId());
+            LOG.debug("adding " + image.getBackupId() + " for bulk load");
+          }
+        }
+      }
+    }
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<TableName> sTableList = Arrays.asList(sTableArray);
+      for (String id : backupIdSet) {
+        LOG.debug("restoring bulk load for " + id);
+        Map<byte[], List<Path>>[] mapForSrc = table.readBulkLoadedFiles(id, sTableList);
+        Map<LoadQueueItem, ByteBuffer> loaderResult;
+        conf.setBoolean(LoadIncrementalHFiles.ALWAYS_COPY_FILES, true);
+        LoadIncrementalHFiles loader = BackupUtils.createLoader(conf);
+        for (int i = 0; i < sTableList.size(); i++) {
+          if (mapForSrc[i] != null && !mapForSrc[i].isEmpty()) {
+            loaderResult = loader.run(null, mapForSrc[i], tTableArray[i]);
+            LOG.debug("bulk loading " + sTableList.get(i) + " to " + tTableArray[i]);
+            if (loaderResult.isEmpty()) {
+              String msg = "Couldn't bulk load for " + sTableList.get(i) + " to " + tTableArray[i];
+              LOG.error(msg);
+              throw new IOException(msg);
+            }
+          }
+        }
+      }
+    }
+    LOG.debug("restoreStage finished");
+  }
+
+  static long getTsFromBackupId(String backupId) {
+    if (backupId == null) {
+      return 0;
+    }
+    return Long.parseLong(backupId.substring(backupId.lastIndexOf("_") + 1));
+  }
+
+  static boolean withinRange(long a, long lower, long upper) {
+    if (a < lower || a > upper) {
+      return false;
+    }
+    return true;
+  }
+
+  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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
new file mode 100644
index 0000000..6eec460
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
@@ -0,0 +1,436 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * 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 {
+
+  public static final String BACKUP_CLIENT_IMPL_CLASS = "backup.client.impl.class";
+
+  @VisibleForTesting
+  public static final String BACKUP_TEST_MODE_STAGE = "backup.test.mode.stage";
+
+  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() {
+  }
+
+  public TableBackupClient(final Connection conn, final String backupId, BackupRequest request)
+      throws IOException {
+    init(conn, backupId, request);
+  }
+
+  public void init(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());
+    }
+    // Start new session
+    backupManager.startBackupSession();
+  }
+
+  /**
+   * Begin the overall backup.
+   * @param backupInfo backup info
+   * @throws IOException exception
+   */
+  protected void beginBackup(BackupManager backupManager, BackupInfo backupInfo)
+      throws IOException {
+
+    BackupSystemTable.snapshot(conn);
+    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.");
+    }
+  }
+
+  protected 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
+   */
+  protected static void deleteSnapshots(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);
+      }
+      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
+   */
+  protected static 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.
+   */
+  protected static 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.debug("Cleaning up uncompleted backup data at " + targetDirPath.toString()
+                + " done.");
+          } else {
+            LOG.debug("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 {
+
+    try {
+      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);
+      cleanupAndRestoreBackupSystem(conn, backupInfo, conf);
+      // If backup session is updated to FAILED state - means we
+      // processed recovery already.
+      backupManager.updateBackupInfo(backupInfo);
+      backupManager.finishBackupSession();
+      LOG.error("Backup " + backupInfo.getBackupId() + " failed.");
+    } catch (IOException ee) {
+      LOG.error("Please run backup repair tool manually to restore backup system integrity");
+      throw ee;
+    }
+  }
+
+  public static void cleanupAndRestoreBackupSystem (Connection conn, BackupInfo backupInfo,
+      Configuration conf) throws IOException
+  {
+    BackupType type = backupInfo.getType();
+     // 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) {
+       deleteSnapshots(conn, backupInfo, conf);
+       cleanupExportSnapshotLog(conf);
+     }
+     BackupSystemTable.restoreFromSnapshot(conn);
+     BackupSystemTable.deleteSnapshot(conn);
+     // 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);
+  }
+
+
+
+  /**
+   * 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
+   */
+  protected 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
+   */
+  protected 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
+   */
+  protected 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);
+
+    // 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);
+    }
+
+    // when full backup is done:
+    // - delete HBase snapshot
+    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
+    // snapshots
+    if (type == BackupType.FULL) {
+      deleteSnapshots(conn, backupInfo, conf);
+      cleanupExportSnapshotLog(conf);
+    } else if (type == BackupType.INCREMENTAL) {
+      cleanupDistCpLog(backupInfo, conf);
+    }
+    BackupSystemTable.deleteSnapshot(conn);
+    backupManager.updateBackupInfo(backupInfo);
+
+    // Finish active session
+    backupManager.finishBackupSession();
+
+    LOG.info("Backup " + backupInfo.getBackupId() + " completed.");
+  }
+
+  /**
+   * Backup request execution
+   * @throws IOException
+   */
+  public abstract void execute() throws IOException;
+
+  @VisibleForTesting
+  protected Stage getTestStage() {
+    return Stage.valueOf("stage_"+ conf.getInt(BACKUP_TEST_MODE_STAGE, 0));
+  }
+
+  @VisibleForTesting
+  protected void failStageIf(Stage stage) throws IOException {
+    Stage current = getTestStage();
+    if (current == stage) {
+      throw new IOException("Failed stage " + stage+" in testing");
+    }
+  }
+
+  public static enum Stage {
+    stage_0, stage_1, stage_2, stage_3, stage_4
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyJob.java
new file mode 100644
index 0000000..016d1a4
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
new file mode 100644
index 0000000..00c5b83
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupMergeJob.java
@@ -0,0 +1,321 @@
+/**
+ * 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 static org.apache.hadoop.hbase.backup.util.BackupUtils.succeeded;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+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.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.BackupMergeJob;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+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.Pair;
+import org.apache.hadoop.util.Tool;
+
+/**
+ * MapReduce implementation of {@link BackupMergeJob}
+ * Must be initialized with configuration of a backup destination cluster
+ *
+ */
+
+@InterfaceAudience.Private
+public class MapReduceBackupMergeJob implements BackupMergeJob {
+  public static final Log LOG = LogFactory.getLog(MapReduceBackupMergeJob.class);
+
+  protected Tool player;
+  protected Configuration conf;
+
+  public MapReduceBackupMergeJob() {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public void run(String[] backupIds) throws IOException {
+    String bulkOutputConfKey;
+
+    // TODO : run player on remote cluster
+    player = new MapReduceHFileSplitterJob();
+    bulkOutputConfKey = MapReduceHFileSplitterJob.BULK_OUTPUT_CONF_KEY;
+    // Player reads all files in arbitrary directory structure and creates
+    // a Map task for each file
+    String bids = StringUtils.join(backupIds, ",");
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Merge backup images " + bids);
+    }
+
+    List<Pair<TableName, Path>> processedTableList = new ArrayList<Pair<TableName, Path>>();
+    boolean finishedTables = false;
+    Connection conn = ConnectionFactory.createConnection(getConf());
+    BackupSystemTable table = new BackupSystemTable(conn);
+    FileSystem fs = FileSystem.get(getConf());
+
+    try {
+
+      // Get exclusive lock on backup system
+      table.startBackupExclusiveOperation();
+      // Start merge operation
+      table.startMergeOperation(backupIds);
+
+      // Select most recent backup id
+      String mergedBackupId = findMostRecentBackupId(backupIds);
+
+      TableName[] tableNames = getTableNamesInBackupImages(backupIds);
+      String backupRoot = null;
+
+      BackupInfo bInfo = table.readBackupInfo(backupIds[0]);
+      backupRoot = bInfo.getBackupRootDir();
+
+      for (int i = 0; i < tableNames.length; i++) {
+
+        LOG.info("Merge backup images for " + tableNames[i]);
+
+        // Find input directories for table
+
+        Path[] dirPaths = findInputDirectories(fs, backupRoot, tableNames[i], backupIds);
+        String dirs = StringUtils.join(dirPaths, ",");
+        Path bulkOutputPath =
+            BackupUtils.getBulkOutputDir(BackupUtils.getFileNameCompatibleString(tableNames[i]),
+              getConf(), false);
+        // Delete content if exists
+        if (fs.exists(bulkOutputPath)) {
+          if (!fs.delete(bulkOutputPath, true)) {
+            LOG.warn("Can not delete: " + bulkOutputPath);
+          }
+        }
+        Configuration conf = getConf();
+        conf.set(bulkOutputConfKey, bulkOutputPath.toString());
+        String[] playerArgs = { dirs, tableNames[i].getNameAsString() };
+
+        int result = 0;
+
+        player.setConf(getConf());
+        result = player.run(playerArgs);
+        if (!succeeded(result)) {
+          throw new IOException("Can not merge backup images for " + dirs
+              + " (check Hadoop/MR and HBase logs). Player return code =" + result);
+        }
+        // Add to processed table list
+        processedTableList.add(new Pair<TableName, Path>(tableNames[i], bulkOutputPath));
+        LOG.debug("Merge Job finished:" + result);
+      }
+      List<TableName> tableList = toTableNameList(processedTableList);
+      table.updateProcessedTablesForMerge(tableList);
+      finishedTables = true;
+
+      // Move data
+      for (Pair<TableName, Path> tn : processedTableList) {
+        moveData(fs, backupRoot, tn.getSecond(), tn.getFirst(), mergedBackupId);
+      }
+
+      // Delete old data and update manifest
+      List<String> backupsToDelete = getBackupIdsToDelete(backupIds, mergedBackupId);
+      deleteBackupImages(backupsToDelete, conn, fs, backupRoot);
+      updateBackupManifest(backupRoot, mergedBackupId, backupsToDelete);
+      // Finish merge session
+      table.finishMergeOperation();
+      // Release lock
+      table.finishBackupExclusiveOperation();
+    } catch (RuntimeException e) {
+
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e);
+      if (!finishedTables) {
+        // cleanup bulk directories and finish merge
+        // merge MUST be repeated (no need for repair)
+        cleanupBulkLoadDirs(fs, toPathList(processedTableList));
+        table.finishMergeOperation();
+        table.finishBackupExclusiveOperation();
+        throw new IOException("Backup merge operation failed, you should try it again", e);
+      } else {
+        // backup repair must be run
+        throw new IOException(
+            "Backup merge operation failed, run backup repair tool to restore system's integrity",
+            e);
+      }
+    } finally {
+      table.close();
+      conn.close();
+    }
+  }
+
+  protected List<Path> toPathList(List<Pair<TableName, Path>> processedTableList) {
+    ArrayList<Path> list = new ArrayList<Path>();
+    for (Pair<TableName, Path> p : processedTableList) {
+      list.add(p.getSecond());
+    }
+    return list;
+  }
+
+  protected List<TableName> toTableNameList(List<Pair<TableName, Path>> processedTableList) {
+    ArrayList<TableName> list = new ArrayList<TableName>();
+    for (Pair<TableName, Path> p : processedTableList) {
+      list.add(p.getFirst());
+    }
+    return list;
+  }
+
+  protected void cleanupBulkLoadDirs(FileSystem fs, List<Path> pathList) throws IOException {
+    for (Path path : pathList) {
+
+      if (!fs.delete(path, true)) {
+        LOG.warn("Can't delete " + path);
+      }
+    }
+  }
+
+  protected void updateBackupManifest(String backupRoot, String mergedBackupId,
+      List<String> backupsToDelete) throws IllegalArgumentException, IOException {
+
+    BackupManifest manifest =
+        HBackupFileSystem.getManifest(conf, new Path(backupRoot), mergedBackupId);
+    manifest.getBackupImage().removeAncestors(backupsToDelete);
+    // save back
+    manifest.store(conf);
+
+  }
+
+  protected void deleteBackupImages(List<String> backupIds, Connection conn, FileSystem fs,
+      String backupRoot) throws IOException {
+
+    // Delete from backup system table
+    try (BackupSystemTable table = new BackupSystemTable(conn);) {
+      for (String backupId : backupIds) {
+        table.deleteBackupInfo(backupId);
+      }
+    }
+
+    // Delete from file system
+    for (String backupId : backupIds) {
+      Path backupDirPath = HBackupFileSystem.getBackupPath(backupRoot, backupId);
+
+      if (!fs.delete(backupDirPath, true)) {
+        LOG.warn("Could not delete " + backupDirPath);
+      }
+    }
+  }
+
+  protected List<String> getBackupIdsToDelete(String[] backupIds, String mergedBackupId) {
+    List<String> list = new ArrayList<String>();
+    for (String id : backupIds) {
+      if (id.equals(mergedBackupId)) {
+        continue;
+      }
+      list.add(id);
+    }
+    return list;
+  }
+
+  protected void moveData(FileSystem fs, String backupRoot, Path bulkOutputPath, TableName tableName,
+      String mergedBackupId) throws IllegalArgumentException, IOException {
+
+    Path dest =
+        new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, mergedBackupId, tableName));
+
+    // Delete all in dest
+    if (!fs.delete(dest, true)) {
+      throw new IOException("Could not delete " + dest);
+    }
+
+    FileStatus[] fsts = fs.listStatus(bulkOutputPath);
+    for (FileStatus fst : fsts) {
+      if (fst.isDirectory()) {
+        fs.rename(fst.getPath().getParent(), dest);
+      }
+    }
+
+  }
+
+  protected String findMostRecentBackupId(String[] backupIds) {
+    long recentTimestamp = Long.MIN_VALUE;
+    for (String backupId : backupIds) {
+      long ts = Long.parseLong(backupId.split("_")[1]);
+      if (ts > recentTimestamp) {
+        recentTimestamp = ts;
+      }
+    }
+    return BackupRestoreConstants.BACKUPID_PREFIX + recentTimestamp;
+  }
+
+  protected TableName[] getTableNamesInBackupImages(String[] backupIds) throws IOException {
+
+    Set<TableName> allSet = new HashSet<TableName>();
+
+    try (Connection conn = ConnectionFactory.createConnection(conf);
+        BackupSystemTable table = new BackupSystemTable(conn);) {
+      for (String backupId : backupIds) {
+        BackupInfo bInfo = table.readBackupInfo(backupId);
+
+        allSet.addAll(bInfo.getTableNames());
+      }
+    }
+
+    TableName[] ret = new TableName[allSet.size()];
+    return allSet.toArray(ret);
+  }
+
+  protected Path[] findInputDirectories(FileSystem fs, String backupRoot, TableName tableName,
+      String[] backupIds) throws IOException {
+
+    List<Path> dirs = new ArrayList<Path>();
+
+    for (String backupId : backupIds) {
+      Path fileBackupDirPath =
+          new Path(HBackupFileSystem.getTableBackupDataDir(backupRoot, backupId, tableName));
+      if (fs.exists(fileBackupDirPath)) {
+        dirs.add(fileBackupDirPath);
+      } else {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("File: " + fileBackupDirPath + " does not exist.");
+        }
+      }
+    }
+    Path[] ret = new Path[dirs.size()];
+    return dirs.toArray(ret);
+  }
+
+}


[09/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 1765bf3..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
+++ /dev/null
@@ -1,550 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.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 = BackupPhase.REQUEST;
-
-  /**
-   * 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;
-    }
-  }
-
-  @Override
-  public String toString() {
-    return backupId;
-  }
-
-  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/2dda3712/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
deleted file mode 100644
index 136782f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Backup merge operation job interface. Concrete implementation is provided by backup provider, see
- * {@link BackupRestoreFactory}
- */
-
-@InterfaceAudience.Private
-public interface BackupMergeJob extends Configurable {
-
-  /**
-   * Run backup merge operation
-   * @param backupIds backup image ids
-   * @throws IOException
-   */
-  void run(String[] backupIds) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
deleted file mode 100644
index 595e862..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.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.Path;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.TableName;
-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.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * An Observer to facilitate backup operations
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class BackupObserver implements RegionObserver {
-  private static final Log LOG = LogFactory.getLog(BackupObserver.class);
-  @Override
-  public boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
-    List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths,
-    boolean hasLoaded) throws IOException {
-    Configuration cfg = ctx.getEnvironment().getConfiguration();
-    if (!hasLoaded) {
-      // there is no need to record state
-      return hasLoaded;
-    }
-    if (finalPaths == null || !BackupManager.isBackupEnabled(cfg)) {
-      LOG.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled");
-      return hasLoaded;
-    }
-    try (Connection connection = ConnectionFactory.createConnection(cfg);
-        BackupSystemTable tbl = new BackupSystemTable(connection)) {
-      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
-      TableName tableName = info.getTable();
-      if (!fullyBackedUpTables.contains(tableName)) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(tableName + " has not gone thru full backup");
-        }
-        return hasLoaded;
-      }
-      tbl.writePathsPostBulkLoad(tableName, info.getEncodedNameAsBytes(), finalPaths);
-      return hasLoaded;
-    } catch (IOException ioe) {
-      LOG.error("Failed to get tables which have been fully backed up", ioe);
-      return false;
-    }
-  }
-  @Override
-  public void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
-    Configuration cfg = ctx.getEnvironment().getConfiguration();
-    if (pairs == null || pairs.isEmpty() || !BackupManager.isBackupEnabled(cfg)) {
-      LOG.debug("skipping recording bulk load in preCommitStoreFile since backup is disabled");
-      return;
-    }
-    try (Connection connection = ConnectionFactory.createConnection(cfg);
-        BackupSystemTable tbl = new BackupSystemTable(connection)) {
-      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
-      TableName tableName = info.getTable();
-      if (!fullyBackedUpTables.contains(tableName)) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(tableName + " has not gone thru full backup");
-        }
-        return;
-      }
-      tbl.writeFilesForBulkLoadPreCommit(tableName, info.getEncodedNameAsBytes(), family, pairs);
-      return;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index dadd861..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.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/2dda3712/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
deleted file mode 100644
index 48e70a1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.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 = "backup:system";
-
-  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;
-
-
-  public static final String BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.attempts.max";
-  public static final int DEFAULT_BACKUP_MAX_ATTEMPTS = 10;
-
-  public static final String BACKUP_ATTEMPTS_PAUSE_MS_KEY = "hbase.backup.attempts.pause.ms";
-  public static final int DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS = 10000;
-
-  /*
-   *  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, REPAIR
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index d72c884..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
-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";
-  public final static String HBASE_BACKUP_MERGE_IMPL_CLASS = "hbase.backup.merge.class";
-
-  private BackupRestoreFactory() {
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  /**
-   * Gets backup restore job
-   * @param conf configuration
-   * @return backup restore job 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 job instance
-   */
-  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;
-  }
-
-  /**
-   * Gets backup merge job
-   * @param conf configuration
-   * @return backup merge job instance
-   */
-  public static BackupMergeJob getBackupMergeJob(Configuration conf) {
-    Class<? extends BackupMergeJob> cls =
-        conf.getClass(HBASE_BACKUP_MERGE_IMPL_CLASS, MapReduceBackupMergeJob.class,
-          BackupMergeJob.class);
-    BackupMergeJob service = ReflectionUtils.newInstance(cls, conf);
-    service.setConf(conf);
-    return service;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index a5390a6..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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/2dda3712/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
deleted file mode 100644
index 1c43e88..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.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;
-  }
-
-  public static String getTableBackupDataDir(String backupRootDir, String backupId,
-      TableName tableName) {
-    return getTableBackupDir(backupRootDir, backupId, tableName) + Path.SEPARATOR + "data";
-  }
-
-  public static Path getBackupPath(String backupRootDir, String backupId) {
-    return new Path(backupRootDir + Path.SEPARATOR + backupId);
-  }
-
-  /**
-   * Given the backup root dir, backup id and the table name, return the backup image location,
-   * which is also where the backup manifest file is. return value look like:
-   * "hdfs://backup.hbase.org:9000/user/biadmin/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));
-  }
-
-  // TODO we do not keep WAL files anymore
-  // Move manifest file to other place
-  private static Path getManifestPath(Configuration conf, Path backupRootPath, String backupId)
-      throws IOException {
-    Path manifestPath = null;
-
-    FileSystem fs = backupRootPath.getFileSystem(conf);
-    manifestPath =
-        new Path(getBackupPath(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(Configuration conf, Path backupRootPath, String backupId) throws IOException {
-    BackupManifest manifest =
-        new BackupManifest(conf, getManifestPath(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(conf, backupRootPath, backupId);
-      backupManifestMap.put(tableName, manifest);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 1becb75..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
-  * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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/2dda3712/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
deleted file mode 100644
index 82a1b56..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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/2dda3712/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
deleted file mode 100644
index 86fb963..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import 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/2dda3712/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
deleted file mode 100644
index de3ad5a..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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;
-  }
-}


[27/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 1765bf3..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
+++ /dev/null
@@ -1,550 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.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 = BackupPhase.REQUEST;
-
-  /**
-   * 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;
-    }
-  }
-
-  @Override
-  public String toString() {
-    return backupId;
-  }
-
-  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/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
deleted file mode 100644
index 136782f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupMergeJob.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Backup merge operation job interface. Concrete implementation is provided by backup provider, see
- * {@link BackupRestoreFactory}
- */
-
-@InterfaceAudience.Private
-public interface BackupMergeJob extends Configurable {
-
-  /**
-   * Run backup merge operation
-   * @param backupIds backup image ids
-   * @throws IOException
-   */
-  void run(String[] backupIds) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
deleted file mode 100644
index 595e862..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupObserver.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.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.Path;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.TableName;
-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.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * An Observer to facilitate backup operations
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class BackupObserver implements RegionObserver {
-  private static final Log LOG = LogFactory.getLog(BackupObserver.class);
-  @Override
-  public boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
-    List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths,
-    boolean hasLoaded) throws IOException {
-    Configuration cfg = ctx.getEnvironment().getConfiguration();
-    if (!hasLoaded) {
-      // there is no need to record state
-      return hasLoaded;
-    }
-    if (finalPaths == null || !BackupManager.isBackupEnabled(cfg)) {
-      LOG.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled");
-      return hasLoaded;
-    }
-    try (Connection connection = ConnectionFactory.createConnection(cfg);
-        BackupSystemTable tbl = new BackupSystemTable(connection)) {
-      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
-      TableName tableName = info.getTable();
-      if (!fullyBackedUpTables.contains(tableName)) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(tableName + " has not gone thru full backup");
-        }
-        return hasLoaded;
-      }
-      tbl.writePathsPostBulkLoad(tableName, info.getEncodedNameAsBytes(), finalPaths);
-      return hasLoaded;
-    } catch (IOException ioe) {
-      LOG.error("Failed to get tables which have been fully backed up", ioe);
-      return false;
-    }
-  }
-  @Override
-  public void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
-      final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
-    Configuration cfg = ctx.getEnvironment().getConfiguration();
-    if (pairs == null || pairs.isEmpty() || !BackupManager.isBackupEnabled(cfg)) {
-      LOG.debug("skipping recording bulk load in preCommitStoreFile since backup is disabled");
-      return;
-    }
-    try (Connection connection = ConnectionFactory.createConnection(cfg);
-        BackupSystemTable tbl = new BackupSystemTable(connection)) {
-      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
-      HRegionInfo info = ctx.getEnvironment().getRegionInfo();
-      TableName tableName = info.getTable();
-      if (!fullyBackedUpTables.contains(tableName)) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(tableName + " has not gone thru full backup");
-        }
-        return;
-      }
-      tbl.writeFilesForBulkLoadPreCommit(tableName, info.getEncodedNameAsBytes(), family, pairs);
-      return;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index dadd861..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.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/37c65946/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
deleted file mode 100644
index 48e70a1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.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 = "backup:system";
-
-  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;
-
-
-  public static final String BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.attempts.max";
-  public static final int DEFAULT_BACKUP_MAX_ATTEMPTS = 10;
-
-  public static final String BACKUP_ATTEMPTS_PAUSE_MS_KEY = "hbase.backup.attempts.pause.ms";
-  public static final int DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS = 10000;
-
-  /*
-   *  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, REPAIR
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index d72c884..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
-import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupMergeJob;
-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";
-  public final static String HBASE_BACKUP_MERGE_IMPL_CLASS = "hbase.backup.merge.class";
-
-  private BackupRestoreFactory() {
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  /**
-   * Gets backup restore job
-   * @param conf configuration
-   * @return backup restore job 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 job instance
-   */
-  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;
-  }
-
-  /**
-   * Gets backup merge job
-   * @param conf configuration
-   * @return backup merge job instance
-   */
-  public static BackupMergeJob getBackupMergeJob(Configuration conf) {
-    Class<? extends BackupMergeJob> cls =
-        conf.getClass(HBASE_BACKUP_MERGE_IMPL_CLASS, MapReduceBackupMergeJob.class,
-          BackupMergeJob.class);
-    BackupMergeJob service = ReflectionUtils.newInstance(cls, conf);
-    service.setConf(conf);
-    return service;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index a5390a6..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupTableInfo.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import 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/37c65946/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
deleted file mode 100644
index 1c43e88..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.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;
-  }
-
-  public static String getTableBackupDataDir(String backupRootDir, String backupId,
-      TableName tableName) {
-    return getTableBackupDir(backupRootDir, backupId, tableName) + Path.SEPARATOR + "data";
-  }
-
-  public static Path getBackupPath(String backupRootDir, String backupId) {
-    return new Path(backupRootDir + Path.SEPARATOR + backupId);
-  }
-
-  /**
-   * Given the backup root dir, backup id and the table name, return the backup image location,
-   * which is also where the backup manifest file is. return value look like:
-   * "hdfs://backup.hbase.org:9000/user/biadmin/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));
-  }
-
-  // TODO we do not keep WAL files anymore
-  // Move manifest file to other place
-  private static Path getManifestPath(Configuration conf, Path backupRootPath, String backupId)
-      throws IOException {
-    Path manifestPath = null;
-
-    FileSystem fs = backupRootPath.getFileSystem(conf);
-    manifestPath =
-        new Path(getBackupPath(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(Configuration conf, Path backupRootPath, String backupId) throws IOException {
-    BackupManifest manifest =
-        new BackupManifest(conf, getManifestPath(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(conf, backupRootPath, backupId);
-      backupManifestMap.put(tableName, manifest);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/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
deleted file mode 100644
index 1becb75..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
-  * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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/37c65946/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
deleted file mode 100644
index 82a1b56..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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/37c65946/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
deleted file mode 100644
index 86fb963..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import 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/37c65946/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
deleted file mode 100644
index de3ad5a..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import 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;
-  }
-}


[12/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
new file mode 100644
index 0000000..0cfe099
--- /dev/null
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -0,0 +1,516 @@
+/**
+ * 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.StoreFileInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.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 static long TABLE_AVAILABILITY_WAIT_TIME = 180000;
+
+  private final String[] ignoreDirs = { HConstants.RECOVERED_EDITS_DIR };
+  protected Configuration conf = null;
+  protected Path backupRootPath;
+  protected String backupId;
+  protected FileSystem fs;
+
+  // 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);
+  }
+
+  /**
+   * return value represent path for:
+   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn"
+   * @param tableName 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 = new HTableDescriptor(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 {
+    createAndRestoreTable(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;
+  }
+
+  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 createAndRestoreTable(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 = new HTableDescriptor(newTableName, tableDescriptor);
+        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 = new HTableDescriptor(newTableName, tableDescriptor);
+    }
+
+    // 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);
+      RestoreJob restoreService = BackupRestoreFactory.getRestoreJob(conf);
+      Path[] paths = new Path[regionPathList.size()];
+      regionPathList.toArray(paths);
+      restoreService.run(paths, new TableName[]{tableName}, new TableName[] {newTableName}, true);
+
+    } catch (Exception e) {
+      LOG.error(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;
+  }
+
+  /**
+   * 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(TableName, String, Admin)}
+   * @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)) {
+        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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
new file mode 100644
index 0000000..7fe9a61
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
@@ -0,0 +1,503 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+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.BackupPhase;
+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.backup.impl.FullTableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager;
+import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.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.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.mapreduce.HadoopSecurityEnabledUserProviderForTesting;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.access.SecureTestUtil;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.AfterClass;
+import org.junit.Before;
+
+/**
+ * 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 HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static HBaseTestingUtility TEST_UTIL2;
+  protected static Configuration conf1 = TEST_UTIL.getConfiguration();
+  protected static Configuration conf2;
+
+  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";
+  protected static boolean secure = false;
+
+  protected static boolean autoRestoreOnFailure = true;
+  protected static boolean setupIsDone = false;
+  protected static boolean useSecondCluster = false;
+
+
+  static class IncrementalTableBackupClientForTest extends IncrementalTableBackupClient
+  {
+
+    public IncrementalTableBackupClientForTest() {
+    }
+
+    public IncrementalTableBackupClientForTest(Connection conn,
+        String backupId, BackupRequest request) throws IOException {
+      super(conn, backupId, request);
+    }
+
+    @Override
+    public void execute() throws IOException
+    {
+      // case INCREMENTAL_COPY:
+      try {
+        // case PREPARE_INCREMENTAL:
+        failStageIf(Stage.stage_0);
+        beginBackup(backupManager, backupInfo);
+
+        failStageIf(Stage.stage_1);
+        backupInfo.setPhase(BackupPhase.PREPARE_INCREMENTAL);
+        LOG.debug("For incremental backup, current table set is "
+            + backupManager.getIncrementalBackupTableSet());
+        newTimestamps = ((IncrementalBackupManager) backupManager).getIncrBackupLogFileMap();
+        // copy out the table and region info files for each table
+        BackupUtils.copyTableRegionInfo(conn, backupInfo, conf);
+        // convert WAL to HFiles and copy them to .tmp under BACKUP_ROOT
+        convertWALsToHFiles(backupInfo);
+        incrementalCopyHFiles(backupInfo);
+        failStageIf(Stage.stage_2);
+        // Save list of WAL files copied
+        backupManager.recordWALFiles(backupInfo.getIncrBackupFileList());
+
+        // 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
+        // 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);
+        failStageIf(Stage.stage_3);
+
+        HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
+            backupManager.readLogTimestampMap();
+
+        Long newStartCode =
+            BackupUtils.getMinValue(BackupUtils.getRSLogTimestampMins(newTableSetTimestampMap));
+        backupManager.writeBackupStartCode(newStartCode);
+
+        handleBulkLoad(backupInfo.getTableNames());
+        failStageIf(Stage.stage_4);
+
+        // backup complete
+        completeBackup(conn, backupInfo, backupManager, BackupType.INCREMENTAL, conf);
+
+      } catch (Exception e) {
+        failBackup(conn, backupInfo, backupManager, e, "Unexpected Exception : ",
+          BackupType.INCREMENTAL, conf);
+        throw new IOException(e);
+      }
+
+    }
+  }
+
+  static class FullTableBackupClientForTest extends FullTableBackupClient
+  {
+
+
+    public FullTableBackupClientForTest() {
+    }
+
+    public FullTableBackupClientForTest(Connection conn, String backupId, BackupRequest request)
+        throws IOException {
+      super(conn, backupId, request);
+    }
+
+    @Override
+    public void execute() throws IOException
+    {
+      // Get the stage ID to fail on
+      try (Admin admin = conn.getAdmin();) {
+        // Begin BACKUP
+        beginBackup(backupManager, backupInfo);
+        failStageIf(Stage.stage_0);
+        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);
+        }
+        failStageIf(Stage.stage_1);
+        // 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);
+        failStageIf(Stage.stage_2);
+        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();
+
+          snapshotTable(admin, tableName, snapshotName);
+          backupInfo.setSnapshotName(tableName, snapshotName);
+        }
+        failStageIf(Stage.stage_3);
+        // 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);
+        failStageIf(Stage.stage_4);
+        // backup complete
+        completeBackup(conn, backupInfo, backupManager, BackupType.FULL, conf);
+
+      } catch (Exception e) {
+
+        if(autoRestoreOnFailure) {
+          failBackup(conn, backupInfo, backupManager, e, "Unexpected BackupException : ",
+            BackupType.FULL, conf);
+        }
+        throw new IOException(e);
+      }
+    }
+
+  }
+
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Before
+  public void setUp() throws Exception {
+    if (setupIsDone) {
+      return;
+    }
+    if (secure) {
+      // set the always on security provider
+      UserProvider.setUserProviderForTesting(TEST_UTIL.getConfiguration(),
+          HadoopSecurityEnabledUserProviderForTesting.class);
+      // setup configuration
+      SecureTestUtil.enableSecurity(TEST_UTIL.getConfiguration());
+    }
+    String coproc = conf1.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY);
+    conf1.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, (coproc == null ? "" : coproc + ",") +
+        BackupObserver.class.getName());
+    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.startMiniCluster();
+
+    if (useSecondCluster) {
+      conf2 = HBaseConfiguration.create(conf1);
+      conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
+      TEST_UTIL2 = new HBaseTestingUtility(conf2);
+      TEST_UTIL2.setZkCluster(TEST_UTIL.getZkCluster());
+      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);
+    if (useSecondCluster) {
+      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);
+    setupIsDone = true;
+  }
+
+  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 tearDown() throws Exception {
+    try{
+      SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
+    } catch (Exception e) {
+    }
+    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
+    if (useSecondCluster) {
+      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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
new file mode 100644
index 0000000..40a01b9
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupCommandLineTool.java
new file mode 100644
index 0000000..08002fb
--- /dev/null
+++ b/hbase-backup/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);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
new file mode 100644
index 0000000..ebfc735
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
new file mode 100644
index 0000000..208e081
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
new file mode 100644
index 0000000..966f519
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteWithFailures.java
@@ -0,0 +1,194 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import static org.junit.Assert.assertNull;
+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.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.BackupSystemTable;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+/**
+ * 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
+ */
+@Category(LargeTests.class)
+public class TestBackupDeleteWithFailures extends TestBackupBase{
+
+  private static final Log LOG = LogFactory.getLog(TestBackupDeleteWithFailures.class);
+
+
+
+  public static enum Failure {
+    NO_FAILURES,
+    PRE_SNAPSHOT_FAILURE,
+    PRE_DELETE_SNAPSHOT_FAILURE,
+    POST_DELETE_SNAPSHOT_FAILURE
+  }
+
+  public static class MasterSnapshotObserver implements MasterObserver {
+
+
+    List<Failure> failures = new ArrayList<Failure>();
+
+    public void setFailures(Failure ... f) {
+      failures.clear();
+      for (int i = 0; i < f.length; i++) {
+        failures.add(f[i]);
+      }
+    }
+
+    @Override
+    public void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+        final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
+        throws IOException
+    {
+       if (failures.contains(Failure.PRE_SNAPSHOT_FAILURE)) {
+         throw new IOException ("preSnapshot");
+       }
+    }
+
+    @Override
+    public void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        SnapshotDescription snapshot) throws IOException {
+      if (failures.contains(Failure.PRE_DELETE_SNAPSHOT_FAILURE)) {
+        throw new IOException ("preDeleteSnapshot");
+      }
+    }
+
+    @Override
+    public void postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        SnapshotDescription snapshot) throws IOException {
+      if (failures.contains(Failure.POST_DELETE_SNAPSHOT_FAILURE)) {
+        throw new IOException ("postDeleteSnapshot");
+      }
+    }
+
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    conf1.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+      MasterSnapshotObserver.class.getName());
+    conf1.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
+    super.setUp();
+  }
+
+
+  private MasterSnapshotObserver getMasterSnapshotObserver() {
+    return (MasterSnapshotObserver)TEST_UTIL.getHBaseCluster().getMaster()
+      .getMasterCoprocessorHost().findCoprocessor(MasterSnapshotObserver.class.getName());
+  }
+
+  @Test
+  public void testBackupDeleteWithFailures() throws Exception
+  {
+     testBackupDeleteWithFailuresAfter(1, Failure.PRE_DELETE_SNAPSHOT_FAILURE);
+     testBackupDeleteWithFailuresAfter(0, Failure.POST_DELETE_SNAPSHOT_FAILURE);
+     testBackupDeleteWithFailuresAfter(1, Failure.PRE_SNAPSHOT_FAILURE);
+  }
+
+  private void testBackupDeleteWithFailuresAfter(int expected, Failure ...failures) throws Exception {
+    LOG.info("test repair backup delete on a single table with data and failures "+ failures[0]);
+    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));
+
+    Connection conn = TEST_UTIL.getConnection();
+    Admin admin = conn.getAdmin();
+    MasterSnapshotObserver observer = getMasterSnapshotObserver();
+
+    observer.setFailures(failures);
+    try {
+      getBackupAdmin().deleteBackups(backupIds);
+    } catch(IOException e) {
+      if(expected != 1) assertTrue(false);
+    }
+
+    // Verify that history length == expected after delete failure
+    assertTrue (table.getBackupHistory().size() == expected);
+
+    String[] ids = table.getListOfBackupIdsFromDeleteOperation();
+
+    // Verify that we still have delete record in backup system table
+    if(expected == 1) {
+      assertTrue(ids.length == 1);
+      assertTrue(ids[0].equals(backupId));
+    } else {
+      assertNull(ids);
+    }
+
+    // Now run repair command to repair "failed" delete operation
+    String[] args = new String[] {"repair"};
+
+    observer.setFailures(Failure.NO_FAILURES);
+
+    // Run repair
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    // Verify that history length == 0
+    assertTrue (table.getBackupHistory().size() == 0);
+    ids = table.getListOfBackupIdsFromDeleteOperation();
+
+    // Verify that we do not have delete record in backup system table
+    assertNull(ids);
+
+    table.close();
+    admin.close();
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
new file mode 100644
index 0000000..0672325
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/2dda3712/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.java
new file mode 100644
index 0000000..dfbe106
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupHFileCleaner.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 static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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.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.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+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.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+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;
+
+@Category({ MasterTests.class, SmallTests.class })
+public class TestBackupHFileCleaner {
+  private static final Log LOG = LogFactory.getLog(TestBackupHFileCleaner.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf = TEST_UTIL.getConfiguration();
+  private static TableName tableName = TableName.valueOf("backup.hfile.cleaner");
+  private static String famName = "fam";
+  static FileSystem fs = null;
+  Path root;
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf.setBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY, true);
+    TEST_UTIL.startMiniZKCluster();
+    TEST_UTIL.startMiniCluster(1);
+    fs = FileSystem.get(conf);
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setup() throws IOException {
+    root = TEST_UTIL.getDataTestDirOnTestFS();
+  }
+
+  @After
+  public void cleanup() {
+    try {
+      fs.delete(root, true);
+    } catch (IOException e) {
+      LOG.warn("Failed to delete files recursively from path " + root);
+    }
+  }
+
+  @Test
+  public void testGetDeletableFiles() throws IOException {
+    // 1. Create a file
+    Path file = new Path(root, "testIsFileDeletableWithNoHFileRefs");
+    fs.createNewFile(file);
+    // 2. Assert file is successfully created
+    assertTrue("Test file not created!", fs.exists(file));
+    BackupHFileCleaner cleaner = new BackupHFileCleaner();
+    cleaner.setConf(conf);
+    cleaner.setCheckForFullyBackedUpTables(false);
+    // 3. Assert that file as is should be deletable
+    List<FileStatus> stats = new ArrayList<>();
+    FileStatus stat = fs.getFileStatus(file);
+    stats.add(stat);
+    Iterable<FileStatus> deletable = cleaner.getDeletableFiles(stats);
+    deletable = cleaner.getDeletableFiles(stats);
+    boolean found = false;
+    for (FileStatus stat1 : deletable) {
+      if (stat.equals(stat1)) found = true;
+    }
+    assertTrue("Cleaner should allow to delete this file as there is no hfile reference "
+        + "for it.", found);
+
+    // 4. Add the file as bulk load
+    List<Path> list = new ArrayList<>(1);
+    list.add(file);
+    try (Connection conn = ConnectionFactory.createConnection(conf);
+        BackupSystemTable sysTbl = new BackupSystemTable(conn)) {
+      List<TableName> sTableList = new ArrayList<>();
+      sTableList.add(tableName);
+      Map<byte[], List<Path>>[] maps = new Map[1];
+      maps[0] = new HashMap<>();
+      maps[0].put(famName.getBytes(), list);
+      sysTbl.writeBulkLoadedFiles(sTableList, maps, "1");
+    }
+
+    // 5. Assert file should not be deletable
+    deletable = cleaner.getDeletableFiles(stats);
+    deletable = cleaner.getDeletableFiles(stats);
+    found = false;
+    for (FileStatus stat1 : deletable) {
+      if (stat.equals(stat1)) found = true;
+    }
+    assertFalse("Cleaner should not allow to delete this file as there is a hfile reference "
+        + "for it.", found);
+  }
+}
\ No newline at end of file


[29/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
new file mode 100644
index 0000000..0944ea2
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
@@ -0,0 +1,158 @@
+/**
+ * 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.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hadoop.hbase.shaded.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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
new file mode 100644
index 0000000..686d34b
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupRepair.java
@@ -0,0 +1,91 @@
+/**
+ * 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.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.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
+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 TestBackupRepair extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupRepair.class);
+
+
+  @Test
+  public void testFullBackupWithFailuresAndRestore() throws Exception {
+
+    autoRestoreOnFailure = false;
+
+    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
+      FullTableBackupClientForTest.class.getName());
+    int maxStage = Stage.values().length -1;
+    // Fail stage in loop between 0 and 4 inclusive
+    for (int stage = 0; stage < maxStage; stage++) {
+      LOG.info("Running stage " + stage);
+      runBackupAndFailAtStageWithRestore(stage);
+    }
+  }
+
+  public void runBackupAndFailAtStageWithRestore(int stage) throws Exception {
+
+    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
+    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);
+      assertFalse(ret == 0);
+
+      // Now run restore
+      args = new String[] {"repair"};
+
+      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();
+        assertFalse(checkSucceeded(backupId));
+      }
+      Set<TableName> tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR);
+      assertTrue(tables.size() == 0);
+    }
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
new file mode 100644
index 0000000..4e922a2
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
new file mode 100644
index 0000000..73d8d9f
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
new file mode 100644
index 0000000..5814d87
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
new file mode 100644
index 0000000..124d19f
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
new file mode 100644
index 0000000..4dc894b
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSetRestoreSet.java
new file mode 100644
index 0000000..6b007f9
--- /dev/null
+++ b/hbase-backup/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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
new file mode 100644
index 0000000..d18de88
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupWithFailures.java
@@ -0,0 +1,79 @@
+/**
+ * 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.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.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.TableBackupClient.Stage;
+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 TestFullBackupWithFailures extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestFullBackupWithFailures.class);
+
+  @Test
+  public void testFullBackupWithFailures() throws Exception {
+    conf1.set(TableBackupClient.BACKUP_CLIENT_IMPL_CLASS,
+      FullTableBackupClientForTest.class.getName());
+    int maxStage = Stage.values().length -1;
+    // Fail stages between 0 and 4 inclusive
+    for (int stage = 0; stage <= maxStage; stage++) {
+      LOG.info("Running stage " + stage);
+      runBackupAndFailAtStage(stage);
+    }
+  }
+
+  public void runBackupAndFailAtStage(int stage) throws Exception {
+
+    conf1.setInt(FullTableBackupClientForTest.BACKUP_TEST_MODE_STAGE, stage);
+    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);
+      assertFalse(ret == 0);
+      List<BackupInfo> backups = table.getBackupHistory();
+      int after = table.getBackupHistory().size();
+
+      assertTrue(after ==  before +1);
+      for (BackupInfo data : backups) {
+        String backupId = data.getBackupId();
+        assertFalse(checkSucceeded(backupId));
+      }
+      Set<TableName> tables = table.getIncrementalBackupTableSet(BACKUP_ROOT_DIR);
+      assertTrue(tables.size() == 0);
+    }
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
new file mode 100644
index 0000000..48a553f
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
new file mode 100644
index 0000000..73598f3
--- /dev/null
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
@@ -0,0 +1,209 @@
+/**
+ * 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.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 org.apache.hadoop.hbase.shaded.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();
+    try {
+      admin.splitRegion(name);
+    } catch (IOException e) {
+      //although split fail, this may not affect following check
+      //In old split without AM2, if region's best split key is not found,
+      //there are not exception thrown. But in current API, exception
+      //will be thrown.
+      LOG.debug("region is not splittable, because " + e);
+    }
+
+    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.getDescriptor());
+    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/37c65946/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
new file mode 100644
index 0000000..747c1dd
--- /dev/null
+++ b/hbase-backup/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 org.apache.hadoop.hbase.shaded.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();
+  }
+
+}


[06/36] hbase git commit: HBASE-17614: Move Backup/Restore into separate module (Vladimir Rodionov)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2dda3712/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
deleted file mode 100644
index 4dab046..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
+++ /dev/null
@@ -1,2051 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-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.CellComparator;
-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.NamespaceDescriptor;
-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.BackupType;
-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.SnapshotDescription;
-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;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * 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 {
-  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
-
-  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 TableName tableName;
-  /**
-   * Stores backup sessions (contexts)
-   */
-  final static byte[] SESSIONS_FAMILY = "session".getBytes();
-  /**
-   * Stores other meta
-   */
-  final static byte[] META_FAMILY = "meta".getBytes();
-  final static byte[] BULK_LOAD_FAMILY = "bulk".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 byte[] ACTIVE_SESSION_ROW = "activesession:".getBytes();
-  private final static byte[] ACTIVE_SESSION_COL = "c".getBytes();
-
-  private final static byte[] ACTIVE_SESSION_YES = "yes".getBytes();
-  private final static byte[] ACTIVE_SESSION_NO = "no".getBytes();
-
-  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 BULK_LOAD_PREFIX = "bulk:";
-  private final static byte[] BULK_LOAD_PREFIX_BYTES = BULK_LOAD_PREFIX.getBytes();
-  private final static byte[] DELETE_OP_ROW = "delete_op_row".getBytes();
-  private final static byte[] MERGE_OP_ROW = "merge_op_row".getBytes();
-
-  final static byte[] TBL_COL = Bytes.toBytes("tbl");
-  final static byte[] FAM_COL = Bytes.toBytes("fam");
-  final static byte[] PATH_COL = Bytes.toBytes("path");
-  final static byte[] STATE_COL = Bytes.toBytes("state");
-  // the two states a bulk loaded file can be
-  final static byte[] BL_PREPARE = Bytes.toBytes("R");
-  final static byte[] BL_COMMIT = Bytes.toBytes("D");
-
-  private final static String WALS_PREFIX = "wals:";
-  private final static String SET_KEY_PREFIX = "backupset:";
-
-  // separator between BULK_LOAD_PREFIX and ordinals
-  protected final static String BLK_LD_DELIM = ":";
-  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();) {
-
-      verifyNamespaceExists(admin);
-
-      if (!admin.tableExists(tableName)) {
-        HTableDescriptor backupHTD =
-            BackupSystemTable.getSystemTableDescriptor(connection.getConfiguration());
-        admin.createTable(backupHTD);
-      }
-      waitForSystemTable(admin);
-    }
-  }
-
-  private void verifyNamespaceExists(Admin admin) throws IOException {
-    String namespaceName = tableName.getNamespaceAsString();
-    NamespaceDescriptor ns = NamespaceDescriptor.create(namespaceName).build();
-    NamespaceDescriptor[] list = admin.listNamespaceDescriptors();
-    boolean exists = false;
-    for (NamespaceDescriptor nsd : list) {
-      if (nsd.getName().equals(ns.getName())) {
-        exists = true;
-        break;
-      }
-    }
-    if (!exists) {
-      admin.createNamespace(ns);
-    }
-  }
-
-  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);
-    }
-  }
-
-  /*
-   * @param backupId the backup Id
-   * @return Map of rows to path of bulk loaded hfile
-   */
-  Map<byte[], String> readBulkLoadedFiles(String backupId) throws IOException {
-    Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId);
-    try (Table table = connection.getTable(tableName);
-        ResultScanner scanner = table.getScanner(scan)) {
-      Result res = null;
-      Map<byte[], String> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        byte[] row = CellUtil.cloneRow(res.listCells().get(0));
-        for (Cell cell : res.listCells()) {
-          if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
-            BackupSystemTable.PATH_COL.length) == 0) {
-            map.put(row, Bytes.toString(CellUtil.cloneValue(cell)));
-          }
-        }
-      }
-      return map;
-    }
-  }
-
-  /*
-   * Used during restore
-   * @param backupId the backup Id
-   * @param sTableList List of tables
-   * @return array of Map of family to List of Paths
-   */
-  public Map<byte[], List<Path>>[] readBulkLoadedFiles(String backupId, List<TableName> sTableList)
-      throws IOException {
-    Scan scan = BackupSystemTable.createScanForBulkLoadedFiles(backupId);
-    Map<byte[], List<Path>>[] mapForSrc = new Map[sTableList == null ? 1 : sTableList.size()];
-    try (Table table = connection.getTable(tableName);
-        ResultScanner scanner = table.getScanner(scan)) {
-      Result res = null;
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        TableName tbl = null;
-        byte[] fam = null;
-        String path = null;
-        for (Cell cell : res.listCells()) {
-          if (CellComparator.compareQualifiers(cell, BackupSystemTable.TBL_COL, 0,
-            BackupSystemTable.TBL_COL.length) == 0) {
-            tbl = TableName.valueOf(CellUtil.cloneValue(cell));
-          } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
-            BackupSystemTable.FAM_COL.length) == 0) {
-            fam = CellUtil.cloneValue(cell);
-          } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
-            BackupSystemTable.PATH_COL.length) == 0) {
-            path = Bytes.toString(CellUtil.cloneValue(cell));
-          }
-        }
-        int srcIdx = IncrementalTableBackupClient.getIndex(tbl, sTableList);
-        if (srcIdx == -1) {
-          // the table is not among the query
-          continue;
-        }
-        if (mapForSrc[srcIdx] == null) {
-          mapForSrc[srcIdx] = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-        }
-        List<Path> files;
-        if (!mapForSrc[srcIdx].containsKey(fam)) {
-          files = new ArrayList<Path>();
-          mapForSrc[srcIdx].put(fam, files);
-        } else {
-          files = mapForSrc[srcIdx].get(fam);
-        }
-        files.add(new Path(path));
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("found bulk loaded file : " + tbl + " " + Bytes.toString(fam) + " " + path);
-        }
-      }
-      ;
-      return mapForSrc;
-    }
-  }
-
-  /*
-   * @param map Map of row keys to path of bulk loaded hfile
-   */
-  void deleteBulkLoadedFiles(Map<byte[], String> map) throws IOException {
-    try (Table table = connection.getTable(tableName)) {
-      List<Delete> dels = new ArrayList<>();
-      for (byte[] row : map.keySet()) {
-        dels.add(new Delete(row).addFamily(BackupSystemTable.META_FAMILY));
-      }
-      table.delete(dels);
-    }
-  }
-
-  /**
-   * 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);
-    }
-  }
-
-  /*
-   * For postBulkLoadHFile() hook.
-   * @param tabName table name
-   * @param region the region receiving hfile
-   * @param finalPaths family and associated hfiles
-   */
-  public void writePathsPostBulkLoad(TableName tabName, byte[] region,
-      Map<byte[], List<Path>> finalPaths) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + finalPaths.size()
-          + " entries");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      List<Put> puts = BackupSystemTable.createPutForCommittedBulkload(tabName, region, finalPaths);
-      table.put(puts);
-      LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
-    }
-  }
-
-  /*
-   * For preCommitStoreFile() hook
-   * @param tabName table name
-   * @param region the region receiving hfile
-   * @param family column family
-   * @param pairs list of paths for hfiles
-   */
-  public void writeFilesForBulkLoadPreCommit(TableName tabName, byte[] region, final byte[] family,
-      final List<Pair<Path, Path>> pairs) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write bulk load descriptor to backup " + tabName + " with " + pairs.size()
-          + " entries");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      List<Put> puts =
-          BackupSystemTable.createPutForPreparedBulkload(tabName, region, family, pairs);
-      table.put(puts);
-      LOG.debug("written " + puts.size() + " rows for bulk load of " + tabName);
-    }
-  }
-
-  /*
-   * Removes rows recording bulk loaded hfiles from backup table
-   * @param lst list of table names
-   * @param rows the rows to be deleted
-   */
-  public void removeBulkLoadedRows(List<TableName> lst, List<byte[]> rows) throws IOException {
-    try (Table table = connection.getTable(tableName)) {
-      List<Delete> lstDels = new ArrayList<>();
-      for (byte[] row : rows) {
-        Delete del = new Delete(row);
-        lstDels.add(del);
-        LOG.debug("orig deleting the row: " + Bytes.toString(row));
-      }
-      table.delete(lstDels);
-      LOG.debug("deleted " + rows.size() + " original bulkload rows for " + lst.size() + " tables");
-    }
-  }
-
-  /*
-   * Reads the rows from backup table recording bulk loaded hfiles
-   * @param tableList list of table names
-   * @return The keys of the Map are table, region and column family. Value of the map reflects
-   * whether the hfile was recorded by preCommitStoreFile hook (true)
-   */
-  public Pair<Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>>, List<byte[]>>
-      readBulkloadRows(List<TableName> tableList) throws IOException {
-    Map<TableName, Map<String, Map<String, List<Pair<String, Boolean>>>>> map = new HashMap<>();
-    List<byte[]> rows = new ArrayList<>();
-    for (TableName tTable : tableList) {
-      Scan scan = BackupSystemTable.createScanForOrigBulkLoadedFiles(tTable);
-      Map<String, Map<String, List<Pair<String, Boolean>>>> tblMap = map.get(tTable);
-      try (Table table = connection.getTable(tableName);
-          ResultScanner scanner = table.getScanner(scan)) {
-        Result res = null;
-        while ((res = scanner.next()) != null) {
-          res.advance();
-          String fam = null;
-          String path = null;
-          boolean raw = false;
-          byte[] row = null;
-          String region = null;
-          for (Cell cell : res.listCells()) {
-            row = CellUtil.cloneRow(cell);
-            rows.add(row);
-            String rowStr = Bytes.toString(row);
-            region = BackupSystemTable.getRegionNameFromOrigBulkLoadRow(rowStr);
-            if (CellComparator.compareQualifiers(cell, BackupSystemTable.FAM_COL, 0,
-              BackupSystemTable.FAM_COL.length) == 0) {
-              fam = Bytes.toString(CellUtil.cloneValue(cell));
-            } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.PATH_COL, 0,
-              BackupSystemTable.PATH_COL.length) == 0) {
-              path = Bytes.toString(CellUtil.cloneValue(cell));
-            } else if (CellComparator.compareQualifiers(cell, BackupSystemTable.STATE_COL, 0,
-              BackupSystemTable.STATE_COL.length) == 0) {
-              byte[] state = CellUtil.cloneValue(cell);
-              if (Bytes.equals(BackupSystemTable.BL_PREPARE, state)) {
-                raw = true;
-              } else raw = false;
-            }
-          }
-          if (map.get(tTable) == null) {
-            map.put(tTable, new HashMap<String, Map<String, List<Pair<String, Boolean>>>>());
-            tblMap = map.get(tTable);
-          }
-          if (tblMap.get(region) == null) {
-            tblMap.put(region, new HashMap<String, List<Pair<String, Boolean>>>());
-          }
-          Map<String, List<Pair<String, Boolean>>> famMap = tblMap.get(region);
-          if (famMap.get(fam) == null) {
-            famMap.put(fam, new ArrayList<Pair<String, Boolean>>());
-          }
-          famMap.get(fam).add(new Pair<>(path, raw));
-          LOG.debug("found orig " + path + " for " + fam + " of table " + region);
-        }
-      }
-    }
-    return new Pair<>(map, rows);
-  }
-
-  /*
-   * @param sTableList List of tables
-   * @param maps array of Map of family to List of Paths
-   * @param backupId the backup Id
-   */
-  public void writeBulkLoadedFiles(List<TableName> sTableList, Map<byte[], List<Path>>[] maps,
-      String backupId) throws IOException {
-    try (Table table = connection.getTable(tableName)) {
-      long ts = EnvironmentEdgeManager.currentTime();
-      int cnt = 0;
-      List<Put> puts = new ArrayList<>();
-      for (int idx = 0; idx < maps.length; idx++) {
-        Map<byte[], List<Path>> map = maps[idx];
-        TableName tn = sTableList.get(idx);
-        if (map == null) continue;
-        for (Map.Entry<byte[], List<Path>> entry : map.entrySet()) {
-          byte[] fam = entry.getKey();
-          List<Path> paths = entry.getValue();
-          for (Path p : paths) {
-            Put put =
-                BackupSystemTable.createPutForBulkLoadedFile(tn, fam, p.toString(), backupId, ts,
-                  cnt++);
-            puts.add(put);
-          }
-        }
-      }
-      if (!puts.isEmpty()) {
-        table.put(puts);
-      }
-    }
-  }
-
-  /**
-   * 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);
-    }
-  }
-
-  /**
-   * Exclusive operations are:
-   * create, delete, merge
-   * @throws IOException
-   */
-  public void startBackupExclusiveOperation() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Start new backup exclusive operation");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Put put = createPutForStartBackupSession();
-      // First try to put if row does not exist
-      if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL, null, put)) {
-        // Row exists, try to put if value == ACTIVE_SESSION_NO
-        if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
-          ACTIVE_SESSION_NO, put)) {
-          throw new IOException("There is an active backup exclusive operation");
-        }
-      }
-    }
-  }
-
-  private Put createPutForStartBackupSession() {
-    Put put = new Put(ACTIVE_SESSION_ROW);
-    put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_YES);
-    return put;
-  }
-
-  public void finishBackupExclusiveOperation() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Finish backup exclusive operation");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Put put = createPutForStopBackupSession();
-      if (!table.checkAndPut(ACTIVE_SESSION_ROW, SESSIONS_FAMILY, ACTIVE_SESSION_COL,
-        ACTIVE_SESSION_YES, put)) {
-        throw new IOException("There is no active backup exclusive operation");
-      }
-    }
-  }
-
-  private Put createPutForStopBackupSession() {
-    Put put = new Put(ACTIVE_SESSION_ROW);
-    put.addColumn(SESSIONS_FAMILY, ACTIVE_SESSION_COL, ACTIVE_SESSION_NO);
-    return 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, if n== -1 - max number
-   *        is ignored
-   * @return list of records
-   * @throws IOException
-   */
-  public List<BackupInfo> getHistory(int n) throws IOException {
-
-    List<BackupInfo> history = getBackupHistory();
-    if (n == -1 || 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, if n == -1 , then max number
-   *        is ignored
-   * @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 (n >= 0 && 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;
-
-  }
-
-  /*
-   * Retrieve TableName's for completed backup of given type
-   * @param type backup type
-   * @return List of table names
-   */
-  public List<TableName> getTablesForBackupType(BackupType type) throws IOException {
-    Set<TableName> names = new HashSet<>();
-    List<BackupInfo> infos = getBackupHistory(true);
-    for (BackupInfo info : infos) {
-      if (info.getType() != type) continue;
-      names.addAll(info.getTableNames());
-    }
-    return new ArrayList(names);
-  }
-
-  /**
-   * 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
-   * TODO: multiple backup destination support
-   */
-  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();
-  }
-
-  public static String getSnapshotName(Configuration conf) {
-    return "snapshot_" + getTableNameAsString(conf).replace(":", "_");
-  }
-
-  /**
-   * 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's for bulk load resulting from running LoadIncrementalHFiles
-   */
-  static List<Put> createPutForCommittedBulkload(TableName table, byte[] region,
-      Map<byte[], List<Path>> finalPaths) {
-    List<Put> puts = new ArrayList<>();
-    for (Map.Entry<byte[], List<Path>> entry : finalPaths.entrySet()) {
-      for (Path path : entry.getValue()) {
-        String file = path.toString();
-        int lastSlash = file.lastIndexOf("/");
-        String filename = file.substring(lastSlash + 1);
-        Put put =
-            new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM,
-              Bytes.toString(region), BLK_LD_DELIM, filename));
-        put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
-        put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, entry.getKey());
-        put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
-        put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_COMMIT);
-        puts.add(put);
-        LOG.debug("writing done bulk path " + file + " for " + table + " " + Bytes.toString(region));
-      }
-    }
-    return puts;
-  }
-
-  public static void snapshot(Connection conn) throws IOException {
-
-    try (Admin admin = conn.getAdmin();) {
-      Configuration conf = conn.getConfiguration();
-      admin.snapshot(BackupSystemTable.getSnapshotName(conf), BackupSystemTable.getTableName(conf));
-    }
-  }
-
-  public static void restoreFromSnapshot(Connection conn) throws IOException {
-
-    Configuration conf = conn.getConfiguration();
-    LOG.debug("Restoring " + BackupSystemTable.getTableNameAsString(conf) + " from snapshot");
-    try (Admin admin = conn.getAdmin();) {
-      String snapshotName = BackupSystemTable.getSnapshotName(conf);
-      if (snapshotExists(admin, snapshotName)) {
-        admin.disableTable(BackupSystemTable.getTableName(conf));
-        admin.restoreSnapshot(snapshotName);
-        admin.enableTable(BackupSystemTable.getTableName(conf));
-        LOG.debug("Done restoring backup system table");
-      } else {
-        // Snapshot does not exists, i.e completeBackup failed after
-        // deleting backup system table snapshot
-        // In this case we log WARN and proceed
-        LOG.warn("Could not restore backup system table. Snapshot " + snapshotName
-            + " does not exists.");
-      }
-    }
-  }
-
-  protected static boolean snapshotExists(Admin admin, String snapshotName) throws IOException {
-
-    List<SnapshotDescription> list = admin.listSnapshots();
-    for (SnapshotDescription desc : list) {
-      if (desc.getName().equals(snapshotName)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public static boolean snapshotExists(Connection conn) throws IOException {
-    return snapshotExists(conn.getAdmin(), getSnapshotName(conn.getConfiguration()));
-  }
-
-  public static void deleteSnapshot(Connection conn) throws IOException {
-
-    Configuration conf = conn.getConfiguration();
-    LOG.debug("Deleting " + BackupSystemTable.getSnapshotName(conf) + " from the system");
-    try (Admin admin = conn.getAdmin();) {
-      String snapshotName = BackupSystemTable.getSnapshotName(conf);
-      if (snapshotExists(admin, snapshotName)) {
-        admin.deleteSnapshot(snapshotName);
-        LOG.debug("Done deleting backup system table snapshot");
-      } else {
-        LOG.error("Snapshot " + snapshotName + " does not exists");
-      }
-    }
-  }
-
-  /*
-   * Creates Put's for bulk load resulting from running LoadIncrementalHFiles
-   */
-  static List<Put> createPutForPreparedBulkload(TableName table, byte[] region,
-      final byte[] family, final List<Pair<Path, Path>> pairs) {
-    List<Put> puts = new ArrayList<>();
-    for (Pair<Path, Path> pair : pairs) {
-      Path path = pair.getSecond();
-      String file = path.toString();
-      int lastSlash = file.lastIndexOf("/");
-      String filename = file.substring(lastSlash + 1);
-      Put put =
-          new Put(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM, Bytes.toString(region),
-            BLK_LD_DELIM, filename));
-      put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, table.getName());
-      put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, family);
-      put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, file.getBytes());
-      put.addColumn(BackupSystemTable.META_FAMILY, STATE_COL, BL_PREPARE);
-      puts.add(put);
-      LOG.debug("writing raw bulk path " + file + " for " + table + " " + Bytes.toString(region));
-    }
-    return puts;
-  }
-
-  public static List<Delete> createDeleteForOrigBulkLoad(List<TableName> lst) {
-    List<Delete> lstDels = new ArrayList<>();
-    for (TableName table : lst) {
-      Delete del = new Delete(rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM));
-      del.addFamily(BackupSystemTable.META_FAMILY);
-      lstDels.add(del);
-    }
-    return lstDels;
-  }
-
-  private Put createPutForDeleteOperation(String[] backupIdList) {
-
-    byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ","));
-    Put put = new Put(DELETE_OP_ROW);
-    put.addColumn(META_FAMILY, FAM_COL, value);
-    return put;
-  }
-
-  private Delete createDeleteForBackupDeleteOperation() {
-
-    Delete delete = new Delete(DELETE_OP_ROW);
-    delete.addFamily(META_FAMILY);
-    return delete;
-  }
-
-  private Get createGetForDeleteOperation() {
-
-    Get get = new Get(DELETE_OP_ROW);
-    get.addFamily(META_FAMILY);
-    return get;
-  }
-
-  public void startDeleteOperation(String[] backupIdList) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Start delete operation for backups: " + StringUtils.join(backupIdList));
-    }
-    Put put = createPutForDeleteOperation(backupIdList);
-    try (Table table = connection.getTable(tableName)) {
-      table.put(put);
-    }
-  }
-
-  public void finishDeleteOperation() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Finsih delete operation for backup ids ");
-    }
-    Delete delete = createDeleteForBackupDeleteOperation();
-    try (Table table = connection.getTable(tableName)) {
-      table.delete(delete);
-    }
-  }
-
-  public String[] getListOfBackupIdsFromDeleteOperation() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Get delete operation for backup ids ");
-    }
-    Get get = createGetForDeleteOperation();
-    try (Table table = connection.getTable(tableName)) {
-      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).split(",");
-    }
-  }
-
-  private Put createPutForMergeOperation(String[] backupIdList) {
-
-    byte[] value = Bytes.toBytes(StringUtils.join(backupIdList, ","));
-    Put put = new Put(MERGE_OP_ROW);
-    put.addColumn(META_FAMILY, FAM_COL, value);
-    return put;
-  }
-
-  public boolean isMergeInProgress() throws IOException {
-    Get get = new Get(MERGE_OP_ROW);
-    try (Table table = connection.getTable(tableName)) {
-      Result res = table.get(get);
-      if (res.isEmpty()) {
-        return false;
-      }
-      return true;
-    }
-  }
-
-  private Put createPutForUpdateTablesForMerge(List<TableName> tables) {
-
-    byte[] value = Bytes.toBytes(StringUtils.join(tables, ","));
-    Put put = new Put(MERGE_OP_ROW);
-    put.addColumn(META_FAMILY, PATH_COL, value);
-    return put;
-  }
-
-  private Delete createDeleteForBackupMergeOperation() {
-
-    Delete delete = new Delete(MERGE_OP_ROW);
-    delete.addFamily(META_FAMILY);
-    return delete;
-  }
-
-  private Get createGetForMergeOperation() {
-
-    Get get = new Get(MERGE_OP_ROW);
-    get.addFamily(META_FAMILY);
-    return get;
-  }
-
-  public void startMergeOperation(String[] backupIdList) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Start merge operation for backups: " + StringUtils.join(backupIdList));
-    }
-    Put put = createPutForMergeOperation(backupIdList);
-    try (Table table = connection.getTable(tableName)) {
-      table.put(put);
-    }
-  }
-
-  public void updateProcessedTablesForMerge(List<TableName> tables) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Update tables for merge : " + StringUtils.join(tables, ","));
-    }
-    Put put = createPutForUpdateTablesForMerge(tables);
-    try (Table table = connection.getTable(tableName)) {
-      table.put(put);
-    }
-  }
-
-  public void finishMergeOperation() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Finsih merge operation for backup ids ");
-    }
-    Delete delete = createDeleteForBackupMergeOperation();
-    try (Table table = connection.getTable(tableName)) {
-      table.delete(delete);
-    }
-  }
-
-  public String[] getListOfBackupIdsFromMergeOperation() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Get backup ids for merge operation");
-    }
-    Get get = createGetForMergeOperation();
-    try (Table table = connection.getTable(tableName)) {
-      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).split(",");
-    }
-  }
-
-  static Scan createScanForOrigBulkLoadedFiles(TableName table) throws IOException {
-    Scan scan = new Scan();
-    byte[] startRow = rowkey(BULK_LOAD_PREFIX, table.toString(), BLK_LD_DELIM);
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.withStartRow(startRow);
-    scan.withStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.META_FAMILY);
-    scan.setMaxVersions(1);
-    return scan;
-  }
-
-  static String getTableNameFromOrigBulkLoadRow(String rowStr) {
-    String[] parts = rowStr.split(BLK_LD_DELIM);
-    return parts[1];
-  }
-
-  static String getRegionNameFromOrigBulkLoadRow(String rowStr) {
-    // format is bulk : namespace : table : region : file
-    String[] parts = rowStr.split(BLK_LD_DELIM);
-    int idx = 3;
-    if (parts.length == 4) {
-      // the table is in default namespace
-      idx = 2;
-    }
-    LOG.debug("bulk row string " + rowStr + " region " + parts[idx]);
-    return parts[idx];
-  }
-
-  /*
-   * Used to query bulk loaded hfiles which have been copied by incremental backup
-   * @param backupId the backup Id. It can be null when querying for all tables
-   * @return the Scan object
-   */
-  static Scan createScanForBulkLoadedFiles(String backupId) throws IOException {
-    Scan scan = new Scan();
-    byte[] startRow =
-        backupId == null ? BULK_LOAD_PREFIX_BYTES : rowkey(BULK_LOAD_PREFIX, backupId
-            + BLK_LD_DELIM);
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    // scan.setTimeRange(lower, Long.MAX_VALUE);
-    scan.addFamily(BackupSystemTable.META_FAMILY);
-    scan.setMaxVersions(1);
-    return scan;
-  }
-
-  static Put createPutForBulkLoadedFile(TableName tn, byte[] fam, String p, String backupId,
-      long ts, int idx) {
-    Put put = new Put(rowkey(BULK_LOAD_PREFIX, backupId + BLK_LD_DELIM + ts + BLK_LD_DELIM + idx));
-    put.addColumn(BackupSystemTable.META_FAMILY, TBL_COL, tn.getName());
-    put.addColumn(BackupSystemTable.META_FAMILY, FAM_COL, fam);
-    put.addColumn(BackupSystemTable.META_FAMILY, PATH_COL, p.getBytes());
-    return put;
-  }
-
-  /**
-   * 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 static 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/2dda3712/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
deleted file mode 100644
index e323e96..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
+++ /dev/null
@@ -1,224 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_ATTEMPTS_PAUSE_MS_KEY;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_MAX_ATTEMPTS_KEY;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS;
-import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.DEFAULT_BACKUP_MAX_ATTEMPTS;
-
-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() {
-  }
-
-  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
-   */
-  protected 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();
-
-        snapshotTable(admin, tableName, snapshotName);
-        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);
-    }
-
-  }
-
-
-  protected void snapshotTable(Admin admin, TableName tableName, String snapshotName)
-      throws IOException {
-
-    int maxAttempts =
-        conf.getInt(BACKUP_MAX_ATTEMPTS_KEY, DEFAULT_BACKUP_MAX_ATTEMPTS);
-    int pause =
-        conf.getInt(BACKUP_ATTEMPTS_PAUSE_MS_KEY, DEFAULT_BACKUP_ATTEMPTS_PAUSE_MS);
-    int attempts = 0;
-
-    while (attempts++ < maxAttempts) {
-      try {
-        admin.snapshot(snapshotName, tableName);
-        return;
-      } catch (IOException ee) {
-        LOG.warn("Snapshot attempt " + attempts + " failed for table " + tableName
-            + ", sleeping for " + pause + "ms", ee);
-        if (attempts < maxAttempts) {
-          try {
-            Thread.sleep(pause);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            break;
-          }
-        }
-      }
-    }
-    throw new IOException("Failed to snapshot table "+ tableName);
-  }
-}