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

Reply via email to