Repository: hbase
Updated Branches:
  refs/heads/HBASE-7912 0ca86556e -> c61f7c342


http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
index 02fcbba..736b8a5 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALInputFormat.java
@@ -27,22 +27,24 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WAL.Reader;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * Simple {@link InputFormat} for {@link org.apache.hadoop.hbase.wal.WAL} 
files.
@@ -231,21 +233,31 @@ public class WALInputFormat extends InputFormat<WALKey, 
WALEdit> {
   List<InputSplit> getSplits(final JobContext context, final String startKey, 
final String endKey)
       throws IOException, InterruptedException {
     Configuration conf = context.getConfiguration();
-    Path inputDir = new 
Path(conf.get("mapreduce.input.fileinputformat.inputdir"));
+    
+    Path[] inputPaths = getInputPaths(conf);
 
     long startTime = conf.getLong(startKey, Long.MIN_VALUE);
     long endTime = conf.getLong(endKey, Long.MAX_VALUE);
 
-    FileSystem fs = inputDir.getFileSystem(conf);
-    List<FileStatus> files = getFiles(fs, inputDir, startTime, endTime);
-
-    List<InputSplit> splits = new ArrayList<InputSplit>(files.size());
-    for (FileStatus file : files) {
+    FileSystem fs = FileSystem.get(conf);
+    
+    List<FileStatus> allFiles = new ArrayList<FileStatus>();
+    for(Path inputPath: inputPaths){
+      List<FileStatus> files = getFiles(fs, inputPath, startTime, endTime);
+      allFiles.addAll(files);
+    }
+    List<InputSplit> splits = new ArrayList<InputSplit>(allFiles.size());
+    for (FileStatus file : allFiles) {
       splits.add(new WALSplit(file.getPath().toString(), file.getLen(), 
startTime, endTime));
     }
     return splits;
   }
 
+  private Path[] getInputPaths(Configuration conf) {
+    String inpDirs = conf.get("mapreduce.input.fileinputformat.inputdir");
+    return StringUtils.stringToPath(inpDirs.split(","));
+  }
+
   private List<FileStatus> getFiles(FileSystem fs, Path dir, long startTime, 
long endTime)
       throws IOException {
     List<FileStatus> result = new ArrayList<FileStatus>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index 2ceeda5..4cdbad3 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -47,6 +47,8 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
@@ -69,9 +71,9 @@ import org.apache.hadoop.util.ToolRunner;
 public class WALPlayer extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(WALPlayer.class);
   final static String NAME = "WALPlayer";
-  final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
-  final static String TABLES_KEY = "wal.input.tables";
-  final static String TABLE_MAP_KEY = "wal.input.tablesmap";
+  public final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
+  public final static String TABLES_KEY = "wal.input.tables";
+  public final static String TABLE_MAP_KEY = "wal.input.tablesmap";
 
   // This relies on Hadoop Configuration to handle warning about deprecated 
configs and
   // to set the correct non-deprecated configs when an old one shows up.
@@ -139,7 +141,8 @@ public class WALPlayer extends Configured implements Tool {
   protected static class WALMapper
     extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
     private Map<TableName, TableName> tables = new TreeMap<TableName, 
TableName>();
-
+    
+    
     @Override
     public void map(WALKey key, WALEdit value, Context context)
     throws IOException {
@@ -152,6 +155,7 @@ public class WALPlayer extends Configured implements Tool {
           Put put = null;
           Delete del = null;
           Cell lastCell = null;
+                    
           for (Cell cell : value.getCells()) {
             // filtering WAL meta entries
             if (WALEdit.isMetaEditFamily(cell)) {
@@ -210,6 +214,13 @@ public class WALPlayer extends Configured implements Tool {
     }
 
     @Override
+    protected void
+        cleanup(Mapper<WALKey, WALEdit, ImmutableBytesWritable, 
Mutation>.Context context)
+            throws IOException, InterruptedException {
+      super.cleanup(context);
+    }
+
+    @Override
     public void setup(Context context) throws IOException {
       String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY);
       String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY);
@@ -261,7 +272,7 @@ public class WALPlayer extends Configured implements Tool {
     Configuration conf = getConf();
     setupTime(conf, HLogInputFormat.START_TIME_KEY);
     setupTime(conf, HLogInputFormat.END_TIME_KEY);
-    Path inputDir = new Path(args[0]);
+    String inputDirs = args[0];
     String[] tables = args[1].split(",");
     String[] tableMap;
     if (args.length > 2) {
@@ -275,13 +286,18 @@ public class WALPlayer extends Configured implements Tool 
{
     }
     conf.setStrings(TABLES_KEY, tables);
     conf.setStrings(TABLE_MAP_KEY, tableMap);
-    Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + 
inputDir));
+    Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + 
System.currentTimeMillis()));
     job.setJarByClass(WALPlayer.class);
-    FileInputFormat.setInputPaths(job, inputDir);
+        
+    FileInputFormat.addInputPaths(job, inputDirs);    
+    
     job.setInputFormatClass(WALInputFormat.class);
     job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+    
     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
     if (hfileOutPath != null) {
+      LOG.debug("add incremental job :"+hfileOutPath);
+
       // the bulk HFile case
       if (tables.length != 1) {
         throw new IOException("Exactly one table must be specified for the 
bulk export option");
@@ -297,6 +313,8 @@ public class WALPlayer extends Configured implements Tool {
           RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
         HFileOutputFormat2.configureIncrementalLoad(job, 
table.getTableDescriptor(), regionLocator);
       }
+      LOG.debug("success configuring load incremental job");
+      
       TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
           com.google.common.base.Preconditions.class);
     } else {
@@ -311,6 +329,7 @@ public class WALPlayer extends Configured implements Tool {
     return job;
   }
 
+ 
   /**
    * Print usage
    * @param errorMsg Error message.  Can be null.
@@ -360,6 +379,7 @@ public class WALPlayer extends Configured implements Tool {
       System.exit(-1);
     }
     Job job = createSubmittableJob(args);
-    return job.waitForCompletion(true) ? 0 : 1;
+    int result =job.waitForCompletion(true) ? 0 : 1;
+    return result; 
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 0d5ed1f..30da779 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -81,9 +81,8 @@ import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.backup.impl.BackupManager;
 import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTableHelper;
-import org.apache.hadoop.hbase.backup.impl.FullTableBackupProcedure;
-import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupProcedure;
+import org.apache.hadoop.hbase.backup.master.FullTableBackupProcedure;
+import org.apache.hadoop.hbase.backup.master.IncrementalTableBackupProcedure;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@@ -2618,10 +2617,14 @@ public class HMaster extends HRegionServer implements 
MasterServices {
         List<TableName> tableList, final String targetRootDir, final int 
workers,
         final long bandwidth) throws IOException {
     long procId;
-    String backupId = BackupRestoreConstants.BACKUPID_PREFIX + 
EnvironmentEdgeManager.currentTime();
+    String backupId = BackupRestoreConstants.BACKUPID_PREFIX + 
+        EnvironmentEdgeManager.currentTime();
     if (type == BackupType.INCREMENTAL) {
-      Set<TableName> incrTableSet =
-          
BackupSystemTableHelper.getIncrementalBackupTableSet(clusterConnection);
+      Set<TableName> incrTableSet = null;
+      try (BackupSystemTable table = new BackupSystemTable(getConnection())) {
+        incrTableSet = table.getIncrementalBackupTableSet(targetRootDir);
+      }
+         
       if (incrTableSet.isEmpty()) {
         LOG.warn("Incremental backup table set contains no table.\n"
             + "Use 'backup create full' or 'backup stop' to \n "

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 0ce8ee4..81e56b2 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
@@ -521,7 +522,7 @@ public class HRegionServer extends HasThread implements
     FSUtils.setupShortCircuitRead(this.conf);
     // Disable usage of meta replicas in the regionserver
     this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
-
+    BackupManager.decorateRSConfiguration(conf);
     // Config'ed params
     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index 1416523..8358e47 100644
--- 
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
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.backup;
 
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -25,18 +26,19 @@ 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.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.TableName;
-import org.apache.hadoop.hbase.backup.impl.BackupContext.BackupState;
-import org.apache.hadoop.hbase.backup.impl.BackupContext;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.impl.BackupUtil;
-import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
-import 
org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -48,7 +50,6 @@ import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import com.google.common.collect.Lists;
 
 /**
  * This class is only a base for other integration-level backup tests. Do not 
add tests here.
@@ -64,17 +65,17 @@ public class TestBackupBase {
 
   protected static HBaseTestingUtility TEST_UTIL;
   protected static HBaseTestingUtility TEST_UTIL2;
-  protected static TableName table1;
-  protected static TableName table2;
-  protected static TableName table3;
-  protected static TableName table4;
+  protected static TableName table1 = TableName.valueOf("table1");
+  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("table1_restore");
   protected static TableName table2_restore = 
TableName.valueOf("table2_restore");
   protected static TableName table3_restore = 
TableName.valueOf("table3_restore");
   protected static TableName table4_restore = 
TableName.valueOf("table4_restore");
 
-  protected static final int NB_ROWS_IN_BATCH = 100;
+  protected static final int NB_ROWS_IN_BATCH = 999;
   protected static final byte[] qualName = Bytes.toBytes("q1");
   protected static final byte[] famName = Bytes.toBytes("f");
 
@@ -91,15 +92,13 @@ public class TestBackupBase {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
-    TEST_UTIL.getConfiguration().set("hbase.procedure.regionserver.classes",
-      LogRollRegionServerProcedureManager.class.getName());
-    TEST_UTIL.getConfiguration().set("hbase.procedure.master.classes",
-      LogRollMasterProcedureManager.class.getName());
-    TEST_UTIL.getConfiguration().set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
+    conf1 = TEST_UTIL.getConfiguration();
+    conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
+    // Set MultiWAL (with 2 default WAL files per RS)
+    //conf1.set(WAL_PROVIDER, "multiwal");
     TEST_UTIL.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
 
-    conf1 = TEST_UTIL.getConfiguration();
     conf2 = HBaseConfiguration.create(conf1);
     conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
     TEST_UTIL2 = new HBaseTestingUtility(conf2);
@@ -113,9 +112,21 @@ public class TestBackupBase {
     LOG.info("ROOTDIR " + BACKUP_ROOT_DIR);
     BACKUP_REMOTE_ROOT_DIR = TEST_UTIL2.getConfiguration().get("fs.defaultFS") 
+ "/backupUT";
     LOG.info("REMOTE ROOTDIR " + BACKUP_REMOTE_ROOT_DIR);
-
+    waitForSystemTable();
     createTables();
   }
+  
+  static void waitForSystemTable() throws Exception
+  {
+    try(Admin admin = TEST_UTIL.getAdmin();) {
+      while (!admin.tableExists(BackupSystemTable.getTableName()) 
+          || !admin.isTableAvailable(BackupSystemTable.getTableName())) {
+        Thread.sleep(1000);
+      }      
+    }
+    LOG.debug("backup table exists and available");
+
+  }
 
   /**
    * @throws java.lang.Exception
@@ -155,6 +166,10 @@ public class TestBackupBase {
     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(HTable table) throws Exception {
 
     Put p; // 100 + 1 row to t1_syncup
@@ -169,7 +184,6 @@ public class TestBackupBase {
 
     long tid = System.currentTimeMillis();
     table1 = TableName.valueOf("test-" + tid);
-    BackupSystemTable backupTable = new 
BackupSystemTable(TEST_UTIL.getConnection());
     HBaseAdmin ha = TEST_UTIL.getHBaseAdmin();
     HTableDescriptor desc = new HTableDescriptor(table1);
     HColumnDescriptor fam = new HColumnDescriptor(famName);
@@ -197,26 +211,28 @@ public class TestBackupBase {
   }
 
   protected boolean checkSucceeded(String backupId) throws IOException {
-    BackupContext status = getBackupContext(backupId);
+    BackupInfo status = getBackupContext(backupId);
     if (status == null) return false;
     return status.getState() == BackupState.COMPLETE;
   }
 
   protected boolean checkFailed(String backupId) throws IOException {
-    BackupContext status = getBackupContext(backupId);
+    BackupInfo status = getBackupContext(backupId);
     if (status == null) return false;
     return status.getState() == BackupState.FAILED;
   }
 
-  private BackupContext getBackupContext(String backupId) throws IOException {
-    Configuration conf = conf1;//BackupClientImpl.getConf();
-    try (Connection connection = ConnectionFactory.createConnection(conf);
-        BackupSystemTable table = new BackupSystemTable(connection)) {
-      BackupContext status = table.readBackupStatus(backupId);
+  private BackupInfo getBackupContext(String backupId) throws IOException {
+    try (BackupSystemTable table = new 
BackupSystemTable(TEST_UTIL.getConnection())) {
+      BackupInfo status = table.readBackupInfo(backupId);
       return status;
     }
   }
 
+  protected BackupClient getBackupClient(){
+    return BackupRestoreFactory.getBackupClient(conf1);
+  }
+  
   protected RestoreClient getRestoreClient()
   {
     return BackupRestoreFactory.getRestoreClient(conf1);
@@ -232,4 +248,15 @@ public class TestBackupBase {
     }
     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("DDEBUG: "+it.next().getPath());
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index 7f5846c..62c47d6 100644
--- 
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
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase.backup;
 
-import static org.junit.Assert.assertTrue;
-
 import java.util.List;
 
 import org.apache.commons.logging.Log;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.java
new file mode 100644
index 0000000..eeb89b5
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDelete.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.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupDelete extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupDelete.class);
+
+  /**
+   * Verify that full backup is created on a single table with data correctly. 
Verify that history
+   * works as expected
+   * @throws Exception
+   */
+  @Test
+  public void testBackupDelete() throws Exception {
+    LOG.info("test backup delete on a single table with data");
+    List<TableName> tableList = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tableList);
+    assertTrue(checkSucceeded(backupId));
+    LOG.info("backup complete");
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(baos));
+
+    String[] backupIds = new String[] { backupId };
+    getBackupClient().deleteBackups(backupIds);
+
+    LOG.info("delete_backup");
+    String output = baos.toString();
+    LOG.info(baos.toString());
+    assertTrue(output.indexOf("Delete backup for backupID=" + backupId + " 
completed.") >= 0);
+  }
+
+  /**
+   * 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("Delete backup for backupID=" + backupId + " 
completed.") >= 0);
+  }  
+  
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.java
new file mode 100644
index 0000000..4f7cb11
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDescribe.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 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.BackupSystemTable;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupDescribe extends TestBackupBase {
+
+  private static final Log LOG = LogFactory.getLog(TestBackupDescribe.class);
+
+  /**
+   * Verify that full backup is created on a single table with data correctly. 
Verify that describe
+   * works as expected
+   * @throws Exception
+   */
+  @Test
+  public void testBackupDescribe() throws Exception {
+
+    LOG.info("test backup describe 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 = getBackupClient().getBackupInfo(backupId);
+    assertTrue(info.getState() == BackupState.COMPLETE);
+
+  }
+
+  @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));
+
+    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/c61f7c34/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
index 175bce0..3ef68e6 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
@@ -65,8 +65,7 @@ public class TestBackupLogCleaner extends TestBackupBase {
 
     List<TableName> tableSetFullList = Lists.newArrayList(table1, table2, 
table3, table4);
 
-    try (Connection connection = 
ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
-        BackupSystemTable systemTable = new BackupSystemTable(connection)) {
+    try (BackupSystemTable systemTable = new 
BackupSystemTable(TEST_UTIL.getConnection())) {
       // Verify that we have no backup sessions yet
       assertFalse(systemTable.hasBackupSessions());
 
@@ -76,16 +75,18 @@ public class TestBackupLogCleaner extends TestBackupBase {
       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(Iterables.size(deletable) == walFiles.size());
+      assertTrue(size == walFiles.size());
 
-      systemTable.addWALFiles(swalFiles, "backup");
+      systemTable.addWALFiles(swalFiles, "backup", "root");
       String backupIdFull = fullTableBackup(tableSetFullList);
-      // getBackupClient().create(BackupType.FULL, tableSetFullList, 
BACKUP_ROOT_DIR);
+      assertTrue(checkSucceeded(backupIdFull));
       // Check one more time
       deletable = cleaner.getDeletableFiles(walFiles);
       // We can delete wal files because they were saved into hbase:backup 
table
-      int size = Iterables.size(deletable);
+      size = Iterables.size(deletable);
       assertTrue(size == walFiles.size());
 
       List<FileStatus> newWalFiles = 
getListOfWALFiles(TEST_UTIL.getConfiguration());
@@ -95,7 +96,6 @@ public class TestBackupLogCleaner extends TestBackupBase {
       // New list of wal files is greater than the previous one,
       // because new wal per RS have been opened after full backup
       assertTrue(walFiles.size() < newWalFiles.size());
-      // TODO : verify that result files are not walFiles collection
       Connection conn = ConnectionFactory.createConnection(conf1);
       // #2 - insert some data to table
       HTable t1 = (HTable) conn.getTable(table1);
@@ -123,6 +123,7 @@ public class TestBackupLogCleaner extends TestBackupBase {
       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());

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
new file mode 100644
index 0000000..716a22a
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupShowHistory extends TestBackupBase {
+
+  private static final Log LOG = 
LogFactory.getLog(TestBackupShowHistory.class);
+
+  /**
+   * 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 = getBackupClient().getHistory(10);
+    assertTrue(history.size() > 0);
+    boolean success = false;
+    for(BackupInfo info: history){
+      if(info.getBackupId().equals(backupId)){
+        success = true; break;
+      }
+    }
+    assertTrue(success);
+    LOG.info("show_history");
+
+  }
+
+  @Test
+  public void testBackupHistoryCommand() throws Exception {
+
+    LOG.info("test backup history 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[]{"history",  "-n", "10" }; 
+    // Run backup
+    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+    assertTrue(ret == 0);
+    LOG.info("show_history");
+    String output = baos.toString();
+    LOG.info(baos.toString());
+    assertTrue(output.indexOf(backupId) > 0);
+  }  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
new file mode 100644
index 0000000..ce04b0b
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupStatusProgress.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category(LargeTests.class)
+public class TestBackupStatusProgress extends TestBackupBase {
+
+  private static final Log LOG = 
LogFactory.getLog(TestBackupStatusProgress.class);
+
+  /**
+   * Verify that full backup is created on a single table with data correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testBackupStatusProgress() throws Exception {
+
+    LOG.info("test backup status/progress on a single table with data");
+    
+    List<TableName> tableList = Lists.newArrayList(table1);
+    String backupId = fullTableBackup(tableList);
+    LOG.info("backup complete");
+    assertTrue(checkSucceeded(backupId));
+
+
+    BackupInfo info = getBackupClient().getBackupInfo(backupId);    
+    assertTrue(info.getState() == BackupState.COMPLETE);
+    int p = getBackupClient().getProgress(backupId);
+    LOG.debug(info.getShortDescription());
+    assertTrue(p > 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/c61f7c34/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
index f775c65..7ea4338 100644
--- 
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
@@ -38,14 +38,10 @@ 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.impl.BackupContext;
-import org.apache.hadoop.hbase.backup.impl.BackupContext.BackupState;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTableHelper;
-import org.apache.hadoop.hbase.backup.impl.BackupUtil.BackupCompleteData;
 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.testclassification.MediumTests;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -69,10 +65,21 @@ public class TestBackupSystemTable {
 
   @BeforeClass
   public static void setUp() throws Exception {
-    cluster = UTIL.startMiniCluster();
-    conn = ConnectionFactory.createConnection(UTIL.getConfiguration());
+    cluster = UTIL.startMiniCluster(); 
+    conn = UTIL.getConnection();
+    waitForSystemTable();
   }
-
+  
+  static void waitForSystemTable() throws Exception
+  {
+    try(Admin admin = UTIL.getAdmin();) {
+      while (!admin.tableExists(BackupSystemTable.getTableName()) 
+          || !admin.isTableAvailable(BackupSystemTable.getTableName())) {
+        Thread.sleep(1000);
+      }      
+    }
+  }
+  
   @Before
   public void before() throws IOException {
     table = new BackupSystemTable(conn);
@@ -83,22 +90,21 @@ public class TestBackupSystemTable {
     if (table != null) {
       table.close();
     }
+
   }
 
   @Test
   public void testUpdateReadDeleteBackupStatus() throws IOException {
-    BackupContext ctx = createBackupContext();
-    table.updateBackupStatus(ctx);
-    BackupContext readCtx = table.readBackupStatus(ctx.getBackupId());
+    BackupInfo ctx = createBackupContext();
+    table.updateBackupInfo(ctx);
+    BackupInfo readCtx = table.readBackupInfo(ctx.getBackupId());
     assertTrue(compare(ctx, readCtx));
-
     // try fake backup id
-    readCtx = table.readBackupStatus("fake");
-
+    readCtx = table.readBackupInfo("fake");
     assertNull(readCtx);
     // delete backup context
-    table.deleteBackupStatus(ctx.getBackupId());
-    readCtx = table.readBackupStatus(ctx.getBackupId());
+    table.deleteBackupInfo(ctx.getBackupId());
+    readCtx = table.readBackupInfo(ctx.getBackupId());
     assertNull(readCtx);
     cleanBackupTable();
   }
@@ -106,8 +112,8 @@ public class TestBackupSystemTable {
   @Test
   public void testWriteReadBackupStartCode() throws IOException {
     Long code = 100L;
-    table.writeBackupStartCode(code);
-    String readCode = table.readBackupStartCode();
+    table.writeBackupStartCode(code, "root");
+    String readCode = table.readBackupStartCode("root");
     assertEquals(code, new Long(Long.parseLong(readCode)));
     cleanBackupTable();
   }
@@ -124,23 +130,23 @@ public class TestBackupSystemTable {
   @Test
   public void testBackupHistory() throws IOException {
     int n = 10;
-    List<BackupContext> list = createBackupContextList(n);
+    List<BackupInfo> list = createBackupContextList(n);
 
     // Load data
-    for (BackupContext bc : list) {
+    for (BackupInfo bc : list) {
       // Make sure we set right status
       bc.setState(BackupState.COMPLETE);
-      table.updateBackupStatus(bc);
+      table.updateBackupInfo(bc);
     }
 
     // Reverse list for comparison
     Collections.reverse(list);
-    ArrayList<BackupCompleteData> history = table.getBackupHistory();
+    ArrayList<BackupInfo> history = table.getBackupHistory();
     assertTrue(history.size() == n);
 
     for (int i = 0; i < n; i++) {
-      BackupContext ctx = list.get(i);
-      BackupCompleteData data = history.get(i);
+      BackupInfo ctx = list.get(i);
+      BackupInfo data = history.get(i);
       assertTrue(compare(ctx, data));
     }
 
@@ -149,15 +155,52 @@ public class TestBackupSystemTable {
   }
 
   @Test
+  public void testBackupDelete() throws IOException {
+    
+    try (BackupSystemTable table = new BackupSystemTable(conn)) {
+
+      int n = 10;
+      List<BackupInfo> list = createBackupContextList(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]);
+      table.writeRegionServerLastLogRollResult(servers[i], timestamps[i], 
"root");
     }
 
-    HashMap<String, Long> result = table.readRegionServerLastLogRollResult();
+    HashMap<String, Long> result = 
table.readRegionServerLastLogRollResult("root");
     assertTrue(servers.length == result.size());
     Set<String> keys = result.keySet();
     String[] keysAsArray = new String[keys.size()];
@@ -188,9 +231,10 @@ public class TestBackupSystemTable {
     tables2.add(TableName.valueOf("t4"));
     tables2.add(TableName.valueOf("t5"));
 
-    table.addIncrementalBackupTableSet(tables1);
-    TreeSet<TableName> res1 = (TreeSet<TableName>) BackupSystemTableHelper
-        .getIncrementalBackupTableSet(conn);
+    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();
@@ -198,9 +242,9 @@ public class TestBackupSystemTable {
       assertEquals(desc1.next(), desc2.next());
     }
 
-    table.addIncrementalBackupTableSet(tables2);
-    TreeSet<TableName> res2 = (TreeSet<TableName>) BackupSystemTableHelper
-        .getIncrementalBackupTableSet(conn);
+    table.addIncrementalBackupTableSet(tables2, "root");
+    TreeSet<TableName> res2 = (TreeSet<TableName>) 
+        table.getIncrementalBackupTableSet("root");
     assertTrue((tables2.size() + tables1.size() - 1) == res2.size());
 
     tables1.addAll(tables2);
@@ -229,9 +273,9 @@ public class TestBackupSystemTable {
     rsTimestampMap.put("rs2", 101L);
     rsTimestampMap.put("rs3", 103L);
 
-    table.writeRegionServerLogTimestamp(tables, rsTimestampMap);
+    table.writeRegionServerLogTimestamp(tables, rsTimestampMap, "root");
 
-    HashMap<TableName, HashMap<String, Long>> result = 
table.readLogTimestampMap();
+    HashMap<TableName, HashMap<String, Long>> result = 
table.readLogTimestampMap("root");
 
     assertTrue(tables.size() == result.size());
 
@@ -255,9 +299,9 @@ public class TestBackupSystemTable {
     rsTimestampMap1.put("rs2", 201L);
     rsTimestampMap1.put("rs3", 203L);
 
-    table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1);
+    table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1, "root");
 
-    result = table.readLogTimestampMap();
+    result = table.readLogTimestampMap("root");
 
     assertTrue(5 == result.size());
 
@@ -295,7 +339,7 @@ public class TestBackupSystemTable {
             "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");
+    table.addWALFiles(files, "backup", "root");
 
     assertTrue(table.checkWALFile(files.get(0)));
     assertTrue(table.checkWALFile(files.get(1)));
@@ -305,26 +349,155 @@ public class TestBackupSystemTable {
     cleanBackupTable();
   }
 
-  private boolean compare(BackupContext ctx, BackupCompleteData data) {
+  
+  /**
+   * 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();
+    }
 
-    return ctx.getBackupId().equals(data.getBackupToken())
-        && ctx.getTargetRootDir().equals(data.getBackupRootPath())
-        && ctx.getType().toString().equals(data.getType())
-        && ctx.getStartTs() == Long.parseLong(data.getStartTime())
-        && ctx.getEndTs() == Long.parseLong(data.getEndTime());
+  }
 
+  @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.size() == 0);
+      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(BackupContext one, BackupContext two) {
+  private boolean compare(BackupInfo one, BackupInfo two) {
     return one.getBackupId().equals(two.getBackupId()) && 
one.getType().equals(two.getType())
         && one.getTargetRootDir().equals(two.getTargetRootDir())
         && one.getStartTs() == two.getStartTs() && one.getEndTs() == 
two.getEndTs();
   }
 
-  private BackupContext createBackupContext() {
+  private BackupInfo createBackupContext() {
 
-    BackupContext ctxt =
-        new BackupContext("backup_" + System.nanoTime(), BackupType.FULL,
+    BackupInfo ctxt =
+        new BackupInfo("backup_" + System.nanoTime(), BackupType.FULL,
           new TableName[] {
               TableName.valueOf("t1"), TableName.valueOf("t2"), 
TableName.valueOf("t3") },
           "/hbase/backup");
@@ -333,8 +506,8 @@ public class TestBackupSystemTable {
     return ctxt;
   }
 
-  private List<BackupContext> createBackupContextList(int size) {
-    List<BackupContext> list = new ArrayList<BackupContext>();
+  private List<BackupInfo> createBackupContextList(int size) {
+    List<BackupInfo> list = new ArrayList<BackupInfo>();
     for (int i = 0; i < size; i++) {
       list.add(createBackupContext());
       try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index 9dda967..0c0bf4a 100644
--- 
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
@@ -11,12 +11,17 @@
 
 package org.apache.hadoop.hbase.backup;
 
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
 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.testclassification.LargeTests;
+import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -36,10 +41,36 @@ public class TestFullBackup extends TestBackupBase {
     LOG.info("test full backup on a single table with data");
     List<TableName> tables = Lists.newArrayList(table1);
     String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
     LOG.info("backup complete for " + backupId);
   }
 
   /**
+   * Verify that full backup is created on a single table with data correctly.
+   * @throws Exception
+   */
+  @Test
+  public void testFullBackupSingleCommand() throws Exception {
+    LOG.info("test full backup on a single table 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,  
table1.getNameAsString() }; 
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      ArrayList<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");
+  }
+
+
+  /**
    * Verify that full backup is created on multiple tables correctly.
    * @throws Exception
    */
@@ -48,8 +79,29 @@ public class TestFullBackup extends TestBackupBase {
     LOG.info("create full backup image on multiple tables with data");
     List<TableName> tables = Lists.newArrayList(table1, table1);
     String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
   }
 
+  @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,  
+          table1.getNameAsString() +","+ table2.getNameAsString() }; 
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      ArrayList<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");
+  }
   /**
    * Verify that full backup is created on all tables correctly.
    * @throws Exception
@@ -58,5 +110,26 @@ public class TestFullBackup extends TestBackupBase {
   public void testFullBackupAll() throws Exception {
     LOG.info("create full backup image on all tables");
     String backupId = fullTableBackup(null);
+    assertTrue(checkSucceeded(backupId));
+
+  }
+  
+  @Test
+  public void testFullBackupAllCommand() throws Exception {
+    LOG.info("create full backup image on all tables: command-line");
+    try(BackupSystemTable table = new 
BackupSystemTable(TEST_UTIL.getConnection())) {    
+      int before = table.getBackupHistory().size();
+      String[] args = new String[]{"create", "full", BACKUP_ROOT_DIR  }; 
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      ArrayList<BackupInfo> backups = table.getBackupHistory(); 
+      int after = table.getBackupHistory().size();
+      assertTrue(after == before +1);
+      for(BackupInfo data : backups){
+        String backupId = data.getBackupId();
+        assertTrue(checkSucceeded(backupId));
+      }
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
new file mode 100644
index 0000000..a4c0fa1
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackupSet.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+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.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("TFBSE 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, 
"-set", name };
+      // Run backup
+      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
+      assertTrue(ret == 0);
+      ArrayList<BackupInfo> backups = table.getBackupHistory();
+      assertTrue(backups.size() == 1);
+      String backupId = backups.get(0).getBackupId();
+      assertTrue(checkSucceeded(backupId));
+      LOG.info("TFBSE backup complete");
+    }
+
+  }
+
+  @Test
+  public void testFullBackupSetDoesNotExist() throws Exception {
+
+    LOG.info("TFBSE test full backup, backup set does not exist");    
+    String name = "name1";    
+    String[] args = new String[]{"create", "full", BACKUP_ROOT_DIR, "-set", 
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/c61f7c34/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
index 3139134..983b850 100644
--- 
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
@@ -16,11 +16,13 @@ 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.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;
 
@@ -42,6 +44,8 @@ public class TestFullRestore extends TestBackupBase {
 
     List<TableName> tables = Lists.newArrayList(table1);
     String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));
+    
     LOG.info("backup complete");
 
     TableName[] tableset = new TableName[] { table1 };
@@ -54,6 +58,29 @@ public class TestFullRestore extends TestBackupBase {
     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[]{"restore",  BACKUP_ROOT_DIR, backupId, 
+        table1.getNameAsString(), 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();
+  }
+  
   /**
    * Verify that multiple tables are restored to new tables.
    * @throws Exception
@@ -63,6 +90,7 @@ public class TestFullRestore extends TestBackupBase {
     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 };
@@ -78,6 +106,38 @@ public class TestFullRestore extends TestBackupBase {
   }
 
   /**
+   * 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[]{"restore",  BACKUP_ROOT_DIR, backupId, 
+        StringUtils.join(restore_tableset, ","), 
+        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
    */
@@ -87,6 +147,8 @@ public class TestFullRestore extends TestBackupBase {
     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 };
@@ -96,6 +158,32 @@ public class TestFullRestore extends TestBackupBase {
   }
 
   /**
+   * 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[]{"restore",  BACKUP_ROOT_DIR, backupId, 
+        StringUtils.join(tableset, ","), "-overwrite" }; 
+    // 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
    */
@@ -105,6 +193,7 @@ public class TestFullRestore extends TestBackupBase {
 
     List<TableName> tables = Lists.newArrayList(table2, table3);
     String backupId = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupId));    
 
     TableName[] restore_tableset = new TableName[] { table2, table3 };
     RestoreClient client = getRestoreClient();
@@ -113,6 +202,32 @@ public class TestFullRestore extends TestBackupBase {
   }
 
   /**
+   * 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[]{"restore",  BACKUP_ROOT_DIR, backupId, 
+        StringUtils.join(restore_tableset, ","), "-overwrite" }; 
+    // 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
    */
@@ -122,6 +237,7 @@ public class TestFullRestore extends TestBackupBase {
     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");
 
@@ -132,6 +248,31 @@ public class TestFullRestore extends TestBackupBase {
       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[]{"restore",  BACKUP_ROOT_DIR, backupId, 
+        StringUtils.join(tableset, ","), 
+        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
@@ -143,6 +284,7 @@ public class TestFullRestore extends TestBackupBase {
 
     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") };
@@ -151,4 +293,28 @@ public class TestFullRestore extends TestBackupBase {
     client.restore(BACKUP_ROOT_DIR, backupId, false,
       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[]{"restore",  BACKUP_ROOT_DIR, backupId, 
+        StringUtils.join(restore_tableset, ","), 
+        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/c61f7c34/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
index 6669612..04ee015 100644
--- 
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
@@ -24,7 +24,10 @@ 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.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -43,7 +46,7 @@ import com.google.common.collect.Lists;
 @Category(LargeTests.class)
 public class TestIncrementalBackup extends TestBackupBase {
   private static final Log LOG = 
LogFactory.getLog(TestIncrementalBackup.class);
-  //implement all testcases in 1 test since incremental backup/restore has 
dependencies
+  //implement all test cases in 1 test since incremental backup/restore has 
dependencies
   @Test
   public void TestIncBackupRestore() throws Exception {
     // #1 - create full backup for all tables
@@ -89,6 +92,7 @@ public class TestIncrementalBackup extends TestBackupBase {
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
     .setTargetRootDir(BACKUP_ROOT_DIR);
     String backupIdIncMultiple = admin.backupTables(request);
+    assertTrue(checkSucceeded(backupIdIncMultiple));
 
     // #4 - restore full backup for all tables, without overwrite
     TableName[] tablesRestoreFull =
@@ -133,7 +137,6 @@ public class TestIncrementalBackup extends TestBackupBase {
         new TableName[] { table1, table2, table3 };
     TableName[] tablesMapIncMultiple =
         new TableName[] { table1_restore, table2_restore, table3_restore };
-    LOG.info("restore inc backup " + backupIdIncMultiple);
     client = getRestoreClient();
     client.restore(BACKUP_ROOT_DIR, backupIdIncMultiple, false, false,
       tablesRestoreIncMultiple, tablesMapIncMultiple, true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java
new file mode 100644
index 0000000..b3cf4ee
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.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.hamcrest.CoreMatchers;
+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 TestIncrementalBackupNoDataLoss extends TestBackupBase {
+  private static final Log LOG = 
LogFactory.getLog(TestIncrementalBackupNoDataLoss.class);
+
+  // implement all test cases in 1 test since incremental backup/restore has 
dependencies
+  @Test
+  public void TestIncBackupRestore() 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);   
+    String backupIdFull = fullTableBackup(tables);
+    assertTrue(checkSucceeded(backupIdFull));
+    Connection conn = ConnectionFactory.createConnection(conf1);
+    // #2 - insert some data to table
+    HTable t1 = (HTable) conn.getTable(table1);
+    Put p1;
+    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
+      p1 = new Put(Bytes.toBytes("row-t1" + i));
+      p1.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t1.put(p1);
+    }
+
+    Assert.assertThat(TEST_UTIL.countRows(t1), 
CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
+    t1.close();
+
+    HTable t2 = (HTable) conn.getTable(table2);
+    Put p2;
+    for (int i = 0; i < 5; i++) {
+      p2 = new Put(Bytes.toBytes("row-t2" + i));
+      p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
+      t2.put(p2);
+    }
+
+    Assert.assertThat(TEST_UTIL.countRows(t2), 
CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
+    t2.close();
+
+    // #3 - incremental backup for table1
+
+    tables = Lists.newArrayList(table1);
+    String backupIdInc1 = incrementalTableBackup(tables);
+    assertTrue(checkSucceeded(backupIdInc1));
+
+    // #4 - incremental backup for table2
+
+    tables = Lists.newArrayList(table2);
+    String backupIdInc2 = incrementalTableBackup(tables);
+    assertTrue(checkSucceeded(backupIdInc2));
+    // #5 - restore incremental backup for table1
+    TableName[] tablesRestoreInc1 = new TableName[] { table1 };
+    TableName[] tablesMapInc1 = new TableName[] { table1_restore };
+
+    if (TEST_UTIL.getAdmin().tableExists(table1_restore)) {
+      TEST_UTIL.deleteTable(table1_restore);
+    }
+    if (TEST_UTIL.getAdmin().tableExists(table2_restore)) {
+      TEST_UTIL.deleteTable(table2_restore);
+    }
+
+    RestoreClient client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupIdInc1, false, true, 
tablesRestoreInc1,
+      tablesMapInc1, false);
+
+    HTable hTable = (HTable) conn.getTable(table1_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), 
CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
+    hTable.close();
+
+    // #5 - restore incremental backup for table2
+    
+    TableName[] tablesRestoreInc2 = new TableName[] { table2 };
+    TableName[] tablesMapInc2 = new TableName[] { table2_restore };
+
+    client = getRestoreClient();
+    client.restore(BACKUP_ROOT_DIR, backupIdInc2, false, true, 
tablesRestoreInc2,
+      tablesMapInc2, false);
+
+    hTable = (HTable) conn.getTable(table2_restore);
+    Assert.assertThat(TEST_UTIL.countRows(hTable), 
CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
+    hTable.close();
+
+    conn.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index f97f30a..e29a4a6 100644
--- 
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
@@ -11,6 +11,8 @@
 
 package org.apache.hadoop.hbase.backup;
 
+import static org.junit.Assert.assertTrue;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -35,6 +37,8 @@ public class TestRemoteBackup extends TestBackupBase {
 
     String backupId = backupTables(BackupType.FULL,
       Lists.newArrayList(table1), BACKUP_REMOTE_ROOT_DIR);
+    assertTrue(checkSucceeded(backupId));
+    
     LOG.info("backup complete " + backupId);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index cd597e1..32a028c 100644
--- 
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
@@ -15,7 +15,6 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -40,7 +39,6 @@ public class TestRemoteRestore extends TestBackupBase {
     LOG.info("backup complete");
     TableName[] tableset = new TableName[] { table1 };
     TableName[] tablemap = new TableName[] { table1_restore };
-    Path path = new Path(BACKUP_REMOTE_ROOT_DIR);
     getRestoreClient().restore(BACKUP_REMOTE_ROOT_DIR, backupId, false, false, 
tableset,
       tablemap, false);
     HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index acded77..652a909 100644
--- 
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
@@ -24,7 +24,6 @@ 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.client.HBaseAdmin;
 import org.apache.hadoop.hbase.testclassification.LargeTests;

Reply via email to