You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/10/05 23:30:07 UTC

[01/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Repository: hbase
Updated Branches:
  refs/heads/HBASE-7912 e35f7b920 -> b14e2ab1c


http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java
index 3da7860..cc2ecdf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreServerUtil.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.backup.util;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -39,26 +38,21 @@ 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.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
 import org.apache.hadoop.hbase.backup.HBackupFileSystem;
-import org.apache.hadoop.hbase.backup.RestoreService;
 import org.apache.hadoop.hbase.backup.RestoreRequest;
+import org.apache.hadoop.hbase.backup.RestoreService;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HStore;
@@ -68,7 +62,6 @@ 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;
-import org.apache.hadoop.hbase.util.Pair;
 
 /**
  * A collection for methods used by multiple classes to restore HBase tables.
@@ -172,33 +165,21 @@ public class RestoreServerUtil {
     return regionDirList;
   }
   
-  static void modifyTableSync(MasterServices svc, HTableDescriptor desc) throws IOException {
-    svc.modifyTable(desc.getTableName(), desc, HConstants.NO_NONCE, HConstants.NO_NONCE);
-    @SuppressWarnings("serial")
-    Pair<Integer, Integer> status = new Pair<Integer, Integer>() {{
-      setFirst(0);
-      setSecond(0);
-    }};
-    int i = 0;
-    do {
-      status = svc.getAssignmentManager().getReopenStatus(desc.getTableName());
-      if (status.getSecond() != 0) {
-        LOG.debug(status.getSecond() - status.getFirst() + "/" + status.getSecond()
-          + " regions updated.");
-        try {
-          Thread.sleep(1 * 1000l);
-        } catch (InterruptedException ie) {
-          InterruptedIOException iie = new InterruptedIOException();
-          iie.initCause(ie);
-          throw iie;
+  static 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");
         }
-      } else {
-        LOG.debug("All regions updated.");
-        break;
       }
-    } while (status.getFirst() != 0 && i++ < 500);
-    if (status.getFirst() != 0) {
-      throw new IOException("Failed to update all regions even after 500 seconds.");
+    } catch (Exception e) {
+      throw new IOException(e);
     }
   }
 
@@ -206,7 +187,7 @@ public class RestoreServerUtil {
    * 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 svc MasterServices
+   * @param conn HBase connection
    * @param tableBackupPath backup path
    * @param logDirs : incremental backup folders, which contains WAL
    * @param tableNames : source tableNames(table names were backuped)
@@ -214,9 +195,10 @@ public class RestoreServerUtil {
    * @param incrBackupId incremental backup Id
    * @throws IOException exception
    */
-  public void incrementalRestoreTable(MasterServices svc, Path tableBackupPath, Path[] logDirs,
+  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!");
     }
@@ -225,7 +207,7 @@ public class RestoreServerUtil {
     // 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 (!MetaTableAccessor.tableExists(svc.getConnection(), tableName)) {
+      if (!admin.tableExists(tableName)) {
         throw new IOException("HBase table " + tableName
             + " does not exist. Create the table first, e.g. by restoring a full backup.");
       }
@@ -237,7 +219,7 @@ public class RestoreServerUtil {
       LOG.debug("Found descriptor " + tableDescriptor + " through " + incrBackupId);
 
       TableName newTableName = newTableNames[i];
-      HTableDescriptor newTableDescriptor = svc.getTableDescriptors().get(newTableName);
+      HTableDescriptor newTableDescriptor = admin.getTableDescriptor(newTableName);
       List<HColumnDescriptor> families = Arrays.asList(tableDescriptor.getColumnFamilies());
       List<HColumnDescriptor> existingFamilies =
           Arrays.asList(newTableDescriptor.getColumnFamilies());
@@ -255,7 +237,7 @@ public class RestoreServerUtil {
         }
       }
       if (schemaChangeNeeded) {
-        RestoreServerUtil.modifyTableSync(svc, newTableDescriptor);
+        RestoreServerUtil.modifyTableSync(conn, newTableDescriptor);
         LOG.info("Changed " + newTableDescriptor.getTableName() + " to: " + newTableDescriptor);
       }
     }
@@ -263,12 +245,13 @@ public class RestoreServerUtil {
         BackupRestoreServerFactory.getRestoreService(conf);
 
     restoreService.run(logDirs, tableNames, newTableNames, false);
+    }
   }
 
-  public void fullRestoreTable(MasterServices svc, Path tableBackupPath, TableName tableName,
+  public void fullRestoreTable(Connection conn, Path tableBackupPath, TableName tableName,
       TableName newTableName, boolean truncateIfExists, String lastIncrBackupId)
           throws IOException {
-    restoreTableAndCreate(svc, tableName, newTableName, tableBackupPath, truncateIfExists,
+    restoreTableAndCreate(conn, tableName, newTableName, tableBackupPath, truncateIfExists,
         lastIncrBackupId);
   }
 
@@ -386,7 +369,7 @@ public class RestoreServerUtil {
     return null;
   }
 
-  private void restoreTableAndCreate(MasterServices svc, TableName tableName,
+  private void restoreTableAndCreate(Connection conn, TableName tableName,
       TableName newTableName, Path tableBackupPath, boolean truncateIfExists,
       String lastIncrBackupId) throws IOException {
     if (newTableName == null || newTableName.equals("")) {
@@ -434,7 +417,7 @@ public class RestoreServerUtil {
               + ", will only create table");
         }
         tableDescriptor.setName(newTableName);
-        checkAndCreateTable(svc, tableBackupPath, tableName, newTableName, null, tableDescriptor,
+        checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, null, tableDescriptor,
           truncateIfExists);
         return;
       } else {
@@ -461,7 +444,7 @@ public class RestoreServerUtil {
 
       // should only try to create the table with all region informations, so we could pre-split
       // the regions in fine grain
-      checkAndCreateTable(svc, tableBackupPath, tableName, newTableName, regionPathList,
+      checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, regionPathList,
         tableDescriptor, truncateIfExists);
 
       // Now get region splits from full backup
@@ -702,18 +685,18 @@ public class RestoreServerUtil {
    * @param htd table descriptor
    * @throws IOException exception
    */
-  private void checkAndCreateTable(MasterServices svc, Path tableBackupPath, TableName tableName,
+  private void checkAndCreateTable(Connection conn, Path tableBackupPath, TableName tableName,
       TableName targetTableName, ArrayList<Path> regionDirList, 
       HTableDescriptor htd, boolean truncateIfExists)
           throws IOException {
-    try {
+    try (Admin admin = conn.getAdmin();){
       boolean createNew = false;
-      if (MetaTableAccessor.tableExists(svc.getConnection(), targetTableName)) {
+      if (admin.tableExists(targetTableName)) {
         if(truncateIfExists) {
           LOG.info("Truncating exising target table '" + targetTableName +
             "', preserving region splits");
-          svc.disableTable(targetTableName, HConstants.NO_NONCE, HConstants.NO_NONCE);
-          svc.truncateTable(targetTableName, true, HConstants.NO_NONCE, HConstants.NO_NONCE);
+          admin.disableTable(targetTableName);
+          admin.truncateTable(targetTableName, true);
         } else{
           LOG.info("Using exising target table '" + targetTableName + "'");
         }
@@ -724,14 +707,14 @@ public class RestoreServerUtil {
         LOG.info("Creating target table '" + targetTableName + "'");
         byte[][] keys = null;
         if (regionDirList == null || regionDirList.size() == 0) {
-          svc.createTable(htd, null, HConstants.NO_NONCE, HConstants.NO_NONCE);
+          admin.createTable(htd, null);
         } else {
           keys = generateBoundaryKeys(regionDirList);
           // create table using table descriptor and region boundaries
-          svc.createTable(htd, keys, HConstants.NO_NONCE, HConstants.NO_NONCE);
+          admin.createTable(htd, keys);
         }
         long startTime = EnvironmentEdgeManager.currentTime();
-        while (!((ClusterConnection)svc.getConnection()).isTableAvailable(targetTableName, keys)) {
+        while (!admin.isTableAvailable(targetTableName, keys)) {
           Thread.sleep(100);
           if (EnvironmentEdgeManager.currentTime() - startTime > TABLE_AVAILABILITY_WAIT_TIME) {
             throw new IOException("Time out "+TABLE_AVAILABILITY_WAIT_TIME+

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/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 a02f011..97f08e0 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
@@ -83,9 +83,9 @@ import org.apache.hadoop.hbase.backup.impl.BackupManager;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest;
 import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.master.FullTableBackupProcedure;
-import org.apache.hadoop.hbase.backup.master.IncrementalTableBackupProcedure;
-import org.apache.hadoop.hbase.backup.master.RestoreTablesProcedure;
+import org.apache.hadoop.hbase.backup.impl.FullTableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
+import org.apache.hadoop.hbase.backup.impl.RestoreTablesClient;
 import org.apache.hadoop.hbase.backup.util.RestoreServerUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
@@ -2617,120 +2617,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     return procInfoList;
   }
 
-  @Override
-  public Pair<Long, String> backupTables(final BackupType type,
-        List<TableName> tableList, final String targetRootDir, final int workers,
-        final long bandwidth, final String setName,
-        final long nonceGroup, final long nonce) throws IOException {
-    long procId;
-    String backupId = (setName == null || setName.length() == 0? 
-        BackupRestoreConstants.BACKUPID_PREFIX: setName + "_") + 
-        EnvironmentEdgeManager.currentTime();
-    if (type == BackupType.INCREMENTAL) {
-      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 "
-            + "change the tables covered by incremental backup.");
-        throw new DoNotRetryIOException("No table covered by incremental backup.");
-      }
-
-      tableList.removeAll(incrTableSet);
-      if (!tableList.isEmpty()) {
-        String extraTables = StringUtils.join(",", tableList);
-        LOG.error("Some tables (" + extraTables + ") haven't gone through full backup");
-        throw new DoNotRetryIOException("Perform full backup on " + extraTables + " first, "
-            + "then retry the command");
-      }
-      LOG.info("Incremental backup for the following table set: " + incrTableSet);
-      tableList = Lists.newArrayList(incrTableSet);
-    }
-    if (tableList != null && !tableList.isEmpty()) {
-      for (TableName table : tableList) {
-        String targetTableBackupDir =
-            HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
-        Path targetTableBackupDirPath = new Path(targetTableBackupDir);
-        FileSystem outputFs = FileSystem.get(targetTableBackupDirPath.toUri(), conf);
-        if (outputFs.exists(targetTableBackupDirPath)) {
-          throw new DoNotRetryIOException("Target backup directory " + targetTableBackupDir
-            + " exists already.");
-        }
-      }
-      ArrayList<TableName> nonExistingTableList = null;
-      for (TableName tableName : tableList) {
-        if (!MetaTableAccessor.tableExists(getConnection(), tableName)) {
-          if (nonExistingTableList == null) {
-            nonExistingTableList = new ArrayList<>();
-          }
-          nonExistingTableList.add(tableName);
-        }
-      }
-      if (nonExistingTableList != null) {
-        if (type == BackupType.INCREMENTAL ) {
-          LOG.warn("Incremental backup table set contains non-exising table: "
-              + nonExistingTableList);
-          // Update incremental backup set 
-          tableList = excludeNonExistingTables(tableList, nonExistingTableList);
-        } else {
-          // Throw exception only in full mode - we try to backup non-existing table
-          throw new DoNotRetryIOException("Non-existing tables found in the table list: "
-              + nonExistingTableList);
-        }
-      }
-    }
-    if (type == BackupType.FULL) {
-      procId = this.procedureExecutor.submitProcedure(
-        new FullTableBackupProcedure(procedureExecutor.getEnvironment(), backupId,
-          tableList, targetRootDir, workers, bandwidth), nonceGroup, nonce);
-    } else {
-      procId = this.procedureExecutor.submitProcedure(
-        new IncrementalTableBackupProcedure(procedureExecutor.getEnvironment(), backupId,
-          tableList, targetRootDir, workers, bandwidth), nonceGroup, nonce);
-    }
-    return new Pair<>(procId, backupId);
-  }
 
-  private List<TableName> excludeNonExistingTables(List<TableName> tableList,
-      List<TableName> nonExistingTableList) {
-    
-    for(TableName table: nonExistingTableList) {
-      tableList.remove(table);
-    }
-    return tableList;
-  }
-
-  @Override
-  public long restoreTables(String backupRootDir, String backupId, boolean check,
-      List<TableName> sTableList, List<TableName> tTableList, boolean isOverwrite,
-      final long nonceGroup, final long nonce) throws IOException {
-    if (check) {
-      HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
-      // check and load backup image manifest for the tables
-      Path rootPath = new Path(backupRootDir);
-      HBackupFileSystem.checkImageManifestExist(backupManifestMap,
-        sTableList.toArray(new TableName[sTableList.size()]),
-        conf, rootPath, backupId);
-
-      // Check and validate the backup image and its dependencies
-      if (check) {
-        if (RestoreServerUtil.validate(backupManifestMap, conf)) {
-          LOG.info("Checking backup images: ok");
-        } else {
-          String errMsg = "Some dependencies are missing for restore";
-          LOG.error(errMsg);
-          throw new IOException(errMsg);
-        }
-      }
-    }
-    long procId = this.procedureExecutor.submitProcedure(
-      new RestoreTablesProcedure(procedureExecutor.getEnvironment(), backupRootDir, backupId,
-        sTableList, tTableList, isOverwrite), nonceGroup, nonce);
-    return procId;
-  }
 
   /**
    * Returns the list of table descriptors that match the specified request

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 8025a67..2c577c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -1053,48 +1053,6 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
-  public MasterProtos.BackupTablesResponse backupTables(
-      RpcController controller,
-      MasterProtos.BackupTablesRequest request)  throws ServiceException {
-    try {
-      BackupTablesResponse.Builder response = BackupTablesResponse.newBuilder();
-      List<TableName> tablesList = new ArrayList<>(request.getTablesList().size());
-      for (HBaseProtos.TableName table : request.getTablesList()) {
-        tablesList.add(ProtobufUtil.toTableName(table));
-      }
-      Pair<Long, String> pair = master.backupTables(
-        BackupType.valueOf(request.getType().name()), tablesList, request.getTargetRootDir(),
-        (int)request.getWorkers(), request.getBandwidth(), request.getBackupSetName(), 
-        request.getNonceGroup(), request.getNonce());
-      return response.setProcId(pair.getFirst()).setBackupId(pair.getSecond()).build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
-  @Override
-  public MasterProtos.RestoreTablesResponse restoreTables(
-      RpcController controller,
-      MasterProtos.RestoreTablesRequest request)  throws ServiceException {
-    try {
-      RestoreTablesResponse.Builder response = RestoreTablesResponse.newBuilder();
-      List<TableName> tablesList = new ArrayList<>(request.getTablesList().size());
-      for (HBaseProtos.TableName table : request.getTablesList()) {
-        tablesList.add(ProtobufUtil.toTableName(table));
-      }
-      List<TableName> targetTablesList = new ArrayList<>(request.getTargetTablesList().size());
-      for (HBaseProtos.TableName table : request.getTargetTablesList()) {
-        targetTablesList.add(ProtobufUtil.toTableName(table));
-      }
-      long procId = master.restoreTables(request.getBackupRootDir(), request.getBackupId(),
-        request.getDependencyCheckOnly(), tablesList, targetTablesList, request.getOverwrite(),
-        request.getNonceGroup(), request.getNonce());
-      return response.setProcId(procId).build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-  @Override
   public ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace(RpcController c,
       ListTableDescriptorsByNamespaceRequest request) throws ServiceException {
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index d147ce2..1d7ef4e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -181,36 +181,6 @@ public interface MasterServices extends Server {
       final long nonce)
       throws IOException;
 
-  /**
-   * Full backup given list of tables
-   * @param type whether the backup is full or incremental
-   * @param tableList list of tables to backup
-   * @param targetRootDir root dir for saving the backup
-   * @param workers number of paralle workers. -1 - system defined
-   * @param bandwidth bandwidth per worker in MB per sec. -1 - unlimited
-   * @param setName - backup set name
-   * @param nonceGroup nonce group
-   * @param nonce nonce
-   * @return pair of procedure Id and backupId
-   * @throws IOException
-   */
-  public Pair<Long, String> backupTables(
-      final BackupType type,
-      List<TableName> tableList,
-      final String targetRootDir,
-      final int workers,
-      final long bandwidth,
-      final String setName,
-      final long nonceGroup,
-      final long nonce) throws IOException;
-
-  /*
-   * Restore table set
-   */
-  public long restoreTables(String backupRootDir,
-      String backupId, boolean check, List<TableName> sTableList,
-      List<TableName> tTableList, boolean isOverwrite, long nonceGroup, long nonce)
-          throws IOException;
 
   /**
    * Enable an existing table

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/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 ec53a64..2da7871 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
@@ -21,7 +21,9 @@ package org.apache.hadoop.hbase.backup;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -39,8 +41,8 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.BackupAdmin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Durability;
@@ -116,6 +118,15 @@ public class TestBackupBase {
     LOG.info("REMOTE ROOTDIR " + BACKUP_REMOTE_ROOT_DIR);
     waitForSystemTable();
     createTables();
+    populateFromMasterConfig(TEST_UTIL.getHBaseCluster().getMaster().getConfiguration(), conf1);
+  }
+  
+  private static void populateFromMasterConfig(Configuration masterConf, Configuration conf) {
+    Iterator<Entry<String,String>> it = masterConf.iterator();  
+    while(it.hasNext()) {
+      Entry<String,String> e = it.next();
+      conf.set(e.getKey(), e.getValue());
+    }
   }
   
   public static void waitForSystemTable() throws Exception
@@ -157,23 +168,18 @@ public class TestBackupBase {
   protected String backupTables(BackupType type, List<TableName> tables, String path)
       throws IOException {
     Connection conn = null;
-    HBaseAdmin admin = null;
     BackupAdmin badmin = null;
     String backupId;
     try {
       conn = ConnectionFactory.createConnection(conf1);
-      admin = (HBaseAdmin) conn.getAdmin();
-      BackupRequest request = new BackupRequest();
+      badmin = new HBaseBackupAdmin(conn);
+      BackupRequest request = new BackupRequest();      
       request.setBackupType(type).setTableList(tables).setTargetRootDir(path);
-      badmin = admin.getBackupAdmin();
       backupId = badmin.backupTables(request);
     } finally {
       if(badmin != null){
         badmin.close();
       }
-      if (admin != null) {
-        admin.close();
-      }
       if (conn != null) {
         conn.close();
       }
@@ -264,7 +270,7 @@ public class TestBackupBase {
   }
 
   protected BackupAdmin getBackupAdmin() throws IOException {
-    return TEST_UTIL.getAdmin().getBackupAdmin();
+    return new HBaseBackupAdmin(TEST_UTIL.getConnection());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/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 62c47d6..280314b 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,11 +18,11 @@
 
 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.DoNotRetryIOException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.Test;
@@ -63,7 +63,7 @@ public class TestBackupBoundaryTests extends TestBackupBase {
    * Verify that full backup fails on a single table that does not exist.
    * @throws Exception
    */
-  @Test(expected = DoNotRetryIOException.class)
+  @Test(expected = IOException.class)
   public void testFullBackupSingleDNE() throws Exception {
 
     LOG.info("test full backup fails on a single table that does not exist");
@@ -75,7 +75,7 @@ public class TestBackupBoundaryTests extends TestBackupBase {
    * Verify that full backup fails on multiple tables that do not exist.
    * @throws Exception
    */
-  @Test(expected = DoNotRetryIOException.class)
+  @Test(expected = IOException.class)
   public void testFullBackupMultipleDNE() throws Exception {
 
     LOG.info("test full backup fails on multiple tables that do not exist");
@@ -87,7 +87,7 @@ public class TestBackupBoundaryTests extends TestBackupBase {
    * Verify that full backup fails on tableset containing real and fake tables.
    * @throws Exception
    */
-  @Test(expected = DoNotRetryIOException.class)
+  @Test(expected = IOException.class)
   public void testFullBackupMixExistAndDNE() throws Exception {
     LOG.info("create full backup fails on tableset containing real and fake table");
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
index abdf3c7..09c3833 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
@@ -20,7 +20,6 @@ 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.RestoreServerUtil;
-import org.apache.hadoop.hbase.client.BackupAdmin;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Table;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
index c39241e..1caba22 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupMultipleDeletes.java
@@ -28,6 +28,7 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -59,9 +60,10 @@ public class TestBackupMultipleDeletes extends TestBackupBase {
     HBaseAdmin admin = null;
     Connection conn = ConnectionFactory.createConnection(conf1);
     admin = (HBaseAdmin) conn.getAdmin();
+    BackupAdmin client = new HBaseBackupAdmin(conn);
     BackupRequest request = new BackupRequest();
     request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdFull = admin.getBackupAdmin().backupTables(request);
+    String backupIdFull = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdFull));
     // #2 - insert some data to table table1
     HTable t1 = (HTable) conn.getTable(table1);
@@ -78,7 +80,7 @@ public class TestBackupMultipleDeletes extends TestBackupBase {
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
         .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdInc1 = admin.getBackupAdmin().backupTables(request);
+    String backupIdInc1 = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdInc1));
     // #4 - insert some data to table table2
     HTable t2 = (HTable) conn.getTable(table2);
@@ -93,7 +95,7 @@ public class TestBackupMultipleDeletes extends TestBackupBase {
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
         .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdInc2 = admin.getBackupAdmin().backupTables(request);
+    String backupIdInc2 = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdInc2));
     // #6 - insert some data to table table1
     t1 = (HTable) conn.getTable(table1);
@@ -107,7 +109,7 @@ public class TestBackupMultipleDeletes extends TestBackupBase {
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
         .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdInc3 = admin.getBackupAdmin().backupTables(request);
+    String backupIdInc3 = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdInc3));
     // #8 - insert some data to table table2
     t2 = (HTable) conn.getTable(table2);
@@ -121,25 +123,25 @@ public class TestBackupMultipleDeletes extends TestBackupBase {
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
         .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdInc4 = admin.getBackupAdmin().backupTables(request);
+    String backupIdInc4 = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdInc4));
     // #10 full backup for table3
     tables = Lists.newArrayList(table3);
     request = new BackupRequest();
     request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdFull2 = admin.getBackupAdmin().backupTables(request);
+    String backupIdFull2 = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdFull2));
     // #11 - incremental backup for table3
     tables = Lists.newArrayList(table3);
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
         .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdInc5 = admin.getBackupAdmin().backupTables(request);
+    String backupIdInc5 = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdInc5));
     LOG.error("Delete backupIdInc2");
-    admin.getBackupAdmin().deleteBackups(new String[] { backupIdInc2 });
+    client.deleteBackups(new String[] { backupIdInc2 });
     LOG.error("Delete backupIdInc2 done");
-    List<BackupInfo> list = admin.getBackupAdmin().getHistory(100);
+    List<BackupInfo> list = client.getHistory(100);
     // First check number of backup images before and after
     assertEquals(4, list.size());
     // then verify that no backupIdInc2,3,4

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/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 639aea4..a01801d 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
@@ -21,7 +21,6 @@ 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.RestoreServerUtil;
-import org.apache.hadoop.hbase.client.BackupAdmin;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.util.ToolRunner;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/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 fe00ac5..705a066 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
@@ -22,16 +22,19 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin;
 import org.apache.hadoop.hbase.backup.util.RestoreServerUtil;
-import org.apache.hadoop.hbase.client.BackupAdmin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -68,7 +71,7 @@ public class TestIncrementalBackup extends TestBackupBase {
   //implement all test cases in 1 test since incremental backup/restore has dependencies
   @Test
   public void TestIncBackupRestore() throws Exception {
-    
+
     int ADD_ROWS = 99;
     // #1 - create full backup for all tables
     LOG.info("create full backup image for all tables");
@@ -84,10 +87,11 @@ public class TestIncrementalBackup extends TestBackupBase {
 
     HBaseAdmin admin = null;
     admin = (HBaseAdmin) conn.getAdmin();
+    HBaseBackupAdmin client = new HBaseBackupAdmin(conn);
 
     BackupRequest request = new BackupRequest();
     request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdFull = admin.getBackupAdmin().backupTables(request);
+    String backupIdFull = client.backupTables(request);
 
     assertTrue(checkSucceeded(backupIdFull));
 
@@ -95,12 +99,12 @@ public class TestIncrementalBackup extends TestBackupBase {
     HTable t1 = insertIntoTable(conn, table1, famName, 1, ADD_ROWS);
     LOG.debug("writing " + ADD_ROWS + " rows to " + table1);
 
-    Assert.assertThat(TEST_UTIL.countRows(t1), CoreMatchers.equalTo(
-        NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_FAM3));
+    Assert.assertThat(TEST_UTIL.countRows(t1),
+      CoreMatchers.equalTo(NB_ROWS_IN_BATCH + ADD_ROWS + NB_ROWS_FAM3));
     t1.close();
     LOG.debug("written " + ADD_ROWS + " rows to " + table1);
 
-    HTable t2 =  (HTable) conn.getTable(table2);
+    HTable t2 = (HTable) conn.getTable(table2);
     Put p2;
     for (int i = 0; i < 5; i++) {
       p2 = new Put(Bytes.toBytes("row-t2" + i));
@@ -111,29 +115,29 @@ public class TestIncrementalBackup extends TestBackupBase {
     Assert.assertThat(TEST_UTIL.countRows(t2), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
     t2.close();
     LOG.debug("written " + 5 + " rows to " + table2);
-    // split table1 
+    // split table1
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     List<HRegion> regions = cluster.getRegions(table1);
 
-    byte[] name = regions.get(0).getRegionInfo().getRegionName();    
+    byte[] name = regions.get(0).getRegionInfo().getRegionName();
     long startSplitTime = EnvironmentEdgeManager.currentTime();
     admin.splitRegion(name);
 
     while (!admin.isTableAvailable(table1)) {
       Thread.sleep(100);
     }
-    
+
     long endSplitTime = EnvironmentEdgeManager.currentTime();
 
-    // split finished 
-    LOG.debug("split finished in ="+ (endSplitTime - startSplitTime));
-    
+    // split finished
+    LOG.debug("split finished in =" + (endSplitTime - startSplitTime));
+
     // #3 - incremental backup for multiple tables
     tables = Lists.newArrayList(table1, table2);
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
-    .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdIncMultiple = admin.getBackupAdmin().backupTables(request);
+        .setTargetRootDir(BACKUP_ROOT_DIR);
+    String backupIdIncMultiple = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdIncMultiple));
 
     // add column family f2 to table1
@@ -150,22 +154,18 @@ public class TestIncrementalBackup extends TestBackupBase {
     // #3 - incremental backup for multiple tables
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
-    .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdIncMultiple2 = admin.getBackupAdmin().backupTables(request);
+        .setTargetRootDir(BACKUP_ROOT_DIR);
+    String backupIdIncMultiple2 = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdIncMultiple2));
 
     // #4 - restore full backup for all tables, without overwrite
-    TableName[] tablesRestoreFull =
-        new TableName[] { table1, table2 };
+    TableName[] tablesRestoreFull = new TableName[] { table1, table2 };
 
-    TableName[] tablesMapFull =
-        new TableName[] { table1_restore, table2_restore };
+    TableName[] tablesMapFull = new TableName[] { table1_restore, table2_restore };
 
-    BackupAdmin client = getBackupAdmin();
     LOG.debug("Restoring full " + backupIdFull);
     client.restore(RestoreServerUtil.createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false,
-      tablesRestoreFull,
-      tablesMapFull, false));
+      tablesRestoreFull, tablesMapFull, false));
 
     // #5.1 - check tables for full restore
     HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
@@ -176,8 +176,8 @@ public class TestIncrementalBackup extends TestBackupBase {
 
     // #5.2 - checking row count of tables for full restore
     HTable hTable = (HTable) conn.getTable(table1_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH +
-        NB_ROWS_FAM3));
+    Assert.assertThat(TEST_UTIL.countRows(hTable),
+      CoreMatchers.equalTo(NB_ROWS_IN_BATCH + NB_ROWS_FAM3));
     hTable.close();
 
     hTable = (HTable) conn.getTable(table2_restore);
@@ -185,18 +185,16 @@ public class TestIncrementalBackup extends TestBackupBase {
     hTable.close();
 
     // #6 - restore incremental backup for multiple tables, with overwrite
-    TableName[] tablesRestoreIncMultiple =
-        new TableName[] { table1, table2 };
-    TableName[] tablesMapIncMultiple =
-        new TableName[] { table1_restore, table2_restore };
+    TableName[] tablesRestoreIncMultiple = new TableName[] { table1, table2 };
+    TableName[] tablesMapIncMultiple = new TableName[] { table1_restore, table2_restore };
     client.restore(RestoreServerUtil.createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple2,
-        false, tablesRestoreIncMultiple, tablesMapIncMultiple, true));
+      false, tablesRestoreIncMultiple, tablesMapIncMultiple, true));
 
     hTable = (HTable) conn.getTable(table1_restore);
     LOG.debug("After incremental restore: " + hTable.getTableDescriptor());
     LOG.debug("f1 has " + TEST_UTIL.countRows(hTable, famName) + " rows");
     Assert.assertThat(TEST_UTIL.countRows(hTable, famName),
-        CoreMatchers.equalTo(NB_ROWS_IN_BATCH + ADD_ROWS));
+      CoreMatchers.equalTo(NB_ROWS_IN_BATCH + ADD_ROWS));
     LOG.debug("f2 has " + TEST_UTIL.countRows(hTable, fam2Name) + " rows");
     Assert.assertThat(TEST_UTIL.countRows(hTable, fam2Name), CoreMatchers.equalTo(NB_ROWS_FAM2));
     hTable.close();
@@ -207,6 +205,7 @@ public class TestIncrementalBackup extends TestBackupBase {
 
     admin.close();
     conn.close();
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
index a7c0713..0a73888 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
@@ -25,8 +25,8 @@ 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.HBaseBackupAdmin;
 import org.apache.hadoop.hbase.backup.util.RestoreServerUtil;
-import org.apache.hadoop.hbase.client.BackupAdmin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -63,10 +63,11 @@ public class TestIncrementalBackupDeleteTable extends TestBackupBase {
     HBaseAdmin admin = null;
     Connection conn = ConnectionFactory.createConnection(conf1);
     admin = (HBaseAdmin) conn.getAdmin();
+    HBaseBackupAdmin client = new HBaseBackupAdmin(conn);
 
     BackupRequest request = new BackupRequest();
     request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdFull = admin.getBackupAdmin().backupTables(request);
+    String backupIdFull = client.backupTables(request);
 
     assertTrue(checkSucceeded(backupIdFull));
 
@@ -91,7 +92,7 @@ public class TestIncrementalBackupDeleteTable extends TestBackupBase {
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
     .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdIncMultiple = admin.getBackupAdmin().backupTables(request);
+    String backupIdIncMultiple = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdIncMultiple));
 
     // #4 - restore full backup for all tables, without overwrite
@@ -101,7 +102,6 @@ public class TestIncrementalBackupDeleteTable extends TestBackupBase {
     TableName[] tablesMapFull =
         new TableName[] { table1_restore, table2_restore };
 
-    BackupAdmin client = getBackupAdmin();
     client.restore(RestoreServerUtil.createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false,
       tablesRestoreFull,
       tablesMapFull, false));

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/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 4f4f7ad..42f0ee7 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
@@ -22,7 +22,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.util.RestoreServerUtil;
-import org.apache.hadoop.hbase.client.BackupAdmin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 2a8e3c9..c88f60d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -431,23 +431,6 @@ public class TestCatalogJanitor {
       return null;  //To change body of implemented methods use File | Settings | File Templates.
     }
 
-    @Override
-    public Pair<Long, String> backupTables(
-        final BackupType type,
-        final List<TableName> tableList,
-        final String targetRootDir, final int workers,
-        final long bandwidth, final String setName,
-        final long nonceGroup, final long nonce) throws IOException {
-      return null;
-    }
-
-    @Override
-    public long restoreTables(String backupRootDir,
-        String backupId, boolean check, List<TableName> sTableList,
-        List<TableName> tTableList, boolean isOverwrite, long nonceGroup, long nonce)
-            throws IOException {
-      return -1;
-    }
 
     @Override
     public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {


[07/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
index 4699c81..c5220cc 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/BackupProtos.java
@@ -9,198 +9,27 @@ public final class BackupProtos {
       com.google.protobuf.ExtensionRegistry registry) {
   }
   /**
-   * Protobuf enum {@code hbase.pb.FullTableBackupState}
-   */
-  public enum FullTableBackupState
-      implements com.google.protobuf.ProtocolMessageEnum {
-    /**
-     * <code>PRE_SNAPSHOT_TABLE = 1;</code>
-     */
-    PRE_SNAPSHOT_TABLE(0, 1),
-    /**
-     * <code>SNAPSHOT_TABLES = 2;</code>
-     */
-    SNAPSHOT_TABLES(1, 2),
-    /**
-     * <code>SNAPSHOT_COPY = 3;</code>
-     */
-    SNAPSHOT_COPY(2, 3),
-    /**
-     * <code>BACKUP_COMPLETE = 4;</code>
-     */
-    BACKUP_COMPLETE(3, 4),
-    ;
-
-    /**
-     * <code>PRE_SNAPSHOT_TABLE = 1;</code>
-     */
-    public static final int PRE_SNAPSHOT_TABLE_VALUE = 1;
-    /**
-     * <code>SNAPSHOT_TABLES = 2;</code>
-     */
-    public static final int SNAPSHOT_TABLES_VALUE = 2;
-    /**
-     * <code>SNAPSHOT_COPY = 3;</code>
-     */
-    public static final int SNAPSHOT_COPY_VALUE = 3;
-    /**
-     * <code>BACKUP_COMPLETE = 4;</code>
-     */
-    public static final int BACKUP_COMPLETE_VALUE = 4;
-
-
-    public final int getNumber() { return value; }
-
-    public static FullTableBackupState valueOf(int value) {
-      switch (value) {
-        case 1: return PRE_SNAPSHOT_TABLE;
-        case 2: return SNAPSHOT_TABLES;
-        case 3: return SNAPSHOT_COPY;
-        case 4: return BACKUP_COMPLETE;
-        default: return null;
-      }
-    }
-
-    public static com.google.protobuf.Internal.EnumLiteMap<FullTableBackupState>
-        internalGetValueMap() {
-      return internalValueMap;
-    }
-    private static com.google.protobuf.Internal.EnumLiteMap<FullTableBackupState>
-        internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap<FullTableBackupState>() {
-            public FullTableBackupState findValueByNumber(int number) {
-              return FullTableBackupState.valueOf(number);
-            }
-          };
-
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
-        getValueDescriptor() {
-      return getDescriptor().getValues().get(index);
-    }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptorForType() {
-      return getDescriptor();
-    }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.getDescriptor().getEnumTypes().get(0);
-    }
-
-    private static final FullTableBackupState[] VALUES = values();
-
-    public static FullTableBackupState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-      if (desc.getType() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "EnumValueDescriptor is not for this type.");
-      }
-      return VALUES[desc.getIndex()];
-    }
-
-    private final int index;
-    private final int value;
-
-    private FullTableBackupState(int index, int value) {
-      this.index = index;
-      this.value = value;
-    }
-
-    // @@protoc_insertion_point(enum_scope:hbase.pb.FullTableBackupState)
-  }
-
-  /**
-   * Protobuf enum {@code hbase.pb.IncrementalTableBackupState}
-   */
-  public enum IncrementalTableBackupState
-      implements com.google.protobuf.ProtocolMessageEnum {
-    /**
-     * <code>PREPARE_INCREMENTAL = 1;</code>
-     */
-    PREPARE_INCREMENTAL(0, 1),
-    /**
-     * <code>INCREMENTAL_COPY = 2;</code>
-     */
-    INCREMENTAL_COPY(1, 2),
-    /**
-     * <code>INCR_BACKUP_COMPLETE = 3;</code>
-     */
-    INCR_BACKUP_COMPLETE(2, 3),
-    ;
-
-    /**
-     * <code>PREPARE_INCREMENTAL = 1;</code>
-     */
-    public static final int PREPARE_INCREMENTAL_VALUE = 1;
-    /**
-     * <code>INCREMENTAL_COPY = 2;</code>
-     */
-    public static final int INCREMENTAL_COPY_VALUE = 2;
-    /**
-     * <code>INCR_BACKUP_COMPLETE = 3;</code>
-     */
-    public static final int INCR_BACKUP_COMPLETE_VALUE = 3;
-
-
-    public final int getNumber() { return value; }
-
-    public static IncrementalTableBackupState valueOf(int value) {
-      switch (value) {
-        case 1: return PREPARE_INCREMENTAL;
-        case 2: return INCREMENTAL_COPY;
-        case 3: return INCR_BACKUP_COMPLETE;
-        default: return null;
-      }
-    }
-
-    public static com.google.protobuf.Internal.EnumLiteMap<IncrementalTableBackupState>
-        internalGetValueMap() {
-      return internalValueMap;
-    }
-    private static com.google.protobuf.Internal.EnumLiteMap<IncrementalTableBackupState>
-        internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap<IncrementalTableBackupState>() {
-            public IncrementalTableBackupState findValueByNumber(int number) {
-              return IncrementalTableBackupState.valueOf(number);
-            }
-          };
-
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
-        getValueDescriptor() {
-      return getDescriptor().getValues().get(index);
-    }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptorForType() {
-      return getDescriptor();
-    }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.getDescriptor().getEnumTypes().get(1);
-    }
-
-    private static final IncrementalTableBackupState[] VALUES = values();
-
-    public static IncrementalTableBackupState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-      if (desc.getType() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "EnumValueDescriptor is not for this type.");
-      }
-      return VALUES[desc.getIndex()];
-    }
-
-    private final int index;
-    private final int value;
-
-    private IncrementalTableBackupState(int index, int value) {
-      this.index = index;
-      this.value = value;
-    }
-
-    // @@protoc_insertion_point(enum_scope:hbase.pb.IncrementalTableBackupState)
-  }
-
-  /**
    * Protobuf enum {@code hbase.pb.BackupType}
+   *
+   * <pre>
+   *enum FullTableBackupState {
+   *PRE_SNAPSHOT_TABLE = 1;
+   *SNAPSHOT_TABLES = 2;
+   *SNAPSHOT_COPY = 3;
+   *BACKUP_COMPLETE = 4;
+   *}
+   *
+   *enum IncrementalTableBackupState {
+   *PREPARE_INCREMENTAL = 1;
+   *INCREMENTAL_COPY = 2;
+   *INCR_BACKUP_COMPLETE = 3;
+   *}
+   *
+   *message SnapshotTableStateData {
+   *required TableName table = 1;
+   *required string snapshotName = 2;
+   *}
+   * </pre>
    */
   public enum BackupType
       implements com.google.protobuf.ProtocolMessageEnum {
@@ -256,7 +85,7 @@ public final class BackupProtos {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.getDescriptor().getEnumTypes().get(2);
+      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.getDescriptor().getEnumTypes().get(0);
     }
 
     private static final BackupType[] VALUES = values();
@@ -281,57 +110,138 @@ public final class BackupProtos {
     // @@protoc_insertion_point(enum_scope:hbase.pb.BackupType)
   }
 
-  public interface SnapshotTableStateDataOrBuilder
+  public interface BackupImageOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // required .hbase.pb.TableName table = 1;
+    // required string backup_id = 1;
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>required string backup_id = 1;</code>
      */
-    boolean hasTable();
+    boolean hasBackupId();
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>required string backup_id = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable();
+    java.lang.String getBackupId();
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>required string backup_id = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder();
+    com.google.protobuf.ByteString
+        getBackupIdBytes();
+
+    // required .hbase.pb.BackupType backup_type = 2;
+    /**
+     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+     */
+    boolean hasBackupType();
+    /**
+     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType();
 
-    // required string snapshotName = 2;
+    // required string root_dir = 3;
     /**
-     * <code>required string snapshotName = 2;</code>
+     * <code>required string root_dir = 3;</code>
      */
-    boolean hasSnapshotName();
+    boolean hasRootDir();
     /**
-     * <code>required string snapshotName = 2;</code>
+     * <code>required string root_dir = 3;</code>
      */
-    java.lang.String getSnapshotName();
+    java.lang.String getRootDir();
     /**
-     * <code>required string snapshotName = 2;</code>
+     * <code>required string root_dir = 3;</code>
      */
     com.google.protobuf.ByteString
-        getSnapshotNameBytes();
+        getRootDirBytes();
+
+    // repeated .hbase.pb.TableName table_list = 4;
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> 
+        getTableListList();
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index);
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    int getTableListCount();
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+        getTableListOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
+        int index);
+
+    // required uint64 start_ts = 5;
+    /**
+     * <code>required uint64 start_ts = 5;</code>
+     */
+    boolean hasStartTs();
+    /**
+     * <code>required uint64 start_ts = 5;</code>
+     */
+    long getStartTs();
+
+    // required uint64 complete_ts = 6;
+    /**
+     * <code>required uint64 complete_ts = 6;</code>
+     */
+    boolean hasCompleteTs();
+    /**
+     * <code>required uint64 complete_ts = 6;</code>
+     */
+    long getCompleteTs();
+
+    // repeated .hbase.pb.BackupImage ancestors = 7;
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> 
+        getAncestorsList();
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index);
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    int getAncestorsCount();
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
+        getAncestorsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
+        int index);
   }
   /**
-   * Protobuf type {@code hbase.pb.SnapshotTableStateData}
+   * Protobuf type {@code hbase.pb.BackupImage}
    */
-  public static final class SnapshotTableStateData extends
+  public static final class BackupImage extends
       com.google.protobuf.GeneratedMessage
-      implements SnapshotTableStateDataOrBuilder {
-    // Use SnapshotTableStateData.newBuilder() to construct.
-    private SnapshotTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements BackupImageOrBuilder {
+    // Use BackupImage.newBuilder() to construct.
+    private BackupImage(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private SnapshotTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private BackupImage(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final SnapshotTableStateData defaultInstance;
-    public static SnapshotTableStateData getDefaultInstance() {
+    private static final BackupImage defaultInstance;
+    public static BackupImage getDefaultInstance() {
       return defaultInstance;
     }
 
-    public SnapshotTableStateData getDefaultInstanceForType() {
+    public BackupImage getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -341,7 +251,7 @@ public final class BackupProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private SnapshotTableStateData(
+    private BackupImage(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -365,21 +275,50 @@ public final class BackupProtos {
               break;
             }
             case 10: {
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = table_.toBuilder();
-              }
-              table_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(table_);
-                table_ = subBuilder.buildPartial();
-              }
               bitField0_ |= 0x00000001;
+              backupId_ = input.readBytes();
               break;
             }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              snapshotName_ = input.readBytes();
+            case 16: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType value = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(2, rawValue);
+              } else {
+                bitField0_ |= 0x00000002;
+                backupType_ = value;
+              }
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              rootDir_ = input.readBytes();
+              break;
+            }
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+                tableList_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>();
+                mutable_bitField0_ |= 0x00000008;
+              }
+              tableList_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry));
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000008;
+              startTs_ = input.readUInt64();
+              break;
+            }
+            case 48: {
+              bitField0_ |= 0x00000010;
+              completeTs_ = input.readUInt64();
+              break;
+            }
+            case 58: {
+              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+                ancestors_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage>();
+                mutable_bitField0_ |= 0x00000040;
+              }
+              ancestors_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.PARSER, extensionRegistry));
               break;
             }
           }
@@ -390,74 +329,117 @@ public final class BackupProtos {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
+        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+          tableList_ = java.util.Collections.unmodifiableList(tableList_);
+        }
+        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+          ancestors_ = java.util.Collections.unmodifiableList(ancestors_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_SnapshotTableStateData_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_SnapshotTableStateData_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<SnapshotTableStateData> PARSER =
-        new com.google.protobuf.AbstractParser<SnapshotTableStateData>() {
-      public SnapshotTableStateData parsePartialFrom(
+    public static com.google.protobuf.Parser<BackupImage> PARSER =
+        new com.google.protobuf.AbstractParser<BackupImage>() {
+      public BackupImage parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new SnapshotTableStateData(input, extensionRegistry);
+        return new BackupImage(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<SnapshotTableStateData> getParserForType() {
+    public com.google.protobuf.Parser<BackupImage> getParserForType() {
       return PARSER;
     }
 
     private int bitField0_;
-    // required .hbase.pb.TableName table = 1;
-    public static final int TABLE_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName table_;
+    // required string backup_id = 1;
+    public static final int BACKUP_ID_FIELD_NUMBER = 1;
+    private java.lang.Object backupId_;
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>required string backup_id = 1;</code>
      */
-    public boolean hasTable() {
+    public boolean hasBackupId() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>required string backup_id = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable() {
-      return table_;
+    public java.lang.String getBackupId() {
+      java.lang.Object ref = backupId_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          backupId_ = s;
+        }
+        return s;
+      }
     }
     /**
-     * <code>required .hbase.pb.TableName table = 1;</code>
+     * <code>required string backup_id = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder() {
-      return table_;
+    public com.google.protobuf.ByteString
+        getBackupIdBytes() {
+      java.lang.Object ref = backupId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        backupId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
     }
 
-    // required string snapshotName = 2;
-    public static final int SNAPSHOTNAME_FIELD_NUMBER = 2;
-    private java.lang.Object snapshotName_;
+    // required .hbase.pb.BackupType backup_type = 2;
+    public static final int BACKUP_TYPE_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType backupType_;
     /**
-     * <code>required string snapshotName = 2;</code>
+     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
      */
-    public boolean hasSnapshotName() {
+    public boolean hasBackupType() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>required string snapshotName = 2;</code>
+     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType() {
+      return backupType_;
+    }
+
+    // required string root_dir = 3;
+    public static final int ROOT_DIR_FIELD_NUMBER = 3;
+    private java.lang.Object rootDir_;
+    /**
+     * <code>required string root_dir = 3;</code>
+     */
+    public boolean hasRootDir() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required string root_dir = 3;</code>
      */
-    public java.lang.String getSnapshotName() {
-      java.lang.Object ref = snapshotName_;
+    public java.lang.String getRootDir() {
+      java.lang.Object ref = rootDir_;
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
@@ -465,61 +447,205 @@ public final class BackupProtos {
             (com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
-          snapshotName_ = s;
+          rootDir_ = s;
         }
         return s;
       }
     }
     /**
-     * <code>required string snapshotName = 2;</code>
+     * <code>required string root_dir = 3;</code>
      */
     public com.google.protobuf.ByteString
-        getSnapshotNameBytes() {
-      java.lang.Object ref = snapshotName_;
+        getRootDirBytes() {
+      java.lang.Object ref = rootDir_;
       if (ref instanceof java.lang.String) {
         com.google.protobuf.ByteString b = 
             com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
-        snapshotName_ = b;
+        rootDir_ = b;
         return b;
       } else {
         return (com.google.protobuf.ByteString) ref;
       }
     }
 
+    // repeated .hbase.pb.TableName table_list = 4;
+    public static final int TABLE_LIST_FIELD_NUMBER = 4;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tableList_;
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTableListList() {
+      return tableList_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+        getTableListOrBuilderList() {
+      return tableList_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public int getTableListCount() {
+      return tableList_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index) {
+      return tableList_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
+        int index) {
+      return tableList_.get(index);
+    }
+
+    // required uint64 start_ts = 5;
+    public static final int START_TS_FIELD_NUMBER = 5;
+    private long startTs_;
+    /**
+     * <code>required uint64 start_ts = 5;</code>
+     */
+    public boolean hasStartTs() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required uint64 start_ts = 5;</code>
+     */
+    public long getStartTs() {
+      return startTs_;
+    }
+
+    // required uint64 complete_ts = 6;
+    public static final int COMPLETE_TS_FIELD_NUMBER = 6;
+    private long completeTs_;
+    /**
+     * <code>required uint64 complete_ts = 6;</code>
+     */
+    public boolean hasCompleteTs() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>required uint64 complete_ts = 6;</code>
+     */
+    public long getCompleteTs() {
+      return completeTs_;
+    }
+
+    // repeated .hbase.pb.BackupImage ancestors = 7;
+    public static final int ANCESTORS_FIELD_NUMBER = 7;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> ancestors_;
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> getAncestorsList() {
+      return ancestors_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
+        getAncestorsOrBuilderList() {
+      return ancestors_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public int getAncestorsCount() {
+      return ancestors_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index) {
+      return ancestors_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
+        int index) {
+      return ancestors_.get(index);
+    }
+
     private void initFields() {
-      table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
-      snapshotName_ = "";
+      backupId_ = "";
+      backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
+      rootDir_ = "";
+      tableList_ = java.util.Collections.emptyList();
+      startTs_ = 0L;
+      completeTs_ = 0L;
+      ancestors_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (!hasTable()) {
+      if (!hasBackupId()) {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasSnapshotName()) {
+      if (!hasBackupType()) {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!getTable().isInitialized()) {
+      if (!hasRootDir()) {
         memoizedIsInitialized = 0;
         return false;
       }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeMessage(1, table_);
+      if (!hasStartTs()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasCompleteTs()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getTableListCount(); i++) {
+        if (!getTableList(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getAncestorsCount(); i++) {
+        if (!getAncestors(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getBackupIdBytes());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, getSnapshotNameBytes());
+        output.writeEnum(2, backupType_.getNumber());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, getRootDirBytes());
+      }
+      for (int i = 0; i < tableList_.size(); i++) {
+        output.writeMessage(4, tableList_.get(i));
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt64(5, startTs_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeUInt64(6, completeTs_);
+      }
+      for (int i = 0; i < ancestors_.size(); i++) {
+        output.writeMessage(7, ancestors_.get(i));
       }
       getUnknownFields().writeTo(output);
     }
@@ -532,11 +658,31 @@ public final class BackupProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, table_);
+          .computeBytesSize(1, getBackupIdBytes());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, getSnapshotNameBytes());
+          .computeEnumSize(2, backupType_.getNumber());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getRootDirBytes());
+      }
+      for (int i = 0; i < tableList_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, tableList_.get(i));
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(5, startTs_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(6, completeTs_);
+      }
+      for (int i = 0; i < ancestors_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(7, ancestors_.get(i));
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -555,22 +701,41 @@ public final class BackupProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData) obj;
+      org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage other = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) obj;
 
       boolean result = true;
-      result = result && (hasTable() == other.hasTable());
-      if (hasTable()) {
-        result = result && getTable()
-            .equals(other.getTable());
+      result = result && (hasBackupId() == other.hasBackupId());
+      if (hasBackupId()) {
+        result = result && getBackupId()
+            .equals(other.getBackupId());
+      }
+      result = result && (hasBackupType() == other.hasBackupType());
+      if (hasBackupType()) {
+        result = result &&
+            (getBackupType() == other.getBackupType());
       }
-      result = result && (hasSnapshotName() == other.hasSnapshotName());
-      if (hasSnapshotName()) {
-        result = result && getSnapshotName()
-            .equals(other.getSnapshotName());
+      result = result && (hasRootDir() == other.hasRootDir());
+      if (hasRootDir()) {
+        result = result && getRootDir()
+            .equals(other.getRootDir());
+      }
+      result = result && getTableListList()
+          .equals(other.getTableListList());
+      result = result && (hasStartTs() == other.hasStartTs());
+      if (hasStartTs()) {
+        result = result && (getStartTs()
+            == other.getStartTs());
+      }
+      result = result && (hasCompleteTs() == other.hasCompleteTs());
+      if (hasCompleteTs()) {
+        result = result && (getCompleteTs()
+            == other.getCompleteTs());
       }
+      result = result && getAncestorsList()
+          .equals(other.getAncestorsList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -584,66 +749,86 @@ public final class BackupProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasTable()) {
-        hash = (37 * hash) + TABLE_FIELD_NUMBER;
-        hash = (53 * hash) + getTable().hashCode();
+      if (hasBackupId()) {
+        hash = (37 * hash) + BACKUP_ID_FIELD_NUMBER;
+        hash = (53 * hash) + getBackupId().hashCode();
+      }
+      if (hasBackupType()) {
+        hash = (37 * hash) + BACKUP_TYPE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getBackupType());
+      }
+      if (hasRootDir()) {
+        hash = (37 * hash) + ROOT_DIR_FIELD_NUMBER;
+        hash = (53 * hash) + getRootDir().hashCode();
       }
-      if (hasSnapshotName()) {
-        hash = (37 * hash) + SNAPSHOTNAME_FIELD_NUMBER;
-        hash = (53 * hash) + getSnapshotName().hashCode();
+      if (getTableListCount() > 0) {
+        hash = (37 * hash) + TABLE_LIST_FIELD_NUMBER;
+        hash = (53 * hash) + getTableListList().hashCode();
+      }
+      if (hasStartTs()) {
+        hash = (37 * hash) + START_TS_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getStartTs());
+      }
+      if (hasCompleteTs()) {
+        hash = (37 * hash) + COMPLETE_TS_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getCompleteTs());
+      }
+      if (getAncestorsCount() > 0) {
+        hash = (37 * hash) + ANCESTORS_FIELD_NUMBER;
+        hash = (53 * hash) + getAncestorsList().hashCode();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -652,7 +837,7 @@ public final class BackupProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -664,24 +849,24 @@ public final class BackupProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.SnapshotTableStateData}
+     * Protobuf type {@code hbase.pb.BackupImage}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateDataOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_SnapshotTableStateData_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_SnapshotTableStateData_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -693,7 +878,8 @@ public final class BackupProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getTableFieldBuilder();
+          getTableListFieldBuilder();
+          getAncestorsFieldBuilder();
         }
       }
       private static Builder create() {
@@ -702,14 +888,28 @@ public final class BackupProtos {
 
       public Builder clear() {
         super.clear();
-        if (tableBuilder_ == null) {
-          table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
-        } else {
-          tableBuilder_.clear();
-        }
+        backupId_ = "";
         bitField0_ = (bitField0_ & ~0x00000001);
-        snapshotName_ = "";
+        backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
         bitField0_ = (bitField0_ & ~0x00000002);
+        rootDir_ = "";
+        bitField0_ = (bitField0_ & ~0x00000004);
+        if (tableListBuilder_ == null) {
+          tableList_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000008);
+        } else {
+          tableListBuilder_.clear();
+        }
+        startTs_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        completeTs_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        if (ancestorsBuilder_ == null) {
+          ancestors_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000040);
+        } else {
+          ancestorsBuilder_.clear();
+        }
         return this;
       }
 
@@ -719,2731 +919,997 @@ public final class BackupProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_SnapshotTableStateData_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData build() {
-        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage build() {
+        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData(this);
+      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage result = new org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        if (tableBuilder_ == null) {
-          result.table_ = table_;
-        } else {
-          result.table_ = tableBuilder_.build();
-        }
+        result.backupId_ = backupId_;
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.snapshotName_ = snapshotName_;
+        result.backupType_ = backupType_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.rootDir_ = rootDir_;
+        if (tableListBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008)) {
+            tableList_ = java.util.Collections.unmodifiableList(tableList_);
+            bitField0_ = (bitField0_ & ~0x00000008);
+          }
+          result.tableList_ = tableList_;
+        } else {
+          result.tableList_ = tableListBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.startTs_ = startTs_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.completeTs_ = completeTs_;
+        if (ancestorsBuilder_ == null) {
+          if (((bitField0_ & 0x00000040) == 0x00000040)) {
+            ancestors_ = java.util.Collections.unmodifiableList(ancestors_);
+            bitField0_ = (bitField0_ & ~0x00000040);
+          }
+          result.ancestors_ = ancestors_;
+        } else {
+          result.ancestors_ = ancestorsBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData.getDefaultInstance()) return this;
-        if (other.hasTable()) {
-          mergeTable(other.getTable());
-        }
-        if (other.hasSnapshotName()) {
-          bitField0_ |= 0x00000002;
-          snapshotName_ = other.snapshotName_;
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.getDefaultInstance()) return this;
+        if (other.hasBackupId()) {
+          bitField0_ |= 0x00000001;
+          backupId_ = other.backupId_;
           onChanged();
         }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasTable()) {
-          
-          return false;
-        }
-        if (!hasSnapshotName()) {
-          
-          return false;
+        if (other.hasBackupType()) {
+          setBackupType(other.getBackupType());
         }
-        if (!getTable().isInitialized()) {
-          
-          return false;
+        if (other.hasRootDir()) {
+          bitField0_ |= 0x00000004;
+          rootDir_ = other.rootDir_;
+          onChanged();
         }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.SnapshotTableStateData) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
+        if (tableListBuilder_ == null) {
+          if (!other.tableList_.isEmpty()) {
+            if (tableList_.isEmpty()) {
+              tableList_ = other.tableList_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+            } else {
+              ensureTableListIsMutable();
+              tableList_.addAll(other.tableList_);
+            }
+            onChanged();
           }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // required .hbase.pb.TableName table = 1;
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableBuilder_;
-      /**
-       * <code>required .hbase.pb.TableName table = 1;</code>
-       */
-      public boolean hasTable() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required .hbase.pb.TableName table = 1;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTable() {
-        if (tableBuilder_ == null) {
-          return table_;
         } else {
-          return tableBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>required .hbase.pb.TableName table = 1;</code>
-       */
-      public Builder setTable(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
+          if (!other.tableList_.isEmpty()) {
+            if (tableListBuilder_.isEmpty()) {
+              tableListBuilder_.dispose();
+              tableListBuilder_ = null;
+              tableList_ = other.tableList_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+              tableListBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getTableListFieldBuilder() : null;
+            } else {
+              tableListBuilder_.addAllMessages(other.tableList_);
+            }
           }
-          table_ = value;
-          onChanged();
-        } else {
-          tableBuilder_.setMessage(value);
         }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>required .hbase.pb.TableName table = 1;</code>
-       */
-      public Builder setTable(
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tableBuilder_ == null) {
-          table_ = builderForValue.build();
-          onChanged();
-        } else {
-          tableBuilder_.setMessage(builderForValue.build());
+        if (other.hasStartTs()) {
+          setStartTs(other.getStartTs());
         }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>required .hbase.pb.TableName table = 1;</code>
-       */
-      public Builder mergeTable(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableBuilder_ == null) {
-          if (((bitField0_ & 0x00000001) == 0x00000001) &&
-              table_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
-            table_ =
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(table_).mergeFrom(value).buildPartial();
-          } else {
-            table_ = value;
+        if (other.hasCompleteTs()) {
+          setCompleteTs(other.getCompleteTs());
+        }
+        if (ancestorsBuilder_ == null) {
+          if (!other.ancestors_.isEmpty()) {
+            if (ancestors_.isEmpty()) {
+              ancestors_ = other.ancestors_;
+              bitField0_ = (bitField0_ & ~0x00000040);
+            } else {
+              ensureAncestorsIsMutable();
+              ancestors_.addAll(other.ancestors_);
+            }
+            onChanged();
           }
-          onChanged();
         } else {
-          tableBuilder_.mergeFrom(value);
+          if (!other.ancestors_.isEmpty()) {
+            if (ancestorsBuilder_.isEmpty()) {
+              ancestorsBuilder_.dispose();
+              ancestorsBuilder_ = null;
+              ancestors_ = other.ancestors_;
+              bitField0_ = (bitField0_ & ~0x00000040);
+              ancestorsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getAncestorsFieldBuilder() : null;
+            } else {
+              ancestorsBuilder_.addAllMessages(other.ancestors_);
+            }
+          }
         }
-        bitField0_ |= 0x00000001;
+        this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
-      /**
-       * <code>required .hbase.pb.TableName table = 1;</code>
-       */
-      public Builder clearTable() {
-        if (tableBuilder_ == null) {
-          table_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
-          onChanged();
-        } else {
-          tableBuilder_.clear();
+
+      public final boolean isInitialized() {
+        if (!hasBackupId()) {
+          
+          return false;
         }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-      /**
-       * <code>required .hbase.pb.TableName table = 1;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableBuilder() {
-        bitField0_ |= 0x00000001;
-        onChanged();
-        return getTableFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>required .hbase.pb.TableName table = 1;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableOrBuilder() {
-        if (tableBuilder_ != null) {
-          return tableBuilder_.getMessageOrBuilder();
-        } else {
-          return table_;
+        if (!hasBackupType()) {
+          
+          return false;
+        }
+        if (!hasRootDir()) {
+          
+          return false;
+        }
+        if (!hasStartTs()) {
+          
+          return false;
+        }
+        if (!hasCompleteTs()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getTableListCount(); i++) {
+          if (!getTableList(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        for (int i = 0; i < getAncestorsCount(); i++) {
+          if (!getAncestors(i).isInitialized()) {
+            
+            return false;
+          }
         }
+        return true;
       }
-      /**
-       * <code>required .hbase.pb.TableName table = 1;</code>
-       */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-          getTableFieldBuilder() {
-        if (tableBuilder_ == null) {
-          tableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
-                  table_,
-                  getParentForChildren(),
-                  isClean());
-          table_ = null;
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
         }
-        return tableBuilder_;
+        return this;
       }
+      private int bitField0_;
 
-      // required string snapshotName = 2;
-      private java.lang.Object snapshotName_ = "";
+      // required string backup_id = 1;
+      private java.lang.Object backupId_ = "";
       /**
-       * <code>required string snapshotName = 2;</code>
+       * <code>required string backup_id = 1;</code>
        */
-      public boolean hasSnapshotName() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
+      public boolean hasBackupId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required string snapshotName = 2;</code>
+       * <code>required string backup_id = 1;</code>
        */
-      public java.lang.String getSnapshotName() {
-        java.lang.Object ref = snapshotName_;
+      public java.lang.String getBackupId() {
+        java.lang.Object ref = backupId_;
         if (!(ref instanceof java.lang.String)) {
           java.lang.String s = ((com.google.protobuf.ByteString) ref)
               .toStringUtf8();
-          snapshotName_ = s;
+          backupId_ = s;
           return s;
         } else {
           return (java.lang.String) ref;
         }
       }
       /**
-       * <code>required string snapshotName = 2;</code>
+       * <code>required string backup_id = 1;</code>
        */
       public com.google.protobuf.ByteString
-          getSnapshotNameBytes() {
-        java.lang.Object ref = snapshotName_;
+          getBackupIdBytes() {
+        java.lang.Object ref = backupId_;
         if (ref instanceof String) {
           com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
-          snapshotName_ = b;
+          backupId_ = b;
           return b;
         } else {
           return (com.google.protobuf.ByteString) ref;
         }
       }
       /**
-       * <code>required string snapshotName = 2;</code>
+       * <code>required string backup_id = 1;</code>
        */
-      public Builder setSnapshotName(
+      public Builder setBackupId(
           java.lang.String value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00000002;
-        snapshotName_ = value;
+  bitField0_ |= 0x00000001;
+        backupId_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required string snapshotName = 2;</code>
+       * <code>required string backup_id = 1;</code>
        */
-      public Builder clearSnapshotName() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        snapshotName_ = getDefaultInstance().getSnapshotName();
+      public Builder clearBackupId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        backupId_ = getDefaultInstance().getBackupId();
         onChanged();
         return this;
       }
       /**
-       * <code>required string snapshotName = 2;</code>
+       * <code>required string backup_id = 1;</code>
        */
-      public Builder setSnapshotNameBytes(
+      public Builder setBackupIdBytes(
           com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00000002;
-        snapshotName_ = value;
+  bitField0_ |= 0x00000001;
+        backupId_ = value;
         onChanged();
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SnapshotTableStateData)
-    }
-
-    static {
-      defaultInstance = new SnapshotTableStateData(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:hbase.pb.SnapshotTableStateData)
-  }
-
-  public interface BackupImageOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required string backup_id = 1;
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    boolean hasBackupId();
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    java.lang.String getBackupId();
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    com.google.protobuf.ByteString
-        getBackupIdBytes();
-
-    // required .hbase.pb.BackupType backup_type = 2;
-    /**
-     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-     */
-    boolean hasBackupType();
-    /**
-     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType();
-
-    // required string root_dir = 3;
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    boolean hasRootDir();
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    java.lang.String getRootDir();
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    com.google.protobuf.ByteString
-        getRootDirBytes();
-
-    // repeated .hbase.pb.TableName table_list = 4;
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> 
-        getTableListList();
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index);
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    int getTableListCount();
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-        getTableListOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
-        int index);
-
-    // required uint64 start_ts = 5;
-    /**
-     * <code>required uint64 start_ts = 5;</code>
-     */
-    boolean hasStartTs();
-    /**
-     * <code>required uint64 start_ts = 5;</code>
-     */
-    long getStartTs();
-
-    // required uint64 complete_ts = 6;
-    /**
-     * <code>required uint64 complete_ts = 6;</code>
-     */
-    boolean hasCompleteTs();
-    /**
-     * <code>required uint64 complete_ts = 6;</code>
-     */
-    long getCompleteTs();
-
-    // repeated .hbase.pb.BackupImage ancestors = 7;
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> 
-        getAncestorsList();
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index);
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    int getAncestorsCount();
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
-        getAncestorsOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
-        int index);
-  }
-  /**
-   * Protobuf type {@code hbase.pb.BackupImage}
-   */
-  public static final class BackupImage extends
-      com.google.protobuf.GeneratedMessage
-      implements BackupImageOrBuilder {
-    // Use BackupImage.newBuilder() to construct.
-    private BackupImage(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private BackupImage(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final BackupImage defaultInstance;
-    public static BackupImage getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public BackupImage getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private BackupImage(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              bitField0_ |= 0x00000001;
-              backupId_ = input.readBytes();
-              break;
-            }
-            case 16: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType value = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(2, rawValue);
-              } else {
-                bitField0_ |= 0x00000002;
-                backupType_ = value;
-              }
-              break;
-            }
-            case 26: {
-              bitField0_ |= 0x00000004;
-              rootDir_ = input.readBytes();
-              break;
-            }
-            case 34: {
-              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-                tableList_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>();
-                mutable_bitField0_ |= 0x00000008;
-              }
-              tableList_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry));
-              break;
-            }
-            case 40: {
-              bitField0_ |= 0x00000008;
-              startTs_ = input.readUInt64();
-              break;
-            }
-            case 48: {
-              bitField0_ |= 0x00000010;
-              completeTs_ = input.readUInt64();
-              break;
-            }
-            case 58: {
-              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
-                ancestors_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage>();
-                mutable_bitField0_ |= 0x00000040;
-              }
-              ancestors_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.PARSER, extensionRegistry));
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-          tableList_ = java.util.Collections.unmodifiableList(tableList_);
-        }
-        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
-          ancestors_ = java.util.Collections.unmodifiableList(ancestors_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.BackupProtos.internal_static_hbase_pb_BackupImage_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.class, org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<BackupImage> PARSER =
-        new com.google.protobuf.AbstractParser<BackupImage>() {
-      public BackupImage parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new BackupImage(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<BackupImage> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // required string backup_id = 1;
-    public static final int BACKUP_ID_FIELD_NUMBER = 1;
-    private java.lang.Object backupId_;
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    public boolean hasBackupId() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    public java.lang.String getBackupId() {
-      java.lang.Object ref = backupId_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          backupId_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string backup_id = 1;</code>
-     */
-    public com.google.protobuf.ByteString
-        getBackupIdBytes() {
-      java.lang.Object ref = backupId_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        backupId_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // required .hbase.pb.BackupType backup_type = 2;
-    public static final int BACKUP_TYPE_FIELD_NUMBER = 2;
-    private org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType backupType_;
-    /**
-     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-     */
-    public boolean hasBackupType() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required .hbase.pb.BackupType backup_type = 2;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getBackupType() {
-      return backupType_;
-    }
-
-    // required string root_dir = 3;
-    public static final int ROOT_DIR_FIELD_NUMBER = 3;
-    private java.lang.Object rootDir_;
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    public boolean hasRootDir() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    public java.lang.String getRootDir() {
-      java.lang.Object ref = rootDir_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          rootDir_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string root_dir = 3;</code>
-     */
-    public com.google.protobuf.ByteString
-        getRootDirBytes() {
-      java.lang.Object ref = rootDir_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        rootDir_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // repeated .hbase.pb.TableName table_list = 4;
-    public static final int TABLE_LIST_FIELD_NUMBER = 4;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tableList_;
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTableListList() {
-      return tableList_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-        getTableListOrBuilderList() {
-      return tableList_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public int getTableListCount() {
-      return tableList_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableList(int index) {
-      return tableList_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName table_list = 4;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableListOrBuilder(
-        int index) {
-      return tableList_.get(index);
-    }
-
-    // required uint64 start_ts = 5;
-    public static final int START_TS_FIELD_NUMBER = 5;
-    private long startTs_;
-    /**
-     * <code>required uint64 start_ts = 5;</code>
-     */
-    public boolean hasStartTs() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>required uint64 start_ts = 5;</code>
-     */
-    public long getStartTs() {
-      return startTs_;
-    }
-
-    // required uint64 complete_ts = 6;
-    public static final int COMPLETE_TS_FIELD_NUMBER = 6;
-    private long completeTs_;
-    /**
-     * <code>required uint64 complete_ts = 6;</code>
-     */
-    public boolean hasCompleteTs() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>required uint64 complete_ts = 6;</code>
-     */
-    public long getCompleteTs() {
-      return completeTs_;
-    }
-
-    // repeated .hbase.pb.BackupImage ancestors = 7;
-    public static final int ANCESTORS_FIELD_NUMBER = 7;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> ancestors_;
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage> getAncestorsList() {
-      return ancestors_;
-    }
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder> 
-        getAncestorsOrBuilderList() {
-      return ancestors_;
-    }
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public int getAncestorsCount() {
-      return ancestors_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage getAncestors(int index) {
-      return ancestors_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.BackupImage ancestors = 7;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImageOrBuilder getAncestorsOrBuilder(
-        int index) {
-      return ancestors_.get(index);
-    }
-
-    private void initFields() {
-      backupId_ = "";
-      backupType_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
-      rootDir_ = "";
-      tableList_ = java.util.Collections.emptyList();
-      startTs_ = 0L;
-      completeTs_ = 0L;
-      ancestors_ = java.util.Collections.emptyList();
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasBackupId()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasBackupType()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasRootDir()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasStartTs()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasCompleteTs()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      for (int i = 0; i < getTableListCount(); i++) {
-        if (!getTableList(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      for (int i = 0; i < getAncestorsCount(); i++) {
-        if (!getAncestors(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, getBackupIdBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeEnum(2, backupType_.getNumber());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeBytes(3, getRootDirBytes());
-      }
-      for (int i = 0; i < tableList_.size(); i++) {
-        output.writeMessage(4, tableList_.get(i));
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeUInt64(5, startTs_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeUInt64(6, completeTs_);
-      }
-      for (int i = 0; i < ancestors_.size(); i++) {
-        output.writeMessage(7, ancestors_.get(i));
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, getBackupIdBytes());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeEnumSize(2, backupType_.getNumber());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(3, getRootDirBytes());
-      }
-      for (int i = 0; i < tableList_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(4, tableList_.get(i));
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(5, startTs_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(6, completeTs_);
-      }
-      for (int i = 0; i < ancestors_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(7, ancestors_.get(i));
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage other = (org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage) obj;
-
-      boolean result = true;
-      result = result && (hasBackupId() == other.hasBackupId());
-      if (hasBackupId()) {
-        result = result && getBackupId()
-            .equals(other.getBackupId());
-      }
-      result = result && (hasBackupType() == other.hasBackupType());
-      if (hasBackupType()) {
-        result = result &&
-            (getBackupType() == other.getBackupType());
-      }
-      result = result && (hasRootDir() == other.hasRootDir());
-      if (hasRootDir()) {
-        result = result && getRootDir()
-            .equals(other.getRootDir());
-      }
-      result = result && getTableListList()
-          .equals(other.getTableListList());
-      result = result && (hasStartTs() == other.hasStartTs());
-      if (hasStartTs()) {
-        result = result && (getStartTs()
-            == other.getStartTs());
-      }
-      result = result && (hasCompleteTs() == other.hasCompleteTs());
-      if (hasCompleteTs()) {
-        result = result && (getCompleteTs()
-            == other.getCompleteTs());
-      }
-      result = result && getAncestorsList()
-          .equals(other.getAncestorsList());
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasBackupId()) {
-        hash = (37 * hash) + BACKUP_ID_FIELD_NUMBER;
-        hash = (53 * hash) + getBackupId().hashCode();
-      }
-      if (hasBackupType()) {
-        hash = (37 * hash) + BACKUP_TYPE_FIELD_NUMBER;
-        hash = (53 * hash) + hashEnum(getBackupType());
-      }
-      if (hasRootDir()) {
-        hash = (37 * hash) + ROOT_DIR_FIELD_NUMBER;
-        hash = (53 * hash) + getRootDir().hashCode();
-      }
-      if (getTableListCount() > 0) {
-        hash = (37 * hash) + TABLE_LIST_FIELD_NUMBER;
-        hash = (53 * hash) + getTableListList().hashCode();
-      }
-      if (hasStartTs()) {
-        hash = (37 * hash) + START_TS_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getStartTs());
-      }
-      if (hasCompleteTs()) {
-        hash = (37 * hash) + COMPLETE_TS_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getCompleteTs());
-      }
-      if (getAncestorsCount() > 0) {
-        hash = (37 * hash) + ANCESTORS_FIELD_NUMBER;
-        hash = (53 * hash) + getAncestorsList().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupImage parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.BackupPro

<TRUNCATED>

[03/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
new file mode 100644
index 0000000..9355d07
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/FullTableBackupClient.java
@@ -0,0 +1,540 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.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.TableName;
+import org.apache.hadoop.hbase.backup.BackupCopyService;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupException;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.backup.util.BackupServerUtil;
+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.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+@InterfaceAudience.Private
+public class FullTableBackupClient {
+  private static final Log LOG = LogFactory.getLog(FullTableBackupClient.class);
+
+  private Configuration conf;
+  private Connection conn;
+  private String backupId;
+  private List<TableName> tableList;
+  HashMap<String, Long> newTimestamps = null;
+
+  private BackupManager backupManager;
+  private BackupInfo backupContext;
+
+  public FullTableBackupClient() {
+    // Required by the Procedure framework to create the procedure on replay
+  }
+
+  public FullTableBackupClient(final Connection conn, final String backupId,
+      BackupRequest request)
+      throws IOException {
+    backupManager = new BackupManager(conn, conn.getConfiguration());
+    this.backupId = backupId;
+    this.tableList = request.getTableList();
+    this.conn = conn;
+    this.conf = conn.getConfiguration();
+    backupContext =
+        backupManager.createBackupContext(backupId, BackupType.FULL, tableList, 
+          request.getTargetRootDir(),
+          request.getWorkers(), request.getBandwidth());
+    if (tableList == null || tableList.isEmpty()) {
+      this.tableList = new ArrayList<>(backupContext.getTables());
+    }
+  }
+
+  /**
+   * Begin the overall backup.
+   * @param backupContext backup context
+   * @throws IOException exception
+   */
+  static void beginBackup(BackupManager backupManager, BackupInfo backupContext) throws IOException {
+    backupManager.setBackupContext(backupContext);
+    // set the start timestamp of the overall backup
+    long startTs = EnvironmentEdgeManager.currentTime();
+    backupContext.setStartTs(startTs);
+    // set overall backup status: ongoing
+    backupContext.setState(BackupState.RUNNING);
+    LOG.info("Backup " + backupContext.getBackupId() + " started at " + startTs + ".");
+
+    backupManager.updateBackupInfo(backupContext);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup session " + backupContext.getBackupId() + " has been started.");
+    }
+  }
+
+  private static String getMessage(Exception e) {
+    String msg = e.getMessage();
+    if (msg == null || msg.equals("")) {
+      msg = e.getClass().getName();
+    }
+    return msg;
+  }
+
+  /**
+   * Delete HBase snapshot for backup.
+   * @param backupCtx backup context
+   * @throws Exception exception
+   */
+  private static void
+      deleteSnapshot(final Connection conn, BackupInfo backupCtx, Configuration conf)
+          throws IOException {
+    LOG.debug("Trying to delete snapshot for full backup.");
+    for (String snapshotName : backupCtx.getSnapshotNames()) {
+      if (snapshotName == null) {
+        continue;
+      }
+      LOG.debug("Trying to delete snapshot: " + snapshotName);
+
+      try (Admin admin = conn.getAdmin();) {
+        admin.deleteSnapshot(snapshotName);
+      } catch (IOException ioe) {
+        LOG.debug("when deleting snapshot " + snapshotName, ioe);
+      }
+      LOG.debug("Deleting the snapshot " + snapshotName + " for backup " + backupCtx.getBackupId()
+          + " succeeded.");
+    }
+  }
+
+  /**
+   * Clean up directories with prefix "exportSnapshot-", which are generated when exporting
+   * snapshots.
+   * @throws IOException exception
+   */
+  private static void cleanupExportSnapshotLog(Configuration conf) throws IOException {
+    FileSystem fs = FSUtils.getCurrentFileSystem(conf);
+    Path stagingDir =
+        new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory()
+            .toString()));
+    FileStatus[] files = FSUtils.listStatus(fs, stagingDir);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      if (file.getPath().getName().startsWith("exportSnapshot-")) {
+        LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName());
+        if (FSUtils.delete(fs, file.getPath(), true) == false) {
+          LOG.warn("Can not delete " + file.getPath());
+        }
+      }
+    }
+  }
+
+  /**
+   * Clean up the uncompleted data at target directory if the ongoing backup has already entered the
+   * copy phase.
+   */
+  static void cleanupTargetDir(BackupInfo backupContext, Configuration conf) {
+    try {
+      // clean up the uncompleted data at target directory if the ongoing backup has already entered
+      // the copy phase
+      LOG.debug("Trying to cleanup up target dir. Current backup phase: "
+          + backupContext.getPhase());
+      if (backupContext.getPhase().equals(BackupPhase.SNAPSHOTCOPY)
+          || backupContext.getPhase().equals(BackupPhase.INCREMENTAL_COPY)
+          || backupContext.getPhase().equals(BackupPhase.STORE_MANIFEST)) {
+        FileSystem outputFs =
+            FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf);
+
+        // now treat one backup as a transaction, clean up data that has been partially copied at
+        // table level
+        for (TableName table : backupContext.getTables()) {
+          Path targetDirPath =
+              new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(),
+                backupContext.getBackupId(), table));
+          if (outputFs.delete(targetDirPath, true)) {
+            LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString()
+                + " done.");
+          } else {
+            LOG.info("No data has been copied to " + targetDirPath.toString() + ".");
+          }
+
+          Path tableDir = targetDirPath.getParent();
+          FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir);
+          if (backups == null || backups.length == 0) {
+            outputFs.delete(tableDir, true);
+            LOG.debug(tableDir.toString() + " is empty, remove it.");
+          }
+        }
+      }
+
+    } catch (IOException e1) {
+      LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at "
+          + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
+    }
+  }
+
+  /**
+   * Fail the overall backup.
+   * @param backupContext backup context
+   * @param e exception
+   * @throws Exception exception
+   */
+  static void failBackup(Connection conn, BackupInfo backupContext, BackupManager backupManager,
+      Exception e, String msg, BackupType type, Configuration conf) throws IOException {
+    LOG.error(msg + getMessage(e), e);
+    // If this is a cancel exception, then we've already cleaned.
+
+    // set the failure timestamp of the overall backup
+    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
+
+    // set failure message
+    backupContext.setFailedMsg(e.getMessage());
+
+    // set overall backup status: failed
+    backupContext.setState(BackupState.FAILED);
+
+    // compose the backup failed data
+    String backupFailedData =
+        "BackupId=" + backupContext.getBackupId() + ",startts=" + backupContext.getStartTs()
+            + ",failedts=" + backupContext.getEndTs() + ",failedphase=" + backupContext.getPhase()
+            + ",failedmessage=" + backupContext.getFailedMsg();
+    LOG.error(backupFailedData);
+
+    backupManager.updateBackupInfo(backupContext);
+
+    // if full backup, then delete HBase snapshots if there already are snapshots taken
+    // and also clean up export snapshot log files if exist
+    if (type == BackupType.FULL) {
+      deleteSnapshot(conn, backupContext, conf);
+      cleanupExportSnapshotLog(conf);
+    }
+
+    // clean up the uncompleted data at target directory if the ongoing backup has already entered
+    // the copy phase
+    // For incremental backup, DistCp logs will be cleaned with the targetDir.
+    cleanupTargetDir(backupContext, conf);
+
+    LOG.info("Backup " + backupContext.getBackupId() + " failed.");
+  }
+
+  /**
+   * Do snapshot copy.
+   * @param backupContext backup context
+   * @throws Exception exception
+   */
+  private void snapshotCopy(BackupInfo backupContext) throws Exception {
+    LOG.info("Snapshot copy is starting.");
+
+    // set overall backup phase: snapshot_copy
+    backupContext.setPhase(BackupPhase.SNAPSHOTCOPY);
+
+    // call ExportSnapshot to copy files based on hbase snapshot for backup
+    // ExportSnapshot only support single snapshot export, need loop for multiple tables case
+    BackupCopyService copyService = BackupRestoreServerFactory.getBackupCopyService(conf);
+
+    // number of snapshots matches number of tables
+    float numOfSnapshots = backupContext.getSnapshotNames().size();
+
+    LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied.");
+
+    for (TableName table : backupContext.getTables()) {
+      // Currently we simply set the sub copy tasks by counting the table snapshot number, we can
+      // calculate the real files' size for the percentage in the future.
+      // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots);
+      int res = 0;
+      String[] args = new String[4];
+      args[0] = "-snapshot";
+      args[1] = backupContext.getSnapshotName(table);
+      args[2] = "-copy-to";
+      args[3] = backupContext.getBackupStatus(table).getTargetDir();
+
+      LOG.debug("Copy snapshot " + args[1] + " to " + args[3]);
+      res = copyService.copy(backupContext, backupManager, conf, BackupCopyService.Type.FULL, args);
+      // if one snapshot export failed, do not continue for remained snapshots
+      if (res != 0) {
+        LOG.error("Exporting Snapshot " + args[1] + " failed with return code: " + res + ".");
+
+        throw new IOException("Failed of exporting snapshot " + args[1] + " to " + args[3]
+            + " with reason code " + res);
+      }
+      LOG.info("Snapshot copy " + args[1] + " finished.");
+    }
+  }
+
+  /**
+   * Add manifest for the current backup. The manifest is stored within the table backup directory.
+   * @param backupContext The current backup context
+   * @throws IOException exception
+   * @throws BackupException exception
+   */
+  private static void addManifest(BackupInfo backupContext, BackupManager backupManager,
+      BackupType type, Configuration conf) throws IOException, BackupException {
+    // set the overall backup phase : store manifest
+    backupContext.setPhase(BackupPhase.STORE_MANIFEST);
+
+    BackupManifest manifest;
+
+    // Since we have each table's backup in its own directory structure,
+    // we'll store its manifest with the table directory.
+    for (TableName table : backupContext.getTables()) {
+      manifest = new BackupManifest(backupContext, table);
+      ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupContext, table);
+      for (BackupImage image : ancestors) {
+        manifest.addDependentImage(image);
+      }
+
+      if (type == BackupType.INCREMENTAL) {
+        // We'll store the log timestamps for this table only in its manifest.
+        HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
+            new HashMap<TableName, HashMap<String, Long>>();
+        tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table));
+        manifest.setIncrTimestampMap(tableTimestampMap);
+        ArrayList<BackupImage> ancestorss = backupManager.getAncestors(backupContext);
+        for (BackupImage image : ancestorss) {
+          manifest.addDependentImage(image);
+        }
+      }
+      manifest.store(conf);
+    }
+
+    // For incremental backup, we store a overall manifest in
+    // <backup-root-dir>/WALs/<backup-id>
+    // This is used when created the next incremental backup
+    if (type == BackupType.INCREMENTAL) {
+      manifest = new BackupManifest(backupContext);
+      // set the table region server start and end timestamps for incremental backup
+      manifest.setIncrTimestampMap(backupContext.getIncrTimestampMap());
+      ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupContext);
+      for (BackupImage image : ancestors) {
+        manifest.addDependentImage(image);
+      }
+      manifest.store(conf);
+    }
+  }
+
+  /**
+   * Get backup request meta data dir as string.
+   * @param backupContext backup context
+   * @return meta data dir
+   */
+  private static String obtainBackupMetaDataStr(BackupInfo backupContext) {
+    StringBuffer sb = new StringBuffer();
+    sb.append("type=" + backupContext.getType() + ",tablelist=");
+    for (TableName table : backupContext.getTables()) {
+      sb.append(table + ";");
+    }
+    if (sb.lastIndexOf(";") > 0) {
+      sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1);
+    }
+    sb.append(",targetRootDir=" + backupContext.getTargetRootDir());
+
+    return sb.toString();
+  }
+
+  /**
+   * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying
+   * hlogs.
+   * @throws IOException exception
+   */
+  private static void cleanupDistCpLog(BackupInfo backupContext, Configuration conf)
+      throws IOException {
+    Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent();
+    FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
+    FileStatus[] files = FSUtils.listStatus(fs, rootPath);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      if (file.getPath().getName().startsWith("_distcp_logs")) {
+        LOG.debug("Delete log files of DistCp: " + file.getPath().getName());
+        FSUtils.delete(fs, file.getPath(), true);
+      }
+    }
+  }
+
+  /**
+   * Complete the overall backup.
+   * @param backupContext backup context
+   * @throws Exception exception
+   */
+  static void completeBackup(final Connection conn, BackupInfo backupContext,
+      BackupManager backupManager, BackupType type, Configuration conf) throws IOException {
+    // set the complete timestamp of the overall backup
+    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
+    // set overall backup status: complete
+    backupContext.setState(BackupState.COMPLETE);
+    backupContext.setProgress(100);
+    // add and store the manifest for the backup
+    addManifest(backupContext, backupManager, type, conf);
+
+    // after major steps done and manifest persisted, do convert if needed for incremental backup
+    /* in-fly convert code here, provided by future jira */
+    LOG.debug("in-fly convert code here, provided by future jira");
+
+    // compose the backup complete data
+    String backupCompleteData =
+        obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs()
+            + ",completets=" + backupContext.getEndTs() + ",bytescopied="
+            + backupContext.getTotalBytesCopied();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData);
+    }
+    backupManager.updateBackupInfo(backupContext);
+
+    // when full backup is done:
+    // - delete HBase snapshot
+    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
+    // snapshots
+    if (type == BackupType.FULL) {
+      deleteSnapshot(conn, backupContext, conf);
+      cleanupExportSnapshotLog(conf);
+    } else if (type == BackupType.INCREMENTAL) {
+      cleanupDistCpLog(backupContext, conf);
+    }
+
+    LOG.info("Backup " + backupContext.getBackupId() + " completed.");
+  }
+
+  /**
+   * Wrap a SnapshotDescription for a target table.
+   * @param table table
+   * @return a SnapshotDescription especially for backup.
+   */
+  static SnapshotDescription wrapSnapshotDescription(TableName tableName, String snapshotName) {
+    // Mock a SnapshotDescription from backupContext to call SnapshotManager function,
+    // Name it in the format "snapshot_<timestamp>_<table>"
+    HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
+    builder.setTable(tableName.getNameAsString());
+    builder.setName(snapshotName);
+    HBaseProtos.SnapshotDescription backupSnapshot = builder.build();
+
+    LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName()
+        + " from backupContext to request snapshot for backup.");
+
+    return backupSnapshot;
+  }
+
+  /**
+   * Backup request execution
+   * @throws IOException
+   */
+  public void execute() throws IOException {
+
+    try (Admin admin = conn.getAdmin();) {
+
+      // Begin BACKUP
+      beginBackup(backupManager, backupContext);
+      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 on ZK so that we can hold the logs
+        // while we do the backup.
+        backupManager.writeBackupStartCode(0L);
+      }
+      // We roll log here before we do the snapshot. It is possible there is duplicate data
+      // in the log that is already in the snapshot. But if we do it after the snapshot, we
+      // could have data loss.
+      // A better approach is to do the roll log on each RS in the same global procedure as
+      // the snapshot.
+      LOG.info("Execute roll log procedure for full backup ...");
+
+      Map<String, String> props = new HashMap<String, String>();
+      props.put("backupRoot", backupContext.getTargetRootDir());
+      admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
+        LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props);
+
+      newTimestamps = backupManager.readRegionServerLastLogRollResult();
+      if (firstBackup) {
+        // Updates registered log files
+        // We record ALL old WAL files as registered, because
+        // this is a first full backup in the system and these
+        // files are not needed for next incremental backup
+        List<String> logFiles = BackupServerUtil.getWALFilesOlderThan(conf, newTimestamps);
+        backupManager.recordWALFiles(logFiles);
+      }
+
+      // SNAPSHOT_TABLES:
+      for (TableName tableName : tableList) {
+        String snapshotName =
+            "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_"
+                + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString();
+
+        admin.snapshot(snapshotName, tableName);
+
+        backupContext.setSnapshotName(tableName, snapshotName);
+      }
+
+      // SNAPSHOT_COPY:
+      // do snapshot copy
+      LOG.debug("snapshot copy for " + backupId);
+      snapshotCopy(backupContext);
+      // Updates incremental backup table set
+      backupManager.addIncrementalBackupTableSet(backupContext.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
+      backupContext.setState(BackupState.COMPLETE);
+      // The table list in backupContext is good for both full backup and incremental backup.
+      // For incremental backup, it contains the incremental backup table set.
+      backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps);
+
+      HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
+          backupManager.readLogTimestampMap();
+
+      Long newStartCode =
+          BackupClientUtil.getMinValue(BackupServerUtil
+              .getRSLogTimestampMins(newTableSetTimestampMap));
+      backupManager.writeBackupStartCode(newStartCode);
+
+      // backup complete
+      completeBackup(conn, backupContext, backupManager, BackupType.FULL, conf);
+    } catch (Exception e) {
+      failBackup(conn, backupContext, backupManager, e, "Unexpected BackupException : ",
+        BackupType.FULL, conf);
+      throw new IOException(e);
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/HBaseBackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/HBaseBackupAdmin.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/HBaseBackupAdmin.java
new file mode 100644
index 0000000..8c63f98
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/HBaseBackupAdmin.java
@@ -0,0 +1,555 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Future;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupAdmin;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.RestoreRequest;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.backup.util.RestoreServerUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import com.google.common.collect.Lists;
+
+/**
+ * The administrative API implementation for HBase Backup . Obtain an instance from 
+ * an {@link Admin#getBackupAdmin()} and call {@link #close()} afterwards.
+ * <p>BackupAdmin can be used to create backups, restore data from backups and for 
+ * other backup-related operations. 
+ *
+ * @see Admin
+ * @since 2.0
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+
+public class HBaseBackupAdmin implements BackupAdmin {
+  private static final Log LOG = LogFactory.getLog(HBaseBackupAdmin.class);
+
+  private final Connection conn;
+
+  public HBaseBackupAdmin(Connection conn) {
+    this.conn = conn;
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public BackupInfo getBackupInfo(String backupId) throws IOException {
+    BackupInfo backupInfo = null;
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      backupInfo = table.readBackupInfo(backupId);
+      return backupInfo;
+    }
+  }
+
+  @Override
+  public int getProgress(String backupId) throws IOException {
+    BackupInfo backupInfo = null;
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      if (backupId == null) {
+        ArrayList<BackupInfo> recentSessions = table.getBackupContexts(BackupState.RUNNING);
+        if (recentSessions.isEmpty()) {
+          LOG.warn("No ongoing sessions found.");
+          return -1;
+        }
+        // else show status for ongoing session
+        // must be one maximum
+        return recentSessions.get(0).getProgress();
+      } else {
+
+        backupInfo = table.readBackupInfo(backupId);
+        if (backupInfo != null) {
+          return backupInfo.getProgress();
+        } else {
+          LOG.warn("No information found for backupID=" + backupId);
+          return -1;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int deleteBackups(String[] backupIds) throws IOException {
+    // TODO: requires FT, failure will leave system
+    // in non-consistent state
+    // see HBASE-15227
+
+    int totalDeleted = 0;
+    Map<String, HashSet<TableName>> allTablesMap = new HashMap<String, HashSet<TableName>>();
+
+    try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) {
+      for (int i = 0; i < backupIds.length; i++) {
+        BackupInfo info = sysTable.readBackupInfo(backupIds[i]);
+        if (info != null) {
+          String rootDir = info.getTargetRootDir();
+          HashSet<TableName> allTables = allTablesMap.get(rootDir);
+          if (allTables == null) {
+            allTables = new HashSet<TableName>();
+            allTablesMap.put(rootDir, allTables);
+          }
+          allTables.addAll(info.getTableNames());
+          totalDeleted += deleteBackup(backupIds[i], sysTable);
+        }
+      }
+      finalizeDelete(allTablesMap, sysTable);
+    }
+    return totalDeleted;
+  }
+
+  /**
+   * Updates incremental backup set for every backupRoot
+   * @param tablesMap - Map [backupRoot: Set<TableName>]
+   * @param table - backup system table
+   * @throws IOException
+   */
+
+  private void finalizeDelete(Map<String, HashSet<TableName>> tablesMap, BackupSystemTable table)
+      throws IOException {
+    for (String backupRoot : tablesMap.keySet()) {
+      Set<TableName> incrTableSet = table.getIncrementalBackupTableSet(backupRoot);
+      Map<TableName, ArrayList<BackupInfo>> tableMap = 
+          table.getBackupHistoryForTableSet(incrTableSet, backupRoot);      
+      for(Map.Entry<TableName, ArrayList<BackupInfo>> entry: tableMap.entrySet()) {
+        if(entry.getValue() == null) {
+          // No more backups for a table
+          incrTableSet.remove(entry.getKey());
+        }
+      }      
+      if (!incrTableSet.isEmpty()) {
+        table.addIncrementalBackupTableSet(incrTableSet, backupRoot);
+      } else { // empty
+        table.deleteIncrementalBackupTableSet(backupRoot);
+      }
+    }
+  }
+  
+  /**
+   * Delete single backup and all related backups
+   * Algorithm:
+   * 
+   * Backup type: FULL or INCREMENTAL
+   * Is this last backup session for table T: YES or NO
+   * For every table T from table list 'tables':
+   * if(FULL, YES) deletes only physical data (PD)
+   * if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo, until
+   * we either reach the most recent backup for T in the system or FULL backup which
+   * includes T
+   * if(INCREMENTAL, YES) deletes only physical data (PD)
+   * if(INCREMENTAL, NO) deletes physical data and for table T scans all backup images
+   * between last FULL backup, which is older than the backup being deleted and the next
+   * FULL backup (if exists) or last one for a particular table T and removes T from list
+   * of backup tables.
+   * @param backupId - backup id
+   * @param sysTable - backup system table
+   * @return total - number of deleted backup images
+   * @throws IOException
+   */
+  private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException {
+
+    BackupInfo backupInfo = sysTable.readBackupInfo(backupId);
+
+    int totalDeleted = 0;
+    if (backupInfo != null) {
+      LOG.info("Deleting backup " + backupInfo.getBackupId() + " ...");
+      BackupClientUtil.cleanupBackupData(backupInfo, conn.getConfiguration());
+      // List of tables in this backup;
+      List<TableName> tables = backupInfo.getTableNames();
+      long startTime = backupInfo.getStartTs();
+      for (TableName tn : tables) {
+        boolean isLastBackupSession = isLastBackupSession(sysTable, tn, startTime);
+        if (isLastBackupSession) {
+          continue;
+        }
+        // else
+        List<BackupInfo> affectedBackups = getAffectedBackupInfos(backupInfo, tn, sysTable);
+        for (BackupInfo info : affectedBackups) {
+          if (info.equals(backupInfo)) {
+            continue;
+          }
+          removeTableFromBackupImage(info, tn, sysTable);
+        }
+      }
+      LOG.debug("Delete backup info "+ backupInfo.getBackupId());  
+
+      sysTable.deleteBackupInfo(backupInfo.getBackupId());
+      LOG.info("Delete backup " + backupInfo.getBackupId() + " completed.");
+      totalDeleted++;
+    } else {
+      LOG.warn("Delete backup failed: no information found for backupID=" + backupId);
+    }
+    return totalDeleted;
+  }
+
+  private void removeTableFromBackupImage(BackupInfo info, TableName tn, BackupSystemTable sysTable)
+      throws IOException {
+    List<TableName> tables = info.getTableNames();
+    LOG.debug("Remove "+ tn +" from " + info.getBackupId() + " tables=" + 
+      info.getTableListAsString());
+    if (tables.contains(tn)) {
+      tables.remove(tn);
+
+      if (tables.isEmpty()) {
+        LOG.debug("Delete backup info "+ info.getBackupId());  
+
+        sysTable.deleteBackupInfo(info.getBackupId());
+        BackupClientUtil.cleanupBackupData(info, conn.getConfiguration());
+      } else {
+        info.setTables(tables);
+        sysTable.updateBackupInfo(info);
+        // Now, clean up directory for table
+        cleanupBackupDir(info, tn, conn.getConfiguration());
+      }
+    }
+  }
+
+  private List<BackupInfo> getAffectedBackupInfos(BackupInfo backupInfo, TableName tn,
+      BackupSystemTable table) throws IOException {
+    LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn);
+    long ts = backupInfo.getStartTs();
+    List<BackupInfo> list = new ArrayList<BackupInfo>();
+    List<BackupInfo> history = table.getBackupHistory(backupInfo.getTargetRootDir());
+    // Scan from most recent to backupInfo
+    // break when backupInfo reached
+    for (BackupInfo info : history) {
+      if (info.getStartTs() == ts) {
+        break;
+      }
+      List<TableName> tables = info.getTableNames();
+      if (tables.contains(tn)) {
+        BackupType bt = info.getType();
+        if (bt == BackupType.FULL) {
+          // Clear list if we encounter FULL backup
+          list.clear();
+        } else {
+          LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn
+              + " added " + info.getBackupId() + " tables=" + info.getTableListAsString());
+          list.add(info);
+        }
+      }
+    }
+    return list;
+  }
+
+  
+  
+  /**
+   * Clean up the data at target directory
+   * @throws IOException 
+   */
+  private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf) 
+      throws IOException {
+    try {
+      // clean up the data at target directory
+      String targetDir = backupInfo.getTargetRootDir();
+      if (targetDir == null) {
+        LOG.warn("No target directory specified for " + backupInfo.getBackupId());
+        return;
+      }
+
+      FileSystem outputFs = FileSystem.get(new Path(backupInfo.getTargetRootDir()).toUri(), conf);
+
+      Path targetDirPath =
+          new Path(BackupClientUtil.getTableBackupDir(backupInfo.getTargetRootDir(),
+            backupInfo.getBackupId(), table));
+      if (outputFs.delete(targetDirPath, true)) {
+        LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done.");
+      } else {
+        LOG.info("No data has been found in " + targetDirPath.toString() + ".");
+      }
+
+    } catch (IOException e1) {
+      LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " for table " + table
+          + "at " + backupInfo.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
+      throw e1;
+    }
+  }
+
+  private boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime)
+      throws IOException {
+    List<BackupInfo> history = table.getBackupHistory();
+    for (BackupInfo info : history) {
+      List<TableName> tables = info.getTableNames();
+      if (!tables.contains(tn)) {
+        continue;
+      }
+      if (info.getStartTs() <= startTime) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public List<BackupInfo> getHistory(int n) throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<BackupInfo> history = table.getBackupHistory();
+      if (history.size() <= n) return history;
+      List<BackupInfo> list = new ArrayList<BackupInfo>();
+      for (int i = 0; i < n; i++) {
+        list.add(history.get(i));
+      }
+      return list;
+    }
+  }
+
+  @Override
+  public List<BackupInfo> getHistory(int n, BackupInfo.Filter ... filters) throws IOException {
+    if (filters.length == 0) return getHistory(n);
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<BackupInfo> history = table.getBackupHistory();
+      List<BackupInfo> result = new ArrayList<BackupInfo>();
+      for(BackupInfo bi: history) {
+        if(result.size() == n) break;
+        boolean passed = true;
+        for(int i=0; i < filters.length; i++) {
+          if(!filters[i].apply(bi)) {
+            passed = false; 
+            break;
+          }
+        }        
+        if(passed) {
+          result.add(bi);
+        }
+      }
+      return result;
+    }
+  }
+
+  @Override
+  public List<BackupSet> listBackupSets() throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<String> list = table.listBackupSets();
+      List<BackupSet> bslist = new ArrayList<BackupSet>();
+      for (String s : list) {
+        List<TableName> tables = table.describeBackupSet(s);
+        if (tables != null) {
+          bslist.add(new BackupSet(s, tables));
+        }
+      }
+      return bslist;
+    }
+  }
+
+  @Override
+  public BackupSet getBackupSet(String name) throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<TableName> list = table.describeBackupSet(name);
+      if (list == null) return null;
+      return new BackupSet(name, list);
+    }
+  }
+
+  @Override
+  public boolean deleteBackupSet(String name) throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      if (table.describeBackupSet(name) == null) {
+        return false;
+      }
+      table.deleteBackupSet(name);
+      return true;
+    }
+  }
+
+  @Override
+  public void addToBackupSet(String name, TableName[] tables) throws IOException {
+    String[] tableNames = new String[tables.length];
+    try (final BackupSystemTable table = new BackupSystemTable(conn);
+         final Admin admin = conn.getAdmin();) {
+      for (int i = 0; i < tables.length; i++) {
+        tableNames[i] = tables[i].getNameAsString();
+        if (!admin.tableExists(TableName.valueOf(tableNames[i]))) {
+          throw new IOException("Cannot add " + tableNames[i] + " because it doesn't exist");
+        }
+      }
+      table.addToBackupSet(name, tableNames);
+      LOG.info("Added tables [" + StringUtils.join(tableNames, " ") + "] to '" + name
+          + "' backup set");
+    }
+  }
+
+  @Override
+  public void removeFromBackupSet(String name, String[] tables) throws IOException {
+    LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + "'");
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      table.removeFromBackupSet(name, tables);
+      LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name
+          + "' completed.");
+    }
+  }
+
+  @Override
+  public void restore(RestoreRequest request) throws IOException {
+    if (request.isCheck()) {
+      HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
+      // check and load backup image manifest for the tables
+      Path rootPath = new Path(request.getBackupRootDir());
+      String backupId = request.getBackupId();
+      TableName[] sTableArray = request.getFromTables();
+      HBackupFileSystem.checkImageManifestExist(backupManifestMap,
+        sTableArray, conn.getConfiguration(), rootPath, backupId);
+
+      // Check and validate the backup image and its dependencies
+     
+        if (RestoreServerUtil.validate(backupManifestMap, conn.getConfiguration())) {
+          LOG.info("Checking backup images: ok");
+        } else {
+          String errMsg = "Some dependencies are missing for restore";
+          LOG.error(errMsg);
+          throw new IOException(errMsg);
+        }
+      
+    }
+    // Execute restore request
+    new RestoreTablesClient(conn, request).execute();
+  }
+
+  @Override
+  public Future<Void> restoreAsync(RestoreRequest request) throws IOException {
+    // TBI
+    return null;
+  }
+
+  @Override
+  public String backupTables(final BackupRequest request) throws IOException {
+    String setName = request.getBackupSetName();
+    BackupType type = request.getBackupType();
+    String targetRootDir = request.getTargetRootDir();
+    List<TableName> tableList = request.getTableList();
+
+    String backupId =
+        (setName == null || setName.length() == 0 ? BackupRestoreConstants.BACKUPID_PREFIX
+            : setName + "_") + EnvironmentEdgeManager.currentTime();
+    if (type == BackupType.INCREMENTAL) {
+      Set<TableName> incrTableSet = null;
+      try (BackupSystemTable table = new BackupSystemTable(conn)) {
+        incrTableSet = table.getIncrementalBackupTableSet(targetRootDir);
+      }
+
+      if (incrTableSet.isEmpty()) {
+        System.err.println("Incremental backup table set contains no table.\n"
+            + "Use 'backup create full' or 'backup stop' to \n "
+            + "change the tables covered by incremental backup.");
+        throw new IOException("No table covered by incremental backup.");
+      }
+
+      tableList.removeAll(incrTableSet);
+      if (!tableList.isEmpty()) {
+        String extraTables = StringUtils.join(tableList, ",");
+        System.err.println("Some tables (" + extraTables + ") haven't gone through full backup");
+        throw new IOException("Perform full backup on " + extraTables + " first, "
+            + "then retry the command");
+      }
+      System.out.println("Incremental backup for the following table set: " + incrTableSet);
+      tableList = Lists.newArrayList(incrTableSet);
+    }
+    if (tableList != null && !tableList.isEmpty()) {
+      for (TableName table : tableList) {
+        String targetTableBackupDir =
+            HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
+        Path targetTableBackupDirPath = new Path(targetTableBackupDir);
+        FileSystem outputFs =
+            FileSystem.get(targetTableBackupDirPath.toUri(), conn.getConfiguration());
+        if (outputFs.exists(targetTableBackupDirPath)) {
+          throw new IOException("Target backup directory " + targetTableBackupDir
+              + " exists already.");
+        }
+      }
+      ArrayList<TableName> nonExistingTableList = null;
+      try (Admin admin = conn.getAdmin();) {
+        for (TableName tableName : tableList) {
+          if (!admin.tableExists(tableName)) {
+            if (nonExistingTableList == null) {
+              nonExistingTableList = new ArrayList<>();
+            }
+            nonExistingTableList.add(tableName);
+          }
+        }
+      }
+      if (nonExistingTableList != null) {
+        if (type == BackupType.INCREMENTAL) {
+          System.err.println("Incremental backup table set contains non-exising table: "
+              + nonExistingTableList);
+          // Update incremental backup set
+          tableList = excludeNonExistingTables(tableList, nonExistingTableList);
+        } else {
+          // Throw exception only in full mode - we try to backup non-existing table
+          throw new IOException("Non-existing tables found in the table list: "
+              + nonExistingTableList);
+        }
+      }
+    }
+
+    // update table list
+    request.setTableList(tableList);
+
+    if (type == BackupType.FULL) {
+      new FullTableBackupClient(conn, backupId, request).execute();
+    } else {
+      new IncrementalTableBackupClient(conn, backupId, request).execute();
+    }
+    return backupId;
+  }
+
+
+  private List<TableName> excludeNonExistingTables(List<TableName> tableList,
+      List<TableName> nonExistingTableList) {
+
+    for (TableName table : nonExistingTableList) {
+      tableList.remove(table);
+    }
+    return tableList;
+  }
+
+  @Override
+  public Future<String> backupTablesAsync(final BackupRequest userRequest) throws IOException {
+    // TBI
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
index 8f6aeb8..be5fd23 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -35,6 +34,7 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable.WALItem;
 import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
 import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
 import org.apache.hadoop.hbase.backup.util.BackupServerUtil;
@@ -42,13 +42,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
-import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
-import org.apache.hadoop.hbase.procedure.ProcedureUtil;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable.WALItem;
 
 /**
  * After a full backup was created, the incremental backup will only store the changes made
@@ -64,12 +59,10 @@ public class IncrementalBackupManager {
   // parent manager
   private final BackupManager backupManager;
   private final Configuration conf;
-  private final Connection conn;
 
   public IncrementalBackupManager(BackupManager bm) {
     this.backupManager = bm;
     this.conf = bm.getConf();
-    this.conn = bm.getConnection();
   }
 
   /**
@@ -80,7 +73,7 @@ public class IncrementalBackupManager {
    * @return The new HashMap of RS log timestamps after the log roll for this incremental backup.
    * @throws IOException exception
    */
-  public HashMap<String, Long> getIncrBackupLogFileList(MasterServices svc,BackupInfo backupContext)
+  public HashMap<String, Long> getIncrBackupLogFileList(Connection conn,BackupInfo backupContext)
       throws IOException {
     List<String> logList;
     HashMap<String, Long> newTimestamps;
@@ -109,19 +102,13 @@ public class IncrementalBackupManager {
     LOG.info("Execute roll log procedure for incremental backup ...");
     HashMap<String, String> props = new HashMap<String, String>();
     props.put("backupRoot", backupContext.getTargetRootDir());
-    MasterProcedureManager mpm = svc.getMasterProcedureManagerHost()
-        .getProcedureManager(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE);
-    long waitTime = ProcedureUtil.execProcedure(mpm,
-        LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
+    
+    try(Admin admin = conn.getAdmin();) {
+    
+      admin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE, 
         LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props);
-    ProcedureUtil.waitForProcedure(mpm,
-        LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
-        LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props, waitTime,
-        conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-            HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER),
-            conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
-                HConstants.DEFAULT_HBASE_CLIENT_PAUSE));
 
+    }
     newTimestamps = backupManager.readRegionServerLastLogRollResult();
 
     logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
new file mode 100644
index 0000000..d9610a2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalTableBackupClient.java
@@ -0,0 +1,235 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupCopyService;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.backup.util.BackupServerUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+
+@InterfaceAudience.Private
+public class IncrementalTableBackupClient {
+  private static final Log LOG = LogFactory.getLog(IncrementalTableBackupClient.class);
+
+  private Configuration conf;
+  private Connection conn;
+  //private String backupId;
+  HashMap<String, Long> newTimestamps = null;
+
+  private String backupId;
+  private BackupManager backupManager;
+  private BackupInfo backupContext;
+
+  public IncrementalTableBackupClient() {
+    // Required by the Procedure framework to create the procedure on replay
+  }
+
+  public IncrementalTableBackupClient(final Connection conn, final String backupId,
+      BackupRequest request)
+      throws IOException {
+
+    this.conn = conn;
+    this.conf = conn.getConfiguration();
+    backupManager = new BackupManager(conn, conf);
+    this.backupId = backupId;
+    backupContext =
+        backupManager.createBackupContext(backupId, BackupType.INCREMENTAL, request.getTableList(),
+          request.getTargetRootDir(), request.getWorkers(), (int) request.getBandwidth());
+  }
+
+  private List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    List<String> list = new ArrayList<String>();
+    for (String file : incrBackupFileList) {
+      if (fs.exists(new Path(file))) {
+        list.add(file);
+      } else {
+        LOG.warn("Can't find file: " + file);
+      }
+    }
+    return list;
+  }
+
+  private List<String> getMissingFiles(List<String> incrBackupFileList) throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    List<String> list = new ArrayList<String>();
+    for (String file : incrBackupFileList) {
+      if (!fs.exists(new Path(file))) {
+        list.add(file);
+      }
+    }
+    return list;
+
+  }
+
+  /**
+   * Do incremental copy.
+   * @param backupContext backup context
+   */
+  private void incrementalCopy(BackupInfo backupContext) throws Exception {
+
+    LOG.info("Incremental copy is starting.");
+    // set overall backup phase: incremental_copy
+    backupContext.setPhase(BackupPhase.INCREMENTAL_COPY);
+    // get incremental backup file list and prepare parms for DistCp
+    List<String> incrBackupFileList = backupContext.getIncrBackupFileList();
+    // filter missing files out (they have been copied by previous backups)
+    incrBackupFileList = filterMissingFiles(incrBackupFileList);
+    String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
+    strArr[strArr.length - 1] = backupContext.getHLogTargetDir();
+
+    BackupCopyService copyService = BackupRestoreServerFactory.getBackupCopyService(conf);
+    int counter = 0;
+    int MAX_ITERAIONS = 2;
+    while (counter++ < MAX_ITERAIONS) {
+      // We run DistCp maximum 2 times
+      // If it fails on a second time, we throw Exception
+      int res =
+          copyService.copy(backupContext, backupManager, conf, BackupCopyService.Type.INCREMENTAL,
+            strArr);
+
+      if (res != 0) {
+        LOG.error("Copy incremental log files failed with return code: " + res + ".");
+        throw new IOException("Failed of Hadoop Distributed Copy from "
+            + StringUtils.join(incrBackupFileList, ",") + " to " + backupContext.getHLogTargetDir());
+      }
+      List<String> missingFiles = getMissingFiles(incrBackupFileList);
+
+      if (missingFiles.isEmpty()) {
+        break;
+      } else {
+        // Repeat DistCp, some files have been moved from WALs to oldWALs during previous run
+        // update backupContext and strAttr
+        if (counter == MAX_ITERAIONS) {
+          String msg =
+              "DistCp could not finish the following files: " + StringUtils.join(missingFiles, ",");
+          LOG.error(msg);
+          throw new IOException(msg);
+        }
+        List<String> converted = convertFilesFromWALtoOldWAL(missingFiles);
+        incrBackupFileList.removeAll(missingFiles);
+        incrBackupFileList.addAll(converted);
+        backupContext.setIncrBackupFileList(incrBackupFileList);
+
+        // Run DistCp only for missing files (which have been moved from WALs to oldWALs
+        // during previous run)
+        strArr = converted.toArray(new String[converted.size() + 1]);
+        strArr[strArr.length - 1] = backupContext.getHLogTargetDir();
+      }
+    }
+
+    LOG.info("Incremental copy from " + StringUtils.join(incrBackupFileList, ",") + " to "
+        + backupContext.getHLogTargetDir() + " finished.");
+  }
+
+  private List<String> convertFilesFromWALtoOldWAL(List<String> missingFiles) throws IOException {
+    List<String> list = new ArrayList<String>();
+    for (String path : missingFiles) {
+      if (path.indexOf(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME) < 0) {
+        LOG.error("Copy incremental log files failed, file is missing : " + path);
+        throw new IOException("Failed of Hadoop Distributed Copy to "
+            + backupContext.getHLogTargetDir() + ", file is missing " + path);
+      }
+      list.add(path.replace(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME, Path.SEPARATOR
+          + HConstants.HREGION_OLDLOGDIR_NAME));
+    }
+    return list;
+  }
+
+  public void execute() throws IOException {
+
+    // case PREPARE_INCREMENTAL:
+    FullTableBackupClient.beginBackup(backupManager, backupContext);
+    LOG.debug("For incremental backup, current table set is "
+        + backupManager.getIncrementalBackupTableSet());
+    try {
+      IncrementalBackupManager incrBackupManager = new IncrementalBackupManager(backupManager);
+
+      newTimestamps = incrBackupManager.getIncrBackupLogFileList(conn, backupContext);
+    } catch (Exception e) {
+      // fail the overall backup and return
+      FullTableBackupClient.failBackup(conn, backupContext, backupManager, e,
+        "Unexpected Exception : ", BackupType.INCREMENTAL, conf);
+    }
+
+    // case INCREMENTAL_COPY:
+    try {
+      // copy out the table and region info files for each table
+      BackupServerUtil.copyTableRegionInfo(conn, backupContext, conf);
+      incrementalCopy(backupContext);
+      // Save list of WAL files copied
+      backupManager.recordWALFiles(backupContext.getIncrBackupFileList());
+    } catch (Exception e) {
+      String msg = "Unexpected exception in incremental-backup: incremental copy " + backupId;
+      // fail the overall backup and return
+      FullTableBackupClient.failBackup(conn, backupContext, backupManager, e, msg,
+        BackupType.INCREMENTAL, conf);
+    }
+    // case INCR_BACKUP_COMPLETE:
+    // set overall backup status: complete. Here we make sure to complete the backup.
+    // After this checkpoint, even if entering cancel process, will let the backup finished
+    try {
+      backupContext.setState(BackupState.COMPLETE);
+      // Set the previousTimestampMap which is before this current log roll to the manifest.
+      HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
+          backupManager.readLogTimestampMap();
+      backupContext.setIncrTimestampMap(previousTimestampMap);
+
+      // The table list in backupContext is good for both full backup and incremental backup.
+      // For incremental backup, it contains the incremental backup table set.
+      backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps);
+
+      HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
+          backupManager.readLogTimestampMap();
+
+      Long newStartCode =
+          BackupClientUtil.getMinValue(BackupServerUtil
+              .getRSLogTimestampMins(newTableSetTimestampMap));
+      backupManager.writeBackupStartCode(newStartCode);
+      // backup complete
+      FullTableBackupClient.completeBackup(conn, backupContext, backupManager,
+        BackupType.INCREMENTAL, conf);
+
+    } catch (IOException e) {
+      FullTableBackupClient.failBackup(conn, backupContext, backupManager, e,
+        "Unexpected Exception : ", BackupType.INCREMENTAL, conf);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
new file mode 100644
index 0000000..91f2d68
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreTablesClient.java
@@ -0,0 +1,236 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.RestoreRequest;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.util.RestoreServerUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+
+@InterfaceAudience.Private
+public class RestoreTablesClient {
+  private static final Log LOG = LogFactory.getLog(RestoreTablesClient.class);
+
+  private Configuration conf;
+  private Connection conn;
+  private String backupId;
+  private TableName[] sTableArray;
+  private TableName[] tTableArray;
+  private String targetRootDir;
+  private boolean isOverwrite;
+
+  public RestoreTablesClient() {
+    // Required by the Procedure framework to create the procedure on replay
+  }
+
+  public RestoreTablesClient(Connection conn, RestoreRequest request)
+      throws IOException {
+    this.targetRootDir = request.getBackupRootDir();
+    this.backupId = request.getBackupId();
+    this.sTableArray = request.getFromTables();
+    this.tTableArray = request.getToTables();
+    if (tTableArray == null || tTableArray.length == 0) {
+      this.tTableArray = sTableArray;
+    }
+    this.isOverwrite = request.isOverwrite();
+    this.conn = conn;
+    this.conf = conn.getConfiguration();
+
+  }
+
+  /**
+   * Validate target Tables
+   * @param conn connection
+   * @param mgr table state manager
+   * @param tTableArray: target tables
+   * @param isOverwrite overwrite existing table
+   * @throws IOException exception
+   */
+  private void checkTargetTables(TableName[] tTableArray, boolean isOverwrite) throws IOException {
+    ArrayList<TableName> existTableList = new ArrayList<>();
+    ArrayList<TableName> disabledTableList = new ArrayList<>();
+
+    // check if the tables already exist
+    try (Admin admin = conn.getAdmin();) {
+      for (TableName tableName : tTableArray) {
+        if (admin.tableExists(tableName)) {
+          existTableList.add(tableName);
+          if (admin.isTableDisabled(tableName)) {
+            disabledTableList.add(tableName);
+          }
+        } else {
+          LOG.info("HBase table " + tableName
+              + " does not exist. It will be created during restore process");
+        }
+      }
+    }
+
+    if (existTableList.size() > 0) {
+      if (!isOverwrite) {
+        LOG.error("Existing table ("
+            + existTableList
+            + ") found in the restore target, please add "
+            + "\"-overwrite\" option in the command if you mean to restore to these existing tables");
+        throw new IOException("Existing table found in target while no \"-overwrite\" "
+            + "option found");
+      } else {
+        if (disabledTableList.size() > 0) {
+          LOG.error("Found offline table in the restore target, "
+              + "please enable them before restore with \"-overwrite\" option");
+          LOG.info("Offline table list in restore target: " + disabledTableList);
+          throw new IOException(
+              "Found offline table in the target when restore with \"-overwrite\" option");
+        }
+      }
+    }
+  }
+
+  /**
+   * Restore operation handle each backupImage in array
+   * @param svc: master services
+   * @param images: array BackupImage
+   * @param sTable: table to be restored
+   * @param tTable: table to be restored to
+   * @param truncateIfExists: truncate table
+   * @throws IOException exception
+   */
+
+  private void restoreImages(BackupImage[] images, TableName sTable, TableName tTable,
+      boolean truncateIfExists) throws IOException {
+
+    // First image MUST be image of a FULL backup
+    BackupImage image = images[0];
+    String rootDir = image.getRootDir();
+    String backupId = image.getBackupId();
+    Path backupRoot = new Path(rootDir);
+    RestoreServerUtil restoreTool = new RestoreServerUtil(conf, backupRoot, backupId);
+    Path tableBackupPath = HBackupFileSystem.getTableBackupPath(sTable, backupRoot, backupId);
+    String lastIncrBackupId = images.length == 1 ? null : images[images.length - 1].getBackupId();
+    // We need hFS only for full restore (see the code)
+    BackupManifest manifest = HBackupFileSystem.getManifest(sTable, conf, backupRoot, backupId);
+    if (manifest.getType() == BackupType.FULL) {
+      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from full" + " backup image "
+          + tableBackupPath.toString());
+      restoreTool.fullRestoreTable(conn, tableBackupPath, sTable, tTable, truncateIfExists,
+        lastIncrBackupId);
+    } else { // incremental Backup
+      throw new IOException("Unexpected backup type " + image.getType());
+    }
+
+    if (images.length == 1) {
+      // full backup restore done
+      return;
+    }
+
+    List<Path> dirList = new ArrayList<Path>();
+    // add full backup path
+    // full backup path comes first
+    for (int i = 1; i < images.length; i++) {
+      BackupImage im = images[i];
+      String logBackupDir = HBackupFileSystem.getLogBackupDir(im.getRootDir(), im.getBackupId());
+      dirList.add(new Path(logBackupDir));
+    }
+
+    String dirs = StringUtils.join(dirList, ",");
+    LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from log dirs: " + dirs);
+    Path[] paths = new Path[dirList.size()];
+    dirList.toArray(paths);
+    restoreTool.incrementalRestoreTable(conn, tableBackupPath, paths, new TableName[] { sTable },
+      new TableName[] { tTable }, lastIncrBackupId);
+    LOG.info(sTable + " has been successfully restored to " + tTable);
+
+  }
+
+  /**
+   * Restore operation. Stage 2: resolved Backup Image dependency
+   * @param backupManifestMap : tableName, Manifest
+   * @param sTableArray The array of tables to be restored
+   * @param tTableArray The array of mapping tables to restore to
+   * @return set of BackupImages restored
+   * @throws IOException exception
+   */
+  private void restore(HashMap<TableName, BackupManifest> backupManifestMap,
+      TableName[] sTableArray, TableName[] tTableArray, boolean isOverwrite) throws IOException {
+    TreeSet<BackupImage> restoreImageSet = new TreeSet<BackupImage>();
+    boolean truncateIfExists = isOverwrite;
+    try {
+      for (int i = 0; i < sTableArray.length; i++) {
+        TableName table = sTableArray[i];
+        BackupManifest manifest = backupManifestMap.get(table);
+        // Get the image list of this backup for restore in time order from old
+        // to new.
+        List<BackupImage> list = new ArrayList<BackupImage>();
+        list.add(manifest.getBackupImage());
+        TreeSet<BackupImage> set = new TreeSet<BackupImage>(list);
+        List<BackupImage> depList = manifest.getDependentListByTable(table);
+        set.addAll(depList);
+        BackupImage[] arr = new BackupImage[set.size()];
+        set.toArray(arr);
+        restoreImages(arr, table, tTableArray[i], truncateIfExists);
+        restoreImageSet.addAll(list);
+        if (restoreImageSet != null && !restoreImageSet.isEmpty()) {
+          LOG.info("Restore includes the following image(s):");
+          for (BackupImage image : restoreImageSet) {
+            LOG.info("Backup: "
+                + image.getBackupId()
+                + " "
+                + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(),
+                  table));
+          }
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Failed", e);
+      throw new IOException(e);
+    }
+    LOG.debug("restoreStage finished");
+  }
+
+  public void execute() throws IOException {
+
+    // case VALIDATION:
+    // check the target tables
+    checkTargetTables(tTableArray, isOverwrite);
+    // case RESTORE_IMAGES:
+    HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
+    // check and load backup image manifest for the tables
+    Path rootPath = new Path(targetRootDir);
+    HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, conf, rootPath,
+      backupId);
+    restore(backupManifestMap, sTableArray, tTableArray, isOverwrite);
+  }
+
+}


[04/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
new file mode 100644
index 0000000..d10713d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
@@ -0,0 +1,791 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+
+/**
+ * Backup manifest Contains all the meta data of a backup image. The manifest info will be bundled
+ * as manifest file together with data. So that each backup image will contain all the info needed
+ * for restore.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupManifest {
+
+  private static final Log LOG = LogFactory.getLog(BackupManifest.class);
+
+  // manifest file name
+  public static final String MANIFEST_FILE_NAME = ".backup.manifest";
+
+  // manifest file version, current is 1.0
+  public static final String MANIFEST_VERSION = "1.0";
+
+  // backup image, the dependency graph is made up by series of backup images
+
+  public static class BackupImage implements Comparable<BackupImage> {
+
+    private String backupId;
+    private BackupType type;
+    private String rootDir;
+    private List<TableName> tableList;
+    private long startTs;
+    private long completeTs;
+    private ArrayList<BackupImage> ancestors;
+
+    public BackupImage() {
+      super();
+    }
+
+    public BackupImage(String backupId, BackupType type, String rootDir,
+        List<TableName> tableList, long startTs, long completeTs) {
+      this.backupId = backupId;
+      this.type = type;
+      this.rootDir = rootDir;
+      this.tableList = tableList;
+      this.startTs = startTs;
+      this.completeTs = completeTs;
+    }
+
+    static BackupImage fromProto(BackupProtos.BackupImage im) {
+      String backupId = im.getBackupId();
+      String rootDir = im.getRootDir();
+      long startTs = im.getStartTs();
+      long completeTs = im.getCompleteTs();
+      List<HBaseProtos.TableName> tableListList = im.getTableListList();
+      List<TableName> tableList = new ArrayList<TableName>();
+      for(HBaseProtos.TableName tn : tableListList) {
+        tableList.add(ProtobufUtil.toTableName(tn));
+      }
+      
+      List<BackupProtos.BackupImage> ancestorList = im.getAncestorsList();
+      
+      BackupType type =
+          im.getBackupType() == BackupProtos.BackupType.FULL ? BackupType.FULL:
+            BackupType.INCREMENTAL;
+
+      BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs);
+      for(BackupProtos.BackupImage img: ancestorList) {
+        image.addAncestor(fromProto(img));
+      }
+      return image;
+    }
+
+    BackupProtos.BackupImage toProto() {
+      BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder();
+      builder.setBackupId(backupId);
+      builder.setCompleteTs(completeTs);
+      builder.setStartTs(startTs);
+      builder.setRootDir(rootDir);
+      if (type == BackupType.FULL) {
+        builder.setBackupType(BackupProtos.BackupType.FULL);
+      } else{
+        builder.setBackupType(BackupProtos.BackupType.INCREMENTAL);
+      }
+
+      for (TableName name: tableList) {
+        builder.addTableList(ProtobufUtil.toProtoTableName(name));
+      }
+
+      if (ancestors != null){
+        for (BackupImage im: ancestors){
+          builder.addAncestors(im.toProto());
+        }
+      }
+
+      return builder.build();
+    }
+
+    public String getBackupId() {
+      return backupId;
+    }
+
+    public void setBackupId(String backupId) {
+      this.backupId = backupId;
+    }
+
+    public BackupType getType() {
+      return type;
+    }
+
+    public void setType(BackupType type) {
+      this.type = type;
+    }
+
+    public String getRootDir() {
+      return rootDir;
+    }
+
+    public void setRootDir(String rootDir) {
+      this.rootDir = rootDir;
+    }
+
+    public List<TableName> getTableNames() {
+      return tableList;
+    }
+
+    public void setTableList(List<TableName> tableList) {
+      this.tableList = tableList;
+    }
+
+    public long getStartTs() {
+      return startTs;
+    }
+
+    public void setStartTs(long startTs) {
+      this.startTs = startTs;
+    }
+
+    public long getCompleteTs() {
+      return completeTs;
+    }
+
+    public void setCompleteTs(long completeTs) {
+      this.completeTs = completeTs;
+    }
+
+    public ArrayList<BackupImage> getAncestors() {
+      if (this.ancestors == null) {
+        this.ancestors = new ArrayList<BackupImage>();
+      }
+      return this.ancestors;
+    }
+
+    public void addAncestor(BackupImage backupImage) {
+      this.getAncestors().add(backupImage);
+    }
+
+    public boolean hasAncestor(String token) {
+      for (BackupImage image : this.getAncestors()) {
+        if (image.getBackupId().equals(token)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public boolean hasTable(TableName table) {
+      for (TableName t : tableList) {
+        if (t.equals(table)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    public int compareTo(BackupImage other) {
+      String thisBackupId = this.getBackupId();
+      String otherBackupId = other.getBackupId();
+      int index1 = thisBackupId.lastIndexOf("_");
+      int index2 = otherBackupId.lastIndexOf("_");
+      String name1 = thisBackupId.substring(0, index1);
+      String name2 = otherBackupId.substring(0, index2);
+      if(name1.equals(name2)) {
+        Long thisTS = new Long(thisBackupId.substring(index1 + 1));
+        Long otherTS = new Long(otherBackupId.substring(index2 + 1));
+        return thisTS.compareTo(otherTS);
+      } else {
+        return name1.compareTo(name2);
+      }
+    }
+  }
+
+  // manifest version
+  private String version = MANIFEST_VERSION;
+
+  // hadoop hbase configuration
+  protected Configuration config = null;
+
+  // backup root directory
+  private String rootDir = null;
+
+  // backup image directory
+  private String tableBackupDir = null;
+
+  // backup log directory if this is an incremental backup
+  private String logBackupDir = null;
+
+  // backup token
+  private String backupId;
+
+  // backup type, full or incremental
+  private BackupType type;
+
+  // the table list for the backup
+  private ArrayList<TableName> tableList;
+
+  // actual start timestamp of the backup process
+  private long startTs;
+
+  // actual complete timestamp of the backup process
+  private long completeTs;
+
+  // the region server timestamp for tables:
+  // <table, <rs, timestamp>>
+  private Map<TableName, HashMap<String, Long>> incrTimeRanges;
+
+  // dependency of this backup, including all the dependent images to do PIT recovery
+  private Map<String, BackupImage> dependency;
+  
+  /**
+   * Construct manifest for a ongoing backup.
+   * @param backupCtx The ongoing backup context
+   */
+  public BackupManifest(BackupInfo backupCtx) {
+    this.backupId = backupCtx.getBackupId();
+    this.type = backupCtx.getType();
+    this.rootDir = backupCtx.getTargetRootDir();
+    if (this.type == BackupType.INCREMENTAL) {
+      this.logBackupDir = backupCtx.getHLogTargetDir();
+    }
+    this.startTs = backupCtx.getStartTs();
+    this.completeTs = backupCtx.getEndTs();
+    this.loadTableList(backupCtx.getTableNames());
+  }
+  
+  
+  /**
+   * Construct a table level manifest for a backup of the named table.
+   * @param backupCtx The ongoing backup context
+   */
+  public BackupManifest(BackupInfo backupCtx, TableName table) {
+    this.backupId = backupCtx.getBackupId();
+    this.type = backupCtx.getType();
+    this.rootDir = backupCtx.getTargetRootDir();
+    this.tableBackupDir = backupCtx.getBackupStatus(table).getTargetDir();
+    if (this.type == BackupType.INCREMENTAL) {
+      this.logBackupDir = backupCtx.getHLogTargetDir();
+    }
+    this.startTs = backupCtx.getStartTs();
+    this.completeTs = backupCtx.getEndTs();
+    List<TableName> tables = new ArrayList<TableName>();
+    tables.add(table);
+    this.loadTableList(tables);
+  }
+
+  /**
+   * Construct manifest from a backup directory.
+   * @param conf configuration
+   * @param backupPath backup path
+   * @throws IOException 
+   */
+
+  public BackupManifest(Configuration conf, Path backupPath) throws IOException {
+    this(backupPath.getFileSystem(conf), backupPath);
+  }
+
+  /**
+   * Construct manifest from a backup directory.
+   * @param conf configuration
+   * @param backupPath backup path
+   * @throws BackupException exception
+   */
+
+  public BackupManifest(FileSystem fs, Path backupPath) throws BackupException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Loading manifest from: " + backupPath.toString());
+    }
+    // The input backupDir may not exactly be the backup table dir.
+    // It could be the backup log dir where there is also a manifest file stored.
+    // This variable's purpose is to keep the correct and original location so
+    // that we can store/persist it.
+    this.tableBackupDir = backupPath.toString();
+    this.config = fs.getConf();
+    try {
+
+      FileStatus[] subFiles = BackupClientUtil.listStatus(fs, backupPath, null);
+      if (subFiles == null) {
+        String errorMsg = backupPath.toString() + " does not exist";
+        LOG.error(errorMsg);
+        throw new IOException(errorMsg);
+      }
+      for (FileStatus subFile : subFiles) {
+        if (subFile.getPath().getName().equals(MANIFEST_FILE_NAME)) {
+
+          // load and set manifest field from file content
+          FSDataInputStream in = fs.open(subFile.getPath());
+          long len = subFile.getLen();
+          byte[] pbBytes = new byte[(int) len];
+          in.readFully(pbBytes);
+          BackupProtos.BackupManifest proto = null;
+          try{
+            proto = parseFrom(pbBytes);
+          } catch(Exception e){
+            throw new BackupException(e);
+          }
+          this.version = proto.getVersion();
+          this.backupId = proto.getBackupId();
+          this.type = BackupType.valueOf(proto.getType().name());
+          // Here the parameter backupDir is where the manifest file is.
+          // There should always be a manifest file under:
+          // backupRootDir/namespace/table/backupId/.backup.manifest
+          this.rootDir = backupPath.getParent().getParent().getParent().toString();
+
+          Path p = backupPath.getParent();
+          if (p.getName().equals(HConstants.HREGION_LOGDIR_NAME)) {
+            this.rootDir = p.getParent().toString();
+          } else {
+            this.rootDir = p.getParent().getParent().toString();
+          }
+
+          loadTableList(proto);
+          this.startTs = proto.getStartTs();
+          this.completeTs = proto.getCompleteTs();
+          loadIncrementalTimestampMap(proto);
+          loadDependency(proto);
+          //TODO: merge will be implemented by future jira
+          LOG.debug("Loaded manifest instance from manifest file: "
+              + BackupClientUtil.getPath(subFile.getPath()));
+          return;
+        }
+      }
+      String errorMsg = "No manifest file found in: " + backupPath.toString();
+      throw new IOException(errorMsg);
+
+    } catch (IOException e) {
+      throw new BackupException(e.getMessage());
+    }
+  }
+  
+  private void loadIncrementalTimestampMap(BackupProtos.BackupManifest proto) {
+    List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList();
+    if(list == null || list.size() == 0) return;
+    this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
+    for(BackupProtos.TableServerTimestamp tst: list){
+      TableName tn = ProtobufUtil.toTableName(tst.getTable());
+      HashMap<String, Long> map = this.incrTimeRanges.get(tn);
+      if(map == null){
+        map = new HashMap<String, Long>();
+        this.incrTimeRanges.put(tn, map);
+      }
+      List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList();
+      for(BackupProtos.ServerTimestamp stm: listSt) {
+        map.put(stm.getServer(), stm.getTimestamp());
+      }
+    }
+  }
+
+  private void loadDependency(BackupProtos.BackupManifest proto) {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("load dependency for: "+proto.getBackupId());
+    }
+
+    dependency = new HashMap<String, BackupImage>();
+    List<BackupProtos.BackupImage> list = proto.getDependentBackupImageList();
+    for (BackupProtos.BackupImage im : list) {
+      BackupImage bim = BackupImage.fromProto(im);
+      if(im.getBackupId() != null){
+        dependency.put(im.getBackupId(), bim);
+      } else{
+        LOG.warn("Load dependency for backup manifest: "+ backupId+ 
+          ". Null backup id in dependent image");
+      }
+    }
+  }
+
+  private void loadTableList(BackupProtos.BackupManifest proto) {
+    this.tableList = new ArrayList<TableName>();
+    List<HBaseProtos.TableName> list = proto.getTableListList();
+    for (HBaseProtos.TableName name: list) {
+      this.tableList.add(ProtobufUtil.toTableName(name));
+    }
+  }
+
+  public BackupType getType() {
+    return type;
+  }
+
+  public void setType(BackupType type) {
+    this.type = type;
+  }
+
+  /**
+   * Loads table list.
+   * @param tableList Table list
+   */
+  private void loadTableList(List<TableName> tableList) {
+
+    this.tableList = this.getTableList();
+    if (this.tableList.size() > 0) {
+      this.tableList.clear();
+    }
+    for (int i = 0; i < tableList.size(); i++) {
+      this.tableList.add(tableList.get(i));
+    }
+
+    LOG.debug(tableList.size() + " tables exist in table set.");
+  }
+
+  /**
+   * Get the table set of this image.
+   * @return The table set list
+   */
+  public ArrayList<TableName> getTableList() {
+    if (this.tableList == null) {
+      this.tableList = new ArrayList<TableName>();
+    }
+    return this.tableList;
+  }
+
+  /**
+   * Persist the manifest file.
+   * @throws IOException IOException when storing the manifest file.
+   */
+
+  public void store(Configuration conf) throws BackupException {
+    byte[] data = toByteArray();
+
+    // write the file, overwrite if already exist
+    Path manifestFilePath =
+        new Path(new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir))
+            ,MANIFEST_FILE_NAME);
+    try {
+      FSDataOutputStream out =
+          manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);
+      out.write(data);
+      out.close();
+    } catch (IOException e) {      
+      throw new BackupException(e.getMessage());
+    }
+
+    LOG.info("Manifest file stored to " + manifestFilePath);
+  }
+
+  /**
+   * Protobuf serialization
+   * @return The filter serialized using pb
+   */
+  public byte[] toByteArray() {
+    BackupProtos.BackupManifest.Builder builder = BackupProtos.BackupManifest.newBuilder();
+    builder.setVersion(this.version);
+    builder.setBackupId(this.backupId);
+    builder.setType(BackupProtos.BackupType.valueOf(this.type.name()));
+    setTableList(builder);
+    builder.setStartTs(this.startTs);
+    builder.setCompleteTs(this.completeTs);
+    setIncrementalTimestampMap(builder);
+    setDependencyMap(builder);
+    return builder.build().toByteArray();
+  }
+
+  private void setIncrementalTimestampMap(BackupProtos.BackupManifest.Builder builder) {
+    if (this.incrTimeRanges == null) {
+      return;
+    }
+    for (Entry<TableName, HashMap<String,Long>> entry: this.incrTimeRanges.entrySet()) {
+      TableName key = entry.getKey();
+      HashMap<String, Long> value = entry.getValue();
+      BackupProtos.TableServerTimestamp.Builder tstBuilder =
+          BackupProtos.TableServerTimestamp.newBuilder();
+      tstBuilder.setTable(ProtobufUtil.toProtoTableName(key));
+
+      for (String s : value.keySet()) {
+        BackupProtos.ServerTimestamp.Builder stBuilder = BackupProtos.ServerTimestamp.newBuilder();
+        stBuilder.setServer(s);
+        stBuilder.setTimestamp(value.get(s));
+        tstBuilder.addServerTimestamp(stBuilder.build());
+      }
+      builder.addTstMap(tstBuilder.build());
+    }
+  }
+
+  private void setDependencyMap(BackupProtos.BackupManifest.Builder builder) {
+    for (BackupImage image: getDependency().values()) {
+      builder.addDependentBackupImage(image.toProto());
+    }
+  }
+
+  private void setTableList(BackupProtos.BackupManifest.Builder builder) {
+    for(TableName name: tableList){
+      builder.addTableList(ProtobufUtil.toProtoTableName(name));
+    }
+  }
+
+  /**
+   * Parse protobuf from byte array
+   * @param pbBytes A pb serialized BackupManifest instance
+   * @return An instance of  made from <code>bytes</code>
+   * @throws DeserializationException
+   */
+  private static BackupProtos.BackupManifest parseFrom(final byte[] pbBytes)
+      throws DeserializationException {
+    BackupProtos.BackupManifest proto;
+    try {
+      proto = BackupProtos.BackupManifest.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return proto;
+  }
+
+  /**
+   * Get manifest file version
+   * @return version
+   */
+  public String getVersion() {
+    return version;
+  }
+
+  /**
+   * Get this backup image.
+   * @return the backup image.
+   */
+  public BackupImage getBackupImage() {
+    return this.getDependency().get(this.backupId);
+  }
+
+  /**
+   * Add dependent backup image for this backup.
+   * @param image The direct dependent backup image
+   */
+  public void addDependentImage(BackupImage image) {
+    this.getDependency().get(this.backupId).addAncestor(image);
+    this.setDependencyMap(this.getDependency(), image);
+  }
+
+
+
+  /**
+   * Get all dependent backup images. The image of this backup is also contained.
+   * @return The dependent backup images map
+   */
+  public Map<String, BackupImage> getDependency() {
+    if (this.dependency == null) {
+      this.dependency = new HashMap<String, BackupImage>();
+      LOG.debug(this.rootDir + " " + this.backupId + " " + this.type);
+      this.dependency.put(this.backupId,
+        new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs,
+            this.completeTs));
+    }
+    return this.dependency;
+  }
+
+  /**
+   * Set the incremental timestamp map directly.
+   * @param incrTimestampMap timestamp map
+   */
+  public void setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> incrTimestampMap) {
+    this.incrTimeRanges = incrTimestampMap;
+  }
+
+
+  public Map<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+    if (this.incrTimeRanges == null) {
+      this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
+    }
+    return this.incrTimeRanges;
+  }
+
+
+  /**
+   * Get the image list of this backup for restore in time order.
+   * @param reverse If true, then output in reverse order, otherwise in time order from old to new
+   * @return the backup image list for restore in time order
+   */
+  public ArrayList<BackupImage> getRestoreDependentList(boolean reverse) {
+    TreeMap<Long, BackupImage> restoreImages = new TreeMap<Long, BackupImage>();
+    for (BackupImage image : this.getDependency().values()) {
+      restoreImages.put(Long.valueOf(image.startTs), image);
+    }
+    return new ArrayList<BackupImage>(reverse ? (restoreImages.descendingMap().values())
+        : (restoreImages.values()));
+  }
+
+  /**
+   * Get the dependent image list for a specific table of this backup in time order from old to new
+   * if want to restore to this backup image level.
+   * @param table table
+   * @return the backup image list for a table in time order
+   */
+  public ArrayList<BackupImage> getDependentListByTable(TableName table) {
+    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
+    ArrayList<BackupImage> imageList = getRestoreDependentList(true);
+    for (BackupImage image : imageList) {
+      if (image.hasTable(table)) {
+        tableImageList.add(image);
+        if (image.getType() == BackupType.FULL) {
+          break;
+        }
+      }
+    }
+    Collections.reverse(tableImageList);
+    return tableImageList;
+  }
+
+  /**
+   * Get the full dependent image list in the whole dependency scope for a specific table of this
+   * backup in time order from old to new.
+   * @param table table
+   * @return the full backup image list for a table in time order in the whole scope of the
+   *         dependency of this image
+   */
+  public ArrayList<BackupImage> getAllDependentListByTable(TableName table) {
+    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
+    ArrayList<BackupImage> imageList = getRestoreDependentList(false);
+    for (BackupImage image : imageList) {
+      if (image.hasTable(table)) {
+        tableImageList.add(image);
+      }
+    }
+    return tableImageList;
+  }
+
+
+  /**
+   * Recursively set the dependency map of the backup images.
+   * @param map The dependency map
+   * @param image The backup image
+   */
+  private void setDependencyMap(Map<String, BackupImage> map, BackupImage image) {
+    if (image == null) {
+      return;
+    } else {
+      map.put(image.getBackupId(), image);
+      for (BackupImage img : image.getAncestors()) {
+        setDependencyMap(map, img);
+      }
+    }
+  }
+
+  /**
+   * Check whether backup image1 could cover backup image2 or not.
+   * @param image1 backup image 1
+   * @param image2 backup image 2
+   * @return true if image1 can cover image2, otherwise false
+   */
+  public static boolean canCoverImage(BackupImage image1, BackupImage image2) {
+    // image1 can cover image2 only when the following conditions are satisfied:
+    // - image1 must not be an incremental image;
+    // - image1 must be taken after image2 has been taken;
+    // - table set of image1 must cover the table set of image2.
+    if (image1.getType() == BackupType.INCREMENTAL) {
+      return false;
+    }
+    if (image1.getStartTs() < image2.getStartTs()) {
+      return false;
+    }
+    List<TableName> image1TableList = image1.getTableNames();
+    List<TableName> image2TableList = image2.getTableNames();
+    boolean found = false;
+    for (int i = 0; i < image2TableList.size(); i++) {
+      found = false;
+      for (int j = 0; j < image1TableList.size(); j++) {
+        if (image2TableList.get(i).equals(image1TableList.get(j))) {
+          found = true;
+          break;
+        }
+      }
+      if (!found) {
+        return false;
+      }
+    }
+
+    LOG.debug("Backup image " + image1.getBackupId() + " can cover " + image2.getBackupId());
+    return true;
+  }
+
+  /**
+   * Check whether backup image set could cover a backup image or not.
+   * @param fullImages The backup image set
+   * @param image The target backup image
+   * @return true if fullImages can cover image, otherwise false
+   */
+  public static boolean canCoverImage(ArrayList<BackupImage> fullImages, BackupImage image) {
+    // fullImages can cover image only when the following conditions are satisfied:
+    // - each image of fullImages must not be an incremental image;
+    // - each image of fullImages must be taken after image has been taken;
+    // - sum table set of fullImages must cover the table set of image.
+    for (BackupImage image1 : fullImages) {
+      if (image1.getType() == BackupType.INCREMENTAL) {
+        return false;
+      }
+      if (image1.getStartTs() < image.getStartTs()) {
+        return false;
+      }
+    }
+
+    ArrayList<String> image1TableList = new ArrayList<String>();
+    for (BackupImage image1 : fullImages) {
+      List<TableName> tableList = image1.getTableNames();
+      for (TableName table : tableList) {
+        image1TableList.add(table.getNameAsString());
+      }
+    }
+    ArrayList<String> image2TableList = new ArrayList<String>();
+    List<TableName> tableList = image.getTableNames();
+    for (TableName table : tableList) {
+      image2TableList.add(table.getNameAsString());
+    }
+
+    for (int i = 0; i < image2TableList.size(); i++) {
+      if (image1TableList.contains(image2TableList.get(i)) == false) {
+        return false;
+      }
+    }
+
+    LOG.debug("Full image set can cover image " + image.getBackupId());
+    return true;
+  }
+  
+  public BackupInfo toBackupInfo()
+  {
+    BackupInfo info = new BackupInfo();
+    info.setType(type);
+    TableName[] tables = new TableName[tableList.size()];
+    info.addTables(getTableList().toArray(tables));
+    info.setBackupId(backupId);
+    info.setStartTs(startTs);
+    info.setTargetRootDir(rootDir);
+    if(type == BackupType.INCREMENTAL) {
+      info.setHlogTargetDir(logBackupDir);
+    }
+    return info;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
new file mode 100644
index 0000000..ac1d2bc
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.impl;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * BackupRestoreConstants holds a bunch of HBase Backup and Restore constants
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public final class BackupRestoreConstants {
+
+
+  // delimiter in tablename list in restore command
+  public static final String TABLENAME_DELIMITER_IN_COMMAND = ",";
+
+  public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root";
+
+  public static final String BACKUPID_PREFIX = "backup_";
+
+  public static enum BackupCommand {
+    CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, PROGRESS, SET,
+    SET_ADD, SET_REMOVE, SET_DELETE, SET_DESCRIBE, SET_LIST
+  }
+
+  private BackupRestoreConstants() {
+    // Can't be instantiated with this ctor.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
new file mode 100644
index 0000000..d05d54c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
@@ -0,0 +1,926 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+
+/**
+ * This class provides 'hbase:backup' table API
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupSystemTable implements Closeable {
+
+  static class WALItem {
+    String backupId;
+    String walFile;
+    String backupRoot;
+
+    WALItem(String backupId, String walFile, String backupRoot) {
+      this.backupId = backupId;
+      this.walFile = walFile;
+      this.backupRoot = backupRoot;
+    }
+
+    public String getBackupId() {
+      return backupId;
+    }
+
+    public String getWalFile() {
+      return walFile;
+    }
+
+    public String getBackupRoot() {
+      return backupRoot;
+    }
+
+    public String toString() {
+      return "/" + backupRoot + "/" + backupId + "/" + walFile;
+    }
+
+  }
+
+  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
+  private final static TableName tableName = TableName.BACKUP_TABLE_NAME;
+  // Stores backup sessions (contexts)
+  final static byte[] SESSIONS_FAMILY = "session".getBytes();
+  // Stores other meta
+  final static byte[] META_FAMILY = "meta".getBytes();
+  // Connection to HBase cluster, shared
+  // among all instances
+  private final Connection connection;
+
+  public BackupSystemTable(Connection conn) throws IOException {
+    this.connection = conn;
+  }
+
+  public void close() {
+    // do nothing
+  }
+
+  /**
+   * Updates status (state) of a backup session in hbase:backup table
+   * @param context context
+   * @throws IOException exception
+   */
+  public void updateBackupInfo(BackupInfo context) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("update backup status in hbase:backup for: " + context.getBackupId()
+          + " set status=" + context.getState());
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = BackupSystemTableHelper.createPutForBackupContext(context);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Deletes backup status from hbase:backup table
+   * @param backupId backup id
+   * @throws IOException exception
+   */
+
+  public void deleteBackupInfo(String backupId) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("delete backup status in hbase:backup for " + backupId);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Delete del = BackupSystemTableHelper.createDeleteForBackupInfo(backupId);
+      table.delete(del);
+    }
+  }
+
+  /**
+   * Reads backup status object (instance of BackupContext) from hbase:backup table
+   * @param backupId - backupId
+   * @return Current status of backup session or null
+   */
+
+  public BackupInfo readBackupInfo(String backupId) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read backup status from hbase:backup for: " + backupId);
+    }
+
+    try (Table table = connection.getTable(tableName)) {
+      Get get = BackupSystemTableHelper.createGetForBackupContext(backupId);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return null;
+      }
+      return BackupSystemTableHelper.resultToBackupInfo(res);
+    }
+  }
+
+  /**
+   * Read the last backup start code (timestamp) of last successful backup. Will return null if
+   * there is no start code stored on hbase or the value is of length 0. These two cases indicate
+   * there is no successful backup completed so far.
+   * @param backupRoot root directory path to backup
+   * @return the timestamp of last successful backup
+   * @throws IOException exception
+   */
+  public String readBackupStartCode(String backupRoot) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read backup start code from hbase:backup");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Get get = BackupSystemTableHelper.createGetForStartCode(backupRoot);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return null;
+      }
+      Cell cell = res.listCells().get(0);
+      byte[] val = CellUtil.cloneValue(cell);
+      if (val.length == 0) {
+        return null;
+      }
+      return new String(val);
+    }
+  }
+
+  /**
+   * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
+   * @param startCode start code
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public void writeBackupStartCode(Long startCode, String backupRoot) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write backup start code to hbase:backup " + startCode);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = BackupSystemTableHelper.createPutForStartCode(startCode.toString(), backupRoot);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Get the Region Servers log information after the last log roll from hbase:backup.
+   * @param backupRoot root directory path to backup
+   * @return RS log info
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> readRegionServerLastLogRollResult(String backupRoot)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read region server last roll log result to hbase:backup");
+    }
+
+    Scan scan = BackupSystemTableHelper.createScanForReadRegionServerLastLogRollResult(backupRoot);
+
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      HashMap<String, Long> rsTimestampMap = new HashMap<String, Long>();
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        Cell cell = res.current();
+        byte[] row = CellUtil.cloneRow(cell);
+        String server =
+            BackupSystemTableHelper.getServerNameForReadRegionServerLastLogRollResult(row);
+        byte[] data = CellUtil.cloneValue(cell);
+        rsTimestampMap.put(server, Long.parseLong(new String(data)));
+      }
+      return rsTimestampMap;
+    }
+  }
+
+  /**
+   * Writes Region Server last roll log result (timestamp) to hbase:backup table
+   * @param server - Region Server name
+   * @param ts- last log timestamp
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public void writeRegionServerLastLogRollResult(String server, Long ts, String backupRoot)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write region server last roll log result to hbase:backup");
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put =
+          BackupSystemTableHelper.createPutForRegionServerLastLogRollResult(server, ts, backupRoot);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Get all completed backup information (in desc order by time)
+   * @param onlyCompeleted, true, if only successfully completed sessions
+   * @return history info of BackupCompleteData
+   * @throws IOException exception
+   */
+  public ArrayList<BackupInfo> getBackupHistory(boolean onlyCompleted) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get backup history from hbase:backup");
+    }
+    ArrayList<BackupInfo> list;
+    BackupState state = onlyCompleted ? BackupState.COMPLETE : BackupState.ANY;
+    list = getBackupContexts(state);
+    return BackupClientUtil.sortHistoryListDesc(list);
+  }
+
+  /**
+   * Get all backups history
+   * @return list of backup info 
+   * @throws IOException
+   */
+  public List<BackupInfo> getBackupHistory() throws IOException {
+    return getBackupHistory(false);
+  }
+
+  /**
+   * Get first n backup history records
+   * @param n - number of records
+   * @return list of records
+   * @throws IOException
+   */
+  public List<BackupInfo> getHistory(int n) throws IOException {
+
+    List<BackupInfo> history = getBackupHistory();
+    if (history.size() <= n) return history;
+    List<BackupInfo> list = new ArrayList<BackupInfo>();
+    for (int i = 0; i < n; i++) {
+      list.add(history.get(i));
+    }
+    return list;
+
+  }
+  
+  /**
+   * Get backup history records filtered by list
+   * of filters.
+   * @param n - max number of records
+   * @param filters - list of filters
+   * @return backup records
+   * @throws IOException
+   */
+  public List<BackupInfo> getBackupHistory(int n, BackupInfo.Filter... filters) throws IOException {
+    if (filters.length == 0) return getHistory(n);
+
+    List<BackupInfo> history = getBackupHistory();
+    List<BackupInfo> result = new ArrayList<BackupInfo>();
+    for (BackupInfo bi : history) {
+      if (result.size() == n) break;
+      boolean passed = true;
+      for (int i = 0; i < filters.length; i++) {
+        if (!filters[i].apply(bi)) {
+          passed = false;
+          break;
+        }
+      }
+      if (passed) {
+        result.add(bi);
+      }
+    }
+    return result;
+
+  }
+
+  /**
+   * Get history for backup destination
+   * @param backupRoot - backup destination
+   * @return List of backup info
+   * @throws IOException
+   */
+  public List<BackupInfo> getBackupHistory(String backupRoot) throws IOException {
+    ArrayList<BackupInfo> history = getBackupHistory(false);
+    for (Iterator<BackupInfo> iterator = history.iterator(); iterator.hasNext();) {
+      BackupInfo info = iterator.next();
+      if (!backupRoot.equals(info.getTargetRootDir())) {
+        iterator.remove();
+      }
+    }
+    return history;
+  }
+  
+  /**
+   * Get history for a table
+   * @param name - table name
+   * @return history for a table
+   * @throws IOException
+   */
+  public List<BackupInfo> getBackupHistoryForTable(TableName name) throws IOException {
+    List<BackupInfo> history = getBackupHistory();
+    List<BackupInfo> tableHistory = new ArrayList<BackupInfo>();
+    for (BackupInfo info : history) {
+      List<TableName> tables = info.getTableNames();
+      if (tables.contains(name)) {
+        tableHistory.add(info);
+      }
+    }
+    return tableHistory;
+  }
+
+  public Map<TableName, ArrayList<BackupInfo>> 
+    getBackupHistoryForTableSet(Set<TableName> set, String backupRoot) throws IOException {
+    List<BackupInfo> history = getBackupHistory(backupRoot);
+    Map<TableName, ArrayList<BackupInfo>> tableHistoryMap = 
+        new HashMap<TableName, ArrayList<BackupInfo>>();
+    for (Iterator<BackupInfo> iterator = history.iterator(); iterator.hasNext();) {
+      BackupInfo info = iterator.next();
+      if (!backupRoot.equals(info.getTargetRootDir())) {
+        continue;
+      }
+      List<TableName> tables = info.getTableNames();
+      for (TableName tableName: tables) {      
+        if (set.contains(tableName)) {
+          ArrayList<BackupInfo> list = tableHistoryMap.get(tableName);
+          if (list == null) {
+            list = new ArrayList<BackupInfo>();
+            tableHistoryMap.put(tableName, list);
+          }
+          list.add(info);
+        }
+      }
+    }
+    return tableHistoryMap;
+  }
+  
+  /**
+   * Get all backup session with a given status (in desc order by time)
+   * @param status status
+   * @return history info of backup contexts
+   * @throws IOException exception
+   */
+  public ArrayList<BackupInfo> getBackupContexts(BackupState status) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get backup contexts from hbase:backup");
+    }
+
+    Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
+    ArrayList<BackupInfo> list = new ArrayList<BackupInfo>();
+
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        BackupInfo context = BackupSystemTableHelper.cellToBackupInfo(res.current());
+        if (status != BackupState.ANY && context.getState() != status) {
+          continue;
+        }
+        list.add(context);
+      }
+      return list;
+    }
+  }
+
+  /**
+   * Write the current timestamps for each regionserver to hbase:backup after a successful full or
+   * incremental backup. The saved timestamp is of the last log file that was backed up already.
+   * @param tables tables
+   * @param newTimestamps timestamps
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public void writeRegionServerLogTimestamp(Set<TableName> tables,
+      HashMap<String, Long> newTimestamps, String backupRoot) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("write RS log time stamps to hbase:backup for tables ["
+          + StringUtils.join(tables, ",") + "]");
+    }
+    List<Put> puts = new ArrayList<Put>();
+    for (TableName table : tables) {
+      byte[] smapData = toTableServerTimestampProto(table, newTimestamps).toByteArray();
+      Put put =
+          BackupSystemTableHelper.createPutForWriteRegionServerLogTimestamp(table, smapData,
+            backupRoot);
+      puts.add(put);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      table.put(puts);
+    }
+  }
+
+  /**
+   * Read the timestamp for each region server log after the last successful backup. Each table has
+   * its own set of the timestamps. The info is stored for each table as a concatenated string of
+   * rs->timestapmp
+   * @param backupRoot root directory path to backup
+   * @return the timestamp for each region server. key: tableName value:
+   *         RegionServer,PreviousTimeStamp
+   * @throws IOException exception
+   */
+  public HashMap<TableName, HashMap<String, Long>> readLogTimestampMap(String backupRoot)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("read RS log ts from hbase:backup for root=" + backupRoot);
+    }
+
+    HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
+        new HashMap<TableName, HashMap<String, Long>>();
+
+    Scan scan = BackupSystemTableHelper.createScanForReadLogTimestampMap(backupRoot);
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        Cell cell = res.current();
+        byte[] row = CellUtil.cloneRow(cell);
+        String tabName = BackupSystemTableHelper.getTableNameForReadLogTimestampMap(row);
+        TableName tn = TableName.valueOf(tabName);
+        byte[] data = CellUtil.cloneValue(cell);
+        if (data == null) {
+          throw new IOException("Data of last backup data from hbase:backup "
+              + "is empty. Create a backup first.");
+        }
+        if (data != null && data.length > 0) {
+          HashMap<String, Long> lastBackup =
+              fromTableServerTimestampProto(BackupProtos.TableServerTimestamp.parseFrom(data));
+          tableTimestampMap.put(tn, lastBackup);
+        }
+      }
+      return tableTimestampMap;
+    }
+  }
+
+  private BackupProtos.TableServerTimestamp toTableServerTimestampProto(TableName table,
+      Map<String, Long> map) {
+    BackupProtos.TableServerTimestamp.Builder tstBuilder =
+        BackupProtos.TableServerTimestamp.newBuilder();
+    tstBuilder.setTable(ProtobufUtil.toProtoTableName(table));
+
+    for (Entry<String, Long> entry : map.entrySet()) {
+      BackupProtos.ServerTimestamp.Builder builder = BackupProtos.ServerTimestamp.newBuilder();
+      builder.setServer(entry.getKey());
+      builder.setTimestamp(entry.getValue());
+      tstBuilder.addServerTimestamp(builder.build());
+    }
+
+    return tstBuilder.build();
+  }
+
+  private HashMap<String, Long> fromTableServerTimestampProto(
+      BackupProtos.TableServerTimestamp proto) {
+    HashMap<String, Long> map = new HashMap<String, Long>();
+    List<BackupProtos.ServerTimestamp> list = proto.getServerTimestampList();
+    for (BackupProtos.ServerTimestamp st : list) {
+      map.put(st.getServer(), st.getTimestamp());
+    }
+    return map;
+  }
+
+  /**
+   * Return the current tables covered by incremental backup.
+   * @param backupRoot root directory path to backup
+   * @return set of tableNames
+   * @throws IOException exception
+   */
+  public Set<TableName> getIncrementalBackupTableSet(String backupRoot) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get incr backup table set from hbase:backup");
+    }
+    TreeSet<TableName> set = new TreeSet<>();
+
+    try (Table table = connection.getTable(tableName)) {
+      Get get = BackupSystemTableHelper.createGetForIncrBackupTableSet(backupRoot);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return set;
+      }
+      List<Cell> cells = res.listCells();
+      for (Cell cell : cells) {
+        // qualifier = table name - we use table names as qualifiers
+        set.add(TableName.valueOf(CellUtil.cloneQualifier(cell)));
+      }
+      return set;
+    }
+  }
+
+  /**
+   * Add tables to global incremental backup set
+   * @param tables - set of tables
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public void addIncrementalBackupTableSet(Set<TableName> tables, String backupRoot)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Add incremental backup table set to hbase:backup. ROOT=" + backupRoot
+          + " tables [" + StringUtils.join(tables, " ") + "]");
+      for (TableName table : tables) {
+        LOG.debug(table);
+      }
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Put put = BackupSystemTableHelper.createPutForIncrBackupTableSet(tables, backupRoot);
+      table.put(put);
+    }
+  }
+
+  /**
+   * Removes incremental backup set
+   * @param backupRoot backup root
+   */
+
+  public void deleteIncrementalBackupTableSet(String backupRoot) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Delete incremental backup table set to hbase:backup. ROOT=" + backupRoot);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Delete delete = BackupSystemTableHelper.createDeleteForIncrBackupTableSet(backupRoot);
+      table.delete(delete);
+    }
+  }
+
+  /**
+   * Register WAL files as eligible for deletion
+   * @param files files
+   * @param backupId backup id
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public void addWALFiles(List<String> files, String backupId, String backupRoot)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("add WAL files to hbase:backup: " + backupId + " " + backupRoot + " files ["
+          + StringUtils.join(files, ",") + "]");
+      for (String f : files) {
+        LOG.debug("add :" + f);
+      }
+    }
+    try (Table table = connection.getTable(tableName)) {
+      List<Put> puts =
+          BackupSystemTableHelper.createPutsForAddWALFiles(files, backupId, backupRoot);
+      table.put(puts);
+    }
+  }
+
+  /**
+   * Register WAL files as eligible for deletion
+   * @param backupRoot root directory path to backup
+   * @throws IOException exception
+   */
+  public Iterator<WALItem> getWALFilesIterator(String backupRoot) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("get WAL files from hbase:backup");
+    }
+    final Table table = connection.getTable(tableName);
+    Scan scan = BackupSystemTableHelper.createScanForGetWALs(backupRoot);
+    final ResultScanner scanner = table.getScanner(scan);
+    final Iterator<Result> it = scanner.iterator();
+    return new Iterator<WALItem>() {
+
+      @Override
+      public boolean hasNext() {
+        boolean next = it.hasNext();
+        if (!next) {
+          // close all
+          try {
+            scanner.close();
+            table.close();
+          } catch (IOException e) {
+            LOG.error("Close WAL Iterator", e);
+          }
+        }
+        return next;
+      }
+
+      @Override
+      public WALItem next() {
+        Result next = it.next();
+        List<Cell> cells = next.listCells();
+        byte[] buf = cells.get(0).getValueArray();
+        int len = cells.get(0).getValueLength();
+        int offset = cells.get(0).getValueOffset();
+        String backupId = new String(buf, offset, len);
+        buf = cells.get(1).getValueArray();
+        len = cells.get(1).getValueLength();
+        offset = cells.get(1).getValueOffset();
+        String walFile = new String(buf, offset, len);
+        buf = cells.get(2).getValueArray();
+        len = cells.get(2).getValueLength();
+        offset = cells.get(2).getValueOffset();
+        String backupRoot = new String(buf, offset, len);
+        return new WALItem(backupId, walFile, backupRoot);
+      }
+
+      @Override
+      public void remove() {
+        // not implemented
+        throw new RuntimeException("remove is not supported");
+      }
+    };
+
+  }
+
+  /**
+   * Check if WAL file is eligible for deletion Future: to support all backup destinations
+   * @param file file
+   * @return true, if - yes.
+   * @throws IOException exception
+   */
+  public boolean isWALFileDeletable(String file) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Check if WAL file has been already backed up in hbase:backup " + file);
+    }
+    try (Table table = connection.getTable(tableName)) {
+      Get get = BackupSystemTableHelper.createGetForCheckWALFile(file);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        return false;
+      }
+      return true;
+    }
+  }
+
+  /**
+   * Checks if we have at least one backup session in hbase:backup This API is used by
+   * BackupLogCleaner
+   * @return true, if - at least one session exists in hbase:backup table
+   * @throws IOException exception
+   */
+  public boolean hasBackupSessions() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Has backup sessions from hbase:backup");
+    }
+    boolean result = false;
+    Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
+    scan.setCaching(1);
+    try (Table table = connection.getTable(tableName);
+        ResultScanner scanner = table.getScanner(scan)) {
+      if (scanner.next() != null) {
+        result = true;
+      }
+      return result;
+    }
+  }
+
+  /**
+   * BACKUP SETS
+   */
+
+  /**
+   * Get backup set list
+   * @return backup set list
+   * @throws IOException
+   */
+  public List<String> listBackupSets() throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(" Backup set list");
+    }
+    List<String> list = new ArrayList<String>();
+    Table table = null;
+    ResultScanner scanner = null;
+    try {
+      table = connection.getTable(tableName);
+      Scan scan = BackupSystemTableHelper.createScanForBackupSetList();
+      scan.setMaxVersions(1);
+      scanner = table.getScanner(scan);
+      Result res = null;
+      while ((res = scanner.next()) != null) {
+        res.advance();
+        list.add(BackupSystemTableHelper.cellKeyToBackupSetName(res.current()));
+      }
+      return list;
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Get backup set description (list of tables)
+   * @param name - set's name
+   * @return list of tables in a backup set
+   * @throws IOException
+   */
+  public List<TableName> describeBackupSet(String name) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(" Backup set describe: " + name);
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Get get = BackupSystemTableHelper.createGetForBackupSet(name);
+      Result res = table.get(get);
+      if (res.isEmpty()) return null;
+      res.advance();
+      String[] tables = BackupSystemTableHelper.cellValueToBackupSet(res.current());
+      return toList(tables);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  private List<TableName> toList(String[] tables) {
+    List<TableName> list = new ArrayList<TableName>(tables.length);
+    for (String name : tables) {
+      list.add(TableName.valueOf(name));
+    }
+    return list;
+  }
+
+  /**
+   * Add backup set (list of tables)
+   * @param name - set name
+   * @param tables - list of tables, comma-separated
+   * @throws IOException
+   */
+  public void addToBackupSet(String name, String[] newTables) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Backup set add: " + name + " tables [" + StringUtils.join(newTables, " ") + "]");
+    }
+    Table table = null;
+    String[] union = null;
+    try {
+      table = connection.getTable(tableName);
+      Get get = BackupSystemTableHelper.createGetForBackupSet(name);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        union = newTables;
+      } else {
+        res.advance();
+        String[] tables = BackupSystemTableHelper.cellValueToBackupSet(res.current());
+        union = merge(tables, newTables);
+      }
+      Put put = BackupSystemTableHelper.createPutForBackupSet(name, union);
+      table.put(put);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  private String[] merge(String[] tables, String[] newTables) {
+    List<String> list = new ArrayList<String>();
+    // Add all from tables
+    for (String t : tables) {
+      list.add(t);
+    }
+    for (String nt : newTables) {
+      if (list.contains(nt)) continue;
+      list.add(nt);
+    }
+    String[] arr = new String[list.size()];
+    list.toArray(arr);
+    return arr;
+  }
+
+  /**
+   * Remove tables from backup set (list of tables)
+   * @param name - set name
+   * @param tables - list of tables, comma-separated
+   * @throws IOException
+   */
+  public void removeFromBackupSet(String name, String[] toRemove) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(" Backup set remove from : " + name + " tables [" + StringUtils.join(toRemove, " ")
+          + "]");
+    }
+    Table table = null;
+    String[] disjoint = null;
+    try {
+      table = connection.getTable(tableName);
+      Get get = BackupSystemTableHelper.createGetForBackupSet(name);
+      Result res = table.get(get);
+      if (res.isEmpty()) {
+        LOG.warn("Backup set '" + name + "' not found.");
+        return;
+      } else {
+        res.advance();
+        String[] tables = BackupSystemTableHelper.cellValueToBackupSet(res.current());
+        disjoint = disjoin(tables, toRemove);
+      }
+      if (disjoint.length > 0) {
+        Put put = BackupSystemTableHelper.createPutForBackupSet(name, disjoint);
+        table.put(put);
+      } else {
+        // Delete
+        // describeBackupSet(name);
+        LOG.warn("Backup set '" + name + "' does not contain tables ["
+            + StringUtils.join(toRemove, " ") + "]");
+      }
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  private String[] disjoin(String[] tables, String[] toRemove) {
+    List<String> list = new ArrayList<String>();
+    // Add all from tables
+    for (String t : tables) {
+      list.add(t);
+    }
+    for (String nt : toRemove) {
+      if (list.contains(nt)) {
+        list.remove(nt);
+      }
+    }
+    String[] arr = new String[list.size()];
+    list.toArray(arr);
+    return arr;
+  }
+
+  /**
+   * Delete backup set
+   * @param name set's name
+   * @throws IOException
+   */
+  public void deleteBackupSet(String name) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(" Backup set delete: " + name);
+    }
+    Table table = null;
+    try {
+      table = connection.getTable(tableName);
+      Delete del = BackupSystemTableHelper.createDeleteForBackupSet(name);
+      table.delete(del);
+    } finally {
+      if (table != null) {
+        table.close();
+      }
+    }
+  }
+
+  /**
+   * Get backup system table descriptor
+   * @return descriptor
+   */
+  public static HTableDescriptor getSystemTableDescriptor() {
+    HTableDescriptor tableDesc = new HTableDescriptor(tableName);
+    HColumnDescriptor colSessionsDesc = new HColumnDescriptor(SESSIONS_FAMILY);
+    colSessionsDesc.setMaxVersions(1);
+    // Time to keep backup sessions (secs)
+    Configuration config = HBaseConfiguration.create();
+    int ttl = config.getInt(HConstants.BACKUP_SYSTEM_TTL_KEY, HConstants.BACKUP_SYSTEM_TTL_DEFAULT);
+    colSessionsDesc.setTimeToLive(ttl);
+    tableDesc.addFamily(colSessionsDesc);
+    HColumnDescriptor colMetaDesc = new HColumnDescriptor(META_FAMILY);
+    // colDesc.setMaxVersions(1);
+    tableDesc.addFamily(colMetaDesc);
+    return tableDesc;
+  }
+
+  public static String getTableNameAsString() {
+    return tableName.getNameAsString();
+  }
+
+  public static TableName getTableName() {
+    return tableName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java
new file mode 100644
index 0000000..37f29f8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java
@@ -0,0 +1,433 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * A collection for methods used by BackupSystemTable.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupSystemTableHelper {
+
+  /**
+   * hbase:backup schema: 
+   * 1. Backup sessions rowkey= "session:" + backupId; value = serialized BackupContext 
+   * 2. Backup start code rowkey = "startcode:" + backupRoot; value = startcode 
+   * 3. Incremental backup set rowkey="incrbackupset:" + backupRoot; value=[list of tables] 
+   * 4. Table-RS-timestamp map rowkey="trslm:"+ backupRoot+table_name; value = map[RS-> last WAL
+   * timestamp] 
+   * 5. RS - WAL ts map rowkey="rslogts:"+backupRoot +server; value = last WAL timestamp
+   * 6. WALs recorded rowkey="wals:"+WAL unique file name; value = backupId and full WAL file name
+   */
+
+  private final static String BACKUP_INFO_PREFIX = "session:";
+  private final static String START_CODE_ROW = "startcode:";
+  private final static String INCR_BACKUP_SET = "incrbackupset:";
+  private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm:";
+  private final static String RS_LOG_TS_PREFIX = "rslogts:";
+  private final static String WALS_PREFIX = "wals:";
+  private final static String SET_KEY_PREFIX = "backupset:";
+
+  private final static byte[] EMPTY_VALUE = new byte[] {};
+
+  // Safe delimiter in a string
+  private final static String NULL = "\u0000";
+
+  private BackupSystemTableHelper() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Creates Put operation for a given backup context object
+   * @param context backup context
+   * @return put operation
+   * @throws IOException exception
+   */
+  static Put createPutForBackupContext(BackupInfo context) throws IOException {
+    Put put = new Put(rowkey(BACKUP_INFO_PREFIX, context.getBackupId()));
+    put.addColumn(BackupSystemTable.SESSIONS_FAMILY, "context".getBytes(), context.toByteArray());
+    return put;
+  }
+
+  /**
+   * Creates Get operation for a given backup id
+   * @param backupId - backup's ID
+   * @return get operation
+   * @throws IOException exception
+   */
+  static Get createGetForBackupContext(String backupId) throws IOException {
+    Get get = new Get(rowkey(BACKUP_INFO_PREFIX, backupId));
+    get.addFamily(BackupSystemTable.SESSIONS_FAMILY);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Delete operation for a given backup id
+   * @param backupId - backup's ID
+   * @return delete operation
+   * @throws IOException exception
+   */
+  public static Delete createDeleteForBackupInfo(String backupId) {
+    Delete del = new Delete(rowkey(BACKUP_INFO_PREFIX, backupId));
+    del.addFamily(BackupSystemTable.SESSIONS_FAMILY);
+    return del;
+  }
+
+  /**
+   * Converts Result to BackupContext
+   * @param res - HBase result
+   * @return backup context instance
+   * @throws IOException exception
+   */
+  static BackupInfo resultToBackupInfo(Result res) throws IOException {
+    res.advance();
+    Cell cell = res.current();
+    return cellToBackupInfo(cell);
+  }
+
+  /**
+   * Creates Get operation to retrieve start code from hbase:backup
+   * @return get operation
+   * @throws IOException exception
+   */
+  static Get createGetForStartCode(String rootPath) throws IOException {
+    Get get = new Get(rowkey(START_CODE_ROW, rootPath));
+    get.addFamily(BackupSystemTable.META_FAMILY);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Put operation to store start code to hbase:backup
+   * @return put operation
+   * @throws IOException exception
+   */
+  static Put createPutForStartCode(String startCode, String rootPath) {
+    Put put = new Put(rowkey(START_CODE_ROW, rootPath));
+    put.addColumn(BackupSystemTable.META_FAMILY, "startcode".getBytes(), startCode.getBytes());
+    return put;
+  }
+
+  /**
+   * Creates Get to retrieve incremental backup table set from hbase:backup
+   * @return get operation
+   * @throws IOException exception
+   */
+  static Get createGetForIncrBackupTableSet(String backupRoot) throws IOException {
+    Get get = new Get(rowkey(INCR_BACKUP_SET, backupRoot));
+    get.addFamily(BackupSystemTable.META_FAMILY);
+    get.setMaxVersions(1);
+    return get;
+  }
+
+  /**
+   * Creates Put to store incremental backup table set
+   * @param tables tables
+   * @return put operation
+   */
+  static Put createPutForIncrBackupTableSet(Set<TableName> tables, String backupRoot) {
+    Put put = new Put(rowkey(INCR_BACKUP_SET, backupRoot));
+    for (TableName table : tables) {
+      put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes(table.getNameAsString()),
+        EMPTY_VALUE);
+    }
+    return put;
+  }
+
+  /**
+   * Creates Delete for incremental backup table set
+   * @param backupRoot backup root
+   * @return delete operation
+   */
+  static Delete createDeleteForIncrBackupTableSet(String backupRoot) {
+    Delete delete = new Delete(rowkey(INCR_BACKUP_SET, backupRoot));
+    delete.addFamily(BackupSystemTable.META_FAMILY);
+    return delete;
+  }
+
+  /**
+   * Creates Scan operation to load backup history
+   * @return scan operation
+   */
+  static Scan createScanForBackupHistory() {
+    Scan scan = new Scan();
+    byte[] startRow = BACKUP_INFO_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.SESSIONS_FAMILY);
+    scan.setMaxVersions(1);
+    return scan;
+  }
+
+  /**
+   * Converts cell to backup context instance.
+   * @param current - cell
+   * @return backup context instance
+   * @throws IOException exception
+   */
+  static BackupInfo cellToBackupInfo(Cell current) throws IOException {
+    byte[] data = CellUtil.cloneValue(current);
+    return BackupInfo.fromByteArray(data);
+  }
+
+  /**
+   * Creates Put to write RS last roll log timestamp map
+   * @param table - table
+   * @param smap - map, containing RS:ts
+   * @return put operation
+   */
+  static Put createPutForWriteRegionServerLogTimestamp(TableName table, byte[] smap,
+      String backupRoot) {
+    Put put = new Put(rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot, NULL, table.getNameAsString()));
+    put.addColumn(BackupSystemTable.META_FAMILY, "log-roll-map".getBytes(), smap);
+    return put;
+  }
+
+  /**
+   * Creates Scan to load table-> { RS -> ts} map of maps
+   * @return scan operation
+   */
+  static Scan createScanForReadLogTimestampMap(String backupRoot) {
+    Scan scan = new Scan();
+    byte[] startRow = rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot);
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.META_FAMILY);
+
+    return scan;
+  }
+
+  /**
+   * Get table name from rowkey
+   * @param cloneRow rowkey
+   * @return table name
+   */
+  static String getTableNameForReadLogTimestampMap(byte[] cloneRow) {
+    String s = new String(cloneRow);
+    int index = s.lastIndexOf(NULL);
+    return s.substring(index + 1);
+  }
+
+  /**
+   * Creates Put to store RS last log result
+   * @param server - server name
+   * @param timestamp - log roll result (timestamp)
+   * @return put operation
+   */
+  static Put createPutForRegionServerLastLogRollResult(String server, Long timestamp,
+      String backupRoot) {
+    Put put = new Put(rowkey(RS_LOG_TS_PREFIX, backupRoot, NULL, server));
+    put.addColumn(BackupSystemTable.META_FAMILY, "rs-log-ts".getBytes(), timestamp.toString()
+        .getBytes());
+    return put;
+  }
+
+  /**
+   * Creates Scan operation to load last RS log roll results
+   * @return scan operation
+   */
+  static Scan createScanForReadRegionServerLastLogRollResult(String backupRoot) {
+    Scan scan = new Scan();
+    byte[] startRow = rowkey(RS_LOG_TS_PREFIX, backupRoot);
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.META_FAMILY);
+    scan.setMaxVersions(1);
+
+    return scan;
+  }
+
+  /**
+   * Get server's name from rowkey
+   * @param row - rowkey
+   * @return server's name
+   */
+  static String getServerNameForReadRegionServerLastLogRollResult(byte[] row) {
+    String s = new String(row);
+    int index = s.lastIndexOf(NULL);
+    return s.substring(index + 1);
+  }
+
+  /**
+   * Creates put list for list of WAL files
+   * @param files list of WAL file paths
+   * @param backupId backup id
+   * @return put list
+   * @throws IOException exception
+   */
+  public static List<Put> createPutsForAddWALFiles(List<String> files, String backupId,
+      String backupRoot) throws IOException {
+
+    List<Put> puts = new ArrayList<Put>();
+    for (String file : files) {
+      Put put = new Put(rowkey(WALS_PREFIX, BackupClientUtil.getUniqueWALFileNamePart(file)));
+      put.addColumn(BackupSystemTable.META_FAMILY, "backupId".getBytes(), backupId.getBytes());
+      put.addColumn(BackupSystemTable.META_FAMILY, "file".getBytes(), file.getBytes());
+      put.addColumn(BackupSystemTable.META_FAMILY, "root".getBytes(), backupRoot.getBytes());
+      puts.add(put);
+    }
+    return puts;
+  }
+
+  /**
+   * Creates Scan operation to load WALs TODO: support for backupRoot
+   * @param backupRoot - path to backup destination
+   * @return scan operation
+   */
+  public static Scan createScanForGetWALs(String backupRoot) {
+    Scan scan = new Scan();
+    byte[] startRow = WALS_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.META_FAMILY);
+    return scan;
+  }
+
+  /**
+   * Creates Get operation for a given wal file name TODO: support for backup destination
+   * @param file file
+   * @return get operation
+   * @throws IOException exception
+   */
+  public static Get createGetForCheckWALFile(String file) throws IOException {
+    Get get = new Get(rowkey(WALS_PREFIX, BackupClientUtil.getUniqueWALFileNamePart(file)));
+    // add backup root column
+    get.addFamily(BackupSystemTable.META_FAMILY);
+    return get;
+  }
+
+  /**
+   * Creates Scan operation to load backup set list
+   * @return scan operation
+   */
+  static Scan createScanForBackupSetList() {
+    Scan scan = new Scan();
+    byte[] startRow = SET_KEY_PREFIX.getBytes();
+    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
+    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
+    scan.setStartRow(startRow);
+    scan.setStopRow(stopRow);
+    scan.addFamily(BackupSystemTable.META_FAMILY);
+    return scan;
+  }
+
+  /**
+   * Creates Get operation to load backup set content
+   * @return get operation
+   */
+  static Get createGetForBackupSet(String name) {
+    Get get = new Get(rowkey(SET_KEY_PREFIX, name));
+    get.addFamily(BackupSystemTable.META_FAMILY);
+    return get;
+  }
+
+  /**
+   * Creates Delete operation to delete backup set content
+   * @param name - backup set's name
+   * @return delete operation
+   */
+  static Delete createDeleteForBackupSet(String name) {
+    Delete del = new Delete(rowkey(SET_KEY_PREFIX, name));
+    del.addFamily(BackupSystemTable.META_FAMILY);
+    return del;
+  }
+
+  /**
+   * Creates Put operation to update backup set content
+   * @param name - backup set's name
+   * @param tables - list of tables
+   * @return put operation
+   */
+  static Put createPutForBackupSet(String name, String[] tables) {
+    Put put = new Put(rowkey(SET_KEY_PREFIX, name));
+    byte[] value = convertToByteArray(tables);
+    put.addColumn(BackupSystemTable.META_FAMILY, "tables".getBytes(), value);
+    return put;
+  }
+
+  private static byte[] convertToByteArray(String[] tables) {
+    return StringUtils.join(tables, ",").getBytes();
+  }
+
+  /**
+   * Converts cell to backup set list.
+   * @param current - cell
+   * @return backup set
+   * @throws IOException
+   */
+  static String[] cellValueToBackupSet(Cell current) throws IOException {
+    byte[] data = CellUtil.cloneValue(current);
+    if (data != null && data.length > 0) {
+      return new String(data).split(",");
+    } else {
+      return new String[0];
+    }
+  }
+
+  /**
+   * Converts cell key to backup set name.
+   * @param current - cell
+   * @return backup set name
+   * @throws IOException
+   */
+  static String cellKeyToBackupSetName(Cell current) throws IOException {
+    byte[] data = CellUtil.cloneRow(current);
+    return new String(data).substring(SET_KEY_PREFIX.length());
+  }
+
+  static byte[] rowkey(String s, String... other) {
+    StringBuilder sb = new StringBuilder(s);
+    for (String ss : other) {
+      sb.append(ss);
+    }
+    return sb.toString().getBytes();
+  }
+
+}


[08/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseBackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseBackupAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseBackupAdmin.java
deleted file mode 100644
index dfa2fb1..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseBackupAdmin.java
+++ /dev/null
@@ -1,439 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.client;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Future;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
-import org.apache.hadoop.hbase.backup.BackupRequest;
-import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.RestoreRequest;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.backup.util.BackupSet;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * The administrative API implementation for HBase Backup . Obtain an instance from 
- * an {@link Admin#getBackupAdmin()} and call {@link #close()} afterwards.
- * <p>BackupAdmin can be used to create backups, restore data from backups and for 
- * other backup-related operations. 
- *
- * @see Admin
- * @since 2.0
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-
-public class HBaseBackupAdmin implements BackupAdmin {
-  private static final Log LOG = LogFactory.getLog(HBaseBackupAdmin.class);
-
-  private final HBaseAdmin admin;
-  private final Connection conn;
-
-  HBaseBackupAdmin(HBaseAdmin admin) {
-    this.admin = admin;
-    this.conn = admin.getConnection();
-  }
-
-  @Override
-  public void close() throws IOException {
-  }
-
-  @Override
-  public BackupInfo getBackupInfo(String backupId) throws IOException {
-    BackupInfo backupInfo = null;
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      backupInfo = table.readBackupInfo(backupId);
-      return backupInfo;
-    }
-  }
-
-  @Override
-  public int getProgress(String backupId) throws IOException {
-    BackupInfo backupInfo = null;
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      if (backupId == null) {
-        ArrayList<BackupInfo> recentSessions = table.getBackupContexts(BackupState.RUNNING);
-        if (recentSessions.isEmpty()) {
-          LOG.warn("No ongoing sessions found.");
-          return -1;
-        }
-        // else show status for ongoing session
-        // must be one maximum
-        return recentSessions.get(0).getProgress();
-      } else {
-
-        backupInfo = table.readBackupInfo(backupId);
-        if (backupInfo != null) {
-          return backupInfo.getProgress();
-        } else {
-          LOG.warn("No information found for backupID=" + backupId);
-          return -1;
-        }
-      }
-    }
-  }
-
-  @Override
-  public int deleteBackups(String[] backupIds) throws IOException {
-    // TODO: requires FT, failure will leave system
-    // in non-consistent state
-    // see HBASE-15227
-
-    int totalDeleted = 0;
-    Map<String, HashSet<TableName>> allTablesMap = new HashMap<String, HashSet<TableName>>();
-
-    try (final BackupSystemTable sysTable = new BackupSystemTable(conn)) {
-      for (int i = 0; i < backupIds.length; i++) {
-        BackupInfo info = sysTable.readBackupInfo(backupIds[i]);
-        if (info != null) {
-          String rootDir = info.getTargetRootDir();
-          HashSet<TableName> allTables = allTablesMap.get(rootDir);
-          if (allTables == null) {
-            allTables = new HashSet<TableName>();
-            allTablesMap.put(rootDir, allTables);
-          }
-          allTables.addAll(info.getTableNames());
-          totalDeleted += deleteBackup(backupIds[i], sysTable);
-        }
-      }
-      finalizeDelete(allTablesMap, sysTable);
-    }
-    return totalDeleted;
-  }
-
-  /**
-   * Updates incremental backup set for every backupRoot
-   * @param tablesMap - Map [backupRoot: Set<TableName>]
-   * @param table - backup system table
-   * @throws IOException
-   */
-
-  private void finalizeDelete(Map<String, HashSet<TableName>> tablesMap, BackupSystemTable table)
-      throws IOException {
-    for (String backupRoot : tablesMap.keySet()) {
-      Set<TableName> incrTableSet = table.getIncrementalBackupTableSet(backupRoot);
-      Map<TableName, ArrayList<BackupInfo>> tableMap = 
-          table.getBackupHistoryForTableSet(incrTableSet, backupRoot);      
-      for(Map.Entry<TableName, ArrayList<BackupInfo>> entry: tableMap.entrySet()) {
-        if(entry.getValue() == null) {
-          // No more backups for a table
-          incrTableSet.remove(entry.getKey());
-        }
-      }      
-      if (!incrTableSet.isEmpty()) {
-        table.addIncrementalBackupTableSet(incrTableSet, backupRoot);
-      } else { // empty
-        table.deleteIncrementalBackupTableSet(backupRoot);
-      }
-    }
-  }
-  
-  /**
-   * Delete single backup and all related backups
-   * Algorithm:
-   * 
-   * Backup type: FULL or INCREMENTAL
-   * Is this last backup session for table T: YES or NO
-   * For every table T from table list 'tables':
-   * if(FULL, YES) deletes only physical data (PD)
-   * if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo, until
-   * we either reach the most recent backup for T in the system or FULL backup which
-   * includes T
-   * if(INCREMENTAL, YES) deletes only physical data (PD)
-   * if(INCREMENTAL, NO) deletes physical data and for table T scans all backup images
-   * between last FULL backup, which is older than the backup being deleted and the next
-   * FULL backup (if exists) or last one for a particular table T and removes T from list
-   * of backup tables.
-   * @param backupId - backup id
-   * @param sysTable - backup system table
-   * @return total - number of deleted backup images
-   * @throws IOException
-   */
-  private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException {
-
-    BackupInfo backupInfo = sysTable.readBackupInfo(backupId);
-
-    int totalDeleted = 0;
-    if (backupInfo != null) {
-      LOG.info("Deleting backup " + backupInfo.getBackupId() + " ...");
-      BackupClientUtil.cleanupBackupData(backupInfo, admin.getConfiguration());
-      // List of tables in this backup;
-      List<TableName> tables = backupInfo.getTableNames();
-      long startTime = backupInfo.getStartTs();
-      for (TableName tn : tables) {
-        boolean isLastBackupSession = isLastBackupSession(sysTable, tn, startTime);
-        if (isLastBackupSession) {
-          continue;
-        }
-        // else
-        List<BackupInfo> affectedBackups = getAffectedBackupInfos(backupInfo, tn, sysTable);
-        for (BackupInfo info : affectedBackups) {
-          if (info.equals(backupInfo)) {
-            continue;
-          }
-          removeTableFromBackupImage(info, tn, sysTable);
-        }
-      }
-      LOG.debug("Delete backup info "+ backupInfo.getBackupId());  
-
-      sysTable.deleteBackupInfo(backupInfo.getBackupId());
-      LOG.info("Delete backup " + backupInfo.getBackupId() + " completed.");
-      totalDeleted++;
-    } else {
-      LOG.warn("Delete backup failed: no information found for backupID=" + backupId);
-    }
-    return totalDeleted;
-  }
-
-  private void removeTableFromBackupImage(BackupInfo info, TableName tn, BackupSystemTable sysTable)
-      throws IOException {
-    List<TableName> tables = info.getTableNames();
-    LOG.debug("Remove "+ tn +" from " + info.getBackupId() + " tables=" + 
-      info.getTableListAsString());
-    if (tables.contains(tn)) {
-      tables.remove(tn);
-
-      if (tables.isEmpty()) {
-        LOG.debug("Delete backup info "+ info.getBackupId());  
-
-        sysTable.deleteBackupInfo(info.getBackupId());
-        BackupClientUtil.cleanupBackupData(info, conn.getConfiguration());
-      } else {
-        info.setTables(tables);
-        sysTable.updateBackupInfo(info);
-        // Now, clean up directory for table
-        cleanupBackupDir(info, tn, conn.getConfiguration());
-      }
-    }
-  }
-
-  private List<BackupInfo> getAffectedBackupInfos(BackupInfo backupInfo, TableName tn,
-      BackupSystemTable table) throws IOException {
-    LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn);
-    long ts = backupInfo.getStartTs();
-    List<BackupInfo> list = new ArrayList<BackupInfo>();
-    List<BackupInfo> history = table.getBackupHistory(backupInfo.getTargetRootDir());
-    // Scan from most recent to backupInfo
-    // break when backupInfo reached
-    for (BackupInfo info : history) {
-      if (info.getStartTs() == ts) {
-        break;
-      }
-      List<TableName> tables = info.getTableNames();
-      if (tables.contains(tn)) {
-        BackupType bt = info.getType();
-        if (bt == BackupType.FULL) {
-          // Clear list if we encounter FULL backup
-          list.clear();
-        } else {
-          LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn
-              + " added " + info.getBackupId() + " tables=" + info.getTableListAsString());
-          list.add(info);
-        }
-      }
-    }
-    return list;
-  }
-
-  
-  
-  /**
-   * Clean up the data at target directory
-   * @throws IOException 
-   */
-  private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf) 
-      throws IOException {
-    try {
-      // clean up the data at target directory
-      String targetDir = backupInfo.getTargetRootDir();
-      if (targetDir == null) {
-        LOG.warn("No target directory specified for " + backupInfo.getBackupId());
-        return;
-      }
-
-      FileSystem outputFs = FileSystem.get(new Path(backupInfo.getTargetRootDir()).toUri(), conf);
-
-      Path targetDirPath =
-          new Path(BackupClientUtil.getTableBackupDir(backupInfo.getTargetRootDir(),
-            backupInfo.getBackupId(), table));
-      if (outputFs.delete(targetDirPath, true)) {
-        LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done.");
-      } else {
-        LOG.info("No data has been found in " + targetDirPath.toString() + ".");
-      }
-
-    } catch (IOException e1) {
-      LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " for table " + table
-          + "at " + backupInfo.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
-      throw e1;
-    }
-  }
-
-  private boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime)
-      throws IOException {
-    List<BackupInfo> history = table.getBackupHistory();
-    for (BackupInfo info : history) {
-      List<TableName> tables = info.getTableNames();
-      if (!tables.contains(tn)) {
-        continue;
-      }
-      if (info.getStartTs() <= startTime) {
-        return true;
-      } else {
-        return false;
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public List<BackupInfo> getHistory(int n) throws IOException {
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      List<BackupInfo> history = table.getBackupHistory();
-      if (history.size() <= n) return history;
-      List<BackupInfo> list = new ArrayList<BackupInfo>();
-      for (int i = 0; i < n; i++) {
-        list.add(history.get(i));
-      }
-      return list;
-    }
-  }
-
-  @Override
-  public List<BackupInfo> getHistory(int n, BackupInfo.Filter ... filters) throws IOException {
-    if (filters.length == 0) return getHistory(n);
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      List<BackupInfo> history = table.getBackupHistory();
-      List<BackupInfo> result = new ArrayList<BackupInfo>();
-      for(BackupInfo bi: history) {
-        if(result.size() == n) break;
-        boolean passed = true;
-        for(int i=0; i < filters.length; i++) {
-          if(!filters[i].apply(bi)) {
-            passed = false; 
-            break;
-          }
-        }        
-        if(passed) {
-          result.add(bi);
-        }
-      }
-      return result;
-    }
-  }
-
-  @Override
-  public List<BackupSet> listBackupSets() throws IOException {
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      List<String> list = table.listBackupSets();
-      List<BackupSet> bslist = new ArrayList<BackupSet>();
-      for (String s : list) {
-        List<TableName> tables = table.describeBackupSet(s);
-        if (tables != null) {
-          bslist.add(new BackupSet(s, tables));
-        }
-      }
-      return bslist;
-    }
-  }
-
-  @Override
-  public BackupSet getBackupSet(String name) throws IOException {
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      List<TableName> list = table.describeBackupSet(name);
-      if (list == null) return null;
-      return new BackupSet(name, list);
-    }
-  }
-
-  @Override
-  public boolean deleteBackupSet(String name) throws IOException {
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      if (table.describeBackupSet(name) == null) {
-        return false;
-      }
-      table.deleteBackupSet(name);
-      return true;
-    }
-  }
-
-  @Override
-  public void addToBackupSet(String name, TableName[] tables) throws IOException {
-    String[] tableNames = new String[tables.length];
-    for (int i = 0; i < tables.length; i++) {
-      tableNames[i] = tables[i].getNameAsString();
-      if (!admin.tableExists(TableName.valueOf(tableNames[i]))) {
-        throw new IOException("Cannot add " + tableNames[i] + " because it doesn't exist");
-      }
-    }
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      table.addToBackupSet(name, tableNames);
-      LOG.info("Added tables [" + StringUtils.join(tableNames, " ") + "] to '" + name
-          + "' backup set");
-    }
-  }
-
-  @Override
-  public void removeFromBackupSet(String name, String[] tables) throws IOException {
-    LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + "'");
-    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
-      table.removeFromBackupSet(name, tables);
-      LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name
-          + "' completed.");
-    }
-  }
-
-  @Override
-  public void restore(RestoreRequest request) throws IOException {
-    admin.restoreTables(request);
-  }
-
-  @Override
-  public Future<Void> restoreAsync(RestoreRequest request) throws IOException {
-    return admin.restoreTablesAsync(request);
-  }
-
-  @Override
-  public String backupTables(final BackupRequest userRequest) throws IOException {
-    return admin.backupTables(userRequest);
-  }
-
-  @Override
-  public Future<String> backupTablesAsync(final BackupRequest userRequest) throws IOException {
-    return admin.backupTablesAsync(userRequest);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index 0d1700e..1f14191 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.RegionUpdateInfo;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath;
@@ -105,7 +104,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableReques
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreTablesRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
@@ -1271,48 +1269,6 @@ public final class RequestConverter {
     return builder.build();
   }
 
-  public static BackupTablesRequest buildBackupTablesRequest(
-      final BackupType type, List<TableName> tableList, String targetRootDir, final int workers,
-      final long bandwidth, String setName, final long nonceGroup, final long nonce) {
-    BackupTablesRequest.Builder builder = BackupTablesRequest.newBuilder();
-    builder.setType(ProtobufUtil.toProtoBackupType(type));
-    builder.setTargetRootDir(targetRootDir);
-    builder.setWorkers(workers);
-    builder.setBandwidth(bandwidth);
-    if(setName != null) {
-      builder.setBackupSetName(setName);
-    }
-    if (tableList != null) {
-      for (TableName table : tableList) {
-        builder.addTables(ProtobufUtil.toProtoTableName(table));
-      }
-    }
-    builder.setNonceGroup(nonceGroup).setNonce(nonce);
-    return builder.build();
-  }
-
-  public static RestoreTablesRequest buildRestoreTablesRequest(String backupRootDir,
-      String backupId, boolean check, TableName[] sTableList,
-      TableName[] tTableList, boolean isOverwrite, final long nonceGroup, final long nonce)
-          throws IOException {
-    RestoreTablesRequest.Builder builder = RestoreTablesRequest.newBuilder();
-    builder.setBackupId(backupId).setBackupRootDir(backupRootDir);
-    builder.setDependencyCheckOnly(check).setOverwrite(isOverwrite);
-    if (sTableList != null) {
-      for (TableName table : sTableList) {
-        builder.addTables(ProtobufUtil.toProtoTableName(table));
-      }
-    } else {
-      throw new IOException("Source table list shouldn't be empty");
-    }
-    if (tTableList != null) {
-      for (TableName table : tTableList) {
-        builder.addTargetTables(ProtobufUtil.toProtoTableName(table));
-      }
-    }
-    builder.setNonceGroup(nonceGroup).setNonce(nonce);
-    return builder.build();
-  }
 
   /**
    * Creates a protocol buffer GetSchemaAlterStatusRequest

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
index 4bcfa4e..bbb878f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
@@ -20,7 +20,6 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -45,7 +44,7 @@ public class ClientSnapshotDescriptionUtils {
       // make sure the table name is valid, this will implicitly check validity
       TableName tableName = TableName.valueOf(snapshot.getTable());
 
-      if (tableName.isSystemTable() && !BackupSystemTable.getTableName().equals(tableName)) {
+      if (tableName.isSystemTable() && !tableName.toString().equals("hbase:backup")) {
         // allow hbase:backup table snapshot
         throw new IllegalArgumentException("System table snapshots are not allowed");
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
index 7e947aa..95363e5 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
@@ -32,12 +32,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.backup.BackupInfo;
 import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupAdmin;
 import org.apache.hadoop.hbase.backup.BackupRequest;
 import org.apache.hadoop.hbase.backup.BackupType;
 import org.apache.hadoop.hbase.backup.RestoreRequest;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.BackupAdmin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
@@ -160,9 +161,11 @@ public class IntegrationTestBackupRestore extends IntegrationTestBase {
     List<TableName> tables = Lists.newArrayList(TABLE_NAME1, TABLE_NAME2);
     HBaseAdmin admin = null;
     admin = (HBaseAdmin) conn.getAdmin();
+    BackupAdmin client = new HBaseBackupAdmin(util.getConnection());
+
     BackupRequest request = new BackupRequest();
     request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdFull = admin.getBackupAdmin().backupTables(request);
+    String backupIdFull = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdFull));
     // #2 - insert some data to table
     loadData(TABLE_NAME1, rowsInBatch);
@@ -178,11 +181,10 @@ public class IntegrationTestBackupRestore extends IntegrationTestBase {
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
         .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdIncMultiple = admin.getBackupAdmin().backupTables(request);
+    String backupIdIncMultiple = client.backupTables(request);
     assertTrue(checkSucceeded(backupIdIncMultiple));
     // #4 - restore full backup for all tables, without overwrite
     TableName[] tablesRestoreFull = new TableName[] { TABLE_NAME1, TABLE_NAME2 };
-    BackupAdmin client = util.getAdmin().getBackupAdmin();
     client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false, tablesRestoreFull,
       null, true));
     // #5.1 - check tables for full restore


[06/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index 4562a7c..d495e84 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -90,88 +90,6 @@ public final class MasterProtos {
     // @@protoc_insertion_point(enum_scope:hbase.pb.MasterSwitchType)
   }
 
-  /**
-   * Protobuf enum {@code hbase.pb.RestoreTablesState}
-   */
-  public enum RestoreTablesState
-      implements com.google.protobuf.ProtocolMessageEnum {
-    /**
-     * <code>VALIDATION = 1;</code>
-     */
-    VALIDATION(0, 1),
-    /**
-     * <code>RESTORE_IMAGES = 2;</code>
-     */
-    RESTORE_IMAGES(1, 2),
-    ;
-
-    /**
-     * <code>VALIDATION = 1;</code>
-     */
-    public static final int VALIDATION_VALUE = 1;
-    /**
-     * <code>RESTORE_IMAGES = 2;</code>
-     */
-    public static final int RESTORE_IMAGES_VALUE = 2;
-
-
-    public final int getNumber() { return value; }
-
-    public static RestoreTablesState valueOf(int value) {
-      switch (value) {
-        case 1: return VALIDATION;
-        case 2: return RESTORE_IMAGES;
-        default: return null;
-      }
-    }
-
-    public static com.google.protobuf.Internal.EnumLiteMap<RestoreTablesState>
-        internalGetValueMap() {
-      return internalValueMap;
-    }
-    private static com.google.protobuf.Internal.EnumLiteMap<RestoreTablesState>
-        internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap<RestoreTablesState>() {
-            public RestoreTablesState findValueByNumber(int number) {
-              return RestoreTablesState.valueOf(number);
-            }
-          };
-
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
-        getValueDescriptor() {
-      return getDescriptor().getValues().get(index);
-    }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptorForType() {
-      return getDescriptor();
-    }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.getDescriptor().getEnumTypes().get(1);
-    }
-
-    private static final RestoreTablesState[] VALUES = values();
-
-    public static RestoreTablesState valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-      if (desc.getType() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "EnumValueDescriptor is not for this type.");
-      }
-      return VALUES[desc.getIndex()];
-    }
-
-    private final int index;
-    private final int value;
-
-    private RestoreTablesState(int index, int value) {
-      this.index = index;
-      this.value = value;
-    }
-
-    // @@protoc_insertion_point(enum_scope:hbase.pb.RestoreTablesState)
-  }
-
   public interface AddColumnRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -58736,4572 +58654,36 @@ public final class MasterProtos {
               break;
             }
             case 8: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability value = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(1, rawValue);
-              } else {
-                if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                  capabilities_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability>();
-                  mutable_bitField0_ |= 0x00000001;
-                }
-                capabilities_.add(value);
-              }
-              break;
-            }
-            case 10: {
-              int length = input.readRawVarint32();
-              int oldLimit = input.pushLimit(length);
-              while(input.getBytesUntilLimit() > 0) {
-                int rawValue = input.readEnum();
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability value = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability.valueOf(rawValue);
-                if (value == null) {
-                  unknownFields.mergeVarintField(1, rawValue);
-                } else {
-                  if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                    capabilities_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability>();
-                    mutable_bitField0_ |= 0x00000001;
-                  }
-                  capabilities_.add(value);
-                }
-              }
-              input.popLimit(oldLimit);
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-          capabilities_ = java.util.Collections.unmodifiableList(capabilities_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<SecurityCapabilitiesResponse> PARSER =
-        new com.google.protobuf.AbstractParser<SecurityCapabilitiesResponse>() {
-      public SecurityCapabilitiesResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new SecurityCapabilitiesResponse(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<SecurityCapabilitiesResponse> getParserForType() {
-      return PARSER;
-    }
-
-    /**
-     * Protobuf enum {@code hbase.pb.SecurityCapabilitiesResponse.Capability}
-     */
-    public enum Capability
-        implements com.google.protobuf.ProtocolMessageEnum {
-      /**
-       * <code>SIMPLE_AUTHENTICATION = 0;</code>
-       */
-      SIMPLE_AUTHENTICATION(0, 0),
-      /**
-       * <code>SECURE_AUTHENTICATION = 1;</code>
-       */
-      SECURE_AUTHENTICATION(1, 1),
-      /**
-       * <code>AUTHORIZATION = 2;</code>
-       */
-      AUTHORIZATION(2, 2),
-      /**
-       * <code>CELL_AUTHORIZATION = 3;</code>
-       */
-      CELL_AUTHORIZATION(3, 3),
-      /**
-       * <code>CELL_VISIBILITY = 4;</code>
-       */
-      CELL_VISIBILITY(4, 4),
-      ;
-
-      /**
-       * <code>SIMPLE_AUTHENTICATION = 0;</code>
-       */
-      public static final int SIMPLE_AUTHENTICATION_VALUE = 0;
-      /**
-       * <code>SECURE_AUTHENTICATION = 1;</code>
-       */
-      public static final int SECURE_AUTHENTICATION_VALUE = 1;
-      /**
-       * <code>AUTHORIZATION = 2;</code>
-       */
-      public static final int AUTHORIZATION_VALUE = 2;
-      /**
-       * <code>CELL_AUTHORIZATION = 3;</code>
-       */
-      public static final int CELL_AUTHORIZATION_VALUE = 3;
-      /**
-       * <code>CELL_VISIBILITY = 4;</code>
-       */
-      public static final int CELL_VISIBILITY_VALUE = 4;
-
-
-      public final int getNumber() { return value; }
-
-      public static Capability valueOf(int value) {
-        switch (value) {
-          case 0: return SIMPLE_AUTHENTICATION;
-          case 1: return SECURE_AUTHENTICATION;
-          case 2: return AUTHORIZATION;
-          case 3: return CELL_AUTHORIZATION;
-          case 4: return CELL_VISIBILITY;
-          default: return null;
-        }
-      }
-
-      public static com.google.protobuf.Internal.EnumLiteMap<Capability>
-          internalGetValueMap() {
-        return internalValueMap;
-      }
-      private static com.google.protobuf.Internal.EnumLiteMap<Capability>
-          internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap<Capability>() {
-              public Capability findValueByNumber(int number) {
-                return Capability.valueOf(number);
-              }
-            };
-
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
-          getValueDescriptor() {
-        return getDescriptor().getValues().get(index);
-      }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptorForType() {
-        return getDescriptor();
-      }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDescriptor().getEnumTypes().get(0);
-      }
-
-      private static final Capability[] VALUES = values();
-
-      public static Capability valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-        if (desc.getType() != getDescriptor()) {
-          throw new java.lang.IllegalArgumentException(
-            "EnumValueDescriptor is not for this type.");
-        }
-        return VALUES[desc.getIndex()];
-      }
-
-      private final int index;
-      private final int value;
-
-      private Capability(int index, int value) {
-        this.index = index;
-        this.value = value;
-      }
-
-      // @@protoc_insertion_point(enum_scope:hbase.pb.SecurityCapabilitiesResponse.Capability)
-    }
-
-    // repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;
-    public static final int CAPABILITIES_FIELD_NUMBER = 1;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability> capabilities_;
-    /**
-     * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability> getCapabilitiesList() {
-      return capabilities_;
-    }
-    /**
-     * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-     */
-    public int getCapabilitiesCount() {
-      return capabilities_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability getCapabilities(int index) {
-      return capabilities_.get(index);
-    }
-
-    private void initFields() {
-      capabilities_ = java.util.Collections.emptyList();
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      for (int i = 0; i < capabilities_.size(); i++) {
-        output.writeEnum(1, capabilities_.get(i).getNumber());
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      {
-        int dataSize = 0;
-        for (int i = 0; i < capabilities_.size(); i++) {
-          dataSize += com.google.protobuf.CodedOutputStream
-            .computeEnumSizeNoTag(capabilities_.get(i).getNumber());
-        }
-        size += dataSize;
-        size += 1 * capabilities_.size();
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse) obj;
-
-      boolean result = true;
-      result = result && getCapabilitiesList()
-          .equals(other.getCapabilitiesList());
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (getCapabilitiesCount() > 0) {
-        hash = (37 * hash) + CAPABILITIES_FIELD_NUMBER;
-        hash = (53 * hash) + hashEnumList(getCapabilitiesList());
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code hbase.pb.SecurityCapabilitiesResponse}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        capabilities_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_SecurityCapabilitiesResponse_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse(this);
-        int from_bitField0_ = bitField0_;
-        if (((bitField0_ & 0x00000001) == 0x00000001)) {
-          capabilities_ = java.util.Collections.unmodifiableList(capabilities_);
-          bitField0_ = (bitField0_ & ~0x00000001);
-        }
-        result.capabilities_ = capabilities_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.getDefaultInstance()) return this;
-        if (!other.capabilities_.isEmpty()) {
-          if (capabilities_.isEmpty()) {
-            capabilities_ = other.capabilities_;
-            bitField0_ = (bitField0_ & ~0x00000001);
-          } else {
-            ensureCapabilitiesIsMutable();
-            capabilities_.addAll(other.capabilities_);
-          }
-          onChanged();
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;
-      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability> capabilities_ =
-        java.util.Collections.emptyList();
-      private void ensureCapabilitiesIsMutable() {
-        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          capabilities_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability>(capabilities_);
-          bitField0_ |= 0x00000001;
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability> getCapabilitiesList() {
-        return java.util.Collections.unmodifiableList(capabilities_);
-      }
-      /**
-       * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-       */
-      public int getCapabilitiesCount() {
-        return capabilities_.size();
-      }
-      /**
-       * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability getCapabilities(int index) {
-        return capabilities_.get(index);
-      }
-      /**
-       * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-       */
-      public Builder setCapabilities(
-          int index, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability value) {
-        if (value == null) {
-          throw new NullPointerException();
-        }
-        ensureCapabilitiesIsMutable();
-        capabilities_.set(index, value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-       */
-      public Builder addCapabilities(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability value) {
-        if (value == null) {
-          throw new NullPointerException();
-        }
-        ensureCapabilitiesIsMutable();
-        capabilities_.add(value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-       */
-      public Builder addAllCapabilities(
-          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability> values) {
-        ensureCapabilitiesIsMutable();
-        super.addAll(values, capabilities_);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.SecurityCapabilitiesResponse.Capability capabilities = 1;</code>
-       */
-      public Builder clearCapabilities() {
-        capabilities_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000001);
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SecurityCapabilitiesResponse)
-    }
-
-    static {
-      defaultInstance = new SecurityCapabilitiesResponse(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:hbase.pb.SecurityCapabilitiesResponse)
-  }
-
-  public interface BackupTablesRequestOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required .hbase.pb.BackupType type = 1;
-    /**
-     * <code>required .hbase.pb.BackupType type = 1;</code>
-     */
-    boolean hasType();
-    /**
-     * <code>required .hbase.pb.BackupType type = 1;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getType();
-
-    // repeated .hbase.pb.TableName tables = 2;
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> 
-        getTablesList();
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTables(int index);
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    int getTablesCount();
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-        getTablesOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTablesOrBuilder(
-        int index);
-
-    // required string target_root_dir = 3;
-    /**
-     * <code>required string target_root_dir = 3;</code>
-     */
-    boolean hasTargetRootDir();
-    /**
-     * <code>required string target_root_dir = 3;</code>
-     */
-    java.lang.String getTargetRootDir();
-    /**
-     * <code>required string target_root_dir = 3;</code>
-     */
-    com.google.protobuf.ByteString
-        getTargetRootDirBytes();
-
-    // optional int64 workers = 4;
-    /**
-     * <code>optional int64 workers = 4;</code>
-     */
-    boolean hasWorkers();
-    /**
-     * <code>optional int64 workers = 4;</code>
-     */
-    long getWorkers();
-
-    // optional int64 bandwidth = 5;
-    /**
-     * <code>optional int64 bandwidth = 5;</code>
-     */
-    boolean hasBandwidth();
-    /**
-     * <code>optional int64 bandwidth = 5;</code>
-     */
-    long getBandwidth();
-
-    // optional string backup_set_name = 6;
-    /**
-     * <code>optional string backup_set_name = 6;</code>
-     */
-    boolean hasBackupSetName();
-    /**
-     * <code>optional string backup_set_name = 6;</code>
-     */
-    java.lang.String getBackupSetName();
-    /**
-     * <code>optional string backup_set_name = 6;</code>
-     */
-    com.google.protobuf.ByteString
-        getBackupSetNameBytes();
-
-    // optional uint64 nonce_group = 7 [default = 0];
-    /**
-     * <code>optional uint64 nonce_group = 7 [default = 0];</code>
-     */
-    boolean hasNonceGroup();
-    /**
-     * <code>optional uint64 nonce_group = 7 [default = 0];</code>
-     */
-    long getNonceGroup();
-
-    // optional uint64 nonce = 8 [default = 0];
-    /**
-     * <code>optional uint64 nonce = 8 [default = 0];</code>
-     */
-    boolean hasNonce();
-    /**
-     * <code>optional uint64 nonce = 8 [default = 0];</code>
-     */
-    long getNonce();
-  }
-  /**
-   * Protobuf type {@code hbase.pb.BackupTablesRequest}
-   */
-  public static final class BackupTablesRequest extends
-      com.google.protobuf.GeneratedMessage
-      implements BackupTablesRequestOrBuilder {
-    // Use BackupTablesRequest.newBuilder() to construct.
-    private BackupTablesRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private BackupTablesRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final BackupTablesRequest defaultInstance;
-    public static BackupTablesRequest getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public BackupTablesRequest getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private BackupTablesRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 8: {
-              int rawValue = input.readEnum();
-              org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType value = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(1, rawValue);
-              } else {
-                bitField0_ |= 0x00000001;
-                type_ = value;
-              }
-              break;
-            }
-            case 18: {
-              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-                tables_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>();
-                mutable_bitField0_ |= 0x00000002;
-              }
-              tables_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry));
-              break;
-            }
-            case 26: {
-              bitField0_ |= 0x00000002;
-              targetRootDir_ = input.readBytes();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000004;
-              workers_ = input.readInt64();
-              break;
-            }
-            case 40: {
-              bitField0_ |= 0x00000008;
-              bandwidth_ = input.readInt64();
-              break;
-            }
-            case 50: {
-              bitField0_ |= 0x00000010;
-              backupSetName_ = input.readBytes();
-              break;
-            }
-            case 56: {
-              bitField0_ |= 0x00000020;
-              nonceGroup_ = input.readUInt64();
-              break;
-            }
-            case 64: {
-              bitField0_ |= 0x00000040;
-              nonce_ = input.readUInt64();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-          tables_ = java.util.Collections.unmodifiableList(tables_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BackupTablesRequest_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BackupTablesRequest_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<BackupTablesRequest> PARSER =
-        new com.google.protobuf.AbstractParser<BackupTablesRequest>() {
-      public BackupTablesRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new BackupTablesRequest(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<BackupTablesRequest> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // required .hbase.pb.BackupType type = 1;
-    public static final int TYPE_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType type_;
-    /**
-     * <code>required .hbase.pb.BackupType type = 1;</code>
-     */
-    public boolean hasType() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required .hbase.pb.BackupType type = 1;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getType() {
-      return type_;
-    }
-
-    // repeated .hbase.pb.TableName tables = 2;
-    public static final int TABLES_FIELD_NUMBER = 2;
-    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tables_;
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTablesList() {
-      return tables_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-        getTablesOrBuilderList() {
-      return tables_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    public int getTablesCount() {
-      return tables_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTables(int index) {
-      return tables_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.TableName tables = 2;</code>
-     */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTablesOrBuilder(
-        int index) {
-      return tables_.get(index);
-    }
-
-    // required string target_root_dir = 3;
-    public static final int TARGET_ROOT_DIR_FIELD_NUMBER = 3;
-    private java.lang.Object targetRootDir_;
-    /**
-     * <code>required string target_root_dir = 3;</code>
-     */
-    public boolean hasTargetRootDir() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required string target_root_dir = 3;</code>
-     */
-    public java.lang.String getTargetRootDir() {
-      java.lang.Object ref = targetRootDir_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          targetRootDir_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>required string target_root_dir = 3;</code>
-     */
-    public com.google.protobuf.ByteString
-        getTargetRootDirBytes() {
-      java.lang.Object ref = targetRootDir_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        targetRootDir_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional int64 workers = 4;
-    public static final int WORKERS_FIELD_NUMBER = 4;
-    private long workers_;
-    /**
-     * <code>optional int64 workers = 4;</code>
-     */
-    public boolean hasWorkers() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional int64 workers = 4;</code>
-     */
-    public long getWorkers() {
-      return workers_;
-    }
-
-    // optional int64 bandwidth = 5;
-    public static final int BANDWIDTH_FIELD_NUMBER = 5;
-    private long bandwidth_;
-    /**
-     * <code>optional int64 bandwidth = 5;</code>
-     */
-    public boolean hasBandwidth() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>optional int64 bandwidth = 5;</code>
-     */
-    public long getBandwidth() {
-      return bandwidth_;
-    }
-
-    // optional string backup_set_name = 6;
-    public static final int BACKUP_SET_NAME_FIELD_NUMBER = 6;
-    private java.lang.Object backupSetName_;
-    /**
-     * <code>optional string backup_set_name = 6;</code>
-     */
-    public boolean hasBackupSetName() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>optional string backup_set_name = 6;</code>
-     */
-    public java.lang.String getBackupSetName() {
-      java.lang.Object ref = backupSetName_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          backupSetName_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string backup_set_name = 6;</code>
-     */
-    public com.google.protobuf.ByteString
-        getBackupSetNameBytes() {
-      java.lang.Object ref = backupSetName_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        backupSetName_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    // optional uint64 nonce_group = 7 [default = 0];
-    public static final int NONCE_GROUP_FIELD_NUMBER = 7;
-    private long nonceGroup_;
-    /**
-     * <code>optional uint64 nonce_group = 7 [default = 0];</code>
-     */
-    public boolean hasNonceGroup() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
-    }
-    /**
-     * <code>optional uint64 nonce_group = 7 [default = 0];</code>
-     */
-    public long getNonceGroup() {
-      return nonceGroup_;
-    }
-
-    // optional uint64 nonce = 8 [default = 0];
-    public static final int NONCE_FIELD_NUMBER = 8;
-    private long nonce_;
-    /**
-     * <code>optional uint64 nonce = 8 [default = 0];</code>
-     */
-    public boolean hasNonce() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
-    }
-    /**
-     * <code>optional uint64 nonce = 8 [default = 0];</code>
-     */
-    public long getNonce() {
-      return nonce_;
-    }
-
-    private void initFields() {
-      type_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
-      tables_ = java.util.Collections.emptyList();
-      targetRootDir_ = "";
-      workers_ = 0L;
-      bandwidth_ = 0L;
-      backupSetName_ = "";
-      nonceGroup_ = 0L;
-      nonce_ = 0L;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasType()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasTargetRootDir()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      for (int i = 0; i < getTablesCount(); i++) {
-        if (!getTables(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeEnum(1, type_.getNumber());
-      }
-      for (int i = 0; i < tables_.size(); i++) {
-        output.writeMessage(2, tables_.get(i));
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(3, getTargetRootDirBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt64(4, workers_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeInt64(5, bandwidth_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeBytes(6, getBackupSetNameBytes());
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeUInt64(7, nonceGroup_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeUInt64(8, nonce_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeEnumSize(1, type_.getNumber());
-      }
-      for (int i = 0; i < tables_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, tables_.get(i));
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(3, getTargetRootDirBytes());
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(4, workers_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(5, bandwidth_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(6, getBackupSetNameBytes());
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(7, nonceGroup_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(8, nonce_);
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest) obj;
-
-      boolean result = true;
-      result = result && (hasType() == other.hasType());
-      if (hasType()) {
-        result = result &&
-            (getType() == other.getType());
-      }
-      result = result && getTablesList()
-          .equals(other.getTablesList());
-      result = result && (hasTargetRootDir() == other.hasTargetRootDir());
-      if (hasTargetRootDir()) {
-        result = result && getTargetRootDir()
-            .equals(other.getTargetRootDir());
-      }
-      result = result && (hasWorkers() == other.hasWorkers());
-      if (hasWorkers()) {
-        result = result && (getWorkers()
-            == other.getWorkers());
-      }
-      result = result && (hasBandwidth() == other.hasBandwidth());
-      if (hasBandwidth()) {
-        result = result && (getBandwidth()
-            == other.getBandwidth());
-      }
-      result = result && (hasBackupSetName() == other.hasBackupSetName());
-      if (hasBackupSetName()) {
-        result = result && getBackupSetName()
-            .equals(other.getBackupSetName());
-      }
-      result = result && (hasNonceGroup() == other.hasNonceGroup());
-      if (hasNonceGroup()) {
-        result = result && (getNonceGroup()
-            == other.getNonceGroup());
-      }
-      result = result && (hasNonce() == other.hasNonce());
-      if (hasNonce()) {
-        result = result && (getNonce()
-            == other.getNonce());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasType()) {
-        hash = (37 * hash) + TYPE_FIELD_NUMBER;
-        hash = (53 * hash) + hashEnum(getType());
-      }
-      if (getTablesCount() > 0) {
-        hash = (37 * hash) + TABLES_FIELD_NUMBER;
-        hash = (53 * hash) + getTablesList().hashCode();
-      }
-      if (hasTargetRootDir()) {
-        hash = (37 * hash) + TARGET_ROOT_DIR_FIELD_NUMBER;
-        hash = (53 * hash) + getTargetRootDir().hashCode();
-      }
-      if (hasWorkers()) {
-        hash = (37 * hash) + WORKERS_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getWorkers());
-      }
-      if (hasBandwidth()) {
-        hash = (37 * hash) + BANDWIDTH_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getBandwidth());
-      }
-      if (hasBackupSetName()) {
-        hash = (37 * hash) + BACKUP_SET_NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getBackupSetName().hashCode();
-      }
-      if (hasNonceGroup()) {
-        hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getNonceGroup());
-      }
-      if (hasNonce()) {
-        hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getNonce());
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code hbase.pb.BackupTablesRequest}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BackupTablesRequest_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BackupTablesRequest_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getTablesFieldBuilder();
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        type_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        if (tablesBuilder_ == null) {
-          tables_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
-        } else {
-          tablesBuilder_.clear();
-        }
-        targetRootDir_ = "";
-        bitField0_ = (bitField0_ & ~0x00000004);
-        workers_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        bandwidth_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        backupSetName_ = "";
-        bitField0_ = (bitField0_ & ~0x00000020);
-        nonceGroup_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        nonce_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000080);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BackupTablesRequest_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.type_ = type_;
-        if (tablesBuilder_ == null) {
-          if (((bitField0_ & 0x00000002) == 0x00000002)) {
-            tables_ = java.util.Collections.unmodifiableList(tables_);
-            bitField0_ = (bitField0_ & ~0x00000002);
-          }
-          result.tables_ = tables_;
-        } else {
-          result.tables_ = tablesBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.targetRootDir_ = targetRootDir_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.workers_ = workers_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.bandwidth_ = bandwidth_;
-        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.backupSetName_ = backupSetName_;
-        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
-          to_bitField0_ |= 0x00000020;
-        }
-        result.nonceGroup_ = nonceGroup_;
-        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
-          to_bitField0_ |= 0x00000040;
-        }
-        result.nonce_ = nonce_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest.getDefaultInstance()) return this;
-        if (other.hasType()) {
-          setType(other.getType());
-        }
-        if (tablesBuilder_ == null) {
-          if (!other.tables_.isEmpty()) {
-            if (tables_.isEmpty()) {
-              tables_ = other.tables_;
-              bitField0_ = (bitField0_ & ~0x00000002);
-            } else {
-              ensureTablesIsMutable();
-              tables_.addAll(other.tables_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.tables_.isEmpty()) {
-            if (tablesBuilder_.isEmpty()) {
-              tablesBuilder_.dispose();
-              tablesBuilder_ = null;
-              tables_ = other.tables_;
-              bitField0_ = (bitField0_ & ~0x00000002);
-              tablesBuilder_ = 
-                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
-                   getTablesFieldBuilder() : null;
-            } else {
-              tablesBuilder_.addAllMessages(other.tables_);
-            }
-          }
-        }
-        if (other.hasTargetRootDir()) {
-          bitField0_ |= 0x00000004;
-          targetRootDir_ = other.targetRootDir_;
-          onChanged();
-        }
-        if (other.hasWorkers()) {
-          setWorkers(other.getWorkers());
-        }
-        if (other.hasBandwidth()) {
-          setBandwidth(other.getBandwidth());
-        }
-        if (other.hasBackupSetName()) {
-          bitField0_ |= 0x00000020;
-          backupSetName_ = other.backupSetName_;
-          onChanged();
-        }
-        if (other.hasNonceGroup()) {
-          setNonceGroup(other.getNonceGroup());
-        }
-        if (other.hasNonce()) {
-          setNonce(other.getNonce());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasType()) {
-          
-          return false;
-        }
-        if (!hasTargetRootDir()) {
-          
-          return false;
-        }
-        for (int i = 0; i < getTablesCount(); i++) {
-          if (!getTables(i).isInitialized()) {
-            
-            return false;
-          }
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // required .hbase.pb.BackupType type = 1;
-      private org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType type_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
-      /**
-       * <code>required .hbase.pb.BackupType type = 1;</code>
-       */
-      public boolean hasType() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required .hbase.pb.BackupType type = 1;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType getType() {
-        return type_;
-      }
-      /**
-       * <code>required .hbase.pb.BackupType type = 1;</code>
-       */
-      public Builder setType(org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType value) {
-        if (value == null) {
-          throw new NullPointerException();
-        }
-        bitField0_ |= 0x00000001;
-        type_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required .hbase.pb.BackupType type = 1;</code>
-       */
-      public Builder clearType() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        type_ = org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupType.FULL;
-        onChanged();
-        return this;
-      }
-
-      // repeated .hbase.pb.TableName tables = 2;
-      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> tables_ =
-        java.util.Collections.emptyList();
-      private void ensureTablesIsMutable() {
-        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
-          tables_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName>(tables_);
-          bitField0_ |= 0x00000002;
-         }
-      }
-
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tablesBuilder_;
-
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> getTablesList() {
-        if (tablesBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(tables_);
-        } else {
-          return tablesBuilder_.getMessageList();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public int getTablesCount() {
-        if (tablesBuilder_ == null) {
-          return tables_.size();
-        } else {
-          return tablesBuilder_.getCount();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTables(int index) {
-        if (tablesBuilder_ == null) {
-          return tables_.get(index);
-        } else {
-          return tablesBuilder_.getMessage(index);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public Builder setTables(
-          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tablesBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureTablesIsMutable();
-          tables_.set(index, value);
-          onChanged();
-        } else {
-          tablesBuilder_.setMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public Builder setTables(
-          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tablesBuilder_ == null) {
-          ensureTablesIsMutable();
-          tables_.set(index, builderForValue.build());
-          onChanged();
-        } else {
-          tablesBuilder_.setMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public Builder addTables(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tablesBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureTablesIsMutable();
-          tables_.add(value);
-          onChanged();
-        } else {
-          tablesBuilder_.addMessage(value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public Builder addTables(
-          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
-        if (tablesBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureTablesIsMutable();
-          tables_.add(index, value);
-          onChanged();
-        } else {
-          tablesBuilder_.addMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public Builder addTables(
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tablesBuilder_ == null) {
-          ensureTablesIsMutable();
-          tables_.add(builderForValue.build());
-          onChanged();
-        } else {
-          tablesBuilder_.addMessage(builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public Builder addTables(
-          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tablesBuilder_ == null) {
-          ensureTablesIsMutable();
-          tables_.add(index, builderForValue.build());
-          onChanged();
-        } else {
-          tablesBuilder_.addMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public Builder addAllTables(
-          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName> values) {
-        if (tablesBuilder_ == null) {
-          ensureTablesIsMutable();
-          super.addAll(values, tables_);
-          onChanged();
-        } else {
-          tablesBuilder_.addAllMessages(values);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public Builder clearTables() {
-        if (tablesBuilder_ == null) {
-          tables_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
-          onChanged();
-        } else {
-          tablesBuilder_.clear();
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public Builder removeTables(int index) {
-        if (tablesBuilder_ == null) {
-          ensureTablesIsMutable();
-          tables_.remove(index);
-          onChanged();
-        } else {
-          tablesBuilder_.remove(index);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTablesBuilder(
-          int index) {
-        return getTablesFieldBuilder().getBuilder(index);
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTablesOrBuilder(
-          int index) {
-        if (tablesBuilder_ == null) {
-          return tables_.get(index);  } else {
-          return tablesBuilder_.getMessageOrBuilder(index);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-           getTablesOrBuilderList() {
-        if (tablesBuilder_ != null) {
-          return tablesBuilder_.getMessageOrBuilderList();
-        } else {
-          return java.util.Collections.unmodifiableList(tables_);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTablesBuilder() {
-        return getTablesFieldBuilder().addBuilder(
-            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder addTablesBuilder(
-          int index) {
-        return getTablesFieldBuilder().addBuilder(
-            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance());
-      }
-      /**
-       * <code>repeated .hbase.pb.TableName tables = 2;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder> 
-           getTablesBuilderList() {
-        return getTablesFieldBuilder().getBuilderList();
-      }
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-          getTablesFieldBuilder() {
-        if (tablesBuilder_ == null) {
-          tablesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
-                  tables_,
-                  ((bitField0_ & 0x00000002) == 0x00000002),
-                  getParentForChildren(),
-                  isClean());
-          tables_ = null;
-        }
-        return tablesBuilder_;
-      }
-
-      // required string target_root_dir = 3;
-      private java.lang.Object targetRootDir_ = "";
-      /**
-       * <code>required string target_root_dir = 3;</code>
-       */
-      public boolean hasTargetRootDir() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>required string target_root_dir = 3;</code>
-       */
-      public java.lang.String getTargetRootDir() {
-        java.lang.Object ref = targetRootDir_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          targetRootDir_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>required string target_root_dir = 3;</code>
-       */
-      public com.google.protobuf.ByteString
-          getTargetRootDirBytes() {
-        java.lang.Object ref = targetRootDir_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          targetRootDir_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>required string target_root_dir = 3;</code>
-       */
-      public Builder setTargetRootDir(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000004;
-        targetRootDir_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string target_root_dir = 3;</code>
-       */
-      public Builder clearTargetRootDir() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        targetRootDir_ = getDefaultInstance().getTargetRootDir();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required string target_root_dir = 3;</code>
-       */
-      public Builder setTargetRootDirBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000004;
-        targetRootDir_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 workers = 4;
-      private long workers_ ;
-      /**
-       * <code>optional int64 workers = 4;</code>
-       */
-      public boolean hasWorkers() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>optional int64 workers = 4;</code>
-       */
-      public long getWorkers() {
-        return workers_;
-      }
-      /**
-       * <code>optional int64 workers = 4;</code>
-       */
-      public Builder setWorkers(long value) {
-        bitField0_ |= 0x00000008;
-        workers_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 workers = 4;</code>
-       */
-      public Builder clearWorkers() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        workers_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 bandwidth = 5;
-      private long bandwidth_ ;
-      /**
-       * <code>optional int64 bandwidth = 5;</code>
-       */
-      public boolean hasBandwidth() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <code>optional int64 bandwidth = 5;</code>
-       */
-      public long getBandwidth() {
-        return bandwidth_;
-      }
-      /**
-       * <code>optional int64 bandwidth = 5;</code>
-       */
-      public Builder setBandwidth(long value) {
-        bitField0_ |= 0x00000010;
-        bandwidth_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 bandwidth = 5;</code>
-       */
-      public Builder clearBandwidth() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        bandwidth_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional string backup_set_name = 6;
-      private java.lang.Object backupSetName_ = "";
-      /**
-       * <code>optional string backup_set_name = 6;</code>
-       */
-      public boolean hasBackupSetName() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
-      }
-      /**
-       * <code>optional string backup_set_name = 6;</code>
-       */
-      public java.lang.String getBackupSetName() {
-        java.lang.Object ref = backupSetName_;
-        if (!(ref instanceof java.lang.String)) {
-          java.lang.String s = ((com.google.protobuf.ByteString) ref)
-              .toStringUtf8();
-          backupSetName_ = s;
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string backup_set_name = 6;</code>
-       */
-      public com.google.protobuf.ByteString
-          getBackupSetNameBytes() {
-        java.lang.Object ref = backupSetName_;
-        if (ref instanceof String) {
-          com.google.protobuf.ByteString b = 
-              com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          backupSetName_ = b;
-          return b;
-        } else {
-          return (com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string backup_set_name = 6;</code>
-       */
-      public Builder setBackupSetName(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000020;
-        backupSetName_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string backup_set_name = 6;</code>
-       */
-      public Builder clearBackupSetName() {
-        bitField0_ = (bitField0_ & ~0x00000020);
-        backupSetName_ = getDefaultInstance().getBackupSetName();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string backup_set_name = 6;</code>
-       */
-      public Builder setBackupSetNameBytes(
-          com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000020;
-        backupSetName_ = value;
-        onChanged();
-        return this;
-      }
-
-      // optional uint64 nonce_group = 7 [default = 0];
-      private long nonceGroup_ ;
-      /**
-       * <code>optional uint64 nonce_group = 7 [default = 0];</code>
-       */
-      public boolean hasNonceGroup() {
-        return ((bitField0_ & 0x00000040) == 0x00000040);
-      }
-      /**
-       * <code>optional uint64 nonce_group = 7 [default = 0];</code>
-       */
-      public long getNonceGroup() {
-        return nonceGroup_;
-      }
-      /**
-       * <code>optional uint64 nonce_group = 7 [default = 0];</code>
-       */
-      public Builder setNonceGroup(long value) {
-        bitField0_ |= 0x00000040;
-        nonceGroup_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional uint64 nonce_group = 7 [default = 0];</code>
-       */
-      public Builder clearNonceGroup() {
-        bitField0_ = (bitField0_ & ~0x00000040);
-        nonceGroup_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional uint64 nonce = 8 [default = 0];
-      private long nonce_ ;
-      /**
-       * <code>optional uint64 nonce = 8 [default = 0];</code>
-       */
-      public boolean hasNonce() {
-        return ((bitField0_ & 0x00000080) == 0x00000080);
-      }
-      /**
-       * <code>optional uint64 nonce = 8 [default = 0];</code>
-       */
-      public long getNonce() {
-        return nonce_;
-      }
-      /**
-       * <code>optional uint64 nonce = 8 [default = 0];</code>
-       */
-      public Builder setNonce(long value) {
-        bitField0_ |= 0x00000080;
-        nonce_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional uint64 nonce = 8 [default = 0];</code>
-       */
-      public Builder clearNonce() {
-        bitField0_ = (bitField0_ & ~0x00000080);
-        nonce_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:hbase.pb.BackupTablesRequest)
-    }
-
-    static {
-      defaultInstance = new BackupTablesRequest(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:hbase.pb.BackupTablesRequest)
-  }
-
-  public interface BackupTablesResponseOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // optional uint64 proc_id = 1;
-    /**
-     * <code>optional uint64 proc_id = 1;</code>
-     */
-    boolean hasProcId();
-    /**
-     * <code>optional uint64 proc_id = 1;</code>
-     */
-    long getProcId();
-
-    // optional string backup_id = 2;
-    /**
-     * <code>optional string backup_id = 2;</code>
-     */
-    boolean hasBackupId();
-    /**
-     * <code>optional string backup_id = 2;</code>
-     */
-    java.lang.String getBackupId();
-    /**
-     * <code>optional string backup_id = 2;</code>
-     */
-    com.google.protobuf.ByteString
-        getBackupIdBytes();
-  }
-  /**
-   * Protobuf type {@code hbase.pb.BackupTablesResponse}
-   */
-  public static final class BackupTablesResponse extends
-      com.google.protobuf.GeneratedMessage
-      implements BackupTablesResponseOrBuilder {
-    // Use BackupTablesResponse.newBuilder() to construct.
-    private BackupTablesResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private BackupTablesResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final BackupTablesResponse defaultInstance;
-    public static BackupTablesResponse getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public BackupTablesResponse getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private BackupTablesResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 8: {
-              bitField0_ |= 0x00000001;
-              procId_ = input.readUInt64();
-              break;
-            }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              backupId_ = input.readBytes();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BackupTablesResponse_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_hbase_pb_BackupTablesResponse_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesResponse.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<BackupTablesResponse> PARSER =
-        new com.google.protobuf.AbstractParser<BackupTablesResponse>() {
-      public BackupTablesResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new BackupTablesResponse(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<BackupTablesResponse> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // optional uint64 proc_id = 1;
-    public static final int PROC_ID_FIELD_NUMBER = 1;
-    private long procId_;
-    /**
-     * <code>optional uint64 proc_id = 1;</code>
-     */
-    public boolean hasProcId() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>optional uint64 proc_id = 1;</code>
-     */
-    public long getProcId() {
-      return procId_;
-    }
-
-    // optional string backup_id = 2;
-    public static final int BACKUP_ID_FIELD_NUMBER = 2;
-    private java.lang.Object backupId_;
-    /**
-     * <code>optional string backup_id = 2;</code>
-     */
-    public boolean hasBackupId() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional string backup_id = 2;</code>
-     */
-    public java.lang.String getBackupId() {
-      java.lang.Object ref = backupId_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        com.google.protobuf.ByteString bs = 
-            (com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          backupId_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string backup_id = 2;</code>
-     */
-    public com.google.protobuf.ByteString
-        getBackupIdBytes() {
-      java.lang.Object ref = backupId_;
-      if (ref instanceof java.lang.String) {
-        com.google.protobuf.ByteString b = 
-            com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        backupId_ = b;
-        return b;
-      } else {
-        return (com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    private void initFields() {
-      procId_ = 0L;
-      backupId_ = "";
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeUInt64(1, procId_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, getBackupIdBytes());
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSi

<TRUNCATED>

[09/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
deleted file mode 100644
index ac1d2bc..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupRestoreConstants.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * BackupRestoreConstants holds a bunch of HBase Backup and Restore constants
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public final class BackupRestoreConstants {
-
-
-  // delimiter in tablename list in restore command
-  public static final String TABLENAME_DELIMITER_IN_COMMAND = ",";
-
-  public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root";
-
-  public static final String BACKUPID_PREFIX = "backup_";
-
-  public static enum BackupCommand {
-    CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, PROGRESS, SET,
-    SET_ADD, SET_REMOVE, SET_DELETE, SET_DESCRIBE, SET_LIST
-  }
-
-  private BackupRestoreConstants() {
-    // Can't be instantiated with this ctor.
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
deleted file mode 100644
index 3066282..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java
+++ /dev/null
@@ -1,873 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
-
-/**
- * This class provides 'hbase:backup' table API
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class BackupSystemTable implements Closeable {
-
-  static class WALItem {
-    String backupId;
-    String walFile;
-    String backupRoot;
-
-    WALItem(String backupId, String walFile, String backupRoot) {
-      this.backupId = backupId;
-      this.walFile = walFile;
-      this.backupRoot = backupRoot;
-    }
-
-    public String getBackupId() {
-      return backupId;
-    }
-
-    public String getWalFile() {
-      return walFile;
-    }
-
-    public String getBackupRoot() {
-      return backupRoot;
-    }
-
-    public String toString() {
-      return "/" + backupRoot + "/" + backupId + "/" + walFile;
-    }
-
-  }
-
-  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
-  private final static TableName tableName = TableName.BACKUP_TABLE_NAME;
-  // Stores backup sessions (contexts)
-  final static byte[] SESSIONS_FAMILY = "session".getBytes();
-  // Stores other meta
-  final static byte[] META_FAMILY = "meta".getBytes();
-  // Connection to HBase cluster, shared
-  // among all instances
-  private final Connection connection;
-
-  public BackupSystemTable(Connection conn) throws IOException {
-    this.connection = conn;
-  }
-
-  public void close() {
-    // do nothing
-  }
-
-  /**
-   * Updates status (state) of a backup session in hbase:backup table
-   * @param context context
-   * @throws IOException exception
-   */
-  public void updateBackupInfo(BackupInfo context) throws IOException {
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("update backup status in hbase:backup for: " + context.getBackupId()
-          + " set status=" + context.getState());
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Put put = BackupSystemTableHelper.createPutForBackupContext(context);
-      table.put(put);
-    }
-  }
-
-  /**
-   * Deletes backup status from hbase:backup table
-   * @param backupId backup id
-   * @throws IOException exception
-   */
-
-  public void deleteBackupInfo(String backupId) throws IOException {
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("delete backup status in hbase:backup for " + backupId);
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Delete del = BackupSystemTableHelper.createDeleteForBackupInfo(backupId);
-      table.delete(del);
-    }
-  }
-
-  /**
-   * Reads backup status object (instance of BackupContext) from hbase:backup table
-   * @param backupId - backupId
-   * @return Current status of backup session or null
-   */
-
-  public BackupInfo readBackupInfo(String backupId) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read backup status from hbase:backup for: " + backupId);
-    }
-
-    try (Table table = connection.getTable(tableName)) {
-      Get get = BackupSystemTableHelper.createGetForBackupContext(backupId);
-      Result res = table.get(get);
-      if (res.isEmpty()) {
-        return null;
-      }
-      return BackupSystemTableHelper.resultToBackupInfo(res);
-    }
-  }
-
-  /**
-   * Read the last backup start code (timestamp) of last successful backup. Will return null if
-   * there is no start code stored on hbase or the value is of length 0. These two cases indicate
-   * there is no successful backup completed so far.
-   * @param backupRoot root directory path to backup
-   * @return the timestamp of last successful backup
-   * @throws IOException exception
-   */
-  public String readBackupStartCode(String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read backup start code from hbase:backup");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Get get = BackupSystemTableHelper.createGetForStartCode(backupRoot);
-      Result res = table.get(get);
-      if (res.isEmpty()) {
-        return null;
-      }
-      Cell cell = res.listCells().get(0);
-      byte[] val = CellUtil.cloneValue(cell);
-      if (val.length == 0) {
-        return null;
-      }
-      return new String(val);
-    }
-  }
-
-  /**
-   * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
-   * @param startCode start code
-   * @param backupRoot root directory path to backup
-   * @throws IOException exception
-   */
-  public void writeBackupStartCode(Long startCode, String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write backup start code to hbase:backup " + startCode);
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Put put = BackupSystemTableHelper.createPutForStartCode(startCode.toString(), backupRoot);
-      table.put(put);
-    }
-  }
-
-  /**
-   * Get the Region Servers log information after the last log roll from hbase:backup.
-   * @param backupRoot root directory path to backup
-   * @return RS log info
-   * @throws IOException exception
-   */
-  public HashMap<String, Long> readRegionServerLastLogRollResult(String backupRoot)
-      throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read region server last roll log result to hbase:backup");
-    }
-
-    Scan scan = BackupSystemTableHelper.createScanForReadRegionServerLastLogRollResult(backupRoot);
-
-    try (Table table = connection.getTable(tableName);
-        ResultScanner scanner = table.getScanner(scan)) {
-      Result res = null;
-      HashMap<String, Long> rsTimestampMap = new HashMap<String, Long>();
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        Cell cell = res.current();
-        byte[] row = CellUtil.cloneRow(cell);
-        String server =
-            BackupSystemTableHelper.getServerNameForReadRegionServerLastLogRollResult(row);
-        byte[] data = CellUtil.cloneValue(cell);
-        rsTimestampMap.put(server, Long.parseLong(new String(data)));
-      }
-      return rsTimestampMap;
-    }
-  }
-
-  /**
-   * Writes Region Server last roll log result (timestamp) to hbase:backup table
-   * @param server - Region Server name
-   * @param ts- last log timestamp
-   * @param backupRoot root directory path to backup
-   * @throws IOException exception
-   */
-  public void writeRegionServerLastLogRollResult(String server, Long ts, String backupRoot)
-      throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write region server last roll log result to hbase:backup");
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Put put =
-          BackupSystemTableHelper.createPutForRegionServerLastLogRollResult(server, ts, backupRoot);
-      table.put(put);
-    }
-  }
-
-  /**
-   * Get all completed backup information (in desc order by time)
-   * @param onlyCompeleted, true, if only successfully completed sessions
-   * @return history info of BackupCompleteData
-   * @throws IOException exception
-   */
-  public ArrayList<BackupInfo> getBackupHistory(boolean onlyCompleted) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get backup history from hbase:backup");
-    }
-    ArrayList<BackupInfo> list;
-    BackupState state = onlyCompleted ? BackupState.COMPLETE : BackupState.ANY;
-    list = getBackupContexts(state);
-    return BackupClientUtil.sortHistoryListDesc(list);
-  }
-
-  public List<BackupInfo> getBackupHistory() throws IOException {
-    return getBackupHistory(false);
-  }
-
-  /**
-   * Get history for backup destination
-   * @param backupRoot - backup destination
-   * @return List of backup info
-   * @throws IOException
-   */
-  public List<BackupInfo> getBackupHistory(String backupRoot) throws IOException {
-    ArrayList<BackupInfo> history = getBackupHistory(false);
-    for (Iterator<BackupInfo> iterator = history.iterator(); iterator.hasNext();) {
-      BackupInfo info = iterator.next();
-      if (!backupRoot.equals(info.getTargetRootDir())) {
-        iterator.remove();
-      }
-    }
-    return history;
-  }
-  
-  /**
-   * Get history for a table
-   * @param name - table name
-   * @return history for a table
-   * @throws IOException
-   */
-  public List<BackupInfo> getBackupHistoryForTable(TableName name) throws IOException {
-    List<BackupInfo> history = getBackupHistory();
-    List<BackupInfo> tableHistory = new ArrayList<BackupInfo>();
-    for (BackupInfo info : history) {
-      List<TableName> tables = info.getTableNames();
-      if (tables.contains(name)) {
-        tableHistory.add(info);
-      }
-    }
-    return tableHistory;
-  }
-
-  public Map<TableName, ArrayList<BackupInfo>> 
-    getBackupHistoryForTableSet(Set<TableName> set, String backupRoot) throws IOException {
-    List<BackupInfo> history = getBackupHistory(backupRoot);
-    Map<TableName, ArrayList<BackupInfo>> tableHistoryMap = 
-        new HashMap<TableName, ArrayList<BackupInfo>>();
-    for (Iterator<BackupInfo> iterator = history.iterator(); iterator.hasNext();) {
-      BackupInfo info = iterator.next();
-      if (!backupRoot.equals(info.getTargetRootDir())) {
-        continue;
-      }
-      List<TableName> tables = info.getTableNames();
-      for (TableName tableName: tables) {      
-        if (set.contains(tableName)) {
-          ArrayList<BackupInfo> list = tableHistoryMap.get(tableName);
-          if (list == null) {
-            list = new ArrayList<BackupInfo>();
-            tableHistoryMap.put(tableName, list);
-          }
-          list.add(info);
-        }
-      }
-    }
-    return tableHistoryMap;
-  }
-  
-  /**
-   * Get all backup session with a given status (in desc order by time)
-   * @param status status
-   * @return history info of backup contexts
-   * @throws IOException exception
-   */
-  public ArrayList<BackupInfo> getBackupContexts(BackupState status) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get backup contexts from hbase:backup");
-    }
-
-    Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
-    ArrayList<BackupInfo> list = new ArrayList<BackupInfo>();
-
-    try (Table table = connection.getTable(tableName);
-        ResultScanner scanner = table.getScanner(scan)) {
-      Result res = null;
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        BackupInfo context = BackupSystemTableHelper.cellToBackupInfo(res.current());
-        if (status != BackupState.ANY && context.getState() != status) {
-          continue;
-        }
-        list.add(context);
-      }
-      return list;
-    }
-  }
-
-  /**
-   * Write the current timestamps for each regionserver to hbase:backup after a successful full or
-   * incremental backup. The saved timestamp is of the last log file that was backed up already.
-   * @param tables tables
-   * @param newTimestamps timestamps
-   * @param backupRoot root directory path to backup
-   * @throws IOException exception
-   */
-  public void writeRegionServerLogTimestamp(Set<TableName> tables,
-      HashMap<String, Long> newTimestamps, String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write RS log time stamps to hbase:backup for tables ["
-          + StringUtils.join(tables, ",") + "]");
-    }
-    List<Put> puts = new ArrayList<Put>();
-    for (TableName table : tables) {
-      byte[] smapData = toTableServerTimestampProto(table, newTimestamps).toByteArray();
-      Put put =
-          BackupSystemTableHelper.createPutForWriteRegionServerLogTimestamp(table, smapData,
-            backupRoot);
-      puts.add(put);
-    }
-    try (Table table = connection.getTable(tableName)) {
-      table.put(puts);
-    }
-  }
-
-  /**
-   * Read the timestamp for each region server log after the last successful backup. Each table has
-   * its own set of the timestamps. The info is stored for each table as a concatenated string of
-   * rs->timestapmp
-   * @param backupRoot root directory path to backup
-   * @return the timestamp for each region server. key: tableName value:
-   *         RegionServer,PreviousTimeStamp
-   * @throws IOException exception
-   */
-  public HashMap<TableName, HashMap<String, Long>> readLogTimestampMap(String backupRoot)
-      throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read RS log ts from hbase:backup for root=" + backupRoot);
-    }
-
-    HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
-        new HashMap<TableName, HashMap<String, Long>>();
-
-    Scan scan = BackupSystemTableHelper.createScanForReadLogTimestampMap(backupRoot);
-    try (Table table = connection.getTable(tableName);
-        ResultScanner scanner = table.getScanner(scan)) {
-      Result res = null;
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        Cell cell = res.current();
-        byte[] row = CellUtil.cloneRow(cell);
-        String tabName = BackupSystemTableHelper.getTableNameForReadLogTimestampMap(row);
-        TableName tn = TableName.valueOf(tabName);
-        byte[] data = CellUtil.cloneValue(cell);
-        if (data == null) {
-          throw new IOException("Data of last backup data from hbase:backup "
-              + "is empty. Create a backup first.");
-        }
-        if (data != null && data.length > 0) {
-          HashMap<String, Long> lastBackup =
-              fromTableServerTimestampProto(BackupProtos.TableServerTimestamp.parseFrom(data));
-          tableTimestampMap.put(tn, lastBackup);
-        }
-      }
-      return tableTimestampMap;
-    }
-  }
-
-  private BackupProtos.TableServerTimestamp toTableServerTimestampProto(TableName table,
-      Map<String, Long> map) {
-    BackupProtos.TableServerTimestamp.Builder tstBuilder =
-        BackupProtos.TableServerTimestamp.newBuilder();
-    tstBuilder.setTable(ProtobufUtil.toProtoTableName(table));
-
-    for (Entry<String, Long> entry : map.entrySet()) {
-      BackupProtos.ServerTimestamp.Builder builder = BackupProtos.ServerTimestamp.newBuilder();
-      builder.setServer(entry.getKey());
-      builder.setTimestamp(entry.getValue());
-      tstBuilder.addServerTimestamp(builder.build());
-    }
-
-    return tstBuilder.build();
-  }
-
-  private HashMap<String, Long> fromTableServerTimestampProto(
-      BackupProtos.TableServerTimestamp proto) {
-    HashMap<String, Long> map = new HashMap<String, Long>();
-    List<BackupProtos.ServerTimestamp> list = proto.getServerTimestampList();
-    for (BackupProtos.ServerTimestamp st : list) {
-      map.put(st.getServer(), st.getTimestamp());
-    }
-    return map;
-  }
-
-  /**
-   * Return the current tables covered by incremental backup.
-   * @param backupRoot root directory path to backup
-   * @return set of tableNames
-   * @throws IOException exception
-   */
-  public Set<TableName> getIncrementalBackupTableSet(String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get incr backup table set from hbase:backup");
-    }
-    TreeSet<TableName> set = new TreeSet<>();
-
-    try (Table table = connection.getTable(tableName)) {
-      Get get = BackupSystemTableHelper.createGetForIncrBackupTableSet(backupRoot);
-      Result res = table.get(get);
-      if (res.isEmpty()) {
-        return set;
-      }
-      List<Cell> cells = res.listCells();
-      for (Cell cell : cells) {
-        // qualifier = table name - we use table names as qualifiers
-        set.add(TableName.valueOf(CellUtil.cloneQualifier(cell)));
-      }
-      return set;
-    }
-  }
-
-  /**
-   * Add tables to global incremental backup set
-   * @param tables - set of tables
-   * @param backupRoot root directory path to backup
-   * @throws IOException exception
-   */
-  public void addIncrementalBackupTableSet(Set<TableName> tables, String backupRoot)
-      throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Add incremental backup table set to hbase:backup. ROOT=" + backupRoot
-          + " tables [" + StringUtils.join(tables, " ") + "]");
-      for (TableName table : tables) {
-        LOG.debug(table);
-      }
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Put put = BackupSystemTableHelper.createPutForIncrBackupTableSet(tables, backupRoot);
-      table.put(put);
-    }
-  }
-
-  /**
-   * Removes incremental backup set
-   * @param backupRoot backup root
-   */
-
-  public void deleteIncrementalBackupTableSet(String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Delete incremental backup table set to hbase:backup. ROOT=" + backupRoot);
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Delete delete = BackupSystemTableHelper.createDeleteForIncrBackupTableSet(backupRoot);
-      table.delete(delete);
-    }
-  }
-
-  /**
-   * Register WAL files as eligible for deletion
-   * @param files files
-   * @param backupId backup id
-   * @param backupRoot root directory path to backup
-   * @throws IOException exception
-   */
-  public void addWALFiles(List<String> files, String backupId, String backupRoot)
-      throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("add WAL files to hbase:backup: " + backupId + " " + backupRoot + " files ["
-          + StringUtils.join(files, ",") + "]");
-      for (String f : files) {
-        LOG.debug("add :" + f);
-      }
-    }
-    try (Table table = connection.getTable(tableName)) {
-      List<Put> puts =
-          BackupSystemTableHelper.createPutsForAddWALFiles(files, backupId, backupRoot);
-      table.put(puts);
-    }
-  }
-
-  /**
-   * Register WAL files as eligible for deletion
-   * @param backupRoot root directory path to backup
-   * @throws IOException exception
-   */
-  public Iterator<WALItem> getWALFilesIterator(String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get WAL files from hbase:backup");
-    }
-    final Table table = connection.getTable(tableName);
-    Scan scan = BackupSystemTableHelper.createScanForGetWALs(backupRoot);
-    final ResultScanner scanner = table.getScanner(scan);
-    final Iterator<Result> it = scanner.iterator();
-    return new Iterator<WALItem>() {
-
-      @Override
-      public boolean hasNext() {
-        boolean next = it.hasNext();
-        if (!next) {
-          // close all
-          try {
-            scanner.close();
-            table.close();
-          } catch (IOException e) {
-            LOG.error("Close WAL Iterator", e);
-          }
-        }
-        return next;
-      }
-
-      @Override
-      public WALItem next() {
-        Result next = it.next();
-        List<Cell> cells = next.listCells();
-        byte[] buf = cells.get(0).getValueArray();
-        int len = cells.get(0).getValueLength();
-        int offset = cells.get(0).getValueOffset();
-        String backupId = new String(buf, offset, len);
-        buf = cells.get(1).getValueArray();
-        len = cells.get(1).getValueLength();
-        offset = cells.get(1).getValueOffset();
-        String walFile = new String(buf, offset, len);
-        buf = cells.get(2).getValueArray();
-        len = cells.get(2).getValueLength();
-        offset = cells.get(2).getValueOffset();
-        String backupRoot = new String(buf, offset, len);
-        return new WALItem(backupId, walFile, backupRoot);
-      }
-
-      @Override
-      public void remove() {
-        // not implemented
-        throw new RuntimeException("remove is not supported");
-      }
-    };
-
-  }
-
-  /**
-   * Check if WAL file is eligible for deletion Future: to support all backup destinations
-   * @param file file
-   * @return true, if - yes.
-   * @throws IOException exception
-   */
-  public boolean isWALFileDeletable(String file) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Check if WAL file has been already backed up in hbase:backup " + file);
-    }
-    try (Table table = connection.getTable(tableName)) {
-      Get get = BackupSystemTableHelper.createGetForCheckWALFile(file);
-      Result res = table.get(get);
-      if (res.isEmpty()) {
-        return false;
-      }
-      return true;
-    }
-  }
-
-  /**
-   * Checks if we have at least one backup session in hbase:backup This API is used by
-   * BackupLogCleaner
-   * @return true, if - at least one session exists in hbase:backup table
-   * @throws IOException exception
-   */
-  public boolean hasBackupSessions() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Has backup sessions from hbase:backup");
-    }
-    boolean result = false;
-    Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
-    scan.setCaching(1);
-    try (Table table = connection.getTable(tableName);
-        ResultScanner scanner = table.getScanner(scan)) {
-      if (scanner.next() != null) {
-        result = true;
-      }
-      return result;
-    }
-  }
-
-  /**
-   * BACKUP SETS
-   */
-
-  /**
-   * Get backup set list
-   * @return backup set list
-   * @throws IOException
-   */
-  public List<String> listBackupSets() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(" Backup set list");
-    }
-    List<String> list = new ArrayList<String>();
-    Table table = null;
-    ResultScanner scanner = null;
-    try {
-      table = connection.getTable(tableName);
-      Scan scan = BackupSystemTableHelper.createScanForBackupSetList();
-      scan.setMaxVersions(1);
-      scanner = table.getScanner(scan);
-      Result res = null;
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        list.add(BackupSystemTableHelper.cellKeyToBackupSetName(res.current()));
-      }
-      return list;
-    } finally {
-      if (scanner != null) {
-        scanner.close();
-      }
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Get backup set description (list of tables)
-   * @param name - set's name
-   * @return list of tables in a backup set
-   * @throws IOException
-   */
-  public List<TableName> describeBackupSet(String name) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(" Backup set describe: " + name);
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Get get = BackupSystemTableHelper.createGetForBackupSet(name);
-      Result res = table.get(get);
-      if (res.isEmpty()) return null;
-      res.advance();
-      String[] tables = BackupSystemTableHelper.cellValueToBackupSet(res.current());
-      return toList(tables);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  private List<TableName> toList(String[] tables) {
-    List<TableName> list = new ArrayList<TableName>(tables.length);
-    for (String name : tables) {
-      list.add(TableName.valueOf(name));
-    }
-    return list;
-  }
-
-  /**
-   * Add backup set (list of tables)
-   * @param name - set name
-   * @param tables - list of tables, comma-separated
-   * @throws IOException
-   */
-  public void addToBackupSet(String name, String[] newTables) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Backup set add: " + name + " tables [" + StringUtils.join(newTables, " ") + "]");
-    }
-    Table table = null;
-    String[] union = null;
-    try {
-      table = connection.getTable(tableName);
-      Get get = BackupSystemTableHelper.createGetForBackupSet(name);
-      Result res = table.get(get);
-      if (res.isEmpty()) {
-        union = newTables;
-      } else {
-        res.advance();
-        String[] tables = BackupSystemTableHelper.cellValueToBackupSet(res.current());
-        union = merge(tables, newTables);
-      }
-      Put put = BackupSystemTableHelper.createPutForBackupSet(name, union);
-      table.put(put);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  private String[] merge(String[] tables, String[] newTables) {
-    List<String> list = new ArrayList<String>();
-    // Add all from tables
-    for (String t : tables) {
-      list.add(t);
-    }
-    for (String nt : newTables) {
-      if (list.contains(nt)) continue;
-      list.add(nt);
-    }
-    String[] arr = new String[list.size()];
-    list.toArray(arr);
-    return arr;
-  }
-
-  /**
-   * Remove tables from backup set (list of tables)
-   * @param name - set name
-   * @param tables - list of tables, comma-separated
-   * @throws IOException
-   */
-  public void removeFromBackupSet(String name, String[] toRemove) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(" Backup set remove from : " + name + " tables [" + StringUtils.join(toRemove, " ")
-          + "]");
-    }
-    Table table = null;
-    String[] disjoint = null;
-    try {
-      table = connection.getTable(tableName);
-      Get get = BackupSystemTableHelper.createGetForBackupSet(name);
-      Result res = table.get(get);
-      if (res.isEmpty()) {
-        LOG.warn("Backup set '" + name + "' not found.");
-        return;
-      } else {
-        res.advance();
-        String[] tables = BackupSystemTableHelper.cellValueToBackupSet(res.current());
-        disjoint = disjoin(tables, toRemove);
-      }
-      if (disjoint.length > 0) {
-        Put put = BackupSystemTableHelper.createPutForBackupSet(name, disjoint);
-        table.put(put);
-      } else {
-        // Delete
-        // describeBackupSet(name);
-        LOG.warn("Backup set '" + name + "' does not contain tables ["
-            + StringUtils.join(toRemove, " ") + "]");
-      }
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  private String[] disjoin(String[] tables, String[] toRemove) {
-    List<String> list = new ArrayList<String>();
-    // Add all from tables
-    for (String t : tables) {
-      list.add(t);
-    }
-    for (String nt : toRemove) {
-      if (list.contains(nt)) {
-        list.remove(nt);
-      }
-    }
-    String[] arr = new String[list.size()];
-    list.toArray(arr);
-    return arr;
-  }
-
-  /**
-   * Delete backup set
-   * @param name set's name
-   * @throws IOException
-   */
-  public void deleteBackupSet(String name) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(" Backup set delete: " + name);
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Delete del = BackupSystemTableHelper.createDeleteForBackupSet(name);
-      table.delete(del);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Get backup system table descriptor
-   * @return descriptor
-   */
-  public static HTableDescriptor getSystemTableDescriptor() {
-    HTableDescriptor tableDesc = new HTableDescriptor(tableName);
-    HColumnDescriptor colSessionsDesc = new HColumnDescriptor(SESSIONS_FAMILY);
-    colSessionsDesc.setMaxVersions(1);
-    // Time to keep backup sessions (secs)
-    Configuration config = HBaseConfiguration.create();
-    int ttl = config.getInt(HConstants.BACKUP_SYSTEM_TTL_KEY, HConstants.BACKUP_SYSTEM_TTL_DEFAULT);
-    colSessionsDesc.setTimeToLive(ttl);
-    tableDesc.addFamily(colSessionsDesc);
-    HColumnDescriptor colMetaDesc = new HColumnDescriptor(META_FAMILY);
-    // colDesc.setMaxVersions(1);
-    tableDesc.addFamily(colMetaDesc);
-    return tableDesc;
-  }
-
-  public static String getTableNameAsString() {
-    return tableName.getNameAsString();
-  }
-
-  public static TableName getTableName() {
-    return tableName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java
deleted file mode 100644
index 37f29f8..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTableHelper.java
+++ /dev/null
@@ -1,433 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.util.Bytes;
-
-
-/**
- * A collection for methods used by BackupSystemTable.
- */
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class BackupSystemTableHelper {
-
-  /**
-   * hbase:backup schema: 
-   * 1. Backup sessions rowkey= "session:" + backupId; value = serialized BackupContext 
-   * 2. Backup start code rowkey = "startcode:" + backupRoot; value = startcode 
-   * 3. Incremental backup set rowkey="incrbackupset:" + backupRoot; value=[list of tables] 
-   * 4. Table-RS-timestamp map rowkey="trslm:"+ backupRoot+table_name; value = map[RS-> last WAL
-   * timestamp] 
-   * 5. RS - WAL ts map rowkey="rslogts:"+backupRoot +server; value = last WAL timestamp
-   * 6. WALs recorded rowkey="wals:"+WAL unique file name; value = backupId and full WAL file name
-   */
-
-  private final static String BACKUP_INFO_PREFIX = "session:";
-  private final static String START_CODE_ROW = "startcode:";
-  private final static String INCR_BACKUP_SET = "incrbackupset:";
-  private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm:";
-  private final static String RS_LOG_TS_PREFIX = "rslogts:";
-  private final static String WALS_PREFIX = "wals:";
-  private final static String SET_KEY_PREFIX = "backupset:";
-
-  private final static byte[] EMPTY_VALUE = new byte[] {};
-
-  // Safe delimiter in a string
-  private final static String NULL = "\u0000";
-
-  private BackupSystemTableHelper() {
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  /**
-   * Creates Put operation for a given backup context object
-   * @param context backup context
-   * @return put operation
-   * @throws IOException exception
-   */
-  static Put createPutForBackupContext(BackupInfo context) throws IOException {
-    Put put = new Put(rowkey(BACKUP_INFO_PREFIX, context.getBackupId()));
-    put.addColumn(BackupSystemTable.SESSIONS_FAMILY, "context".getBytes(), context.toByteArray());
-    return put;
-  }
-
-  /**
-   * Creates Get operation for a given backup id
-   * @param backupId - backup's ID
-   * @return get operation
-   * @throws IOException exception
-   */
-  static Get createGetForBackupContext(String backupId) throws IOException {
-    Get get = new Get(rowkey(BACKUP_INFO_PREFIX, backupId));
-    get.addFamily(BackupSystemTable.SESSIONS_FAMILY);
-    get.setMaxVersions(1);
-    return get;
-  }
-
-  /**
-   * Creates Delete operation for a given backup id
-   * @param backupId - backup's ID
-   * @return delete operation
-   * @throws IOException exception
-   */
-  public static Delete createDeleteForBackupInfo(String backupId) {
-    Delete del = new Delete(rowkey(BACKUP_INFO_PREFIX, backupId));
-    del.addFamily(BackupSystemTable.SESSIONS_FAMILY);
-    return del;
-  }
-
-  /**
-   * Converts Result to BackupContext
-   * @param res - HBase result
-   * @return backup context instance
-   * @throws IOException exception
-   */
-  static BackupInfo resultToBackupInfo(Result res) throws IOException {
-    res.advance();
-    Cell cell = res.current();
-    return cellToBackupInfo(cell);
-  }
-
-  /**
-   * Creates Get operation to retrieve start code from hbase:backup
-   * @return get operation
-   * @throws IOException exception
-   */
-  static Get createGetForStartCode(String rootPath) throws IOException {
-    Get get = new Get(rowkey(START_CODE_ROW, rootPath));
-    get.addFamily(BackupSystemTable.META_FAMILY);
-    get.setMaxVersions(1);
-    return get;
-  }
-
-  /**
-   * Creates Put operation to store start code to hbase:backup
-   * @return put operation
-   * @throws IOException exception
-   */
-  static Put createPutForStartCode(String startCode, String rootPath) {
-    Put put = new Put(rowkey(START_CODE_ROW, rootPath));
-    put.addColumn(BackupSystemTable.META_FAMILY, "startcode".getBytes(), startCode.getBytes());
-    return put;
-  }
-
-  /**
-   * Creates Get to retrieve incremental backup table set from hbase:backup
-   * @return get operation
-   * @throws IOException exception
-   */
-  static Get createGetForIncrBackupTableSet(String backupRoot) throws IOException {
-    Get get = new Get(rowkey(INCR_BACKUP_SET, backupRoot));
-    get.addFamily(BackupSystemTable.META_FAMILY);
-    get.setMaxVersions(1);
-    return get;
-  }
-
-  /**
-   * Creates Put to store incremental backup table set
-   * @param tables tables
-   * @return put operation
-   */
-  static Put createPutForIncrBackupTableSet(Set<TableName> tables, String backupRoot) {
-    Put put = new Put(rowkey(INCR_BACKUP_SET, backupRoot));
-    for (TableName table : tables) {
-      put.addColumn(BackupSystemTable.META_FAMILY, Bytes.toBytes(table.getNameAsString()),
-        EMPTY_VALUE);
-    }
-    return put;
-  }
-
-  /**
-   * Creates Delete for incremental backup table set
-   * @param backupRoot backup root
-   * @return delete operation
-   */
-  static Delete createDeleteForIncrBackupTableSet(String backupRoot) {
-    Delete delete = new Delete(rowkey(INCR_BACKUP_SET, backupRoot));
-    delete.addFamily(BackupSystemTable.META_FAMILY);
-    return delete;
-  }
-
-  /**
-   * Creates Scan operation to load backup history
-   * @return scan operation
-   */
-  static Scan createScanForBackupHistory() {
-    Scan scan = new Scan();
-    byte[] startRow = BACKUP_INFO_PREFIX.getBytes();
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.SESSIONS_FAMILY);
-    scan.setMaxVersions(1);
-    return scan;
-  }
-
-  /**
-   * Converts cell to backup context instance.
-   * @param current - cell
-   * @return backup context instance
-   * @throws IOException exception
-   */
-  static BackupInfo cellToBackupInfo(Cell current) throws IOException {
-    byte[] data = CellUtil.cloneValue(current);
-    return BackupInfo.fromByteArray(data);
-  }
-
-  /**
-   * Creates Put to write RS last roll log timestamp map
-   * @param table - table
-   * @param smap - map, containing RS:ts
-   * @return put operation
-   */
-  static Put createPutForWriteRegionServerLogTimestamp(TableName table, byte[] smap,
-      String backupRoot) {
-    Put put = new Put(rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot, NULL, table.getNameAsString()));
-    put.addColumn(BackupSystemTable.META_FAMILY, "log-roll-map".getBytes(), smap);
-    return put;
-  }
-
-  /**
-   * Creates Scan to load table-> { RS -> ts} map of maps
-   * @return scan operation
-   */
-  static Scan createScanForReadLogTimestampMap(String backupRoot) {
-    Scan scan = new Scan();
-    byte[] startRow = rowkey(TABLE_RS_LOG_MAP_PREFIX, backupRoot);
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.META_FAMILY);
-
-    return scan;
-  }
-
-  /**
-   * Get table name from rowkey
-   * @param cloneRow rowkey
-   * @return table name
-   */
-  static String getTableNameForReadLogTimestampMap(byte[] cloneRow) {
-    String s = new String(cloneRow);
-    int index = s.lastIndexOf(NULL);
-    return s.substring(index + 1);
-  }
-
-  /**
-   * Creates Put to store RS last log result
-   * @param server - server name
-   * @param timestamp - log roll result (timestamp)
-   * @return put operation
-   */
-  static Put createPutForRegionServerLastLogRollResult(String server, Long timestamp,
-      String backupRoot) {
-    Put put = new Put(rowkey(RS_LOG_TS_PREFIX, backupRoot, NULL, server));
-    put.addColumn(BackupSystemTable.META_FAMILY, "rs-log-ts".getBytes(), timestamp.toString()
-        .getBytes());
-    return put;
-  }
-
-  /**
-   * Creates Scan operation to load last RS log roll results
-   * @return scan operation
-   */
-  static Scan createScanForReadRegionServerLastLogRollResult(String backupRoot) {
-    Scan scan = new Scan();
-    byte[] startRow = rowkey(RS_LOG_TS_PREFIX, backupRoot);
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.META_FAMILY);
-    scan.setMaxVersions(1);
-
-    return scan;
-  }
-
-  /**
-   * Get server's name from rowkey
-   * @param row - rowkey
-   * @return server's name
-   */
-  static String getServerNameForReadRegionServerLastLogRollResult(byte[] row) {
-    String s = new String(row);
-    int index = s.lastIndexOf(NULL);
-    return s.substring(index + 1);
-  }
-
-  /**
-   * Creates put list for list of WAL files
-   * @param files list of WAL file paths
-   * @param backupId backup id
-   * @return put list
-   * @throws IOException exception
-   */
-  public static List<Put> createPutsForAddWALFiles(List<String> files, String backupId,
-      String backupRoot) throws IOException {
-
-    List<Put> puts = new ArrayList<Put>();
-    for (String file : files) {
-      Put put = new Put(rowkey(WALS_PREFIX, BackupClientUtil.getUniqueWALFileNamePart(file)));
-      put.addColumn(BackupSystemTable.META_FAMILY, "backupId".getBytes(), backupId.getBytes());
-      put.addColumn(BackupSystemTable.META_FAMILY, "file".getBytes(), file.getBytes());
-      put.addColumn(BackupSystemTable.META_FAMILY, "root".getBytes(), backupRoot.getBytes());
-      puts.add(put);
-    }
-    return puts;
-  }
-
-  /**
-   * Creates Scan operation to load WALs TODO: support for backupRoot
-   * @param backupRoot - path to backup destination
-   * @return scan operation
-   */
-  public static Scan createScanForGetWALs(String backupRoot) {
-    Scan scan = new Scan();
-    byte[] startRow = WALS_PREFIX.getBytes();
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.META_FAMILY);
-    return scan;
-  }
-
-  /**
-   * Creates Get operation for a given wal file name TODO: support for backup destination
-   * @param file file
-   * @return get operation
-   * @throws IOException exception
-   */
-  public static Get createGetForCheckWALFile(String file) throws IOException {
-    Get get = new Get(rowkey(WALS_PREFIX, BackupClientUtil.getUniqueWALFileNamePart(file)));
-    // add backup root column
-    get.addFamily(BackupSystemTable.META_FAMILY);
-    return get;
-  }
-
-  /**
-   * Creates Scan operation to load backup set list
-   * @return scan operation
-   */
-  static Scan createScanForBackupSetList() {
-    Scan scan = new Scan();
-    byte[] startRow = SET_KEY_PREFIX.getBytes();
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.META_FAMILY);
-    return scan;
-  }
-
-  /**
-   * Creates Get operation to load backup set content
-   * @return get operation
-   */
-  static Get createGetForBackupSet(String name) {
-    Get get = new Get(rowkey(SET_KEY_PREFIX, name));
-    get.addFamily(BackupSystemTable.META_FAMILY);
-    return get;
-  }
-
-  /**
-   * Creates Delete operation to delete backup set content
-   * @param name - backup set's name
-   * @return delete operation
-   */
-  static Delete createDeleteForBackupSet(String name) {
-    Delete del = new Delete(rowkey(SET_KEY_PREFIX, name));
-    del.addFamily(BackupSystemTable.META_FAMILY);
-    return del;
-  }
-
-  /**
-   * Creates Put operation to update backup set content
-   * @param name - backup set's name
-   * @param tables - list of tables
-   * @return put operation
-   */
-  static Put createPutForBackupSet(String name, String[] tables) {
-    Put put = new Put(rowkey(SET_KEY_PREFIX, name));
-    byte[] value = convertToByteArray(tables);
-    put.addColumn(BackupSystemTable.META_FAMILY, "tables".getBytes(), value);
-    return put;
-  }
-
-  private static byte[] convertToByteArray(String[] tables) {
-    return StringUtils.join(tables, ",").getBytes();
-  }
-
-  /**
-   * Converts cell to backup set list.
-   * @param current - cell
-   * @return backup set
-   * @throws IOException
-   */
-  static String[] cellValueToBackupSet(Cell current) throws IOException {
-    byte[] data = CellUtil.cloneValue(current);
-    if (data != null && data.length > 0) {
-      return new String(data).split(",");
-    } else {
-      return new String[0];
-    }
-  }
-
-  /**
-   * Converts cell key to backup set name.
-   * @param current - cell
-   * @return backup set name
-   * @throws IOException
-   */
-  static String cellKeyToBackupSetName(Cell current) throws IOException {
-    byte[] data = CellUtil.cloneRow(current);
-    return new String(data).substring(SET_KEY_PREFIX.length());
-  }
-
-  static byte[] rowkey(String s, String... other) {
-    StringBuilder sb = new StringBuilder(s);
-    for (String ss : other) {
-      sb.append(ss);
-    }
-    return sb.toString().getBytes();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/util/BackupClientUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/util/BackupClientUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/util/BackupClientUtil.java
deleted file mode 100644
index c22f51b..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/util/BackupClientUtil.java
+++ /dev/null
@@ -1,437 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.util;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.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.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.impl.BackupManifest;
-import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * A collection of methods used by multiple classes to backup HBase tables.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class BackupClientUtil {
-  protected static final Log LOG = LogFactory.getLog(BackupClientUtil.class);
-  public static final String LOGNAME_SEPARATOR = ".";
-
-  private BackupClientUtil() {
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  /**
-   * Check whether the backup path exist
-   * @param backupStr backup
-   * @param conf configuration
-   * @return Yes if path exists
-   * @throws IOException exception
-   */
-  public static boolean checkPathExist(String backupStr, Configuration conf) throws IOException {
-    boolean isExist = false;
-    Path backupPath = new Path(backupStr);
-    FileSystem fileSys = backupPath.getFileSystem(conf);
-    String targetFsScheme = fileSys.getUri().getScheme();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Schema of given url: " + backupStr + " is: " + targetFsScheme);
-    }
-    if (fileSys.exists(backupPath)) {
-      isExist = true;
-    }
-    return isExist;
-  }
-
-  // check target path first, confirm it doesn't exist before backup
-  public static void checkTargetDir(String backupRootPath, Configuration conf) throws IOException {
-    boolean targetExists = false;
-    try {
-      targetExists = checkPathExist(backupRootPath, conf);
-    } catch (IOException e) {
-      String expMsg = e.getMessage();
-      String newMsg = null;
-      if (expMsg.contains("No FileSystem for scheme")) {
-        newMsg =
-            "Unsupported filesystem scheme found in the backup target url. Error Message: "
-                + newMsg;
-        LOG.error(newMsg);
-        throw new IOException(newMsg);
-      } else {
-        throw e;
-      }
-    }
-
-    if (targetExists) {
-      LOG.info("Using existing backup root dir: " + backupRootPath);
-    } else {
-      LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created.");
-    }
-  }
-
-  /**
-   * Get the min value for all the Values a map.
-   * @param map map
-   * @return the min value
-   */
-  public static <T> Long getMinValue(HashMap<T, Long> map) {
-    Long minTimestamp = null;
-    if (map != null) {
-      ArrayList<Long> timestampList = new ArrayList<Long>(map.values());
-      Collections.sort(timestampList);
-      // The min among all the RS log timestamps will be kept in hbase:backup table.
-      minTimestamp = timestampList.get(0);
-    }
-    return minTimestamp;
-  }
-
-  /**
-   * Parses host name:port from archived WAL path
-   * @param p path
-   * @return host name
-   * @throws IOException exception
-   */
-  public static String parseHostFromOldLog(Path p) {
-    try {
-      String n = p.getName();
-      int idx = n.lastIndexOf(LOGNAME_SEPARATOR);
-      String s = URLDecoder.decode(n.substring(0, idx), "UTF8");
-      return ServerName.parseHostname(s) + ":" + ServerName.parsePort(s);
-    } catch (Exception e) {
-      LOG.warn("Skip log file (can't parse): " + p);
-      return null;
-    }
-  }
-
-  /**
-   * Given the log file, parse the timestamp from the file name. The timestamp is the last number.
-   * @param p a path to the log file
-   * @return the timestamp
-   * @throws IOException exception
-   */
-  public static Long getCreationTime(Path p) throws IOException {
-    int idx = p.getName().lastIndexOf(LOGNAME_SEPARATOR);
-    if (idx < 0) {
-      throw new IOException("Cannot parse timestamp from path " + p);
-    }
-    String ts = p.getName().substring(idx + 1);
-    return Long.parseLong(ts);
-  }
-
-  public static List<String> getFiles(FileSystem fs, Path rootDir, List<String> files,
-      PathFilter filter) throws FileNotFoundException, IOException {
-    RemoteIterator<LocatedFileStatus> it = fs.listFiles(rootDir, true);
-
-    while (it.hasNext()) {
-      LocatedFileStatus lfs = it.next();
-      if (lfs.isDirectory()) {
-        continue;
-      }
-      // apply filter
-      if (filter.accept(lfs.getPath())) {
-        files.add(lfs.getPath().toString());
-      }
-    }
-    return files;
-  }
-
-  public static void cleanupBackupData(BackupInfo context, Configuration conf) throws IOException {
-    cleanupHLogDir(context, conf);
-    cleanupTargetDir(context, conf);
-  }
-
-  /**
-   * Clean up directories which are generated when DistCp copying hlogs.
-   * @throws IOException
-   */
-  private static void cleanupHLogDir(BackupInfo backupContext, Configuration conf)
-      throws IOException {
-
-    String logDir = backupContext.getHLogTargetDir();
-    if (logDir == null) {
-      LOG.warn("No log directory specified for " + backupContext.getBackupId());
-      return;
-    }
-
-    Path rootPath = new Path(logDir).getParent();
-    FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
-    FileStatus[] files = listStatus(fs, rootPath, null);
-    if (files == null) {
-      return;
-    }
-    for (FileStatus file : files) {
-      LOG.debug("Delete log files: " + file.getPath().getName());
-      fs.delete(file.getPath(), true);
-    }
-  }
-
-  /**
-   * Clean up the data at target directory
-   */
-  private static void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) {
-    try {
-      // clean up the data at target directory
-      LOG.debug("Trying to cleanup up target dir : " + backupInfo.getBackupId());
-      String targetDir = backupInfo.getTargetRootDir();
-      if (targetDir == null) {
-        LOG.warn("No target directory specified for " + backupInfo.getBackupId());
-        return;
-      }
-
-      FileSystem outputFs = FileSystem.get(new Path(backupInfo.getTargetRootDir()).toUri(), conf);
-
-      for (TableName table : backupInfo.getTables()) {
-        Path targetDirPath =
-            new Path(getTableBackupDir(backupInfo.getTargetRootDir(), backupInfo.getBackupId(),
-              table));
-        if (outputFs.delete(targetDirPath, true)) {
-          LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done.");
-        } else {
-          LOG.info("No data has been found in " + targetDirPath.toString() + ".");
-        }
-
-        Path tableDir = targetDirPath.getParent();
-        FileStatus[] backups = listStatus(outputFs, tableDir, null);
-        if (backups == null || backups.length == 0) {
-          outputFs.delete(tableDir, true);
-          LOG.debug(tableDir.toString() + " is empty, remove it.");
-        }
-      }
-      outputFs.delete(new Path(targetDir, backupInfo.getBackupId()), true);
-    } catch (IOException e1) {
-      LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " at "
-          + backupInfo.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
-    }
-  }
-
-  /**
-   * Given the backup root dir, backup id and the table name, return the backup image location,
-   * which is also where the backup manifest file is. return value look like:
-   * "hdfs://backup.hbase.org:9000/user/biadmin/backup1/backup_1396650096738/default/t1_dn/"
-   * @param backupRootDir backup root directory
-   * @param backupId backup id
-   * @param table table name
-   * @return backupPath String for the particular table
-   */
-  public static String
-      getTableBackupDir(String backupRootDir, String backupId, TableName tableName) {
-    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
-        + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString()
-        + Path.SEPARATOR;
-  }
-
-  public static TableName[] parseTableNames(String tables) {
-    if (tables == null) {
-      return null;
-    }
-    String[] tableArray = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
-
-    TableName[] ret = new TableName[tableArray.length];
-    for (int i = 0; i < tableArray.length; i++) {
-      ret[i] = TableName.valueOf(tableArray[i]);
-    }
-    return ret;
-  }
-
-  /**
-   * Sort history list by start time in descending order.
-   * @param historyList history list
-   * @return sorted list of BackupCompleteData
-   */
-  public static ArrayList<BackupInfo> sortHistoryListDesc(ArrayList<BackupInfo> historyList) {
-    ArrayList<BackupInfo> list = new ArrayList<BackupInfo>();
-    TreeMap<String, BackupInfo> map = new TreeMap<String, BackupInfo>();
-    for (BackupInfo h : historyList) {
-      map.put(Long.toString(h.getStartTs()), h);
-    }
-    Iterator<String> i = map.descendingKeySet().iterator();
-    while (i.hasNext()) {
-      list.add(map.get(i.next()));
-    }
-    return list;
-  }
-
-  /**
-   * Returns WAL file name
-   * @param walFileName WAL file name
-   * @return WAL file name
-   * @throws IOException exception
-   * @throws IllegalArgumentException exception
-   */
-  public static String getUniqueWALFileNamePart(String walFileName) throws IOException {
-    return getUniqueWALFileNamePart(new Path(walFileName));
-  }
-
-  /**
-   * Returns WAL file name
-   * @param p - WAL file path
-   * @return WAL file name
-   * @throws IOException exception
-   */
-  public static String getUniqueWALFileNamePart(Path p) throws IOException {
-    return p.getName();
-  }
-
-  /**
-   * Calls fs.listStatus() and treats FileNotFoundException as non-fatal This accommodates
-   * differences between hadoop versions, where hadoop 1 does not throw a FileNotFoundException, and
-   * return an empty FileStatus[] while Hadoop 2 will throw FileNotFoundException.
-   * @param fs file system
-   * @param dir directory
-   * @param filter path filter
-   * @return null if dir is empty or doesn't exist, otherwise FileStatus array
-   */
-  public static FileStatus[]
-      listStatus(final FileSystem fs, final Path dir, final PathFilter filter) throws IOException {
-    FileStatus[] status = null;
-    try {
-      status = filter == null ? fs.listStatus(dir) : fs.listStatus(dir, filter);
-    } catch (FileNotFoundException fnfe) {
-      // if directory doesn't exist, return null
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(dir + " doesn't exist");
-      }
-    }
-    if (status == null || status.length < 1) return null;
-    return status;
-  }
-
-  /**
-   * Return the 'path' component of a Path. In Hadoop, Path is an URI. This method returns the
-   * 'path' component of a Path's URI: e.g. If a Path is
-   * <code>hdfs://example.org:9000/hbase_trunk/TestTable/compaction.dir</code>, this method returns
-   * <code>/hbase_trunk/TestTable/compaction.dir</code>. This method is useful if you want to print
-   * out a Path without qualifying Filesystem instance.
-   * @param p Filesystem Path whose 'path' component we are to return.
-   * @return Path portion of the Filesystem
-   */
-  public static String getPath(Path p) {
-    return p.toUri().getPath();
-  }
-
-  /**
-   * Given the backup root dir and the backup id, return the log file location for an incremental
-   * backup.
-   * @param backupRootDir backup root directory
-   * @param backupId backup id
-   * @return logBackupDir: ".../user/biadmin/backup1/WALs/backup_1396650096738"
-   */
-  public static String getLogBackupDir(String backupRootDir, String backupId) {
-    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
-        + HConstants.HREGION_LOGDIR_NAME;
-  }
-
-  private static List<BackupInfo> getHistory(Configuration conf, Path backupRootPath)
-      throws IOException {
-    // Get all (n) history from backup root destination
-    FileSystem fs = FileSystem.get(conf);
-    RemoteIterator<LocatedFileStatus> it = fs.listLocatedStatus(backupRootPath);
-
-    List<BackupInfo> infos = new ArrayList<BackupInfo>();
-    while (it.hasNext()) {
-      LocatedFileStatus lfs = it.next();
-      if (!lfs.isDirectory()) continue;
-      String backupId = lfs.getPath().getName();
-      try {
-        BackupInfo info = loadBackupInfo(backupRootPath, backupId, fs);
-        infos.add(info);
-      } catch(IOException e) {
-        LOG.error("Can not load backup info from: "+ lfs.getPath(), e);
-      }
-    }
-    // Sort
-    Collections.sort(infos, new Comparator<BackupInfo>() {
-
-      @Override
-      public int compare(BackupInfo o1, BackupInfo o2) {
-        long ts1 = getTimestamp(o1.getBackupId());
-        long ts2 = getTimestamp(o2.getBackupId());
-        if (ts1 == ts2) return 0;
-        return ts1 < ts2 ? 1 : -1;
-      }
-
-      private long getTimestamp(String backupId) {
-        String[] split = backupId.split("_");
-        return Long.parseLong(split[1]);
-      }
-    });
-    return infos;
-  }
-
-  public static List<BackupInfo> getHistory(Configuration conf, int n, Path backupRootPath,
-      BackupInfo.Filter... filters) throws IOException {
-    List<BackupInfo> infos = getHistory(conf, backupRootPath);
-    List<BackupInfo> ret = new ArrayList<BackupInfo>();
-    for (BackupInfo info : infos) {
-      if (ret.size() == n) {
-        break;
-      }
-      boolean passed = true;
-      for (int i = 0; i < filters.length; i++) {
-        if (!filters[i].apply(info)) {
-          passed = false;
-          break;
-        }
-      }
-      if (passed) {
-        ret.add(info);
-      }
-    }
-    return ret;
-  }
-  
-  public static BackupInfo loadBackupInfo(Path backupRootPath, String backupId, FileSystem fs)
-      throws IOException {
-    Path backupPath = new Path(backupRootPath, backupId);
-
-    RemoteIterator<LocatedFileStatus> it = fs.listFiles(backupPath, true);
-    while (it.hasNext()) {
-      LocatedFileStatus lfs = it.next();
-      if (lfs.getPath().getName().equals(BackupManifest.MANIFEST_FILE_NAME)) {
-        // Load BackupManifest
-        BackupManifest manifest = new BackupManifest(fs, lfs.getPath().getParent());
-        BackupInfo info = manifest.toBackupInfo();
-        return info;
-      }
-    }
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
deleted file mode 100644
index 76402c7..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.util;
-import java.util.List;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-/**
- * Backup set is a named group of HBase tables,
- * which are managed together by Backup/Restore  
- * framework. Instead of using list of tables in backup or restore 
- * operation, one can use set's name instead.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class BackupSet {
-  private final String name;
-  private final List<TableName> tables;
-
-  public BackupSet(String name, List<TableName> tables) {
-    this.name = name;
-    this.tables = tables;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public List<TableName> getTables() {
-    return tables;
-  }
-
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(name).append("={");
-    for (int i = 0; i < tables.size(); i++) {
-      sb.append(tables.get(i));
-      if (i < tables.size() - 1) {
-        sb.append(",");
-      }
-    }
-    sb.append("}");
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 343dad4..17d5e78 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -37,8 +37,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.backup.BackupRequest;
-import org.apache.hadoop.hbase.backup.RestoreRequest;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
@@ -1696,13 +1694,6 @@ public interface Admin extends Abortable, Closeable {
    * @return true if the switch is enabled, false otherwise.
    */
   boolean isSplitOrMergeEnabled(final MasterSwitchType switchType) throws IOException;
-
-  /**
-   * Get Backup Admin interface 
-   * @return backup admin object
-   * @throws IOException exception
-   */
-  BackupAdmin getBackupAdmin() throws IOException;
   
   /**
    * Currently, there are only two compact types:

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BackupAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BackupAdmin.java
deleted file mode 100644
index 2e5ca2a..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BackupAdmin.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.Future;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupRequest;
-import org.apache.hadoop.hbase.backup.RestoreRequest;
-import org.apache.hadoop.hbase.backup.util.BackupSet;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-/**
- * The administrative API for HBase Backup. Obtain an instance from 
- * an {@link Admin#getBackupAdmin()} and call {@link #close()} afterwards.
- * <p>BackupAdmin can be used to create backups, restore data from backups and for 
- * other backup-related operations. 
- *
- * @see Admin
- * @since 2.0
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-
-public interface BackupAdmin extends Closeable{
-
-  /**
-   * Backs up given list of tables fully. Synchronous operation.
-   * 
-   * @param request BackupRequest instance which contains the following members:
-   *  type whether the backup is full or incremental
-   *  tableList list of tables to backup
-   *  targetRootDir root directory for saving the backup
-   *  workers number of parallel workers. -1 - system defined
-   *  bandwidth bandwidth per worker in MB per second. -1 - unlimited
-   * @return the backup Id
-   */
-  
-  public String backupTables(final BackupRequest userRequest) throws IOException;
-  
-  /**
-   * Backs up given list of tables fully. Asynchronous operation.
-   * 
-   * @param request BackupRequest instance which contains the following members:
-   *  type whether the backup is full or incremental
-   *  tableList list of tables to backup
-   *  targetRootDir root dir for saving the backup
-   *  workers number of paralle workers. -1 - system defined
-   *  bandwidth bandwidth per worker in MB per sec. -1 - unlimited
-   * @return the backup Id future
-   */
-  public Future<String> backupTablesAsync(final BackupRequest userRequest) throws IOException;
-
-  /**
-   * Restore backup
-   * @param request - restore request
-   * @throws IOException exception
-   */
-  public void restore(RestoreRequest request) throws IOException;
-
-  /**
-   * Restore backup
-   * @param request - restore request
-   * @return Future which client can wait on
-   * @throws IOException exception
-   */
-  public Future<Void> restoreAsync(RestoreRequest request) throws IOException;
-
-  /**
-   * Describe backup image command
-   * @param backupId - backup id
-   * @return backup info
-   * @throws IOException exception
-   */
-  public BackupInfo getBackupInfo(String backupId) throws IOException;
-
-  /**
-   * Show backup progress command
-   * @param backupId - backup id (may be null)
-   * @return backup progress (0-100%), -1 if no active sessions
-   *  or session not found
-   * @throws IOException exception
-   */
-  public int getProgress(String backupId) throws IOException;
-
-  /**
-   * Delete backup image command
-   * @param backupIds - backup id
-   * @return total number of deleted sessions
-   * @throws IOException exception
-   */
-  public int deleteBackups(String[] backupIds) throws IOException;
-
-  /**
-   * Show backup history command
-   * @param n - last n backup sessions
-   * @return list of backup infos
-   * @throws IOException exception
-   */
-  public List<BackupInfo> getHistory(int n) throws IOException;
-
-
-  /**
-   * Show backup history command with filters
-   * @param n - last n backup sessions
-   * @param f - list of filters
-   * @return list of backup infos
-   * @throws IOException exception
-   */
-  public List<BackupInfo> getHistory(int n, BackupInfo.Filter ... f) throws IOException;
-
-  
-  /**
-   * Backup sets list command - list all backup sets. Backup set is 
-   * a named group of tables. 
-   * @return all registered backup sets
-   * @throws IOException exception
-   */
-  public List<BackupSet> listBackupSets() throws IOException;
-
-  /**
-   * Backup set describe command. Shows list of tables in
-   * this particular backup set.
-   * @param name set name
-   * @return backup set description or null
-   * @throws IOException exception
-   */
-  public BackupSet getBackupSet(String name) throws IOException;
-
-  /**
-   * Delete backup set command
-   * @param name - backup set name
-   * @return true, if success, false - otherwise 
-   * @throws IOException exception
-   */
-  public boolean deleteBackupSet(String name) throws IOException;
-
-  /**
-   * Add tables to backup set command
-   * @param name - name of backup set.
-   * @param tables - list of tables to be added to this set.
-   * @throws IOException exception
-   */
-  public void addToBackupSet(String name, TableName[] tables) throws IOException;
-
-  /**
-   * Remove tables from backup set
-   * @param name - name of backup set.
-   * @param tables - list of tables to be removed from this set.
-   * @throws IOException exception
-   */
-  public void removeFromBackupSet(String name, String[] tables) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 1ac43f9..c8367b9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1400,19 +1400,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
         return stub.listProcedures(controller, request);
       }
 
-      @Override
-      public MasterProtos.BackupTablesResponse backupTables(
-          RpcController controller,
-          MasterProtos.BackupTablesRequest request)  throws ServiceException {
-        return stub.backupTables(controller, request);
-      }
-
-      @Override
-      public MasterProtos.RestoreTablesResponse restoreTables(
-          RpcController controller,
-          MasterProtos.RestoreTablesRequest request)  throws ServiceException {
-        return stub.restoreTables(controller, request);
-      }
 
       @Override
       public MasterProtos.AddColumnResponse addColumn(

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index aa4b3f6..f6ee79a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -61,9 +61,6 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.backup.BackupRequest;
-import org.apache.hadoop.hbase.backup.RestoreRequest;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
@@ -86,8 +83,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterReque
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesResponse;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
@@ -149,8 +144,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRespon
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreTablesRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreTablesResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
@@ -219,9 +212,7 @@ public class HBaseAdmin implements Admin {
   // numRetries is for 'normal' stuff... Multiply by this factor when
   // want to wait a long time.
   private final int retryLongerMultiplier;
-  private final int syncWaitTimeout;
-  private final long backupWaitTimeout;
-  private final long restoreWaitTimeout;
+  private final int syncWaitTimeout; 
   private boolean aborted;
   private int operationTimeout;
 
@@ -248,10 +239,6 @@ public class HBaseAdmin implements Admin {
         HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
     this.syncWaitTimeout = this.conf.getInt(
       "hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
-    this.backupWaitTimeout = this.conf.getInt(
-      "hbase.client.backup.wait.timeout.sec", 24 * 3600); // 24 h
-    this.restoreWaitTimeout = this.conf.getInt(
-        "hbase.client.restore.wait.timeout.sec", 24 * 3600); // 24 h
     this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
 
     this.ng = this.connection.getNonceGenerator();
@@ -1571,112 +1558,112 @@ public class HBaseAdmin implements Admin {
     ProtobufUtil.split(admin, hri, splitPoint);
   }
 
-  Future<String> backupTablesAsync(final BackupRequest userRequest) throws IOException {
-    BackupClientUtil.checkTargetDir(userRequest.getTargetRootDir(), conf);
-    if (userRequest.getTableList() != null) {
-      for (TableName table : userRequest.getTableList()) {
-        if (!tableExists(table)) {
-          throw new DoNotRetryIOException(table + "does not exist");
-        }
-      }
-    }
-    BackupTablesResponse response = executeCallable(
-      new MasterCallable<BackupTablesResponse>(getConnection()) {
-        @Override
-        public BackupTablesResponse call(int callTimeout) throws ServiceException {
-          BackupTablesRequest request = RequestConverter.buildBackupTablesRequest(
-            userRequest.getBackupType(), userRequest.getTableList(), userRequest.getTargetRootDir(),
-            userRequest.getWorkers(), userRequest.getBandwidth(), 
-            userRequest.getBackupSetName(), ng.getNonceGroup(),ng.newNonce());
-          return master.backupTables(null, request);
-        }
-      }, (int) backupWaitTimeout);
-    return new TableBackupFuture(this, TableName.BACKUP_TABLE_NAME, response);
-  }
-
-  String backupTables(final BackupRequest userRequest) throws IOException {
-    return get(
-      backupTablesAsync(userRequest),
-      backupWaitTimeout,
-      TimeUnit.SECONDS);
-  }
-
-  public static class TableBackupFuture extends TableFuture<String> {
-    String backupId;
-    public TableBackupFuture(final HBaseAdmin admin, final TableName tableName,
-        final BackupTablesResponse response) {
-      super(admin, tableName,
-          (response != null && response.hasProcId()) ? response.getProcId() : null);
-      backupId = response.getBackupId();
-    }
-
-    String getBackupId() {
-      return backupId;
-    }
-
-    @Override
-    public String getOperationType() {
-      return "BACKUP";
-    }
-
-    @Override
-    protected String convertResult(final GetProcedureResultResponse response) throws IOException {
-      if (response.hasException()) {
-        throw ForeignExceptionUtil.toIOException(response.getException());
-      }
-      ByteString result = response.getResult();
-      if (result == null) return null;
-      return Bytes.toStringBinary(result.toByteArray());
-    }
-
-    @Override
-    protected String postOperationResult(final String result,
-      final long deadlineTs) throws IOException, TimeoutException {
-      return result;
-    }
-  }
-
-  /**
-   * Restore operation.
-   * @param request RestoreRequest instance
-   * @throws IOException
-   */
-  public Future<Void> restoreTablesAsync(final RestoreRequest userRequest) throws IOException {
-    RestoreTablesResponse response = executeCallable(
-      new MasterCallable<RestoreTablesResponse>(getConnection()) {
-        @Override
-        public RestoreTablesResponse call(int callTimeout) throws ServiceException {
-          try {
-            RestoreTablesRequest request = RequestConverter.buildRestoreTablesRequest(
-                userRequest.getBackupRootDir(), userRequest.getBackupId(),
-                userRequest.isCheck(), userRequest.getFromTables(), userRequest.getToTables(),
-                userRequest.isOverwrite(), ng.getNonceGroup(), ng.newNonce());
-            return master.restoreTables(null, request);
-          } catch (IOException ioe) {
-            throw new ServiceException(ioe);
-          }
-        }
-      });
-    return new TableRestoreFuture(this, TableName.BACKUP_TABLE_NAME, response);
-  }
-
-  public void restoreTables(final RestoreRequest userRequest) throws IOException {
-    get(restoreTablesAsync(userRequest),
-        restoreWaitTimeout, TimeUnit.SECONDS);
-  }
-
-  private static class TableRestoreFuture extends TableFuture<Void> {
-    public TableRestoreFuture(final HBaseAdmin admin, final TableName tableName,
-        final RestoreTablesResponse response) {
-      super(admin, tableName,
-          (response != null) ? response.getProcId() : null);
-    }
-
-    @Override
-    public String getOperationType() {
-      return "RESTORE";
-    }
-  }
+//  Future<String> backupTablesAsync(final BackupRequest userRequest) throws IOException {
+//    BackupClientUtil.checkTargetDir(userRequest.getTargetRootDir(), conf);
+//    if (userRequest.getTableList() != null) {
+//      for (TableName table : userRequest.getTableList()) {
+//        if (!tableExists(table)) {
+//          throw new DoNotRetryIOException(table + "does not exist");
+//        }
+//      }
+//    }
+//    BackupTablesResponse response = executeCallable(
+//      new MasterCallable<BackupTablesResponse>(getConnection()) {
+//        @Override
+//        public BackupTablesResponse call(int callTimeout) throws ServiceException {
+//          BackupTablesRequest request = RequestConverter.buildBackupTablesRequest(
+//            userRequest.getBackupType(), userRequest.getTableList(), userRequest.getTargetRootDir(),
+//            userRequest.getWorkers(), userRequest.getBandwidth(), 
+//            userRequest.getBackupSetName(), ng.getNonceGroup(),ng.newNonce());
+//          return master.backupTables(null, request);
+//        }
+//      }, (int) backupWaitTimeout);
+//    return new TableBackupFuture(this, TableName.BACKUP_TABLE_NAME, response);
+//  }
+//
+//  String backupTables(final BackupRequest userRequest) throws IOException {
+//    return get(
+//      backupTablesAsync(userRequest),
+//      backupWaitTimeout,
+//      TimeUnit.SECONDS);
+//  }
+//
+//  public static class TableBackupFuture extends TableFuture<String> {
+//    String backupId;
+//    public TableBackupFuture(final HBaseAdmin admin, final TableName tableName,
+//        final BackupTablesResponse response) {
+//      super(admin, tableName,
+//          (response != null && response.hasProcId()) ? response.getProcId() : null);
+//      backupId = response.getBackupId();
+//    }
+//
+//    String getBackupId() {
+//      return backupId;
+//    }
+//
+//    @Override
+//    public String getOperationType() {
+//      return "BACKUP";
+//    }
+//
+//    @Override
+//    protected String convertResult(final GetProcedureResultResponse response) throws IOException {
+//      if (response.hasException()) {
+//        throw ForeignExceptionUtil.toIOException(response.getException());
+//      }
+//      ByteString result = response.getResult();
+//      if (result == null) return null;
+//      return Bytes.toStringBinary(result.toByteArray());
+//    }
+//
+//    @Override
+//    protected String postOperationResult(final String result,
+//      final long deadlineTs) throws IOException, TimeoutException {
+//      return result;
+//    }
+//  }
+//
+//  /**
+//   * Restore operation.
+//   * @param request RestoreRequest instance
+//   * @throws IOException
+//   */
+//  public Future<Void> restoreTablesAsync(final RestoreRequest userRequest) throws IOException {
+//    RestoreTablesResponse response = executeCallable(
+//      new MasterCallable<RestoreTablesResponse>(getConnection()) {
+//        @Override
+//        public RestoreTablesResponse call(int callTimeout) throws ServiceException {
+//          try {
+//            RestoreTablesRequest request = RequestConverter.buildRestoreTablesRequest(
+//                userRequest.getBackupRootDir(), userRequest.getBackupId(),
+//                userRequest.isCheck(), userRequest.getFromTables(), userRequest.getToTables(),
+//                userRequest.isOverwrite(), ng.getNonceGroup(), ng.newNonce());
+//            return master.restoreTables(null, request);
+//          } catch (IOException ioe) {
+//            throw new ServiceException(ioe);
+//          }
+//        }
+//      });
+//    return new TableRestoreFuture(this, TableName.BACKUP_TABLE_NAME, response);
+//  }
+//
+//  public void restoreTables(final RestoreRequest userRequest) throws IOException {
+//    get(restoreTablesAsync(userRequest),
+//        restoreWaitTimeout, TimeUnit.SECONDS);
+//  }
+//
+//  private static class TableRestoreFuture extends TableFuture<Void> {
+//    public TableRestoreFuture(final HBaseAdmin admin, final TableName tableName,
+//        final RestoreTablesResponse response) {
+//      super(admin, tableName,
+//          (response != null) ? response.getProcId() : null);
+//    }
+//
+//    @Override
+//    public String getOperationType() {
+//      return "RESTORE";
+//    }
+//  }
 
   @Override
   public Future<Void> modifyTable(final TableName tableName, final HTableDescriptor htd)
@@ -3544,9 +3531,5 @@ public class HBaseAdmin implements Admin {
             HConstants.EMPTY_END_ROW, false, 0);
   }
 
-  @Override
-  public BackupAdmin getBackupAdmin() throws IOException {
-    return new HBaseBackupAdmin(this);
-  }
 
 }


[10/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Posted by te...@apache.org.
HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)


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

Branch: refs/heads/HBASE-7912
Commit: b14e2ab1c24e65ff88dd4c579acf83cb4ed0605e
Parents: e35f7b9
Author: tedyu <yu...@gmail.com>
Authored: Wed Oct 5 16:29:40 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Oct 5 16:29:40 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/backup/BackupInfo.java  |   504 -
 .../hadoop/hbase/backup/BackupRequest.java      |    91 -
 .../hadoop/hbase/backup/BackupStatus.java       |   104 -
 .../hadoop/hbase/backup/RestoreRequest.java     |    94 -
 .../hbase/backup/impl/BackupCommands.java       |   717 -
 .../hbase/backup/impl/BackupException.java      |    86 -
 .../hbase/backup/impl/BackupManifest.java       |   791 -
 .../backup/impl/BackupRestoreConstants.java     |    47 -
 .../hbase/backup/impl/BackupSystemTable.java    |   873 -
 .../backup/impl/BackupSystemTableHelper.java    |   433 -
 .../hbase/backup/util/BackupClientUtil.java     |   437 -
 .../hadoop/hbase/backup/util/BackupSet.java     |    62 -
 .../org/apache/hadoop/hbase/client/Admin.java   |     9 -
 .../apache/hadoop/hbase/client/BackupAdmin.java |   174 -
 .../hbase/client/ConnectionImplementation.java  |    13 -
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   231 +-
 .../hadoop/hbase/client/HBaseBackupAdmin.java   |   439 -
 .../hadoop/hbase/protobuf/RequestConverter.java |    44 -
 .../ClientSnapshotDescriptionUtils.java         |     3 +-
 .../hbase/IntegrationTestBackupRestore.java     |    10 +-
 .../hbase/protobuf/generated/BackupProtos.java  | 15632 ++++++++---------
 .../hbase/protobuf/generated/MasterProtos.java  |  5893 +------
 hbase-protocol/src/main/protobuf/Backup.proto   |     8 +-
 hbase-protocol/src/main/protobuf/Master.proto   |    44 -
 .../apache/hadoop/hbase/backup/BackupAdmin.java |   171 +
 .../apache/hadoop/hbase/backup/BackupInfo.java  |   504 +
 .../hadoop/hbase/backup/BackupRequest.java      |    91 +
 .../hadoop/hbase/backup/BackupStatus.java       |   104 +
 .../hadoop/hbase/backup/RestoreDriver.java      |     4 +-
 .../hadoop/hbase/backup/RestoreRequest.java     |    94 +
 .../hbase/backup/impl/BackupCommands.java       |   720 +
 .../hbase/backup/impl/BackupException.java      |    86 +
 .../hbase/backup/impl/BackupManifest.java       |   791 +
 .../backup/impl/BackupRestoreConstants.java     |    47 +
 .../hbase/backup/impl/BackupSystemTable.java    |   926 +
 .../backup/impl/BackupSystemTableHelper.java    |   433 +
 .../backup/impl/FullTableBackupClient.java      |   540 +
 .../hbase/backup/impl/HBaseBackupAdmin.java     |   555 +
 .../backup/impl/IncrementalBackupManager.java   |    27 +-
 .../impl/IncrementalTableBackupClient.java      |   235 +
 .../hbase/backup/impl/RestoreTablesClient.java  |   236 +
 .../backup/master/FullTableBackupProcedure.java |   777 -
 .../master/IncrementalTableBackupProcedure.java |   400 -
 .../backup/master/RestoreTablesProcedure.java   |   387 -
 .../hbase/backup/util/BackupClientUtil.java     |   437 +
 .../hbase/backup/util/BackupServerUtil.java     |     3 +-
 .../hadoop/hbase/backup/util/BackupSet.java     |    62 +
 .../hbase/backup/util/RestoreServerUtil.java    |    85 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   119 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |    42 -
 .../hadoop/hbase/master/MasterServices.java     |    30 -
 .../hadoop/hbase/backup/TestBackupBase.java     |    24 +-
 .../hbase/backup/TestBackupBoundaryTests.java   |     8 +-
 .../hbase/backup/TestBackupDeleteRestore.java   |     1 -
 .../hbase/backup/TestBackupMultipleDeletes.java |    20 +-
 .../hadoop/hbase/backup/TestFullRestore.java    |     1 -
 .../hbase/backup/TestIncrementalBackup.java     |    61 +-
 .../TestIncrementalBackupDeleteTable.java       |     8 +-
 .../hadoop/hbase/backup/TestRemoteBackup.java   |     1 -
 .../hadoop/hbase/master/TestCatalogJanitor.java |    17 -
 60 files changed, 13746 insertions(+), 21040 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
deleted file mode 100644
index be5ffea..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
+++ /dev/null
@@ -1,504 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupInfo.Builder;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.TableBackupStatus;
-import org.apache.hadoop.hbase.util.Bytes;
-
-
-/**
- * An object to encapsulate the information for each backup request
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BackupInfo implements Comparable<BackupInfo> {
-  private static final Log LOG = LogFactory.getLog(BackupInfo.class);
-
-  public static interface Filter {
-    
-    /**
-     * Filter interface
-     * @param info: backup info
-     * @return true if info passes filter, false otherwise 
-     */
-    public boolean apply(BackupInfo info);
-  }
-  // backup status flag
-  public static enum BackupState {
-    WAITING, RUNNING, COMPLETE, FAILED, ANY;
-  }
-
-  // backup phase
-  public static enum BackupPhase {
-    SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
-  }
-
-  // backup id: a timestamp when we request the backup
-  private String backupId;
-
-  // backup type, full or incremental
-  private BackupType type;
-
-  // target root directory for storing the backup files
-  private String targetRootDir;
-
-  // overall backup state
-  private BackupState state;
-
-  // overall backup phase
-  private BackupPhase phase;
-
-  // overall backup failure message
-  private String failedMsg;
-
-  // backup status map for all tables
-  private Map<TableName, BackupStatus> backupStatusMap;
-
-  // actual start timestamp of the backup process
-  private long startTs;
-
-  // actual end timestamp of the backup process, could be fail or complete
-  private long endTs;
-
-  // the total bytes of incremental logs copied
-  private long totalBytesCopied;
-
-  // for incremental backup, the location of the backed-up hlogs
-  private String hlogTargetDir = null;
-
-  // incremental backup file list
-  transient private List<String> incrBackupFileList;
-
-  // new region server log timestamps for table set after distributed log roll
-  // key - table name, value - map of RegionServer hostname -> last log rolled timestamp
-  transient private HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap;
-
-  // backup progress in %% (0-100)
-  private int progress;
-
-  // distributed job id
-  private String jobId;
-
-  // Number of parallel workers. -1 - system defined
-  private int workers = -1;
-
-  // Bandwidth per worker in MB per sec. -1 - unlimited
-  private long bandwidth = -1;
-
-  public BackupInfo() {
-    backupStatusMap = new HashMap<TableName, BackupStatus>();
-  }
-
-  public BackupInfo(String backupId, BackupType type, TableName[] tables, String targetRootDir) {
-    this();
-    this.backupId = backupId;
-    this.type = type;
-    this.targetRootDir = targetRootDir;
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("CreateBackupContext: " + tables.length + " " + tables[0]);
-    }
-    this.addTables(tables);
-
-    if (type == BackupType.INCREMENTAL) {
-      setHlogTargetDir(BackupClientUtil.getLogBackupDir(targetRootDir, backupId));
-    }
-
-    this.startTs = 0;
-    this.endTs = 0;
-  }
-
-  public String getJobId() {
-    return jobId;
-  }
-
-  public void setJobId(String jobId) {
-    this.jobId = jobId;
-  }
-
-  public int getWorkers() {
-    return workers;
-  }
-
-  public void setWorkers(int workers) {
-    this.workers = workers;
-  }
-
-  public long getBandwidth() {
-    return bandwidth;
-  }
-
-  public void setBandwidth(long bandwidth) {
-    this.bandwidth = bandwidth;
-  }
-
-  public void setBackupStatusMap(Map<TableName, BackupStatus> backupStatusMap) {
-    this.backupStatusMap = backupStatusMap;
-  }
-
-  public HashMap<TableName, HashMap<String, Long>> getTableSetTimestampMap() {
-    return tableSetTimestampMap;
-  }
-
-  public void
-      setTableSetTimestampMap(HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap) {
-    this.tableSetTimestampMap = tableSetTimestampMap;
-  }
-
-  public String getHlogTargetDir() {
-    return hlogTargetDir;
-  }
-
-  public void setType(BackupType type) {
-    this.type = type;
-  }
-
-  public void setTargetRootDir(String targetRootDir) {
-    this.targetRootDir = targetRootDir;
-  }
-
-  public void setTotalBytesCopied(long totalBytesCopied) {
-    this.totalBytesCopied = totalBytesCopied;
-  }
-
-  /**
-   * Set progress (0-100%)
-   * @param msg progress value
-   */
-
-  public void setProgress(int p) {
-    this.progress = p;
-  }
-
-  /**
-   * Get current progress
-   */
-  public int getProgress() {
-    return progress;
-  }
-
-  public String getBackupId() {
-    return backupId;
-  }
-
-  public void setBackupId(String backupId) {
-    this.backupId = backupId;
-  }
-
-  public BackupStatus getBackupStatus(TableName table) {
-    return this.backupStatusMap.get(table);
-  }
-
-  public String getFailedMsg() {
-    return failedMsg;
-  }
-
-  public void setFailedMsg(String failedMsg) {
-    this.failedMsg = failedMsg;
-  }
-
-  public long getStartTs() {
-    return startTs;
-  }
-
-  public void setStartTs(long startTs) {
-    this.startTs = startTs;
-  }
-
-  public long getEndTs() {
-    return endTs;
-  }
-
-  public void setEndTs(long endTs) {
-    this.endTs = endTs;
-  }
-
-  public long getTotalBytesCopied() {
-    return totalBytesCopied;
-  }
-
-  public BackupState getState() {
-    return state;
-  }
-
-  public void setState(BackupState flag) {
-    this.state = flag;
-  }
-
-  public BackupPhase getPhase() {
-    return phase;
-  }
-
-  public void setPhase(BackupPhase phase) {
-    this.phase = phase;
-  }
-
-  public BackupType getType() {
-    return type;
-  }
-
-  public void setSnapshotName(TableName table, String snapshotName) {
-    this.backupStatusMap.get(table).setSnapshotName(snapshotName);
-  }
-
-  public String getSnapshotName(TableName table) {
-    return this.backupStatusMap.get(table).getSnapshotName();
-  }
-
-  public List<String> getSnapshotNames() {
-    List<String> snapshotNames = new ArrayList<String>();
-    for (BackupStatus backupStatus : this.backupStatusMap.values()) {
-      snapshotNames.add(backupStatus.getSnapshotName());
-    }
-    return snapshotNames;
-  }
-
-  public Set<TableName> getTables() {
-    return this.backupStatusMap.keySet();
-  }
-
-  public List<TableName> getTableNames() {
-    return new ArrayList<TableName>(backupStatusMap.keySet());
-  }
-
-  public void addTables(TableName[] tables) {
-    for (TableName table : tables) {
-      BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId);
-      this.backupStatusMap.put(table, backupStatus);
-    }
-  }
-
-  public void setTables(List<TableName> tables) {
-    this.backupStatusMap.clear();
-    for (TableName table : tables) {
-      BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId);
-      this.backupStatusMap.put(table, backupStatus);
-    }
-  }
-
-  public String getTargetRootDir() {
-    return targetRootDir;
-  }
-
-  public void setHlogTargetDir(String hlogTagetDir) {
-    this.hlogTargetDir = hlogTagetDir;
-  }
-
-  public String getHLogTargetDir() {
-    return hlogTargetDir;
-  }
-
-  public List<String> getIncrBackupFileList() {
-    return incrBackupFileList;
-  }
-
-  public void setIncrBackupFileList(List<String> incrBackupFileList) {
-    this.incrBackupFileList = incrBackupFileList;
-  }
-
-  /**
-   * Set the new region server log timestamps after distributed log roll
-   * @param newTableSetTimestampMap table timestamp map
-   */
-  public void
-      setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap) {
-    this.tableSetTimestampMap = newTableSetTimestampMap;
-  }
-
-  /**
-   * Get new region server log timestamps after distributed log roll
-   * @return new region server log timestamps
-   */
-  public HashMap<TableName, HashMap<String, Long>> getIncrTimestampMap() {
-    return this.tableSetTimestampMap;
-  }
-
-  public TableName getTableBySnapshot(String snapshotName) {
-    for (Entry<TableName, BackupStatus> entry : this.backupStatusMap.entrySet()) {
-      if (snapshotName.equals(entry.getValue().getSnapshotName())) {
-        return entry.getKey();
-      }
-    }
-    return null;
-  }
-
-  public BackupProtos.BackupInfo toProtosBackupInfo() {
-    BackupProtos.BackupInfo.Builder builder = BackupProtos.BackupInfo.newBuilder();
-    builder.setBackupId(getBackupId());
-    setBackupStatusMap(builder);
-    builder.setEndTs(getEndTs());
-    if (getFailedMsg() != null) {
-      builder.setFailedMessage(getFailedMsg());
-    }
-    if (getState() != null) {
-      builder.setState(BackupProtos.BackupInfo.BackupState.valueOf(getState().name()));
-    }
-    if (getPhase() != null) {
-      builder.setPhase(BackupProtos.BackupInfo.BackupPhase.valueOf(getPhase().name()));
-    }
-
-    builder.setProgress(getProgress());
-    builder.setStartTs(getStartTs());
-    builder.setTargetRootDir(getTargetRootDir());
-    builder.setType(BackupProtos.BackupType.valueOf(getType().name()));
-    builder.setWorkersNumber(workers);
-    builder.setBandwidth(bandwidth);
-    if (jobId != null) {
-      builder.setJobId(jobId);
-    }
-    return builder.build();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof BackupInfo) {
-      BackupInfo other = (BackupInfo) obj;
-      try {
-        return Bytes.equals(toByteArray(), other.toByteArray());
-      } catch (IOException e) {
-        LOG.error(e);
-        return false;
-      }
-    } else {
-      return false;
-    }
-  }
-
-  public byte[] toByteArray() throws IOException {
-    return toProtosBackupInfo().toByteArray();
-  }
-
-  private void setBackupStatusMap(Builder builder) {
-    for (Entry<TableName, BackupStatus> entry : backupStatusMap.entrySet()) {
-      builder.addTableBackupStatus(entry.getValue().toProto());
-    }
-  }
-
-  public static BackupInfo fromByteArray(byte[] data) throws IOException {
-    return fromProto(BackupProtos.BackupInfo.parseFrom(data));
-  }
-
-  public static BackupInfo fromStream(final InputStream stream) throws IOException {
-    return fromProto(BackupProtos.BackupInfo.parseDelimitedFrom(stream));
-  }
-
-  public static BackupInfo fromProto(BackupProtos.BackupInfo proto) {
-    BackupInfo context = new BackupInfo();
-    context.setBackupId(proto.getBackupId());
-    context.setBackupStatusMap(toMap(proto.getTableBackupStatusList()));
-    context.setEndTs(proto.getEndTs());
-    if (proto.hasFailedMessage()) {
-      context.setFailedMsg(proto.getFailedMessage());
-    }
-    if (proto.hasState()) {
-      context.setState(BackupInfo.BackupState.valueOf(proto.getState().name()));
-    }
-
-    context.setHlogTargetDir(BackupClientUtil.getLogBackupDir(proto.getTargetRootDir(),
-      proto.getBackupId()));
-
-    if (proto.hasPhase()) {
-      context.setPhase(BackupPhase.valueOf(proto.getPhase().name()));
-    }
-    if (proto.hasProgress()) {
-      context.setProgress(proto.getProgress());
-    }
-    context.setStartTs(proto.getStartTs());
-    context.setTargetRootDir(proto.getTargetRootDir());
-    context.setType(BackupType.valueOf(proto.getType().name()));
-    context.setWorkers(proto.getWorkersNumber());
-    context.setBandwidth(proto.getBandwidth());
-    if (proto.hasJobId()) {
-      context.setJobId(proto.getJobId());
-    }
-    return context;
-  }
-
-  private static Map<TableName, BackupStatus> toMap(List<TableBackupStatus> list) {
-    HashMap<TableName, BackupStatus> map = new HashMap<>();
-    for (TableBackupStatus tbs : list) {
-      map.put(ProtobufUtil.toTableName(tbs.getTable()), BackupStatus.convert(tbs));
-    }
-    return map;
-  }
-
-  public String getShortDescription() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("ID             : " + backupId).append("\n");
-    sb.append("Type           : " + getType()).append("\n");
-    sb.append("Tables         : " + getTableListAsString()).append("\n");
-    sb.append("State          : " + getState()).append("\n");
-    Date date = null;
-    Calendar cal = Calendar.getInstance();
-    cal.setTimeInMillis(getStartTs());
-    date = cal.getTime();
-    sb.append("Start time     : " + date).append("\n");
-    if (state == BackupState.FAILED) {
-      sb.append("Failed message : " + getFailedMsg()).append("\n");
-    } else if (state == BackupState.RUNNING) {
-      sb.append("Phase          : " + getPhase()).append("\n");
-    } else if (state == BackupState.COMPLETE) {
-      cal = Calendar.getInstance();
-      cal.setTimeInMillis(getEndTs());
-      date = cal.getTime();
-      sb.append("End time       : " + date).append("\n");
-    }
-    sb.append("Progress       : " + getProgress()).append("\n");
-    return sb.toString();
-  }
-
-  public String getStatusAndProgressAsString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("id: ").append(getBackupId()).append(" state: ").append(getState())
-        .append(" progress: ").append(getProgress());
-    return sb.toString();
-  }
-
-  public String getTableListAsString() {
-    return StringUtils.join(backupStatusMap.keySet(), ",");
-  }
-
-  @Override
-  public int compareTo(BackupInfo o) {
-    Long thisTS = new Long(this.getBackupId().substring(this.getBackupId().lastIndexOf("_") + 1));
-    Long otherTS = new Long(o.getBackupId().substring(o.getBackupId().lastIndexOf("_") + 1));
-    return thisTS.compareTo(otherTS);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
deleted file mode 100644
index d141239..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.util.List;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * POJO class for backup request
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class BackupRequest {
-  private BackupType type;
-  private List<TableName> tableList;
-  private String targetRootDir;
-  private int workers = -1;
-  private long bandwidth = -1L;
-  private String backupSetName;
-
-  public BackupRequest() {
-  }
-
-  public BackupRequest setBackupType(BackupType type) {
-    this.type = type;
-    return this;
-  }
-  public BackupType getBackupType() {
-    return this.type;
-  }
-
-  public BackupRequest setTableList(List<TableName> tableList) {
-    this.tableList = tableList;
-    return this;
-  }
-  public List<TableName> getTableList() {
-    return this.tableList;
-  }
-
-  public BackupRequest setTargetRootDir(String targetRootDir) {
-    this.targetRootDir = targetRootDir;
-    return this;
-  }
-  public String getTargetRootDir() {
-    return this.targetRootDir;
-  }
-
-  public BackupRequest setWorkers(int workers) {
-    this.workers = workers;
-    return this;
-  }
-  public int getWorkers() {
-    return this.workers;
-  }
-
-  public BackupRequest setBandwidth(long bandwidth) {
-    this.bandwidth = bandwidth;
-    return this;
-  }
-  public long getBandwidth() {
-    return this.bandwidth;
-  }
-
-  public String getBackupSetName() {
-    return backupSetName;
-  }
-
-  public void setBackupSetName(String backupSetName) {
-    this.backupSetName = backupSetName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
deleted file mode 100644
index c82e05a..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.Serializable;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
-
-/**
- * Backup status and related information encapsulated for a table.
- * At this moment only TargetDir and SnapshotName is encapsulated here.
- */
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BackupStatus implements Serializable {
-
-  private static final long serialVersionUID = -5968397963548535982L;
-
-  // table name for backup
-  private TableName table;
-
-  // target directory of the backup image for this table
-  private String targetDir;
-
-  // snapshot name for offline/online snapshot
-  private String snapshotName = null;
-
-  public BackupStatus() {
-
-  }
-
-  public BackupStatus(TableName table, String targetRootDir, String backupId) {
-    this.table = table;
-    this.targetDir = BackupClientUtil.getTableBackupDir(targetRootDir, backupId, table);
-  }
-
-  public String getSnapshotName() {
-    return snapshotName;
-  }
-
-  public void setSnapshotName(String snapshotName) {
-    this.snapshotName = snapshotName;
-  }
-
-  public String getTargetDir() {
-    return targetDir;
-  }
-
-  public TableName getTable() {
-    return table;
-  }
-
-  public void setTable(TableName table) {
-    this.table = table;
-  }
-
-  public void setTargetDir(String targetDir) {
-    this.targetDir = targetDir;
-  }
-
-  public static BackupStatus convert(BackupProtos.TableBackupStatus proto)
-  {
-    BackupStatus bs = new BackupStatus();
-    bs.setTable(ProtobufUtil.toTableName(proto.getTable()));
-    bs.setTargetDir(proto.getTargetDir());
-    if(proto.hasSnapshot()){
-      bs.setSnapshotName(proto.getSnapshot());
-    }
-    return bs;
-  }
-
-  public BackupProtos.TableBackupStatus toProto() {
-    BackupProtos.TableBackupStatus.Builder builder =
-        BackupProtos.TableBackupStatus.newBuilder();
-    if(snapshotName != null) {
-      builder.setSnapshot(snapshotName);
-    }
-    builder.setTable(ProtobufUtil.toProtoTableName(table));
-    builder.setTargetDir(targetDir);
-    return builder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
deleted file mode 100644
index 7490d20..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * POJO class for restore request
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class RestoreRequest {
-
-  private String backupRootDir;
-  private String backupId;
-  private boolean check = false;
-  private TableName[] fromTables;
-  private TableName[] toTables;
-  private boolean overwrite = false;
-
-  public RestoreRequest() {
-  }
-
-  public String getBackupRootDir() {
-    return backupRootDir;
-  }
-
-  public RestoreRequest setBackupRootDir(String backupRootDir) {
-    this.backupRootDir = backupRootDir;
-    return this;
-  }
-
-  public String getBackupId() {
-    return backupId;
-  }
-
-  public RestoreRequest setBackupId(String backupId) {
-    this.backupId = backupId;
-    return this;
-  }
-
-  public boolean isCheck() {
-    return check;
-  }
-
-  public RestoreRequest setCheck(boolean check) {
-    this.check = check;
-    return this;
-  }
-
-  public TableName[] getFromTables() {
-    return fromTables;
-  }
-
-  public RestoreRequest setFromTables(TableName[] fromTables) {
-    this.fromTables = fromTables;
-    return this;
-  }
-
-  public TableName[] getToTables() {
-    return toTables;
-  }
-
-  public RestoreRequest setToTables(TableName[] toTables) {
-    this.toTables = toTables;
-    return this;
-  }
-
-  public boolean isOverwrite() {
-    return overwrite;
-  }
-
-  public RestoreRequest setOverwrite(boolean overwrite) {
-    this.overwrite = overwrite;
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
deleted file mode 100644
index 2ff5756..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
+++ /dev/null
@@ -1,717 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupRequest;
-import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants.BackupCommand;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.backup.util.BackupSet;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.BackupAdmin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-
-import com.google.common.collect.Lists;
-
-/**
- * General backup commands, options and usage messages
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class BackupCommands {
-  
-  public final static String INCORRECT_USAGE = "Incorrect usage";
-
-  public static final String USAGE = "Usage: hbase backup COMMAND [command-specific arguments]\n"
-      + "where COMMAND is one of:\n" 
-      + "  create     create a new backup image\n"
-      + "  delete     delete an existing backup image\n"
-      + "  describe   show the detailed information of a backup image\n"
-      + "  history    show history of all successful backups\n"
-      + "  progress   show the progress of the latest backup request\n"
-      + "  set        backup set management\n"
-      + "Run \'hbase backup COMMAND -h\' to see help message for each command\n";
-
-  public static final String CREATE_CMD_USAGE =
-      "Usage: hbase backup create <type> <BACKUP_ROOT> [tables] [-set name] "
-          + "[-w workers][-b bandwith]\n" 
-          + " type           \"full\" to create a full backup image\n"
-          + "                \"incremental\" to create an incremental backup image\n"
-          + " BACKUP_ROOT     The full root path to store the backup image,\n"
-          + "                 the prefix can be hdfs, webhdfs or gpfs\n" 
-          + "Options:\n"
-          + " tables          If no tables (\"\") are specified, all tables are backed up.\n"
-          + "                 Otherwise it is a comma separated list of tables.\n"
-          + " -w              number of parallel workers (MapReduce tasks).\n" 
-          + " -b              bandwith per one worker (MapReduce task) in MBs per sec\n" 
-          + " -set            name of backup set to use (mutually exclusive with [tables])" ;
-
-  public static final String PROGRESS_CMD_USAGE = "Usage: hbase backup progress <backupId>\n"
-          + " backupId        backup image id\n";
-  public static final String NO_INFO_FOUND = "No info was found for backup id: ";
-
-  public static final String DESCRIBE_CMD_USAGE = "Usage: hbase backup decsribe <backupId>\n"
-          + " backupId        backup image id\n";
-
-  public static final String HISTORY_CMD_USAGE = 
-      "Usage: hbase backup history [-path BACKUP_ROOT] [-n N] [-t table]\n"
-       + " -n N            show up to N last backup sessions, default - 10\n"
-       + " -path           backup root path\n"
-       + " -t table        table name. If specified, only backup images which contain this table\n"
-       + "                 will be listed."  ;
-  
-
-  public static final String DELETE_CMD_USAGE = "Usage: hbase backup delete <backupId>\n"
-          + " backupId        backup image id\n";
-
-  public static final String CANCEL_CMD_USAGE = "Usage: hbase backup cancel <backupId>\n"
-          + " backupId        backup image id\n";
-
-  public static final String SET_CMD_USAGE = "Usage: hbase backup set COMMAND [name] [tables]\n"
-         + " name            Backup set name\n"
-         + " tables          If no tables (\"\") are specified, all tables will belong to the set.\n"
-         + "                 Otherwise it is a comma separated list of tables.\n"
-         + "COMMAND is one of:\n" 
-         + " add             add tables to a set, create a set if needed\n"
-         + " remove          remove tables from a set\n"
-         + " list            list all backup sets in the system\n"
-         + " describe        describe set\n"
-         + " delete          delete backup set\n";
-
-  public static abstract class Command extends Configured {
-    CommandLine cmdline;
-    
-    Command(Configuration conf) {
-      super(conf);
-    }
-    
-    public void execute() throws IOException
-    {
-      if (cmdline.hasOption("h") || cmdline.hasOption("help")) {
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-    }
-    
-    protected abstract void printUsage();
-  }
-
-  private BackupCommands() {
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  public static Command createCommand(Configuration conf, BackupCommand type, CommandLine cmdline) {
-    Command cmd = null;
-    switch (type) {
-    case CREATE:
-      cmd = new CreateCommand(conf, cmdline);
-      break;
-    case DESCRIBE:
-      cmd = new DescribeCommand(conf, cmdline);
-      break;
-    case PROGRESS:
-      cmd = new ProgressCommand(conf, cmdline);
-      break;
-    case DELETE:
-      cmd = new DeleteCommand(conf, cmdline);
-      break;
-    case CANCEL:
-      cmd = new CancelCommand(conf, cmdline);
-      break;
-    case HISTORY:
-      cmd = new HistoryCommand(conf, cmdline);
-      break;
-    case SET:
-      cmd = new BackupSetCommand(conf, cmdline);
-      break;
-    case HELP:
-    default:
-      cmd = new HelpCommand(conf, cmdline);
-      break;
-    }
-    return cmd;
-  }
-
-  static int numOfArgs(String[] args) {
-    if (args == null) return 0;
-    return args.length;
-  }
-
-  public static class CreateCommand extends Command {
-
-    CreateCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-    
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-      if (cmdline == null || cmdline.getArgs() == null) {
-        System.err.println("ERROR: missing arguments");
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-      String[] args = cmdline.getArgs();
-      if (args.length < 3 || args.length > 4) {
-        System.err.println("ERROR: wrong number of arguments: "+ args.length);
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      if (!BackupType.FULL.toString().equalsIgnoreCase(args[1])
-          && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[1])) {
-        System.err.println("ERROR: invalid backup type: "+ args[1]);
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-            
-      String tables = null;
-      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
-
-      // Check backup set
-      String setName = null;
-      if (cmdline.hasOption("set")) {
-        setName = cmdline.getOptionValue("set");
-        tables = getTablesForSet(setName, conf);
-
-        if (tables == null) {
-          System.err.println("ERROR: Backup set '" + setName+ "' is either empty or does not exist");
-          printUsage();
-          throw new IOException(INCORRECT_USAGE);
-        }
-      } else {
-        tables = (args.length == 4) ? args[3] : null;
-      }
-      int bandwidth = cmdline.hasOption('b') ? Integer.parseInt(cmdline.getOptionValue('b')) : -1;
-      int workers = cmdline.hasOption('w') ? Integer.parseInt(cmdline.getOptionValue('w')) : -1;
-
-      try (Connection conn = ConnectionFactory.createConnection(getConf());
-          Admin admin = conn.getAdmin();
-          BackupAdmin backupAdmin = admin.getBackupAdmin();) {
-        BackupRequest request = new BackupRequest();
-        request.setBackupType(BackupType.valueOf(args[1].toUpperCase()))
-        .setTableList(tables != null?Lists.newArrayList(BackupClientUtil.parseTableNames(tables)): null)
-        .setTargetRootDir(args[2]).setWorkers(workers).setBandwidth(bandwidth)
-        .setBackupSetName(setName);
-        String backupId = backupAdmin.backupTables(request);
-        System.out.println("Backup session "+ backupId+" finished. Status: SUCCESS");
-      } catch (IOException e) {
-        System.err.println("Backup session finished. Status: FAILURE");
-        throw e;
-      }
-    }
-    private String getTablesForSet(String name, Configuration conf)
-        throws IOException {
-      try (final Connection conn = ConnectionFactory.createConnection(conf);
-          final BackupSystemTable table = new BackupSystemTable(conn)) {
-        List<TableName> tables = table.describeBackupSet(name);
-        if (tables == null) return null;
-        return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);        
-      }
-    }
-
-    @Override
-    protected void printUsage() {
-      System.err.println(CREATE_CMD_USAGE);      
-    }
-  }
-
-  private static class HelpCommand extends Command {
-
-    HelpCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-      if (cmdline == null) {
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      String[] args = cmdline.getArgs();
-      if (args == null || args.length == 0) {
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      if (args.length != 2) {
-        System.err.println("Only supports help message of a single command type");
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      String type = args[1];
-
-      if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) {
-        System.out.println(CREATE_CMD_USAGE);
-      } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(type)) {
-        System.out.println(DESCRIBE_CMD_USAGE);
-      } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(type)) {
-        System.out.println(HISTORY_CMD_USAGE);
-      } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(type)) {
-        System.out.println(PROGRESS_CMD_USAGE);
-      } else if (BackupCommand.DELETE.name().equalsIgnoreCase(type)) {
-        System.out.println(DELETE_CMD_USAGE);
-      } else if (BackupCommand.CANCEL.name().equalsIgnoreCase(type)) {
-        System.out.println(CANCEL_CMD_USAGE);
-      } else if (BackupCommand.SET.name().equalsIgnoreCase(type)) {
-        System.out.println(SET_CMD_USAGE);
-      } else {
-        System.out.println("Unknown command : " + type);
-        printUsage();
-      }
-    }
-
-    @Override
-    protected void printUsage() {
-      System.err.println(USAGE);      
-    }
-  }
-
-  private static class DescribeCommand extends Command {
-
-    DescribeCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-      if (cmdline == null || cmdline.getArgs() == null) {
-        System.err.println("ERROR: missing arguments");
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-      String[] args = cmdline.getArgs();
-      if (args.length != 2) {
-        System.err.println("ERROR: wrong number of arguments");
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-            
-      String backupId = args[1];
-      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
-      try (final Connection conn = ConnectionFactory.createConnection(conf);
-          final BackupAdmin admin = conn.getAdmin().getBackupAdmin();) {
-        BackupInfo info = admin.getBackupInfo(backupId);
-        if (info == null) {
-          System.err.println("ERROR: " + backupId + " does not exist");
-          printUsage();
-          throw new IOException(INCORRECT_USAGE);
-        }
-        System.out.println(info.getShortDescription());
-      }
-    }
-
-    @Override
-    protected void printUsage() {
-      System.err.println(DESCRIBE_CMD_USAGE);
-    }
-  }
-
-  private static class ProgressCommand extends Command {
-
-    ProgressCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-      
-      if (cmdline == null || cmdline.getArgs() == null ||
-          cmdline.getArgs().length == 1) {
-        System.err.println("No backup id was specified, "
-            + "will retrieve the most recent (ongoing) sessions");
-      }
-      String[] args = cmdline.getArgs();
-      if (args.length > 2) {
-        System.err.println("ERROR: wrong number of arguments: " + args.length);
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-
-      String backupId = (args == null || args.length <= 1) ? null : args[1];
-      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
-      try(final Connection conn = ConnectionFactory.createConnection(conf); 
-          final BackupAdmin admin = conn.getAdmin().getBackupAdmin();){
-        int progress = admin.getProgress(backupId);
-        if(progress < 0){
-          System.err.println(NO_INFO_FOUND + backupId);
-        } else{
-          System.out.println(backupId+" progress=" + progress+"%");
-        }
-      } 
-    }
-
-    @Override
-    protected void printUsage() {
-      System.err.println(PROGRESS_CMD_USAGE);      
-    }
-  }
-
-  private static class DeleteCommand extends Command {
-    
-    DeleteCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
-        System.err.println("No backup id(s) was specified");
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-            
-      String[] args = cmdline.getArgs();
-
-      String[] backupIds = new String[args.length - 1];
-      System.arraycopy(args, 1, backupIds, 0, backupIds.length);
-      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
-      try (final Connection conn = ConnectionFactory.createConnection(conf);
-          final BackupAdmin admin = conn.getAdmin().getBackupAdmin();) {
-        int deleted = admin.deleteBackups(args);
-        System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
-      }
-
-    }
-
-    @Override
-    protected void printUsage() {
-      System.err.println(DELETE_CMD_USAGE);      
-    }
-  }
-
-// TODO Cancel command  
-  
-  private static class CancelCommand extends Command {
-
-    CancelCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();
-      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
-        System.out.println("No backup id(s) was specified, will use the most recent one");
-      }
-      String[] args = cmdline.getArgs();
-      String backupId = args == null || args.length == 0 ? null : args[1];
-      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
-      try (final Connection conn = ConnectionFactory.createConnection(conf);
-          final BackupAdmin admin = conn.getAdmin().getBackupAdmin();) {
-        // TODO cancel backup
-      }
-    }
-
-    @Override
-    protected void printUsage() {
-    }
-  }
-
-  private static class HistoryCommand extends Command {
-    
-    private final static int DEFAULT_HISTORY_LENGTH = 10;
-    
-    HistoryCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    public void execute() throws IOException {
-
-      super.execute();
-
-      int n = parseHistoryLength();
-      final TableName tableName = getTableName();
-      final String setName = getTableSetName();
-      BackupInfo.Filter tableNameFilter = new BackupInfo.Filter() {
-        @Override
-        public boolean apply(BackupInfo info) {
-          if (tableName == null) return true;
-          List<TableName> names = info.getTableNames();
-          return names.contains(tableName);
-        }
-      };
-      BackupInfo.Filter tableSetFilter = new BackupInfo.Filter() {
-        @Override
-        public boolean apply(BackupInfo info) {
-          if (setName == null) return true;
-          String backupId = info.getBackupId();
-          return backupId.startsWith(setName);
-        }
-      };                
-      Path backupRootPath = getBackupRootPath();
-      List<BackupInfo> history = null;
-      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
-      if (backupRootPath == null) {
-        // Load from hbase:backup
-        try (final Connection conn = ConnectionFactory.createConnection(conf);
-            final BackupAdmin admin = conn.getAdmin().getBackupAdmin();) {
- 
-          history = admin.getHistory(n, tableNameFilter, tableSetFilter);
-        }
-      } else {
-        // load from backup FS
-        history = BackupClientUtil.getHistory(conf, n, backupRootPath, 
-          tableNameFilter, tableSetFilter);
-      }
-      for (BackupInfo info : history) {
-        System.out.println(info.getShortDescription());
-      }
-    }
-    
-    private Path getBackupRootPath() throws IOException {
-      String value = null;
-      try{
-        value = cmdline.getOptionValue("path");
-        if (value == null) return null;
-        return new Path(value);
-      } catch (IllegalArgumentException e) {
-        System.err.println("ERROR: Illegal argument for backup root path: "+ value);
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-    }
-
-    private TableName getTableName() throws IOException {
-      String value = cmdline.getOptionValue("t"); 
-      if (value == null) return null;
-      try{
-        return TableName.valueOf(value);
-      } catch (IllegalArgumentException e){
-        System.err.println("Illegal argument for table name: "+ value);
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-    }
-
-    private String getTableSetName() throws IOException {
-      String value = cmdline.getOptionValue("set"); 
-      return value;
-    }
-    
-    private int parseHistoryLength() throws IOException {
-      String value = cmdline.getOptionValue("n");
-      try{
-        if (value == null) return DEFAULT_HISTORY_LENGTH;
-        return Integer.parseInt(value);
-      } catch(NumberFormatException e) {
-        System.err.println("Illegal argument for history length: "+ value);
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-    }
-
-    @Override
-    protected void printUsage() {
-      System.err.println(HISTORY_CMD_USAGE);      
-    }
-  }
-
-  private static class BackupSetCommand extends Command {
-    private final static String SET_ADD_CMD = "add";
-    private final static String SET_REMOVE_CMD = "remove";
-    private final static String SET_DELETE_CMD = "delete";
-    private final static String SET_DESCRIBE_CMD = "describe";
-    private final static String SET_LIST_CMD = "list";
-
-    BackupSetCommand(Configuration conf, CommandLine cmdline) {
-      super(conf);
-      this.cmdline = cmdline;
-    }
-
-    @Override
-    public void execute() throws IOException {
-      super.execute();      
-      // Command-line must have at least one element
-      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
-        System.err.println("ERROR: Command line format");
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-            
-      String[] args = cmdline.getArgs();
-      String cmdStr = args[1];
-      BackupCommand cmd = getCommand(cmdStr);
-
-      switch (cmd) {
-      case SET_ADD:
-        processSetAdd(args);
-        break;
-      case SET_REMOVE:
-        processSetRemove(args);
-        break;
-      case SET_DELETE:
-        processSetDelete(args);
-        break;
-      case SET_DESCRIBE:
-        processSetDescribe(args);
-        break;
-      case SET_LIST:
-        processSetList(args);
-        break;
-      default:
-        break;
-
-      }
-    }
-
-    private void processSetList(String[] args) throws IOException {
-      // List all backup set names
-      // does not expect any args
-      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
-      try(final Connection conn = ConnectionFactory.createConnection(conf); 
-          final BackupAdmin admin = conn.getAdmin().getBackupAdmin();){
-        List<BackupSet> list = admin.listBackupSets();
-        for(BackupSet bs: list){
-          System.out.println(bs);
-        }
-      }
-    }
-
-    private void processSetDescribe(String[] args) throws IOException {
-      if (args == null || args.length != 3) {
-        System.err.println("ERROR: Wrong number of args for 'set describe' command: "
-            + numOfArgs(args));
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-      String setName = args[2];
-      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
-      try(final Connection conn = ConnectionFactory.createConnection(conf); 
-          final BackupAdmin admin = conn.getAdmin().getBackupAdmin();){
-        BackupSet set = admin.getBackupSet(setName);
-        if(set == null) {
-          System.out.println("Set '"+setName+"' does not exist.");
-        } else{
-          System.out.println(set);
-        }
-      }
-    }
-
-    private void processSetDelete(String[] args) throws IOException {
-      if (args == null || args.length != 3) {
-        System.err.println("ERROR: Wrong number of args for 'set delete' command: "
-            + numOfArgs(args));
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-      String setName = args[2];
-      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
-      try(final Connection conn = ConnectionFactory.createConnection(conf); 
-          final BackupAdmin admin = conn.getAdmin().getBackupAdmin();){
-        boolean result = admin.deleteBackupSet(setName);
-        if(result){
-          System.out.println("Delete set "+setName+" OK.");
-        } else{
-          System.out.println("Set "+setName+" does not exist");
-        }
-      }
-    }
-
-    private void processSetRemove(String[] args) throws IOException {
-      if (args == null || args.length != 4) {
-        System.err.println("ERROR: Wrong number of args for 'set remove' command: "
-            + numOfArgs(args));
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-      
-      String setName = args[2];
-      String[] tables = args[3].split(",");
-      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
-      try(final Connection conn = ConnectionFactory.createConnection(conf); 
-          final BackupAdmin admin = conn.getAdmin().getBackupAdmin();){
-        admin.removeFromBackupSet(setName, tables);
-      }
-    }
-
-    private void processSetAdd(String[] args) throws IOException {
-      if (args == null || args.length != 4) {
-        System.err.println("ERROR: Wrong number of args for 'set add' command: "
-            + numOfArgs(args));
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-      String setName = args[2];
-      String[] tables = args[3].split(",");
-      TableName[] tableNames = new TableName[tables.length];
-      for(int i=0; i < tables.length; i++){
-        tableNames[i] = TableName.valueOf(tables[i]);
-      }
-      Configuration conf = getConf() != null? getConf():HBaseConfiguration.create();
-      try(final Connection conn = ConnectionFactory.createConnection(conf); 
-          final BackupAdmin admin = conn.getAdmin().getBackupAdmin();){
-        admin.addToBackupSet(setName, tableNames);
-      }
-      
-    }
-
-    private BackupCommand getCommand(String cmdStr) throws IOException {
-      if (cmdStr.equals(SET_ADD_CMD)) {
-        return BackupCommand.SET_ADD;
-      } else if (cmdStr.equals(SET_REMOVE_CMD)) {
-        return BackupCommand.SET_REMOVE;
-      } else if (cmdStr.equals(SET_DELETE_CMD)) {
-        return BackupCommand.SET_DELETE;
-      } else if (cmdStr.equals(SET_DESCRIBE_CMD)) {
-        return BackupCommand.SET_DESCRIBE;
-      } else if (cmdStr.equals(SET_LIST_CMD)) {
-        return BackupCommand.SET_LIST;
-      } else {
-        System.err.println("ERROR: Unknown command for 'set' :" + cmdStr);
-        printUsage();
-        throw new IOException(INCORRECT_USAGE);
-      }
-    }
-
-    @Override
-    protected void printUsage() {
-      System.err.println(SET_CMD_USAGE);
-    }
-
-  }  
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
deleted file mode 100644
index ca204b4..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * Backup exception
- */
-@SuppressWarnings("serial")
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BackupException extends HBaseIOException {
-  private BackupInfo description;
-
-  /**
-   * Some exception happened for a backup and don't even know the backup that it was about
-   * @param msg Full description of the failure
-   */
-  public BackupException(String msg) {
-    super(msg);
-  }
-
-  /**
-   * Some exception happened for a backup with a cause
-   * @param cause the cause
-   */
-  public BackupException(Throwable cause) {
-    super(cause);
-  }
-
-  /**
-   * Exception for the given backup that has no previous root cause
-   * @param msg reason why the backup failed
-   * @param desc description of the backup that is being failed
-   */
-  public BackupException(String msg, BackupInfo desc) {
-    super(msg);
-    this.description = desc;
-  }
-
-  /**
-   * Exception for the given backup due to another exception
-   * @param msg reason why the backup failed
-   * @param cause root cause of the failure
-   * @param desc description of the backup that is being failed
-   */
-  public BackupException(String msg, Throwable cause, BackupInfo desc) {
-    super(msg, cause);
-    this.description = desc;
-  }
-
-  /**
-   * Exception when the description of the backup cannot be determined, due to some other root
-   * cause
-   * @param message description of what caused the failure
-   * @param e root cause
-   */
-  public BackupException(String message, Exception e) {
-    super(message, e);
-  }
-
-  public BackupInfo getBackupContext() {
-    return this.description;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
deleted file mode 100644
index d10713d..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManifest.java
+++ /dev/null
@@ -1,791 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
-
-import com.google.protobuf.InvalidProtocolBufferException;
-
-
-/**
- * Backup manifest Contains all the meta data of a backup image. The manifest info will be bundled
- * as manifest file together with data. So that each backup image will contain all the info needed
- * for restore.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BackupManifest {
-
-  private static final Log LOG = LogFactory.getLog(BackupManifest.class);
-
-  // manifest file name
-  public static final String MANIFEST_FILE_NAME = ".backup.manifest";
-
-  // manifest file version, current is 1.0
-  public static final String MANIFEST_VERSION = "1.0";
-
-  // backup image, the dependency graph is made up by series of backup images
-
-  public static class BackupImage implements Comparable<BackupImage> {
-
-    private String backupId;
-    private BackupType type;
-    private String rootDir;
-    private List<TableName> tableList;
-    private long startTs;
-    private long completeTs;
-    private ArrayList<BackupImage> ancestors;
-
-    public BackupImage() {
-      super();
-    }
-
-    public BackupImage(String backupId, BackupType type, String rootDir,
-        List<TableName> tableList, long startTs, long completeTs) {
-      this.backupId = backupId;
-      this.type = type;
-      this.rootDir = rootDir;
-      this.tableList = tableList;
-      this.startTs = startTs;
-      this.completeTs = completeTs;
-    }
-
-    static BackupImage fromProto(BackupProtos.BackupImage im) {
-      String backupId = im.getBackupId();
-      String rootDir = im.getRootDir();
-      long startTs = im.getStartTs();
-      long completeTs = im.getCompleteTs();
-      List<HBaseProtos.TableName> tableListList = im.getTableListList();
-      List<TableName> tableList = new ArrayList<TableName>();
-      for(HBaseProtos.TableName tn : tableListList) {
-        tableList.add(ProtobufUtil.toTableName(tn));
-      }
-      
-      List<BackupProtos.BackupImage> ancestorList = im.getAncestorsList();
-      
-      BackupType type =
-          im.getBackupType() == BackupProtos.BackupType.FULL ? BackupType.FULL:
-            BackupType.INCREMENTAL;
-
-      BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs);
-      for(BackupProtos.BackupImage img: ancestorList) {
-        image.addAncestor(fromProto(img));
-      }
-      return image;
-    }
-
-    BackupProtos.BackupImage toProto() {
-      BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder();
-      builder.setBackupId(backupId);
-      builder.setCompleteTs(completeTs);
-      builder.setStartTs(startTs);
-      builder.setRootDir(rootDir);
-      if (type == BackupType.FULL) {
-        builder.setBackupType(BackupProtos.BackupType.FULL);
-      } else{
-        builder.setBackupType(BackupProtos.BackupType.INCREMENTAL);
-      }
-
-      for (TableName name: tableList) {
-        builder.addTableList(ProtobufUtil.toProtoTableName(name));
-      }
-
-      if (ancestors != null){
-        for (BackupImage im: ancestors){
-          builder.addAncestors(im.toProto());
-        }
-      }
-
-      return builder.build();
-    }
-
-    public String getBackupId() {
-      return backupId;
-    }
-
-    public void setBackupId(String backupId) {
-      this.backupId = backupId;
-    }
-
-    public BackupType getType() {
-      return type;
-    }
-
-    public void setType(BackupType type) {
-      this.type = type;
-    }
-
-    public String getRootDir() {
-      return rootDir;
-    }
-
-    public void setRootDir(String rootDir) {
-      this.rootDir = rootDir;
-    }
-
-    public List<TableName> getTableNames() {
-      return tableList;
-    }
-
-    public void setTableList(List<TableName> tableList) {
-      this.tableList = tableList;
-    }
-
-    public long getStartTs() {
-      return startTs;
-    }
-
-    public void setStartTs(long startTs) {
-      this.startTs = startTs;
-    }
-
-    public long getCompleteTs() {
-      return completeTs;
-    }
-
-    public void setCompleteTs(long completeTs) {
-      this.completeTs = completeTs;
-    }
-
-    public ArrayList<BackupImage> getAncestors() {
-      if (this.ancestors == null) {
-        this.ancestors = new ArrayList<BackupImage>();
-      }
-      return this.ancestors;
-    }
-
-    public void addAncestor(BackupImage backupImage) {
-      this.getAncestors().add(backupImage);
-    }
-
-    public boolean hasAncestor(String token) {
-      for (BackupImage image : this.getAncestors()) {
-        if (image.getBackupId().equals(token)) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    public boolean hasTable(TableName table) {
-      for (TableName t : tableList) {
-        if (t.equals(table)) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    public int compareTo(BackupImage other) {
-      String thisBackupId = this.getBackupId();
-      String otherBackupId = other.getBackupId();
-      int index1 = thisBackupId.lastIndexOf("_");
-      int index2 = otherBackupId.lastIndexOf("_");
-      String name1 = thisBackupId.substring(0, index1);
-      String name2 = otherBackupId.substring(0, index2);
-      if(name1.equals(name2)) {
-        Long thisTS = new Long(thisBackupId.substring(index1 + 1));
-        Long otherTS = new Long(otherBackupId.substring(index2 + 1));
-        return thisTS.compareTo(otherTS);
-      } else {
-        return name1.compareTo(name2);
-      }
-    }
-  }
-
-  // manifest version
-  private String version = MANIFEST_VERSION;
-
-  // hadoop hbase configuration
-  protected Configuration config = null;
-
-  // backup root directory
-  private String rootDir = null;
-
-  // backup image directory
-  private String tableBackupDir = null;
-
-  // backup log directory if this is an incremental backup
-  private String logBackupDir = null;
-
-  // backup token
-  private String backupId;
-
-  // backup type, full or incremental
-  private BackupType type;
-
-  // the table list for the backup
-  private ArrayList<TableName> tableList;
-
-  // actual start timestamp of the backup process
-  private long startTs;
-
-  // actual complete timestamp of the backup process
-  private long completeTs;
-
-  // the region server timestamp for tables:
-  // <table, <rs, timestamp>>
-  private Map<TableName, HashMap<String, Long>> incrTimeRanges;
-
-  // dependency of this backup, including all the dependent images to do PIT recovery
-  private Map<String, BackupImage> dependency;
-  
-  /**
-   * Construct manifest for a ongoing backup.
-   * @param backupCtx The ongoing backup context
-   */
-  public BackupManifest(BackupInfo backupCtx) {
-    this.backupId = backupCtx.getBackupId();
-    this.type = backupCtx.getType();
-    this.rootDir = backupCtx.getTargetRootDir();
-    if (this.type == BackupType.INCREMENTAL) {
-      this.logBackupDir = backupCtx.getHLogTargetDir();
-    }
-    this.startTs = backupCtx.getStartTs();
-    this.completeTs = backupCtx.getEndTs();
-    this.loadTableList(backupCtx.getTableNames());
-  }
-  
-  
-  /**
-   * Construct a table level manifest for a backup of the named table.
-   * @param backupCtx The ongoing backup context
-   */
-  public BackupManifest(BackupInfo backupCtx, TableName table) {
-    this.backupId = backupCtx.getBackupId();
-    this.type = backupCtx.getType();
-    this.rootDir = backupCtx.getTargetRootDir();
-    this.tableBackupDir = backupCtx.getBackupStatus(table).getTargetDir();
-    if (this.type == BackupType.INCREMENTAL) {
-      this.logBackupDir = backupCtx.getHLogTargetDir();
-    }
-    this.startTs = backupCtx.getStartTs();
-    this.completeTs = backupCtx.getEndTs();
-    List<TableName> tables = new ArrayList<TableName>();
-    tables.add(table);
-    this.loadTableList(tables);
-  }
-
-  /**
-   * Construct manifest from a backup directory.
-   * @param conf configuration
-   * @param backupPath backup path
-   * @throws IOException 
-   */
-
-  public BackupManifest(Configuration conf, Path backupPath) throws IOException {
-    this(backupPath.getFileSystem(conf), backupPath);
-  }
-
-  /**
-   * Construct manifest from a backup directory.
-   * @param conf configuration
-   * @param backupPath backup path
-   * @throws BackupException exception
-   */
-
-  public BackupManifest(FileSystem fs, Path backupPath) throws BackupException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Loading manifest from: " + backupPath.toString());
-    }
-    // The input backupDir may not exactly be the backup table dir.
-    // It could be the backup log dir where there is also a manifest file stored.
-    // This variable's purpose is to keep the correct and original location so
-    // that we can store/persist it.
-    this.tableBackupDir = backupPath.toString();
-    this.config = fs.getConf();
-    try {
-
-      FileStatus[] subFiles = BackupClientUtil.listStatus(fs, backupPath, null);
-      if (subFiles == null) {
-        String errorMsg = backupPath.toString() + " does not exist";
-        LOG.error(errorMsg);
-        throw new IOException(errorMsg);
-      }
-      for (FileStatus subFile : subFiles) {
-        if (subFile.getPath().getName().equals(MANIFEST_FILE_NAME)) {
-
-          // load and set manifest field from file content
-          FSDataInputStream in = fs.open(subFile.getPath());
-          long len = subFile.getLen();
-          byte[] pbBytes = new byte[(int) len];
-          in.readFully(pbBytes);
-          BackupProtos.BackupManifest proto = null;
-          try{
-            proto = parseFrom(pbBytes);
-          } catch(Exception e){
-            throw new BackupException(e);
-          }
-          this.version = proto.getVersion();
-          this.backupId = proto.getBackupId();
-          this.type = BackupType.valueOf(proto.getType().name());
-          // Here the parameter backupDir is where the manifest file is.
-          // There should always be a manifest file under:
-          // backupRootDir/namespace/table/backupId/.backup.manifest
-          this.rootDir = backupPath.getParent().getParent().getParent().toString();
-
-          Path p = backupPath.getParent();
-          if (p.getName().equals(HConstants.HREGION_LOGDIR_NAME)) {
-            this.rootDir = p.getParent().toString();
-          } else {
-            this.rootDir = p.getParent().getParent().toString();
-          }
-
-          loadTableList(proto);
-          this.startTs = proto.getStartTs();
-          this.completeTs = proto.getCompleteTs();
-          loadIncrementalTimestampMap(proto);
-          loadDependency(proto);
-          //TODO: merge will be implemented by future jira
-          LOG.debug("Loaded manifest instance from manifest file: "
-              + BackupClientUtil.getPath(subFile.getPath()));
-          return;
-        }
-      }
-      String errorMsg = "No manifest file found in: " + backupPath.toString();
-      throw new IOException(errorMsg);
-
-    } catch (IOException e) {
-      throw new BackupException(e.getMessage());
-    }
-  }
-  
-  private void loadIncrementalTimestampMap(BackupProtos.BackupManifest proto) {
-    List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList();
-    if(list == null || list.size() == 0) return;
-    this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
-    for(BackupProtos.TableServerTimestamp tst: list){
-      TableName tn = ProtobufUtil.toTableName(tst.getTable());
-      HashMap<String, Long> map = this.incrTimeRanges.get(tn);
-      if(map == null){
-        map = new HashMap<String, Long>();
-        this.incrTimeRanges.put(tn, map);
-      }
-      List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList();
-      for(BackupProtos.ServerTimestamp stm: listSt) {
-        map.put(stm.getServer(), stm.getTimestamp());
-      }
-    }
-  }
-
-  private void loadDependency(BackupProtos.BackupManifest proto) {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("load dependency for: "+proto.getBackupId());
-    }
-
-    dependency = new HashMap<String, BackupImage>();
-    List<BackupProtos.BackupImage> list = proto.getDependentBackupImageList();
-    for (BackupProtos.BackupImage im : list) {
-      BackupImage bim = BackupImage.fromProto(im);
-      if(im.getBackupId() != null){
-        dependency.put(im.getBackupId(), bim);
-      } else{
-        LOG.warn("Load dependency for backup manifest: "+ backupId+ 
-          ". Null backup id in dependent image");
-      }
-    }
-  }
-
-  private void loadTableList(BackupProtos.BackupManifest proto) {
-    this.tableList = new ArrayList<TableName>();
-    List<HBaseProtos.TableName> list = proto.getTableListList();
-    for (HBaseProtos.TableName name: list) {
-      this.tableList.add(ProtobufUtil.toTableName(name));
-    }
-  }
-
-  public BackupType getType() {
-    return type;
-  }
-
-  public void setType(BackupType type) {
-    this.type = type;
-  }
-
-  /**
-   * Loads table list.
-   * @param tableList Table list
-   */
-  private void loadTableList(List<TableName> tableList) {
-
-    this.tableList = this.getTableList();
-    if (this.tableList.size() > 0) {
-      this.tableList.clear();
-    }
-    for (int i = 0; i < tableList.size(); i++) {
-      this.tableList.add(tableList.get(i));
-    }
-
-    LOG.debug(tableList.size() + " tables exist in table set.");
-  }
-
-  /**
-   * Get the table set of this image.
-   * @return The table set list
-   */
-  public ArrayList<TableName> getTableList() {
-    if (this.tableList == null) {
-      this.tableList = new ArrayList<TableName>();
-    }
-    return this.tableList;
-  }
-
-  /**
-   * Persist the manifest file.
-   * @throws IOException IOException when storing the manifest file.
-   */
-
-  public void store(Configuration conf) throws BackupException {
-    byte[] data = toByteArray();
-
-    // write the file, overwrite if already exist
-    Path manifestFilePath =
-        new Path(new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir))
-            ,MANIFEST_FILE_NAME);
-    try {
-      FSDataOutputStream out =
-          manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);
-      out.write(data);
-      out.close();
-    } catch (IOException e) {      
-      throw new BackupException(e.getMessage());
-    }
-
-    LOG.info("Manifest file stored to " + manifestFilePath);
-  }
-
-  /**
-   * Protobuf serialization
-   * @return The filter serialized using pb
-   */
-  public byte[] toByteArray() {
-    BackupProtos.BackupManifest.Builder builder = BackupProtos.BackupManifest.newBuilder();
-    builder.setVersion(this.version);
-    builder.setBackupId(this.backupId);
-    builder.setType(BackupProtos.BackupType.valueOf(this.type.name()));
-    setTableList(builder);
-    builder.setStartTs(this.startTs);
-    builder.setCompleteTs(this.completeTs);
-    setIncrementalTimestampMap(builder);
-    setDependencyMap(builder);
-    return builder.build().toByteArray();
-  }
-
-  private void setIncrementalTimestampMap(BackupProtos.BackupManifest.Builder builder) {
-    if (this.incrTimeRanges == null) {
-      return;
-    }
-    for (Entry<TableName, HashMap<String,Long>> entry: this.incrTimeRanges.entrySet()) {
-      TableName key = entry.getKey();
-      HashMap<String, Long> value = entry.getValue();
-      BackupProtos.TableServerTimestamp.Builder tstBuilder =
-          BackupProtos.TableServerTimestamp.newBuilder();
-      tstBuilder.setTable(ProtobufUtil.toProtoTableName(key));
-
-      for (String s : value.keySet()) {
-        BackupProtos.ServerTimestamp.Builder stBuilder = BackupProtos.ServerTimestamp.newBuilder();
-        stBuilder.setServer(s);
-        stBuilder.setTimestamp(value.get(s));
-        tstBuilder.addServerTimestamp(stBuilder.build());
-      }
-      builder.addTstMap(tstBuilder.build());
-    }
-  }
-
-  private void setDependencyMap(BackupProtos.BackupManifest.Builder builder) {
-    for (BackupImage image: getDependency().values()) {
-      builder.addDependentBackupImage(image.toProto());
-    }
-  }
-
-  private void setTableList(BackupProtos.BackupManifest.Builder builder) {
-    for(TableName name: tableList){
-      builder.addTableList(ProtobufUtil.toProtoTableName(name));
-    }
-  }
-
-  /**
-   * Parse protobuf from byte array
-   * @param pbBytes A pb serialized BackupManifest instance
-   * @return An instance of  made from <code>bytes</code>
-   * @throws DeserializationException
-   */
-  private static BackupProtos.BackupManifest parseFrom(final byte[] pbBytes)
-      throws DeserializationException {
-    BackupProtos.BackupManifest proto;
-    try {
-      proto = BackupProtos.BackupManifest.parseFrom(pbBytes);
-    } catch (InvalidProtocolBufferException e) {
-      throw new DeserializationException(e);
-    }
-    return proto;
-  }
-
-  /**
-   * Get manifest file version
-   * @return version
-   */
-  public String getVersion() {
-    return version;
-  }
-
-  /**
-   * Get this backup image.
-   * @return the backup image.
-   */
-  public BackupImage getBackupImage() {
-    return this.getDependency().get(this.backupId);
-  }
-
-  /**
-   * Add dependent backup image for this backup.
-   * @param image The direct dependent backup image
-   */
-  public void addDependentImage(BackupImage image) {
-    this.getDependency().get(this.backupId).addAncestor(image);
-    this.setDependencyMap(this.getDependency(), image);
-  }
-
-
-
-  /**
-   * Get all dependent backup images. The image of this backup is also contained.
-   * @return The dependent backup images map
-   */
-  public Map<String, BackupImage> getDependency() {
-    if (this.dependency == null) {
-      this.dependency = new HashMap<String, BackupImage>();
-      LOG.debug(this.rootDir + " " + this.backupId + " " + this.type);
-      this.dependency.put(this.backupId,
-        new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs,
-            this.completeTs));
-    }
-    return this.dependency;
-  }
-
-  /**
-   * Set the incremental timestamp map directly.
-   * @param incrTimestampMap timestamp map
-   */
-  public void setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> incrTimestampMap) {
-    this.incrTimeRanges = incrTimestampMap;
-  }
-
-
-  public Map<TableName, HashMap<String, Long>> getIncrTimestampMap() {
-    if (this.incrTimeRanges == null) {
-      this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
-    }
-    return this.incrTimeRanges;
-  }
-
-
-  /**
-   * Get the image list of this backup for restore in time order.
-   * @param reverse If true, then output in reverse order, otherwise in time order from old to new
-   * @return the backup image list for restore in time order
-   */
-  public ArrayList<BackupImage> getRestoreDependentList(boolean reverse) {
-    TreeMap<Long, BackupImage> restoreImages = new TreeMap<Long, BackupImage>();
-    for (BackupImage image : this.getDependency().values()) {
-      restoreImages.put(Long.valueOf(image.startTs), image);
-    }
-    return new ArrayList<BackupImage>(reverse ? (restoreImages.descendingMap().values())
-        : (restoreImages.values()));
-  }
-
-  /**
-   * Get the dependent image list for a specific table of this backup in time order from old to new
-   * if want to restore to this backup image level.
-   * @param table table
-   * @return the backup image list for a table in time order
-   */
-  public ArrayList<BackupImage> getDependentListByTable(TableName table) {
-    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
-    ArrayList<BackupImage> imageList = getRestoreDependentList(true);
-    for (BackupImage image : imageList) {
-      if (image.hasTable(table)) {
-        tableImageList.add(image);
-        if (image.getType() == BackupType.FULL) {
-          break;
-        }
-      }
-    }
-    Collections.reverse(tableImageList);
-    return tableImageList;
-  }
-
-  /**
-   * Get the full dependent image list in the whole dependency scope for a specific table of this
-   * backup in time order from old to new.
-   * @param table table
-   * @return the full backup image list for a table in time order in the whole scope of the
-   *         dependency of this image
-   */
-  public ArrayList<BackupImage> getAllDependentListByTable(TableName table) {
-    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
-    ArrayList<BackupImage> imageList = getRestoreDependentList(false);
-    for (BackupImage image : imageList) {
-      if (image.hasTable(table)) {
-        tableImageList.add(image);
-      }
-    }
-    return tableImageList;
-  }
-
-
-  /**
-   * Recursively set the dependency map of the backup images.
-   * @param map The dependency map
-   * @param image The backup image
-   */
-  private void setDependencyMap(Map<String, BackupImage> map, BackupImage image) {
-    if (image == null) {
-      return;
-    } else {
-      map.put(image.getBackupId(), image);
-      for (BackupImage img : image.getAncestors()) {
-        setDependencyMap(map, img);
-      }
-    }
-  }
-
-  /**
-   * Check whether backup image1 could cover backup image2 or not.
-   * @param image1 backup image 1
-   * @param image2 backup image 2
-   * @return true if image1 can cover image2, otherwise false
-   */
-  public static boolean canCoverImage(BackupImage image1, BackupImage image2) {
-    // image1 can cover image2 only when the following conditions are satisfied:
-    // - image1 must not be an incremental image;
-    // - image1 must be taken after image2 has been taken;
-    // - table set of image1 must cover the table set of image2.
-    if (image1.getType() == BackupType.INCREMENTAL) {
-      return false;
-    }
-    if (image1.getStartTs() < image2.getStartTs()) {
-      return false;
-    }
-    List<TableName> image1TableList = image1.getTableNames();
-    List<TableName> image2TableList = image2.getTableNames();
-    boolean found = false;
-    for (int i = 0; i < image2TableList.size(); i++) {
-      found = false;
-      for (int j = 0; j < image1TableList.size(); j++) {
-        if (image2TableList.get(i).equals(image1TableList.get(j))) {
-          found = true;
-          break;
-        }
-      }
-      if (!found) {
-        return false;
-      }
-    }
-
-    LOG.debug("Backup image " + image1.getBackupId() + " can cover " + image2.getBackupId());
-    return true;
-  }
-
-  /**
-   * Check whether backup image set could cover a backup image or not.
-   * @param fullImages The backup image set
-   * @param image The target backup image
-   * @return true if fullImages can cover image, otherwise false
-   */
-  public static boolean canCoverImage(ArrayList<BackupImage> fullImages, BackupImage image) {
-    // fullImages can cover image only when the following conditions are satisfied:
-    // - each image of fullImages must not be an incremental image;
-    // - each image of fullImages must be taken after image has been taken;
-    // - sum table set of fullImages must cover the table set of image.
-    for (BackupImage image1 : fullImages) {
-      if (image1.getType() == BackupType.INCREMENTAL) {
-        return false;
-      }
-      if (image1.getStartTs() < image.getStartTs()) {
-        return false;
-      }
-    }
-
-    ArrayList<String> image1TableList = new ArrayList<String>();
-    for (BackupImage image1 : fullImages) {
-      List<TableName> tableList = image1.getTableNames();
-      for (TableName table : tableList) {
-        image1TableList.add(table.getNameAsString());
-      }
-    }
-    ArrayList<String> image2TableList = new ArrayList<String>();
-    List<TableName> tableList = image.getTableNames();
-    for (TableName table : tableList) {
-      image2TableList.add(table.getNameAsString());
-    }
-
-    for (int i = 0; i < image2TableList.size(); i++) {
-      if (image1TableList.contains(image2TableList.get(i)) == false) {
-        return false;
-      }
-    }
-
-    LOG.debug("Full image set can cover image " + image.getBackupId());
-    return true;
-  }
-  
-  public BackupInfo toBackupInfo()
-  {
-    BackupInfo info = new BackupInfo();
-    info.setType(type);
-    TableName[] tables = new TableName[tableList.size()];
-    info.addTables(getTableList().toArray(tables));
-    info.setBackupId(backupId);
-    info.setStartTs(startTs);
-    info.setTargetRootDir(rootDir);
-    if(type == BackupType.INCREMENTAL) {
-      info.setHlogTargetDir(logBackupDir);
-    }
-    return info;
-  }
-}


[02/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java
deleted file mode 100644
index 2d41423..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/FullTableBackupProcedure.java
+++ /dev/null
@@ -1,777 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.master;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-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.HConstants;
-import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupCopyService;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
-import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
-import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.HBackupFileSystem;
-import org.apache.hadoop.hbase.backup.impl.BackupException;
-import org.apache.hadoop.hbase.backup.impl.BackupManager;
-import org.apache.hadoop.hbase.backup.impl.BackupManifest;
-import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
-import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.backup.util.BackupServerUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
-import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
-import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
-import org.apache.hadoop.hbase.procedure.ProcedureUtil;
-import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.FullTableBackupState;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-
-@InterfaceAudience.Private
-public class FullTableBackupProcedure
-    extends StateMachineProcedure<MasterProcedureEnv, FullTableBackupState>
-    implements TableProcedureInterface {
-  private static final Log LOG = LogFactory.getLog(FullTableBackupProcedure.class);
-  
-  private static final String SNAPSHOT_BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.snapshot.attempts.max";
-  private static final int DEFAULT_SNAPSHOT_BACKUP_MAX_ATTEMPTS = 10;
-  
-  private static final String SNAPSHOT_BACKUP_ATTEMPTS_DELAY_KEY = "hbase.backup.snapshot.attempts.delay";
-  private static final int DEFAULT_SNAPSHOT_BACKUP_ATTEMPTS_DELAY = 10000;
-  
-  private final AtomicBoolean aborted = new AtomicBoolean(false);
-  private Configuration conf;
-  private String backupId;
-  private List<TableName> tableList;
-  private String targetRootDir;
-  HashMap<String, Long> newTimestamps = null;
-
-  private BackupManager backupManager;
-  private BackupInfo backupContext;
-
-  public FullTableBackupProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
-  }
-
-  public FullTableBackupProcedure(final MasterProcedureEnv env,
-      final String backupId, List<TableName> tableList, String targetRootDir, final int workers,
-      final long bandwidth) throws IOException {
-    backupManager = new BackupManager(env.getMasterServices().getConnection(),
-        env.getMasterConfiguration());
-    this.backupId = backupId;
-    this.tableList = tableList;
-    this.targetRootDir = targetRootDir;
-    backupContext =
-        backupManager.createBackupContext(backupId, BackupType.FULL,
-            tableList, targetRootDir, workers, bandwidth);
-    if (tableList == null || tableList.isEmpty()) {
-      this.tableList = new ArrayList<>(backupContext.getTables());
-    }
-    this.setOwner(env.getRequestUser().getUGI().getShortUserName());
-  }
-
-  @Override
-  public byte[] getResult() {
-    return backupId.getBytes();
-  }
-
-  /**
-   * Begin the overall backup.
-   * @param backupContext backup context
-   * @throws IOException exception
-   */
-  static void beginBackup(BackupManager backupManager, BackupInfo backupContext)
-      throws IOException {
-    backupManager.setBackupContext(backupContext);
-    // set the start timestamp of the overall backup
-    long startTs = EnvironmentEdgeManager.currentTime();
-    backupContext.setStartTs(startTs);
-    // set overall backup status: ongoing
-    backupContext.setState(BackupState.RUNNING);
-    LOG.info("Backup " + backupContext.getBackupId() + " started at " + startTs + ".");
-
-    backupManager.updateBackupInfo(backupContext);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Backup session " + backupContext.getBackupId() + " has been started.");
-    }
-  }
-  
-  private static String getMessage(Exception e) {
-    String msg = e.getMessage();
-    if (msg == null || msg.equals("")) {
-      msg = e.getClass().getName();
-    }
-    return msg;
-  }
-
-  /**
-   * Delete HBase snapshot for backup.
-   * @param backupCtx backup context
-   * @throws Exception exception
-   */
-  private static void deleteSnapshot(final MasterProcedureEnv env,
-      BackupInfo backupCtx, Configuration conf)
-      throws IOException {
-    LOG.debug("Trying to delete snapshot for full backup.");
-    for (String snapshotName : backupCtx.getSnapshotNames()) {
-      if (snapshotName == null) {
-        continue;
-      }
-      LOG.debug("Trying to delete snapshot: " + snapshotName);
-      HBaseProtos.SnapshotDescription.Builder builder =
-          HBaseProtos.SnapshotDescription.newBuilder();
-      builder.setName(snapshotName);
-      try {
-        env.getMasterServices().getSnapshotManager().deleteSnapshot(builder.build());
-      } catch (IOException ioe) {
-        LOG.debug("when deleting snapshot " + snapshotName, ioe);
-      }
-      LOG.debug("Deleting the snapshot " + snapshotName + " for backup "
-          + backupCtx.getBackupId() + " succeeded.");
-    }
-  }
-
-  /**
-   * Clean up directories with prefix "exportSnapshot-", which are generated when exporting
-   * snapshots.
-   * @throws IOException exception
-   */
-  private static void cleanupExportSnapshotLog(Configuration conf) throws IOException {
-    FileSystem fs = FSUtils.getCurrentFileSystem(conf);
-    Path stagingDir =
-        new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory()
-          .toString()));
-    FileStatus[] files = FSUtils.listStatus(fs, stagingDir);
-    if (files == null) {
-      return;
-    }
-    for (FileStatus file : files) {
-      if (file.getPath().getName().startsWith("exportSnapshot-")) {
-        LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName());
-        if (FSUtils.delete(fs, file.getPath(), true) == false) {
-          LOG.warn("Can not delete " + file.getPath());
-        }
-      }
-    }
-  }
-
-  /**
-   * Clean up the uncompleted data at target directory if the ongoing backup has already entered the
-   * copy phase.
-   */
-  static void cleanupTargetDir(BackupInfo backupContext, Configuration conf) {
-    try {
-      // clean up the uncompleted data at target directory if the ongoing backup has already entered
-      // the copy phase
-      LOG.debug("Trying to cleanup up target dir. Current backup phase: "
-          + backupContext.getPhase());
-      if (backupContext.getPhase().equals(BackupPhase.SNAPSHOTCOPY)
-          || backupContext.getPhase().equals(BackupPhase.INCREMENTAL_COPY)
-          || backupContext.getPhase().equals(BackupPhase.STORE_MANIFEST)) {
-        FileSystem outputFs =
-            FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf);
-
-        // now treat one backup as a transaction, clean up data that has been partially copied at
-        // table level
-        for (TableName table : backupContext.getTables()) {
-          Path targetDirPath =
-              new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(),
-                backupContext.getBackupId(), table));
-          if (outputFs.delete(targetDirPath, true)) {
-            LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString()
-              + " done.");
-          } else {
-            LOG.info("No data has been copied to " + targetDirPath.toString() + ".");
-          }
-
-          Path tableDir = targetDirPath.getParent();
-          FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir);
-          if (backups == null || backups.length == 0) {
-            outputFs.delete(tableDir, true);
-            LOG.debug(tableDir.toString() + " is empty, remove it.");
-          }
-        }
-      }
-
-    } catch (IOException e1) {
-      LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at "
-          + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
-    }
-  }
-
-  /**
-   * Fail the overall backup.
-   * @param backupContext backup context
-   * @param e exception
-   * @throws Exception exception
-   */
-  static void failBackup(final MasterProcedureEnv env, BackupInfo backupContext,
-      BackupManager backupManager, Exception e,
-      String msg, BackupType type, Configuration conf) throws IOException {
-    LOG.error(msg + getMessage(e));
-    // If this is a cancel exception, then we've already cleaned.
-
-    // set the failure timestamp of the overall backup
-    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
-
-    // set failure message
-    backupContext.setFailedMsg(e.getMessage());
-
-    // set overall backup status: failed
-    backupContext.setState(BackupState.FAILED);
-
-    // compose the backup failed data
-    String backupFailedData =
-        "BackupId=" + backupContext.getBackupId() + ",startts=" + backupContext.getStartTs()
-        + ",failedts=" + backupContext.getEndTs() + ",failedphase=" + backupContext.getPhase()
-        + ",failedmessage=" + backupContext.getFailedMsg();
-    LOG.error(backupFailedData);
-
-    backupManager.updateBackupInfo(backupContext);
-
-    // if full backup, then delete HBase snapshots if there already are snapshots taken
-    // and also clean up export snapshot log files if exist
-    if (type == BackupType.FULL) {
-      deleteSnapshot(env, backupContext, conf);
-      cleanupExportSnapshotLog(conf);
-    }
-
-    // clean up the uncompleted data at target directory if the ongoing backup has already entered
-    // the copy phase
-    // For incremental backup, DistCp logs will be cleaned with the targetDir.
-    cleanupTargetDir(backupContext, conf);
-
-    LOG.info("Backup " + backupContext.getBackupId() + " failed.");
-  }
-
-  /**
-   * Do snapshot copy.
-   * @param backupContext backup context
-   * @throws Exception exception
-   */
-  private void snapshotCopy(BackupInfo backupContext) throws Exception {
-    LOG.info("Snapshot copy is starting.");
-
-    // set overall backup phase: snapshot_copy
-    backupContext.setPhase(BackupPhase.SNAPSHOTCOPY);
-
-    // call ExportSnapshot to copy files based on hbase snapshot for backup
-    // ExportSnapshot only support single snapshot export, need loop for multiple tables case
-    BackupCopyService copyService = BackupRestoreServerFactory.getBackupCopyService(conf);
-
-    // number of snapshots matches number of tables
-    float numOfSnapshots = backupContext.getSnapshotNames().size();
-
-    LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied.");
-
-    for (TableName table : backupContext.getTables()) {
-      // Currently we simply set the sub copy tasks by counting the table snapshot number, we can
-      // calculate the real files' size for the percentage in the future.
-      // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots);
-      int res = 0;
-      String[] args = new String[4];
-      args[0] = "-snapshot";
-      args[1] = backupContext.getSnapshotName(table);
-      args[2] = "-copy-to";
-      args[3] = backupContext.getBackupStatus(table).getTargetDir();
-
-      LOG.debug("Copy snapshot " + args[1] + " to " + args[3]);
-      res = copyService.copy(backupContext, backupManager, conf, BackupCopyService.Type.FULL, args);
-      // if one snapshot export failed, do not continue for remained snapshots
-      if (res != 0) {
-        LOG.error("Exporting Snapshot " + args[1] + " failed with return code: " + res + ".");
-
-        throw new IOException("Failed of exporting snapshot " + args[1] + " to " + args[3]
-            + " with reason code " + res);
-      }
-      LOG.info("Snapshot copy " + args[1] + " finished.");      
-    }
-  }
-  
-  /**
-   * Add manifest for the current backup. The manifest is stored
-   * within the table backup directory.
-   * @param backupContext The current backup context
-   * @throws IOException exception
-   * @throws BackupException exception
-   */
-  private static void addManifest(BackupInfo backupContext, BackupManager backupManager,
-      BackupType type, Configuration conf) throws IOException, BackupException {
-    // set the overall backup phase : store manifest
-    backupContext.setPhase(BackupPhase.STORE_MANIFEST);
-
-    BackupManifest manifest;
-
-    // Since we have each table's backup in its own directory structure,
-    // we'll store its manifest with the table directory.
-    for (TableName table : backupContext.getTables()) {
-      manifest = new BackupManifest(backupContext, table);
-      ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupContext, table);
-      for (BackupImage image : ancestors) {
-        manifest.addDependentImage(image);
-      }
-
-      if (type == BackupType.INCREMENTAL) {
-        // We'll store the log timestamps for this table only in its manifest.
-        HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
-            new HashMap<TableName, HashMap<String, Long>>();
-        tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table));
-        manifest.setIncrTimestampMap(tableTimestampMap);
-        ArrayList<BackupImage> ancestorss = backupManager.getAncestors(backupContext);
-        for (BackupImage image : ancestorss) {
-          manifest.addDependentImage(image);
-        }
-      }
-      manifest.store(conf);
-    }
-
-    // For incremental backup, we store a overall manifest in
-    // <backup-root-dir>/WALs/<backup-id>
-    // This is used when created the next incremental backup
-    if (type == BackupType.INCREMENTAL) {
-      manifest = new BackupManifest(backupContext);
-      // set the table region server start and end timestamps for incremental backup
-      manifest.setIncrTimestampMap(backupContext.getIncrTimestampMap());
-      ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupContext);
-      for (BackupImage image : ancestors) {
-        manifest.addDependentImage(image);
-      }
-      manifest.store(conf);
-    }
-  }
-
-  /**
-   * Get backup request meta data dir as string.
-   * @param backupContext backup context
-   * @return meta data dir
-   */
-  private static String obtainBackupMetaDataStr(BackupInfo backupContext) {
-    StringBuffer sb = new StringBuffer();
-    sb.append("type=" + backupContext.getType() + ",tablelist=");
-    for (TableName table : backupContext.getTables()) {
-      sb.append(table + ";");
-    }
-    if (sb.lastIndexOf(";") > 0) {
-      sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1);
-    }
-    sb.append(",targetRootDir=" + backupContext.getTargetRootDir());
-
-    return sb.toString();
-  }
-
-  /**
-   * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying
-   * hlogs.
-   * @throws IOException exception
-   */
-  private static void cleanupDistCpLog(BackupInfo backupContext, Configuration conf)
-      throws IOException {
-    Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent();
-    FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
-    FileStatus[] files = FSUtils.listStatus(fs, rootPath);
-    if (files == null) {
-      return;
-    }
-    for (FileStatus file : files) {
-      if (file.getPath().getName().startsWith("_distcp_logs")) {
-        LOG.debug("Delete log files of DistCp: " + file.getPath().getName());
-        FSUtils.delete(fs, file.getPath(), true);
-      }
-    }
-  }
-
-  /**
-   * Complete the overall backup.
-   * @param backupContext backup context
-   * @throws Exception exception
-   */
-  static void completeBackup(final MasterProcedureEnv env, BackupInfo backupContext,
-      BackupManager backupManager, BackupType type, Configuration conf) throws IOException {
-    // set the complete timestamp of the overall backup
-    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
-    // set overall backup status: complete
-    backupContext.setState(BackupState.COMPLETE);
-    backupContext.setProgress(100);
-    // add and store the manifest for the backup
-    addManifest(backupContext, backupManager, type, conf);
-
-    // after major steps done and manifest persisted, do convert if needed for incremental backup
-    /* in-fly convert code here, provided by future jira */
-    LOG.debug("in-fly convert code here, provided by future jira");
-
-    // compose the backup complete data
-    String backupCompleteData =
-        obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs()
-        + ",completets=" + backupContext.getEndTs() + ",bytescopied="
-        + backupContext.getTotalBytesCopied();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData);
-    }
-    backupManager.updateBackupInfo(backupContext);
-
-    // when full backup is done:
-    // - delete HBase snapshot
-    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
-    // snapshots
-    if (type == BackupType.FULL) {
-      deleteSnapshot(env, backupContext, conf);
-      cleanupExportSnapshotLog(conf);
-    } else if (type == BackupType.INCREMENTAL) {
-      cleanupDistCpLog(backupContext, conf);
-    }
-
-    LOG.info("Backup " + backupContext.getBackupId() + " completed.");
-  }
-
-  /**
-   * Wrap a SnapshotDescription for a target table.
-   * @param table table
-   * @return a SnapshotDescription especially for backup.
-   */
-  static SnapshotDescription wrapSnapshotDescription(TableName tableName, String snapshotName) {
-    // Mock a SnapshotDescription from backupContext to call SnapshotManager function,
-    // Name it in the format "snapshot_<timestamp>_<table>"
-    HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
-    builder.setTable(tableName.getNameAsString());
-    builder.setName(snapshotName);
-    HBaseProtos.SnapshotDescription backupSnapshot = builder.build();
-
-    LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName()
-      + " from backupContext to request snapshot for backup.");
-
-    return backupSnapshot;
-  }
-
-  @Override
-  protected Flow executeFromState(final MasterProcedureEnv env, final FullTableBackupState state)
-      throws InterruptedException {
-    if (conf == null) {
-      conf = env.getMasterConfiguration();
-    }
-    if (backupManager == null) {
-      try {
-        backupManager = new BackupManager(env.getMasterServices().getConnection(),
-            env.getMasterConfiguration());
-      } catch (IOException ioe) {
-        setFailure("full backup", ioe);
-        return Flow.NO_MORE_STATE;
-      }
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + " execute state=" + state);
-    }
-    try {
-      switch (state) {
-        case PRE_SNAPSHOT_TABLE:
-          beginBackup(backupManager, backupContext);
-          String savedStartCode = null;
-          boolean firstBackup = false;
-          // do snapshot for full table backup
-
-          try {
-            savedStartCode = backupManager.readBackupStartCode();
-            firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L;
-            if (firstBackup) {
-              // This is our first backup. Let's put some marker on ZK so that we can hold the logs
-              // while we do the backup.
-              backupManager.writeBackupStartCode(0L);
-            }
-            // We roll log here before we do the snapshot. It is possible there is duplicate data
-            // in the log that is already in the snapshot. But if we do it after the snapshot, we
-            // could have data loss.
-            // A better approach is to do the roll log on each RS in the same global procedure as
-            // the snapshot.
-            LOG.info("Execute roll log procedure for full backup ...");
-            MasterProcedureManager mpm = env.getMasterServices().getMasterProcedureManagerHost()
-                .getProcedureManager(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE);
-            Map<String, String> props= new HashMap<String, String>();
-            props.put("backupRoot", backupContext.getTargetRootDir());
-            long waitTime = ProcedureUtil.execProcedure(mpm,
-              LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
-              LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props);
-            ProcedureUtil.waitForProcedure(mpm,
-              LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
-              LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props, waitTime,
-              conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-                HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER),
-              conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
-                HConstants.DEFAULT_HBASE_CLIENT_PAUSE));
-
-            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 = BackupServerUtil.getWALFilesOlderThan(conf, newTimestamps);
-              backupManager.recordWALFiles(logFiles);
-            }
-          } catch (BackupException e) {
-            setFailure("Failure in full-backup: pre-snapshot phase", e);
-            // fail the overall backup and return
-            failBackup(env, backupContext, backupManager, e, "Unexpected BackupException : ",
-              BackupType.FULL, conf);
-            return Flow.NO_MORE_STATE;
-          }
-          setNextState(FullTableBackupState.SNAPSHOT_TABLES);
-          break;
-        case SNAPSHOT_TABLES:
-          for (TableName tableName : tableList) {
-            String snapshotName = "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime())
-                + "_" + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString();
-            HBaseProtos.SnapshotDescription backupSnapshot;
-
-            // wrap a SnapshotDescription for offline/online snapshot
-            backupSnapshot = wrapSnapshotDescription(tableName,snapshotName);
-            try {
-              env.getMasterServices().getSnapshotManager().deleteSnapshot(backupSnapshot);
-            } catch (IOException e) {
-              LOG.debug("Unable to delete " + snapshotName, e);
-            }
-            // Kick off snapshot for backup
-            snapshotTable(env, backupSnapshot);  
-            backupContext.setSnapshotName(tableName, backupSnapshot.getName());
-          }
-          setNextState(FullTableBackupState.SNAPSHOT_COPY);
-          break;
-        case SNAPSHOT_COPY:
-          // do snapshot copy
-          LOG.debug("snapshot copy for " + backupId);
-          try {
-            this.snapshotCopy(backupContext);                        
-          } catch (Exception e) {
-            setFailure("Failure in full-backup: snapshot copy phase" + backupId, e);
-            // fail the overall backup and return
-            failBackup(env, backupContext, backupManager, e, "Unexpected BackupException : ",
-              BackupType.FULL, conf);
-            return Flow.NO_MORE_STATE;
-          }
-          // Updates incremental backup table set
-          backupManager.addIncrementalBackupTableSet(backupContext.getTables());
-          setNextState(FullTableBackupState.BACKUP_COMPLETE);
-          break;
-
-        case 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
-          backupContext.setState(BackupState.COMPLETE);
-          // The table list in backupContext is good for both full backup and incremental backup.
-          // For incremental backup, it contains the incremental backup table set.
-          backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps);
-
-          HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
-              backupManager.readLogTimestampMap();
-
-          Long newStartCode =
-            BackupClientUtil.getMinValue(BackupServerUtil.getRSLogTimestampMins(newTableSetTimestampMap));
-          backupManager.writeBackupStartCode(newStartCode);
-
-          // backup complete
-          completeBackup(env, backupContext, backupManager, BackupType.FULL, conf);
-          return Flow.NO_MORE_STATE;
-
-        default:
-          throw new UnsupportedOperationException("unhandled state=" + state);
-      }
-    } catch (IOException e) {
-      LOG.error("Backup failed in " + state);
-      setFailure("snapshot-table", e);
-    }
-    return Flow.HAS_MORE_STATE;
-  }
-
-  private void snapshotTable(final MasterProcedureEnv env, SnapshotDescription backupSnapshot)
-    throws IOException
-  {
-    
-    int maxAttempts = env.getMasterConfiguration().getInt(SNAPSHOT_BACKUP_MAX_ATTEMPTS_KEY, 
-      DEFAULT_SNAPSHOT_BACKUP_MAX_ATTEMPTS);
-    int delay = env.getMasterConfiguration().getInt(SNAPSHOT_BACKUP_ATTEMPTS_DELAY_KEY, 
-      DEFAULT_SNAPSHOT_BACKUP_ATTEMPTS_DELAY);    
-    int attempts = 0;
-    
-    while (attempts++ < maxAttempts) {
-      try {
-        env.getMasterServices().getSnapshotManager().takeSnapshot(backupSnapshot);
-        long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(
-          env.getMasterConfiguration(),
-          backupSnapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
-        BackupServerUtil.waitForSnapshot(backupSnapshot, waitTime,
-          env.getMasterServices().getSnapshotManager(), env.getMasterConfiguration());
-        break;
-      } catch( NotServingRegionException ee) {
-        LOG.warn("Snapshot attempt "+attempts +" failed for table "+backupSnapshot.getTable() +
-          ", sleeping for " + delay+"ms", ee);        
-        if(attempts < maxAttempts) {
-          try {
-            Thread.sleep(delay);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-          }
-        }
-      } 
-    }    
-  }
-  @Override
-  protected void rollbackState(final MasterProcedureEnv env, final FullTableBackupState state)
-      throws IOException {
-    if (state != FullTableBackupState.PRE_SNAPSHOT_TABLE) {
-      deleteSnapshot(env, backupContext, conf);
-      cleanupExportSnapshotLog(conf);
-    }
-
-    // clean up the uncompleted data at target directory if the ongoing backup has already entered
-    // the copy phase
-    // For incremental backup, DistCp logs will be cleaned with the targetDir.
-    if (state == FullTableBackupState.SNAPSHOT_COPY) {
-      cleanupTargetDir(backupContext, conf);
-    }
-  }
-
-  @Override
-  protected FullTableBackupState getState(final int stateId) {
-    return FullTableBackupState.valueOf(stateId);
-  }
-
-  @Override
-  protected int getStateId(final FullTableBackupState state) {
-    return state.getNumber();
-  }
-
-  @Override
-  protected FullTableBackupState getInitialState() {
-    return FullTableBackupState.PRE_SNAPSHOT_TABLE;
-  }
-
-  @Override
-  protected void setNextState(final FullTableBackupState state) {
-    if (aborted.get()) {
-      setAbortFailure("backup-table", "abort requested");
-    } else {
-      super.setNextState(state);
-    }
-  }
-
-  @Override
-  public boolean abort(final MasterProcedureEnv env) {
-    aborted.set(true);
-    return true;
-  }
-
-  @Override
-  public void toStringClassDetails(StringBuilder sb) {
-    sb.append(getClass().getSimpleName());
-    sb.append(" (targetRootDir=");
-    sb.append(targetRootDir);
-    sb.append("; backupId=").append(backupId);
-    sb.append("; tables=");
-    int len = tableList.size();
-    for (int i = 0; i < len-1; i++) {
-      sb.append(tableList.get(i)).append(",");
-    }
-    sb.append(tableList.get(len-1));
-    sb.append(")");
-  }
-
-  BackupProtos.BackupProcContext toBackupContext() {
-    BackupProtos.BackupProcContext.Builder ctxBuilder = BackupProtos.BackupProcContext.newBuilder();
-    ctxBuilder.setCtx(backupContext.toProtosBackupInfo());
-    if (newTimestamps != null && !newTimestamps.isEmpty()) {
-      BackupProtos.ServerTimestamp.Builder tsBuilder = ServerTimestamp.newBuilder();
-      for (Entry<String, Long> entry : newTimestamps.entrySet()) {
-        tsBuilder.clear().setServer(entry.getKey()).setTimestamp(entry.getValue());
-        ctxBuilder.addServerTimestamp(tsBuilder.build());
-      }
-    }
-    return ctxBuilder.build();
-  }
-
-  @Override
-  public void serializeStateData(final OutputStream stream) throws IOException {
-    super.serializeStateData(stream);
-
-    BackupProtos.BackupProcContext backupProcCtx = toBackupContext();
-    backupProcCtx.writeDelimitedTo(stream);
-  }
-
-  @Override
-  public void deserializeStateData(final InputStream stream) throws IOException {
-    super.deserializeStateData(stream);
-
-    BackupProtos.BackupProcContext proto =BackupProtos.BackupProcContext.parseDelimitedFrom(stream);
-    backupContext = BackupInfo.fromProto(proto.getCtx());
-    backupId = backupContext.getBackupId();
-    targetRootDir = backupContext.getTargetRootDir();
-    tableList = backupContext.getTableNames();
-    List<ServerTimestamp> svrTimestamps = proto.getServerTimestampList();
-    if (svrTimestamps != null && !svrTimestamps.isEmpty()) {
-      newTimestamps = new HashMap<>();
-      for (ServerTimestamp ts : svrTimestamps) {
-        newTimestamps.put(ts.getServer(), ts.getTimestamp());
-      }
-    }
-  }
-
-  @Override
-  public TableName getTableName() {
-    return TableName.BACKUP_TABLE_NAME; 
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.EDIT;
-  }
-
-  @Override
-  protected boolean acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) {
-      return false;
-    }
-    return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME);
-  }
-
-  @Override
-  protected void releaseLock(final MasterProcedureEnv env) {
-    env.getProcedureQueue().releaseTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/IncrementalTableBackupProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/IncrementalTableBackupProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/IncrementalTableBackupProcedure.java
deleted file mode 100644
index e877ebd..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/IncrementalTableBackupProcedure.java
+++ /dev/null
@@ -1,400 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.master;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupCopyService;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
-import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.BackupCopyService.Type;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
-import org.apache.hadoop.hbase.backup.impl.BackupManager;
-import org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager;
-import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.backup.util.BackupServerUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
-import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.IncrementalTableBackupState;
-import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp;
-import org.apache.hadoop.security.UserGroupInformation;
-
-@InterfaceAudience.Private
-public class IncrementalTableBackupProcedure
-    extends StateMachineProcedure<MasterProcedureEnv, IncrementalTableBackupState> 
-    implements TableProcedureInterface {
-  private static final Log LOG = LogFactory.getLog(IncrementalTableBackupProcedure.class);
-
-  private final AtomicBoolean aborted = new AtomicBoolean(false);
-  private Configuration conf;
-  private String backupId;
-  private List<TableName> tableList;
-  private String targetRootDir;
-  HashMap<String, Long> newTimestamps = null;
-
-  private BackupManager backupManager;
-  private BackupInfo backupContext;
-
-  public IncrementalTableBackupProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
-  }
-
-  public IncrementalTableBackupProcedure(final MasterProcedureEnv env,
-      final String backupId,
-      List<TableName> tableList, String targetRootDir, final int workers,
-      final long bandwidth) throws IOException {
-    backupManager = new BackupManager(env.getMasterServices().getConnection(),
-        env.getMasterConfiguration());
-    this.backupId = backupId;
-    this.tableList = tableList;
-    this.targetRootDir = targetRootDir;
-    backupContext = backupManager.createBackupContext(backupId, 
-      BackupType.INCREMENTAL, tableList, targetRootDir, workers, (int)bandwidth);
-    this.setOwner(env.getRequestUser().getUGI().getShortUserName());
-  }
-
-  @Override
-  public byte[] getResult() {
-    return backupId.getBytes();
-  }
-
-  private List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
-    FileSystem fs = FileSystem.get(conf);
-    List<String> list = new ArrayList<String>();
-    for (String file : incrBackupFileList) {
-      if (fs.exists(new Path(file))) {
-        list.add(file);
-      } else {
-        LOG.warn("Can't find file: " + file);
-      }
-    }
-    return list;
-  }
-  
-  private List<String> getMissingFiles(List<String> incrBackupFileList) throws IOException {
-    FileSystem fs = FileSystem.get(conf);
-    List<String> list = new ArrayList<String>();
-    for (String file : incrBackupFileList) {
-      if (!fs.exists(new Path(file))) {
-        list.add(file);
-      }
-    }
-    return list;
-    
-  }
-
-  /**
-   * Do incremental copy.
-   * @param backupContext backup context
-   */
-  private void incrementalCopy(BackupInfo backupContext) throws Exception {
-
-    LOG.info("Incremental copy is starting.");
-    // set overall backup phase: incremental_copy
-    backupContext.setPhase(BackupPhase.INCREMENTAL_COPY);
-    // get incremental backup file list and prepare parms for DistCp
-    List<String> incrBackupFileList = backupContext.getIncrBackupFileList();
-    // filter missing files out (they have been copied by previous backups)
-    incrBackupFileList = filterMissingFiles(incrBackupFileList);
-    String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
-    strArr[strArr.length - 1] = backupContext.getHLogTargetDir();
-
-    BackupCopyService copyService = BackupRestoreServerFactory.getBackupCopyService(conf);
-    int counter = 0;
-    int MAX_ITERAIONS = 2;
-    while (counter++ < MAX_ITERAIONS) { 
-      // We run DistCp maximum 2 times
-      // If it fails on a second time, we throw Exception
-      int res = copyService.copy(backupContext, backupManager, conf,
-        BackupCopyService.Type.INCREMENTAL, strArr);
-
-      if (res != 0) {
-        LOG.error("Copy incremental log files failed with return code: " + res + ".");
-        throw new IOException("Failed of Hadoop Distributed Copy from "+
-            StringUtils.join(incrBackupFileList, ",") +" to "
-          + backupContext.getHLogTargetDir());
-      }
-      List<String> missingFiles = getMissingFiles(incrBackupFileList);
-
-      if(missingFiles.isEmpty()) {
-        break;
-      } else {
-        // Repeat DistCp, some files have been moved from WALs to oldWALs during previous run
-        // update backupContext and strAttr
-        if(counter == MAX_ITERAIONS){
-          String msg = "DistCp could not finish the following files: " +
-           StringUtils.join(missingFiles, ",");
-          LOG.error(msg);
-          throw new IOException(msg);
-        }
-        List<String> converted = convertFilesFromWALtoOldWAL(missingFiles);
-        incrBackupFileList.removeAll(missingFiles);
-        incrBackupFileList.addAll(converted);
-        backupContext.setIncrBackupFileList(incrBackupFileList);
-        
-        // Run DistCp only for missing files (which have been moved from WALs to oldWALs 
-        // during previous run)
-        strArr = converted.toArray(new String[converted.size() + 1]);
-        strArr[strArr.length - 1] = backupContext.getHLogTargetDir();
-      }
-    }
-    
-    
-    LOG.info("Incremental copy from " + StringUtils.join(incrBackupFileList, ",") + " to "
-        + backupContext.getHLogTargetDir() + " finished.");
-  }
-
-
-  private List<String> convertFilesFromWALtoOldWAL(List<String> missingFiles) throws IOException {
-    List<String> list = new ArrayList<String>();
-    for(String path: missingFiles){
-      if(path.indexOf(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME) < 0) {
-        LOG.error("Copy incremental log files failed, file is missing : " + path);
-        throw new IOException("Failed of Hadoop Distributed Copy to "
-          + backupContext.getHLogTargetDir()+", file is missing "+ path);
-      }
-      list.add(path.replace(Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME, 
-        Path.SEPARATOR + HConstants.HREGION_OLDLOGDIR_NAME));
-    }
-    return list;
-  }
-
-  @Override
-  protected Flow executeFromState(final MasterProcedureEnv env,
-      final IncrementalTableBackupState state)
-      throws InterruptedException {
-    if (conf == null) {
-      conf = env.getMasterConfiguration();
-    }
-    if (backupManager == null) {
-      try {
-        backupManager = new BackupManager(env.getMasterServices().getConnection(),
-            env.getMasterConfiguration());
-      } catch (IOException ioe) {
-        setFailure("incremental backup", ioe);
-      }
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + " execute state=" + state);
-    }
-    try {
-      switch (state) {
-        case PREPARE_INCREMENTAL:
-          FullTableBackupProcedure.beginBackup(backupManager, backupContext);
-          LOG.debug("For incremental backup, current table set is "
-              + backupManager.getIncrementalBackupTableSet());
-          try {
-            IncrementalBackupManager incrBackupManager =new IncrementalBackupManager(backupManager);
-
-            newTimestamps = incrBackupManager.getIncrBackupLogFileList(env.getMasterServices(),
-                backupContext);
-          } catch (Exception e) {
-            setFailure("Failure in incremental-backup: preparation phase " + backupId, e);
-            // fail the overall backup and return
-            FullTableBackupProcedure.failBackup(env, backupContext, backupManager, e,
-              "Unexpected Exception : ", BackupType.INCREMENTAL, conf);
-          }
-
-          setNextState(IncrementalTableBackupState.INCREMENTAL_COPY);
-          break;
-        case INCREMENTAL_COPY:
-          try {
-            // copy out the table and region info files for each table
-            BackupServerUtil.copyTableRegionInfo(env.getMasterServices(), backupContext, conf);
-            incrementalCopy(backupContext);
-            // Save list of WAL files copied
-            backupManager.recordWALFiles(backupContext.getIncrBackupFileList());
-          } catch (Exception e) {
-            String msg = "Unexpected exception in incremental-backup: incremental copy " + backupId;
-            setFailure(msg, e);
-            // fail the overall backup and return
-            FullTableBackupProcedure.failBackup(env, backupContext, backupManager, e,
-              msg, BackupType.INCREMENTAL, conf);
-          }
-          setNextState(IncrementalTableBackupState.INCR_BACKUP_COMPLETE);
-          break;
-        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
-          backupContext.setState(BackupState.COMPLETE);
-          // Set the previousTimestampMap which is before this current log roll to the manifest.
-          HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
-              backupManager.readLogTimestampMap();
-          backupContext.setIncrTimestampMap(previousTimestampMap);
-
-          // The table list in backupContext is good for both full backup and incremental backup.
-          // For incremental backup, it contains the incremental backup table set.
-          backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps);
-
-          HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
-              backupManager.readLogTimestampMap();
-
-          Long newStartCode = BackupClientUtil
-              .getMinValue(BackupServerUtil.getRSLogTimestampMins(newTableSetTimestampMap));
-          backupManager.writeBackupStartCode(newStartCode);
-          // backup complete
-          FullTableBackupProcedure.completeBackup(env, backupContext, backupManager,
-            BackupType.INCREMENTAL, conf);
-          return Flow.NO_MORE_STATE;
-
-        default:
-          throw new UnsupportedOperationException("unhandled state=" + state);
-      }
-    } catch (IOException e) {
-      setFailure("snapshot-table", e);
-    }
-    return Flow.HAS_MORE_STATE;
-  }
-
-  @Override
-  protected void rollbackState(final MasterProcedureEnv env,
-      final IncrementalTableBackupState state) throws IOException {
-    // clean up the uncompleted data at target directory if the ongoing backup has already entered
-    // the copy phase
-    // For incremental backup, DistCp logs will be cleaned with the targetDir.
-    FullTableBackupProcedure.cleanupTargetDir(backupContext, conf);
-  }
-
-  @Override
-  protected IncrementalTableBackupState getState(final int stateId) {
-    return IncrementalTableBackupState.valueOf(stateId);
-  }
-
-  @Override
-  protected int getStateId(final IncrementalTableBackupState state) {
-    return state.getNumber();
-  }
-
-  @Override
-  protected IncrementalTableBackupState getInitialState() {
-    return IncrementalTableBackupState.PREPARE_INCREMENTAL;
-  }
-
-  @Override
-  protected void setNextState(final IncrementalTableBackupState state) {
-    if (aborted.get()) {
-      setAbortFailure("snapshot-table", "abort requested");
-    } else {
-      super.setNextState(state);
-    }
-  }
-
-  @Override
-  public boolean abort(final MasterProcedureEnv env) {
-    aborted.set(true);
-    return true;
-  }
-
-  @Override
-  public void toStringClassDetails(StringBuilder sb) {
-    sb.append(getClass().getSimpleName());
-    sb.append(" (targetRootDir=");
-    sb.append(targetRootDir);
-    sb.append("; backupId=").append(backupId);
-    sb.append("; tables=");
-    int len = tableList.size();
-    for (int i = 0; i < len-1; i++) {
-      sb.append(tableList.get(i)).append(",");
-    }
-    sb.append(tableList.get(len-1));
-    sb.append(")");
-  }
-
-  BackupProtos.BackupProcContext toBackupContext() {
-    BackupProtos.BackupProcContext.Builder ctxBuilder = BackupProtos.BackupProcContext.newBuilder();
-    ctxBuilder.setCtx(backupContext.toProtosBackupInfo());
-    if (newTimestamps != null && !newTimestamps.isEmpty()) {
-      BackupProtos.ServerTimestamp.Builder tsBuilder = ServerTimestamp.newBuilder();
-      for (Entry<String, Long> entry : newTimestamps.entrySet()) {
-        tsBuilder.clear().setServer(entry.getKey()).setTimestamp(entry.getValue());
-        ctxBuilder.addServerTimestamp(tsBuilder.build());
-      }
-    }
-    return ctxBuilder.build();
-  }
-
-  @Override
-  public void serializeStateData(final OutputStream stream) throws IOException {
-    super.serializeStateData(stream);
-
-    BackupProtos.BackupProcContext backupProcCtx = toBackupContext();
-    backupProcCtx.writeDelimitedTo(stream);
-  }
-
-  @Override
-  public void deserializeStateData(final InputStream stream) throws IOException {
-    super.deserializeStateData(stream);
-
-    BackupProtos.BackupProcContext proto =BackupProtos.BackupProcContext.parseDelimitedFrom(stream);
-    backupContext = BackupInfo.fromProto(proto.getCtx());
-    backupId = backupContext.getBackupId();
-    targetRootDir = backupContext.getTargetRootDir();
-    tableList = backupContext.getTableNames();
-    List<ServerTimestamp> svrTimestamps = proto.getServerTimestampList();
-    if (svrTimestamps != null && !svrTimestamps.isEmpty()) {
-      newTimestamps = new HashMap<>();
-      for (ServerTimestamp ts : svrTimestamps) {
-        newTimestamps.put(ts.getServer(), ts.getTimestamp());
-      }
-    }
-  }
-
-  @Override
-  public TableName getTableName() {
-    return TableName.BACKUP_TABLE_NAME;
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.EDIT;
-  }
-
-  @Override
-  protected boolean acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) {
-      return false;
-    }
-    return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME);
-  }
-
-  @Override
-  protected void releaseLock(final MasterProcedureEnv env) {
-    env.getProcedureQueue().releaseTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/RestoreTablesProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/RestoreTablesProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/RestoreTablesProcedure.java
deleted file mode 100644
index 8fd7621..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/RestoreTablesProcedure.java
+++ /dev/null
@@ -1,387 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.master;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.HBackupFileSystem;
-import org.apache.hadoop.hbase.backup.impl.BackupManifest;
-import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
-import org.apache.hadoop.hbase.backup.util.RestoreServerUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.TableStateManager;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
-import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreTablesState;
-
-@InterfaceAudience.Private
-public class RestoreTablesProcedure
-    extends StateMachineProcedure<MasterProcedureEnv, RestoreTablesState>
-    implements TableProcedureInterface {
-  private static final Log LOG = LogFactory.getLog(RestoreTablesProcedure.class);
-
-  private final AtomicBoolean aborted = new AtomicBoolean(false);
-  private Configuration conf;
-  private String backupId;
-  private List<TableName> sTableList;
-  private List<TableName> tTableList;
-  private String targetRootDir;
-  private boolean isOverwrite;
-
-  public RestoreTablesProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
-  }
-
-  public RestoreTablesProcedure(final MasterProcedureEnv env,
-      final String targetRootDir, String backupId, List<TableName> sTableList,
-      List<TableName> tTableList, boolean isOverwrite) throws IOException {
-    this.targetRootDir = targetRootDir;
-    this.backupId = backupId;
-    this.sTableList = sTableList;
-    this.tTableList = tTableList;
-    if (tTableList == null || tTableList.isEmpty()) {
-      this.tTableList = sTableList;
-    }
-    this.isOverwrite = isOverwrite;
-    this.setOwner(env.getRequestUser().getUGI().getShortUserName());
-  }
-
-  @Override
-  public byte[] getResult() {
-    return null;
-  }
-
-  /**
-   * Validate target Tables
-   * @param conn connection
-   * @param mgr table state manager
-   * @param tTableArray: target tables
-   * @param isOverwrite overwrite existing table
-   * @throws IOException exception
-   */
-  private void checkTargetTables(Connection conn, TableStateManager mgr, TableName[] tTableArray,
-      boolean isOverwrite)
-      throws IOException {
-    ArrayList<TableName> existTableList = new ArrayList<>();
-    ArrayList<TableName> disabledTableList = new ArrayList<>();
-
-    // check if the tables already exist
-    for (TableName tableName : tTableArray) {
-      if (MetaTableAccessor.tableExists(conn, tableName)) {
-        existTableList.add(tableName);
-        if (mgr.isTableState(tableName, TableState.State.DISABLED, TableState.State.DISABLING)) {
-          disabledTableList.add(tableName);
-        }
-      } else {
-        LOG.info("HBase table " + tableName
-            + " does not exist. It will be created during restore process");
-      }
-    }
-
-    if (existTableList.size() > 0) {
-      if (!isOverwrite) {
-        LOG.error("Existing table (" + existTableList + ") found in the restore target, please add "
-          + "\"-overwrite\" option in the command if you mean to restore to these existing tables");
-        throw new IOException("Existing table found in target while no \"-overwrite\" "
-            + "option found");
-      } else {
-        if (disabledTableList.size() > 0) {
-          LOG.error("Found offline table in the restore target, "
-              + "please enable them before restore with \"-overwrite\" option");
-          LOG.info("Offline table list in restore target: " + disabledTableList);
-          throw new IOException(
-              "Found offline table in the target when restore with \"-overwrite\" option");
-        }
-      }
-    }
-  }
-
-  /**
-   * Restore operation handle each backupImage in array
-   * @param svc: master services
-   * @param images: array BackupImage
-   * @param sTable: table to be restored
-   * @param tTable: table to be restored to
-   * @param truncateIfExists: truncate table
-   * @throws IOException exception
-   */
-
-  private void restoreImages(MasterServices svc, BackupImage[] images, TableName sTable, TableName tTable,
-      boolean truncateIfExists) throws IOException {
-
-    // First image MUST be image of a FULL backup
-    BackupImage image = images[0];
-    String rootDir = image.getRootDir();
-    String backupId = image.getBackupId();
-    Path backupRoot = new Path(rootDir);
-    RestoreServerUtil restoreTool = new RestoreServerUtil(conf, backupRoot, backupId);
-    Path tableBackupPath = HBackupFileSystem.getTableBackupPath(sTable, backupRoot, backupId);
-    String lastIncrBackupId = images.length == 1 ? null : images[images.length - 1].getBackupId();
-    // We need hFS only for full restore (see the code)
-    BackupManifest manifest = HBackupFileSystem.getManifest(sTable, conf, backupRoot, backupId);
-    if (manifest.getType() == BackupType.FULL) {
-      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from full"
-          + " backup image " + tableBackupPath.toString());
-      restoreTool.fullRestoreTable(svc, tableBackupPath, sTable, tTable, truncateIfExists,
-        lastIncrBackupId);
-    } else { // incremental Backup
-      throw new IOException("Unexpected backup type " + image.getType());
-    }
-
-    if (images.length == 1) {
-      // full backup restore done
-      return;
-    }
-
-    List<Path> dirList = new ArrayList<Path>();
-    // add full backup path
-    // full backup path comes first
-    for (int i = 1; i < images.length; i++) {
-      BackupImage im = images[i];
-      String logBackupDir = HBackupFileSystem.getLogBackupDir(im.getRootDir(), im.getBackupId());
-      dirList.add(new Path(logBackupDir));
-    }
-
-    String dirs = StringUtils.join(dirList, ",");
-    LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from log dirs: " + dirs);
-    Path[] paths = new Path[dirList.size()];
-    dirList.toArray(paths);
-    restoreTool.incrementalRestoreTable(svc, tableBackupPath, paths, new TableName[] { sTable },
-      new TableName[] { tTable }, lastIncrBackupId);
-    LOG.info(sTable + " has been successfully restored to " + tTable);
-
-  }
-
-  /**
-   * Restore operation. Stage 2: resolved Backup Image dependency
-   * @param svc: master services
-   * @param backupManifestMap : tableName,  Manifest
-   * @param sTableArray The array of tables to be restored
-   * @param tTableArray The array of mapping tables to restore to
-   * @return set of BackupImages restored
-   * @throws IOException exception
-   */
-  private void restore(MasterServices svc, HashMap<TableName, BackupManifest> backupManifestMap,
-      TableName[] sTableArray, TableName[] tTableArray, boolean isOverwrite) throws IOException {
-    TreeSet<BackupImage> restoreImageSet = new TreeSet<BackupImage>();
-    boolean truncateIfExists = isOverwrite;
-    try {
-      for (int i = 0; i < sTableArray.length; i++) {
-        TableName table = sTableArray[i];
-        BackupManifest manifest = backupManifestMap.get(table);
-        // Get the image list of this backup for restore in time order from old
-        // to new.
-        List<BackupImage> list = new ArrayList<BackupImage>();
-        list.add(manifest.getBackupImage());
-        TreeSet<BackupImage> set = new TreeSet<BackupImage>(list);
-        List<BackupImage> depList = manifest.getDependentListByTable(table);
-        set.addAll(depList);
-        BackupImage[] arr = new BackupImage[set.size()];
-        set.toArray(arr);
-        restoreImages(svc, arr, table, tTableArray[i], truncateIfExists);
-        restoreImageSet.addAll(list);
-        if (restoreImageSet != null && !restoreImageSet.isEmpty()) {
-          LOG.info("Restore includes the following image(s):");
-          for (BackupImage image : restoreImageSet) {
-            LOG.info("Backup: "
-                + image.getBackupId()
-                + " "
-                + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(),
-                  table));
-          }
-        }
-      }
-    } catch (Exception e) {
-      LOG.error("Failed", e);
-      throw new IOException(e);
-    }
-    LOG.debug("restoreStage finished");
-  }
-
-  @Override
-  protected Flow executeFromState(final MasterProcedureEnv env, final RestoreTablesState state)
-      throws InterruptedException {
-    if (conf == null) {
-      conf = env.getMasterConfiguration();
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + " execute state=" + state);
-    }
-    TableName[] tTableArray = tTableList.toArray(new TableName[tTableList.size()]);
-    try {
-      switch (state) {
-        case VALIDATION:
-
-          // check the target tables
-          checkTargetTables(env.getMasterServices().getConnection(),
-              env.getMasterServices().getTableStateManager(), tTableArray, isOverwrite);
-
-          setNextState(RestoreTablesState.RESTORE_IMAGES);
-          break;
-        case RESTORE_IMAGES:
-          TableName[] sTableArray = sTableList.toArray(new TableName[sTableList.size()]);
-          HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
-          // check and load backup image manifest for the tables
-          Path rootPath = new Path(targetRootDir);
-          HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, conf, rootPath,
-            backupId);
-          restore(env.getMasterServices(), backupManifestMap, sTableArray, tTableArray, isOverwrite);
-          return Flow.NO_MORE_STATE;
-        default:
-          throw new UnsupportedOperationException("unhandled state=" + state);
-      }
-    } catch (IOException e) {
-      setFailure("restore-table", e);
-    }
-    return Flow.HAS_MORE_STATE;
-  }
-
-  @Override
-  protected void rollbackState(final MasterProcedureEnv env, final RestoreTablesState state)
-      throws IOException {
-  }
-
-  @Override
-  protected RestoreTablesState getState(final int stateId) {
-    return RestoreTablesState.valueOf(stateId);
-  }
-
-  @Override
-  protected int getStateId(final RestoreTablesState state) {
-    return state.getNumber();
-  }
-
-  @Override
-  protected RestoreTablesState getInitialState() {
-    return RestoreTablesState.VALIDATION;
-  }
-
-  @Override
-  protected void setNextState(final RestoreTablesState state) {
-    if (aborted.get()) {
-      setAbortFailure("snapshot-table", "abort requested");
-    } else {
-      super.setNextState(state);
-    }
-  }
-
-  @Override
-  public boolean abort(final MasterProcedureEnv env) {
-    aborted.set(true);
-    return true;
-  }
-
-  @Override
-  public void toStringClassDetails(StringBuilder sb) {
-    sb.append(getClass().getSimpleName());
-    sb.append(" (targetRootDir=");
-    sb.append(targetRootDir);
-    sb.append(" isOverwrite= ");
-    sb.append(isOverwrite);
-    sb.append(" backupId= ");
-    sb.append(backupId);
-    sb.append(")");
-  }
-
-  MasterProtos.RestoreTablesRequest toRestoreTables() {
-    MasterProtos.RestoreTablesRequest.Builder bldr = MasterProtos.RestoreTablesRequest.newBuilder();
-    bldr.setOverwrite(isOverwrite).setBackupId(backupId);
-    bldr.setBackupRootDir(targetRootDir);
-    for (TableName table : sTableList) {
-      bldr.addTables(ProtobufUtil.toProtoTableName(table));
-    }
-    for (TableName table : tTableList) {
-      bldr.addTargetTables(ProtobufUtil.toProtoTableName(table));
-    }
-    return bldr.build();
-  }
-
-  @Override
-  public void serializeStateData(final OutputStream stream) throws IOException {
-    super.serializeStateData(stream);
-
-    MasterProtos.RestoreTablesRequest restoreTables = toRestoreTables();
-    restoreTables.writeDelimitedTo(stream);
-  }
-
-  @Override
-  public void deserializeStateData(final InputStream stream) throws IOException {
-    super.deserializeStateData(stream);
-
-    MasterProtos.RestoreTablesRequest proto =
-        MasterProtos.RestoreTablesRequest.parseDelimitedFrom(stream);
-    backupId = proto.getBackupId();
-    targetRootDir = proto.getBackupRootDir();
-    isOverwrite = proto.getOverwrite();
-    sTableList = new ArrayList<>(proto.getTablesList().size());
-    for (HBaseProtos.TableName table : proto.getTablesList()) {
-      sTableList.add(ProtobufUtil.toTableName(table));
-    }
-    tTableList = new ArrayList<>(proto.getTargetTablesList().size());
-    for (HBaseProtos.TableName table : proto.getTargetTablesList()) {
-      tTableList.add(ProtobufUtil.toTableName(table));
-    }
-  }
-
-  @Override
-  public TableName getTableName() {
-    return TableName.BACKUP_TABLE_NAME;
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.EDIT;
-  }
-
-  @Override
-  protected boolean acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) {
-      return false;
-    }
-    return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME);
-  }
-
-  @Override
-  protected void releaseLock(final MasterProcedureEnv env) {
-    env.getProcedureQueue().releaseTableExclusiveLock(this, TableName.BACKUP_TABLE_NAME);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupClientUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupClientUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupClientUtil.java
new file mode 100644
index 0000000..c22f51b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupClientUtil.java
@@ -0,0 +1,437 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.util;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.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.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A collection of methods used by multiple classes to backup HBase tables.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupClientUtil {
+  protected static final Log LOG = LogFactory.getLog(BackupClientUtil.class);
+  public static final String LOGNAME_SEPARATOR = ".";
+
+  private BackupClientUtil() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Check whether the backup path exist
+   * @param backupStr backup
+   * @param conf configuration
+   * @return Yes if path exists
+   * @throws IOException exception
+   */
+  public static boolean checkPathExist(String backupStr, Configuration conf) throws IOException {
+    boolean isExist = false;
+    Path backupPath = new Path(backupStr);
+    FileSystem fileSys = backupPath.getFileSystem(conf);
+    String targetFsScheme = fileSys.getUri().getScheme();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Schema of given url: " + backupStr + " is: " + targetFsScheme);
+    }
+    if (fileSys.exists(backupPath)) {
+      isExist = true;
+    }
+    return isExist;
+  }
+
+  // check target path first, confirm it doesn't exist before backup
+  public static void checkTargetDir(String backupRootPath, Configuration conf) throws IOException {
+    boolean targetExists = false;
+    try {
+      targetExists = checkPathExist(backupRootPath, conf);
+    } catch (IOException e) {
+      String expMsg = e.getMessage();
+      String newMsg = null;
+      if (expMsg.contains("No FileSystem for scheme")) {
+        newMsg =
+            "Unsupported filesystem scheme found in the backup target url. Error Message: "
+                + newMsg;
+        LOG.error(newMsg);
+        throw new IOException(newMsg);
+      } else {
+        throw e;
+      }
+    }
+
+    if (targetExists) {
+      LOG.info("Using existing backup root dir: " + backupRootPath);
+    } else {
+      LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created.");
+    }
+  }
+
+  /**
+   * Get the min value for all the Values a map.
+   * @param map map
+   * @return the min value
+   */
+  public static <T> Long getMinValue(HashMap<T, Long> map) {
+    Long minTimestamp = null;
+    if (map != null) {
+      ArrayList<Long> timestampList = new ArrayList<Long>(map.values());
+      Collections.sort(timestampList);
+      // The min among all the RS log timestamps will be kept in hbase:backup table.
+      minTimestamp = timestampList.get(0);
+    }
+    return minTimestamp;
+  }
+
+  /**
+   * Parses host name:port from archived WAL path
+   * @param p path
+   * @return host name
+   * @throws IOException exception
+   */
+  public static String parseHostFromOldLog(Path p) {
+    try {
+      String n = p.getName();
+      int idx = n.lastIndexOf(LOGNAME_SEPARATOR);
+      String s = URLDecoder.decode(n.substring(0, idx), "UTF8");
+      return ServerName.parseHostname(s) + ":" + ServerName.parsePort(s);
+    } catch (Exception e) {
+      LOG.warn("Skip log file (can't parse): " + p);
+      return null;
+    }
+  }
+
+  /**
+   * Given the log file, parse the timestamp from the file name. The timestamp is the last number.
+   * @param p a path to the log file
+   * @return the timestamp
+   * @throws IOException exception
+   */
+  public static Long getCreationTime(Path p) throws IOException {
+    int idx = p.getName().lastIndexOf(LOGNAME_SEPARATOR);
+    if (idx < 0) {
+      throw new IOException("Cannot parse timestamp from path " + p);
+    }
+    String ts = p.getName().substring(idx + 1);
+    return Long.parseLong(ts);
+  }
+
+  public static List<String> getFiles(FileSystem fs, Path rootDir, List<String> files,
+      PathFilter filter) throws FileNotFoundException, IOException {
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(rootDir, true);
+
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (lfs.isDirectory()) {
+        continue;
+      }
+      // apply filter
+      if (filter.accept(lfs.getPath())) {
+        files.add(lfs.getPath().toString());
+      }
+    }
+    return files;
+  }
+
+  public static void cleanupBackupData(BackupInfo context, Configuration conf) throws IOException {
+    cleanupHLogDir(context, conf);
+    cleanupTargetDir(context, conf);
+  }
+
+  /**
+   * Clean up directories which are generated when DistCp copying hlogs.
+   * @throws IOException
+   */
+  private static void cleanupHLogDir(BackupInfo backupContext, Configuration conf)
+      throws IOException {
+
+    String logDir = backupContext.getHLogTargetDir();
+    if (logDir == null) {
+      LOG.warn("No log directory specified for " + backupContext.getBackupId());
+      return;
+    }
+
+    Path rootPath = new Path(logDir).getParent();
+    FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
+    FileStatus[] files = listStatus(fs, rootPath, null);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      LOG.debug("Delete log files: " + file.getPath().getName());
+      fs.delete(file.getPath(), true);
+    }
+  }
+
+  /**
+   * Clean up the data at target directory
+   */
+  private static void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) {
+    try {
+      // clean up the data at target directory
+      LOG.debug("Trying to cleanup up target dir : " + backupInfo.getBackupId());
+      String targetDir = backupInfo.getTargetRootDir();
+      if (targetDir == null) {
+        LOG.warn("No target directory specified for " + backupInfo.getBackupId());
+        return;
+      }
+
+      FileSystem outputFs = FileSystem.get(new Path(backupInfo.getTargetRootDir()).toUri(), conf);
+
+      for (TableName table : backupInfo.getTables()) {
+        Path targetDirPath =
+            new Path(getTableBackupDir(backupInfo.getTargetRootDir(), backupInfo.getBackupId(),
+              table));
+        if (outputFs.delete(targetDirPath, true)) {
+          LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done.");
+        } else {
+          LOG.info("No data has been found in " + targetDirPath.toString() + ".");
+        }
+
+        Path tableDir = targetDirPath.getParent();
+        FileStatus[] backups = listStatus(outputFs, tableDir, null);
+        if (backups == null || backups.length == 0) {
+          outputFs.delete(tableDir, true);
+          LOG.debug(tableDir.toString() + " is empty, remove it.");
+        }
+      }
+      outputFs.delete(new Path(targetDir, backupInfo.getBackupId()), true);
+    } catch (IOException e1) {
+      LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " at "
+          + backupInfo.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
+    }
+  }
+
+  /**
+   * Given the backup root dir, backup id and the table name, return the backup image location,
+   * which is also where the backup manifest file is. return value look like:
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup1/backup_1396650096738/default/t1_dn/"
+   * @param backupRootDir backup root directory
+   * @param backupId backup id
+   * @param table table name
+   * @return backupPath String for the particular table
+   */
+  public static String
+      getTableBackupDir(String backupRootDir, String backupId, TableName tableName) {
+    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
+        + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString()
+        + Path.SEPARATOR;
+  }
+
+  public static TableName[] parseTableNames(String tables) {
+    if (tables == null) {
+      return null;
+    }
+    String[] tableArray = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+
+    TableName[] ret = new TableName[tableArray.length];
+    for (int i = 0; i < tableArray.length; i++) {
+      ret[i] = TableName.valueOf(tableArray[i]);
+    }
+    return ret;
+  }
+
+  /**
+   * Sort history list by start time in descending order.
+   * @param historyList history list
+   * @return sorted list of BackupCompleteData
+   */
+  public static ArrayList<BackupInfo> sortHistoryListDesc(ArrayList<BackupInfo> historyList) {
+    ArrayList<BackupInfo> list = new ArrayList<BackupInfo>();
+    TreeMap<String, BackupInfo> map = new TreeMap<String, BackupInfo>();
+    for (BackupInfo h : historyList) {
+      map.put(Long.toString(h.getStartTs()), h);
+    }
+    Iterator<String> i = map.descendingKeySet().iterator();
+    while (i.hasNext()) {
+      list.add(map.get(i.next()));
+    }
+    return list;
+  }
+
+  /**
+   * Returns WAL file name
+   * @param walFileName WAL file name
+   * @return WAL file name
+   * @throws IOException exception
+   * @throws IllegalArgumentException exception
+   */
+  public static String getUniqueWALFileNamePart(String walFileName) throws IOException {
+    return getUniqueWALFileNamePart(new Path(walFileName));
+  }
+
+  /**
+   * Returns WAL file name
+   * @param p - WAL file path
+   * @return WAL file name
+   * @throws IOException exception
+   */
+  public static String getUniqueWALFileNamePart(Path p) throws IOException {
+    return p.getName();
+  }
+
+  /**
+   * Calls fs.listStatus() and treats FileNotFoundException as non-fatal This accommodates
+   * differences between hadoop versions, where hadoop 1 does not throw a FileNotFoundException, and
+   * return an empty FileStatus[] while Hadoop 2 will throw FileNotFoundException.
+   * @param fs file system
+   * @param dir directory
+   * @param filter path filter
+   * @return null if dir is empty or doesn't exist, otherwise FileStatus array
+   */
+  public static FileStatus[]
+      listStatus(final FileSystem fs, final Path dir, final PathFilter filter) throws IOException {
+    FileStatus[] status = null;
+    try {
+      status = filter == null ? fs.listStatus(dir) : fs.listStatus(dir, filter);
+    } catch (FileNotFoundException fnfe) {
+      // if directory doesn't exist, return null
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(dir + " doesn't exist");
+      }
+    }
+    if (status == null || status.length < 1) return null;
+    return status;
+  }
+
+  /**
+   * Return the 'path' component of a Path. In Hadoop, Path is an URI. This method returns the
+   * 'path' component of a Path's URI: e.g. If a Path is
+   * <code>hdfs://example.org:9000/hbase_trunk/TestTable/compaction.dir</code>, this method returns
+   * <code>/hbase_trunk/TestTable/compaction.dir</code>. This method is useful if you want to print
+   * out a Path without qualifying Filesystem instance.
+   * @param p Filesystem Path whose 'path' component we are to return.
+   * @return Path portion of the Filesystem
+   */
+  public static String getPath(Path p) {
+    return p.toUri().getPath();
+  }
+
+  /**
+   * Given the backup root dir and the backup id, return the log file location for an incremental
+   * backup.
+   * @param backupRootDir backup root directory
+   * @param backupId backup id
+   * @return logBackupDir: ".../user/biadmin/backup1/WALs/backup_1396650096738"
+   */
+  public static String getLogBackupDir(String backupRootDir, String backupId) {
+    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
+        + HConstants.HREGION_LOGDIR_NAME;
+  }
+
+  private static List<BackupInfo> getHistory(Configuration conf, Path backupRootPath)
+      throws IOException {
+    // Get all (n) history from backup root destination
+    FileSystem fs = FileSystem.get(conf);
+    RemoteIterator<LocatedFileStatus> it = fs.listLocatedStatus(backupRootPath);
+
+    List<BackupInfo> infos = new ArrayList<BackupInfo>();
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (!lfs.isDirectory()) continue;
+      String backupId = lfs.getPath().getName();
+      try {
+        BackupInfo info = loadBackupInfo(backupRootPath, backupId, fs);
+        infos.add(info);
+      } catch(IOException e) {
+        LOG.error("Can not load backup info from: "+ lfs.getPath(), e);
+      }
+    }
+    // Sort
+    Collections.sort(infos, new Comparator<BackupInfo>() {
+
+      @Override
+      public int compare(BackupInfo o1, BackupInfo o2) {
+        long ts1 = getTimestamp(o1.getBackupId());
+        long ts2 = getTimestamp(o2.getBackupId());
+        if (ts1 == ts2) return 0;
+        return ts1 < ts2 ? 1 : -1;
+      }
+
+      private long getTimestamp(String backupId) {
+        String[] split = backupId.split("_");
+        return Long.parseLong(split[1]);
+      }
+    });
+    return infos;
+  }
+
+  public static List<BackupInfo> getHistory(Configuration conf, int n, Path backupRootPath,
+      BackupInfo.Filter... filters) throws IOException {
+    List<BackupInfo> infos = getHistory(conf, backupRootPath);
+    List<BackupInfo> ret = new ArrayList<BackupInfo>();
+    for (BackupInfo info : infos) {
+      if (ret.size() == n) {
+        break;
+      }
+      boolean passed = true;
+      for (int i = 0; i < filters.length; i++) {
+        if (!filters[i].apply(info)) {
+          passed = false;
+          break;
+        }
+      }
+      if (passed) {
+        ret.add(info);
+      }
+    }
+    return ret;
+  }
+  
+  public static BackupInfo loadBackupInfo(Path backupRootPath, String backupId, FileSystem fs)
+      throws IOException {
+    Path backupPath = new Path(backupRootPath, backupId);
+
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(backupPath, true);
+    while (it.hasNext()) {
+      LocatedFileStatus lfs = it.next();
+      if (lfs.getPath().getName().equals(BackupManifest.MANIFEST_FILE_NAME)) {
+        // Load BackupManifest
+        BackupManifest manifest = new BackupManifest(fs, lfs.getPath().getParent());
+        BackupInfo info = manifest.toBackupInfo();
+        return info;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java
index 486fd2b..d9bf749 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupServerUtil.java
@@ -158,14 +158,13 @@ public final class BackupServerUtil {
    * @throws IOException exception
    * @throws InterruptedException exception
    */
-  public static void copyTableRegionInfo(MasterServices svc, BackupInfo backupContext,
+  public static void copyTableRegionInfo(Connection conn, BackupInfo backupContext,
       Configuration conf) throws IOException, InterruptedException {
     Path rootDir = FSUtils.getRootDir(conf);
     FileSystem fs = rootDir.getFileSystem(conf);
 
     // for each table in the table set, copy out the table info and region 
     // info files in the correct directory structure
-    Connection conn = svc.getConnection();
     for (TableName table : backupContext.getTables()) {
 
       if(!MetaTableAccessor.tableExists(conn, table)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
new file mode 100644
index 0000000..76402c7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/BackupSet.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup.util;
+import java.util.List;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+/**
+ * Backup set is a named group of HBase tables,
+ * which are managed together by Backup/Restore  
+ * framework. Instead of using list of tables in backup or restore 
+ * operation, one can use set's name instead.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BackupSet {
+  private final String name;
+  private final List<TableName> tables;
+
+  public BackupSet(String name, List<TableName> tables) {
+    this.name = name;
+    this.tables = tables;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public List<TableName> getTables() {
+    return tables;
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(name).append("={");
+    for (int i = 0; i < tables.size(); i++) {
+      sb.append(tables.get(i));
+      if (i < tables.size() - 1) {
+        sb.append(",");
+      }
+    }
+    sb.append("}");
+    return sb.toString();
+  }
+
+}


[05/10] hbase git commit: HBASE-16727 Backup refactoring: remove MR dependencies from HMaster (Vladimir Rodionov)

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-protocol/src/main/protobuf/Backup.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Backup.proto b/hbase-protocol/src/main/protobuf/Backup.proto
index 7d1ec4b..2b3feeb 100644
--- a/hbase-protocol/src/main/protobuf/Backup.proto
+++ b/hbase-protocol/src/main/protobuf/Backup.proto
@@ -27,7 +27,7 @@ option optimize_for = SPEED;
 
 import "HBase.proto";
 
-enum FullTableBackupState {
+/*enum FullTableBackupState {
   PRE_SNAPSHOT_TABLE = 1;
   SNAPSHOT_TABLES = 2;
   SNAPSHOT_COPY = 3;
@@ -44,7 +44,7 @@ message SnapshotTableStateData {
   required TableName table = 1;
   required string snapshotName = 2;
 }
-
+*/
 enum BackupType {
   FULL = 0;
   INCREMENTAL = 1;
@@ -119,9 +119,9 @@ message BackupInfo {
     STORE_MANIFEST = 5;
   } 
 }
-
+/*
 message BackupProcContext {
   required BackupInfo ctx = 1;
   repeated ServerTimestamp server_timestamp = 2;
 }
-
+*/

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-protocol/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index 13dbd28..54d6c93 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -27,7 +27,6 @@ option java_generate_equals_and_hash = true;
 option optimize_for = SPEED;
 
 import "HBase.proto";
-import "Backup.proto";
 import "Client.proto";
 import "ClusterStatus.proto";
 import "ErrorHandling.proto";
@@ -541,42 +540,6 @@ message SecurityCapabilitiesResponse {
   repeated Capability capabilities = 1;
 }
 
-message BackupTablesRequest {
-  required BackupType type = 1;
-  repeated TableName tables = 2;
-  required string target_root_dir = 3;
-  optional int64 workers = 4;
-  optional int64 bandwidth = 5;
-  optional string backup_set_name = 6;
-  optional uint64 nonce_group = 7 [default = 0];
-  optional uint64 nonce = 8 [default = 0];
-}
-
-message BackupTablesResponse {
-  optional uint64 proc_id = 1;
-  optional string backup_id = 2;
-}
-
-enum RestoreTablesState {
-  VALIDATION = 1;
-  RESTORE_IMAGES = 2;
-}
-
-message RestoreTablesRequest {
-  required string backup_id = 1;
-  repeated TableName tables = 2;
-  repeated TableName target_tables = 3;
-  required string backup_root_dir = 4;
-  optional bool dependency_check_only = 5;
-  optional bool overwrite = 6;
-  optional uint64 nonce_group = 7 [default = 0];
-  optional uint64 nonce = 8 [default = 0];
-}
-
-message RestoreTablesResponse {
-  optional uint64 proc_id = 1;
-}
-
 service MasterService {
   /** Used by the client to get the number of regions that have received the updated schema */
   rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
@@ -852,11 +815,4 @@ service MasterService {
   rpc ListProcedures(ListProceduresRequest)
     returns(ListProceduresResponse);
 
-  /** backup table set */
-  rpc backupTables(BackupTablesRequest)
-    returns(BackupTablesResponse);
-
-  /** restore table set */
-  rpc restoreTables(RestoreTablesRequest)
-    returns(RestoreTablesResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
new file mode 100644
index 0000000..82bdd4e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
@@ -0,0 +1,171 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Admin;
+/**
+ * The administrative API for HBase Backup. Obtain an instance from 
+ * an {@link Admin#getBackupAdmin()} and call {@link #close()} afterwards.
+ * <p>BackupAdmin can be used to create backups, restore data from backups and for 
+ * other backup-related operations. 
+ *
+ * @see Admin
+ * @since 2.0
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+
+public interface BackupAdmin extends Closeable{
+
+  /**
+   * Backs up given list of tables fully. Synchronous operation.
+   * 
+   * @param request BackupRequest instance which contains the following members:
+   *  type whether the backup is full or incremental
+   *  tableList list of tables to backup
+   *  targetRootDir root directory for saving the backup
+   *  workers number of parallel workers. -1 - system defined
+   *  bandwidth bandwidth per worker in MB per second. -1 - unlimited
+   * @return the backup Id
+   */
+  
+  public String backupTables(final BackupRequest userRequest) throws IOException;
+  
+  /**
+   * Backs up given list of tables fully. Asynchronous operation.
+   * 
+   * @param request BackupRequest instance which contains the following members:
+   *  type whether the backup is full or incremental
+   *  tableList list of tables to backup
+   *  targetRootDir root dir for saving the backup
+   *  workers number of paralle workers. -1 - system defined
+   *  bandwidth bandwidth per worker in MB per sec. -1 - unlimited
+   * @return the backup Id future
+   */
+  public Future<String> backupTablesAsync(final BackupRequest userRequest) throws IOException;
+
+  /**
+   * Restore backup
+   * @param request - restore request
+   * @throws IOException exception
+   */
+  public void restore(RestoreRequest request) throws IOException;
+
+  /**
+   * Restore backup
+   * @param request - restore request
+   * @return Future which client can wait on
+   * @throws IOException exception
+   */
+  public Future<Void> restoreAsync(RestoreRequest request) throws IOException;
+
+  /**
+   * Describe backup image command
+   * @param backupId - backup id
+   * @return backup info
+   * @throws IOException exception
+   */
+  public BackupInfo getBackupInfo(String backupId) throws IOException;
+
+  /**
+   * Show backup progress command
+   * @param backupId - backup id (may be null)
+   * @return backup progress (0-100%), -1 if no active sessions
+   *  or session not found
+   * @throws IOException exception
+   */
+  public int getProgress(String backupId) throws IOException;
+
+  /**
+   * Delete backup image command
+   * @param backupIds - backup id
+   * @return total number of deleted sessions
+   * @throws IOException exception
+   */
+  public int deleteBackups(String[] backupIds) throws IOException;
+
+  /**
+   * Show backup history command
+   * @param n - last n backup sessions
+   * @return list of backup infos
+   * @throws IOException exception
+   */
+  public List<BackupInfo> getHistory(int n) throws IOException;
+
+
+  /**
+   * Show backup history command with filters
+   * @param n - last n backup sessions
+   * @param f - list of filters
+   * @return list of backup infos
+   * @throws IOException exception
+   */
+  public List<BackupInfo> getHistory(int n, BackupInfo.Filter ... f) throws IOException;
+
+  
+  /**
+   * Backup sets list command - list all backup sets. Backup set is 
+   * a named group of tables. 
+   * @return all registered backup sets
+   * @throws IOException exception
+   */
+  public List<BackupSet> listBackupSets() throws IOException;
+
+  /**
+   * Backup set describe command. Shows list of tables in
+   * this particular backup set.
+   * @param name set name
+   * @return backup set description or null
+   * @throws IOException exception
+   */
+  public BackupSet getBackupSet(String name) throws IOException;
+
+  /**
+   * Delete backup set command
+   * @param name - backup set name
+   * @return true, if success, false - otherwise 
+   * @throws IOException exception
+   */
+  public boolean deleteBackupSet(String name) throws IOException;
+
+  /**
+   * Add tables to backup set command
+   * @param name - name of backup set.
+   * @param tables - list of tables to be added to this set.
+   * @throws IOException exception
+   */
+  public void addToBackupSet(String name, TableName[] tables) throws IOException;
+
+  /**
+   * Remove tables from backup set
+   * @param name - name of backup set.
+   * @param tables - list of tables to be removed from this set.
+   * @throws IOException exception
+   */
+  public void removeFromBackupSet(String name, String[] tables) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
new file mode 100644
index 0000000..be5ffea
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
@@ -0,0 +1,504 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupInfo.Builder;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.TableBackupStatus;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * An object to encapsulate the information for each backup request
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupInfo implements Comparable<BackupInfo> {
+  private static final Log LOG = LogFactory.getLog(BackupInfo.class);
+
+  public static interface Filter {
+    
+    /**
+     * Filter interface
+     * @param info: backup info
+     * @return true if info passes filter, false otherwise 
+     */
+    public boolean apply(BackupInfo info);
+  }
+  // backup status flag
+  public static enum BackupState {
+    WAITING, RUNNING, COMPLETE, FAILED, ANY;
+  }
+
+  // backup phase
+  public static enum BackupPhase {
+    SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
+  }
+
+  // backup id: a timestamp when we request the backup
+  private String backupId;
+
+  // backup type, full or incremental
+  private BackupType type;
+
+  // target root directory for storing the backup files
+  private String targetRootDir;
+
+  // overall backup state
+  private BackupState state;
+
+  // overall backup phase
+  private BackupPhase phase;
+
+  // overall backup failure message
+  private String failedMsg;
+
+  // backup status map for all tables
+  private Map<TableName, BackupStatus> backupStatusMap;
+
+  // actual start timestamp of the backup process
+  private long startTs;
+
+  // actual end timestamp of the backup process, could be fail or complete
+  private long endTs;
+
+  // the total bytes of incremental logs copied
+  private long totalBytesCopied;
+
+  // for incremental backup, the location of the backed-up hlogs
+  private String hlogTargetDir = null;
+
+  // incremental backup file list
+  transient private List<String> incrBackupFileList;
+
+  // new region server log timestamps for table set after distributed log roll
+  // key - table name, value - map of RegionServer hostname -> last log rolled timestamp
+  transient private HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap;
+
+  // backup progress in %% (0-100)
+  private int progress;
+
+  // distributed job id
+  private String jobId;
+
+  // Number of parallel workers. -1 - system defined
+  private int workers = -1;
+
+  // Bandwidth per worker in MB per sec. -1 - unlimited
+  private long bandwidth = -1;
+
+  public BackupInfo() {
+    backupStatusMap = new HashMap<TableName, BackupStatus>();
+  }
+
+  public BackupInfo(String backupId, BackupType type, TableName[] tables, String targetRootDir) {
+    this();
+    this.backupId = backupId;
+    this.type = type;
+    this.targetRootDir = targetRootDir;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("CreateBackupContext: " + tables.length + " " + tables[0]);
+    }
+    this.addTables(tables);
+
+    if (type == BackupType.INCREMENTAL) {
+      setHlogTargetDir(BackupClientUtil.getLogBackupDir(targetRootDir, backupId));
+    }
+
+    this.startTs = 0;
+    this.endTs = 0;
+  }
+
+  public String getJobId() {
+    return jobId;
+  }
+
+  public void setJobId(String jobId) {
+    this.jobId = jobId;
+  }
+
+  public int getWorkers() {
+    return workers;
+  }
+
+  public void setWorkers(int workers) {
+    this.workers = workers;
+  }
+
+  public long getBandwidth() {
+    return bandwidth;
+  }
+
+  public void setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+  }
+
+  public void setBackupStatusMap(Map<TableName, BackupStatus> backupStatusMap) {
+    this.backupStatusMap = backupStatusMap;
+  }
+
+  public HashMap<TableName, HashMap<String, Long>> getTableSetTimestampMap() {
+    return tableSetTimestampMap;
+  }
+
+  public void
+      setTableSetTimestampMap(HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap) {
+    this.tableSetTimestampMap = tableSetTimestampMap;
+  }
+
+  public String getHlogTargetDir() {
+    return hlogTargetDir;
+  }
+
+  public void setType(BackupType type) {
+    this.type = type;
+  }
+
+  public void setTargetRootDir(String targetRootDir) {
+    this.targetRootDir = targetRootDir;
+  }
+
+  public void setTotalBytesCopied(long totalBytesCopied) {
+    this.totalBytesCopied = totalBytesCopied;
+  }
+
+  /**
+   * Set progress (0-100%)
+   * @param msg progress value
+   */
+
+  public void setProgress(int p) {
+    this.progress = p;
+  }
+
+  /**
+   * Get current progress
+   */
+  public int getProgress() {
+    return progress;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  public void setBackupId(String backupId) {
+    this.backupId = backupId;
+  }
+
+  public BackupStatus getBackupStatus(TableName table) {
+    return this.backupStatusMap.get(table);
+  }
+
+  public String getFailedMsg() {
+    return failedMsg;
+  }
+
+  public void setFailedMsg(String failedMsg) {
+    this.failedMsg = failedMsg;
+  }
+
+  public long getStartTs() {
+    return startTs;
+  }
+
+  public void setStartTs(long startTs) {
+    this.startTs = startTs;
+  }
+
+  public long getEndTs() {
+    return endTs;
+  }
+
+  public void setEndTs(long endTs) {
+    this.endTs = endTs;
+  }
+
+  public long getTotalBytesCopied() {
+    return totalBytesCopied;
+  }
+
+  public BackupState getState() {
+    return state;
+  }
+
+  public void setState(BackupState flag) {
+    this.state = flag;
+  }
+
+  public BackupPhase getPhase() {
+    return phase;
+  }
+
+  public void setPhase(BackupPhase phase) {
+    this.phase = phase;
+  }
+
+  public BackupType getType() {
+    return type;
+  }
+
+  public void setSnapshotName(TableName table, String snapshotName) {
+    this.backupStatusMap.get(table).setSnapshotName(snapshotName);
+  }
+
+  public String getSnapshotName(TableName table) {
+    return this.backupStatusMap.get(table).getSnapshotName();
+  }
+
+  public List<String> getSnapshotNames() {
+    List<String> snapshotNames = new ArrayList<String>();
+    for (BackupStatus backupStatus : this.backupStatusMap.values()) {
+      snapshotNames.add(backupStatus.getSnapshotName());
+    }
+    return snapshotNames;
+  }
+
+  public Set<TableName> getTables() {
+    return this.backupStatusMap.keySet();
+  }
+
+  public List<TableName> getTableNames() {
+    return new ArrayList<TableName>(backupStatusMap.keySet());
+  }
+
+  public void addTables(TableName[] tables) {
+    for (TableName table : tables) {
+      BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId);
+      this.backupStatusMap.put(table, backupStatus);
+    }
+  }
+
+  public void setTables(List<TableName> tables) {
+    this.backupStatusMap.clear();
+    for (TableName table : tables) {
+      BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, this.backupId);
+      this.backupStatusMap.put(table, backupStatus);
+    }
+  }
+
+  public String getTargetRootDir() {
+    return targetRootDir;
+  }
+
+  public void setHlogTargetDir(String hlogTagetDir) {
+    this.hlogTargetDir = hlogTagetDir;
+  }
+
+  public String getHLogTargetDir() {
+    return hlogTargetDir;
+  }
+
+  public List<String> getIncrBackupFileList() {
+    return incrBackupFileList;
+  }
+
+  public void setIncrBackupFileList(List<String> incrBackupFileList) {
+    this.incrBackupFileList = incrBackupFileList;
+  }
+
+  /**
+   * Set the new region server log timestamps after distributed log roll
+   * @param newTableSetTimestampMap table timestamp map
+   */
+  public void
+      setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap) {
+    this.tableSetTimestampMap = newTableSetTimestampMap;
+  }
+
+  /**
+   * Get new region server log timestamps after distributed log roll
+   * @return new region server log timestamps
+   */
+  public HashMap<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+    return this.tableSetTimestampMap;
+  }
+
+  public TableName getTableBySnapshot(String snapshotName) {
+    for (Entry<TableName, BackupStatus> entry : this.backupStatusMap.entrySet()) {
+      if (snapshotName.equals(entry.getValue().getSnapshotName())) {
+        return entry.getKey();
+      }
+    }
+    return null;
+  }
+
+  public BackupProtos.BackupInfo toProtosBackupInfo() {
+    BackupProtos.BackupInfo.Builder builder = BackupProtos.BackupInfo.newBuilder();
+    builder.setBackupId(getBackupId());
+    setBackupStatusMap(builder);
+    builder.setEndTs(getEndTs());
+    if (getFailedMsg() != null) {
+      builder.setFailedMessage(getFailedMsg());
+    }
+    if (getState() != null) {
+      builder.setState(BackupProtos.BackupInfo.BackupState.valueOf(getState().name()));
+    }
+    if (getPhase() != null) {
+      builder.setPhase(BackupProtos.BackupInfo.BackupPhase.valueOf(getPhase().name()));
+    }
+
+    builder.setProgress(getProgress());
+    builder.setStartTs(getStartTs());
+    builder.setTargetRootDir(getTargetRootDir());
+    builder.setType(BackupProtos.BackupType.valueOf(getType().name()));
+    builder.setWorkersNumber(workers);
+    builder.setBandwidth(bandwidth);
+    if (jobId != null) {
+      builder.setJobId(jobId);
+    }
+    return builder.build();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof BackupInfo) {
+      BackupInfo other = (BackupInfo) obj;
+      try {
+        return Bytes.equals(toByteArray(), other.toByteArray());
+      } catch (IOException e) {
+        LOG.error(e);
+        return false;
+      }
+    } else {
+      return false;
+    }
+  }
+
+  public byte[] toByteArray() throws IOException {
+    return toProtosBackupInfo().toByteArray();
+  }
+
+  private void setBackupStatusMap(Builder builder) {
+    for (Entry<TableName, BackupStatus> entry : backupStatusMap.entrySet()) {
+      builder.addTableBackupStatus(entry.getValue().toProto());
+    }
+  }
+
+  public static BackupInfo fromByteArray(byte[] data) throws IOException {
+    return fromProto(BackupProtos.BackupInfo.parseFrom(data));
+  }
+
+  public static BackupInfo fromStream(final InputStream stream) throws IOException {
+    return fromProto(BackupProtos.BackupInfo.parseDelimitedFrom(stream));
+  }
+
+  public static BackupInfo fromProto(BackupProtos.BackupInfo proto) {
+    BackupInfo context = new BackupInfo();
+    context.setBackupId(proto.getBackupId());
+    context.setBackupStatusMap(toMap(proto.getTableBackupStatusList()));
+    context.setEndTs(proto.getEndTs());
+    if (proto.hasFailedMessage()) {
+      context.setFailedMsg(proto.getFailedMessage());
+    }
+    if (proto.hasState()) {
+      context.setState(BackupInfo.BackupState.valueOf(proto.getState().name()));
+    }
+
+    context.setHlogTargetDir(BackupClientUtil.getLogBackupDir(proto.getTargetRootDir(),
+      proto.getBackupId()));
+
+    if (proto.hasPhase()) {
+      context.setPhase(BackupPhase.valueOf(proto.getPhase().name()));
+    }
+    if (proto.hasProgress()) {
+      context.setProgress(proto.getProgress());
+    }
+    context.setStartTs(proto.getStartTs());
+    context.setTargetRootDir(proto.getTargetRootDir());
+    context.setType(BackupType.valueOf(proto.getType().name()));
+    context.setWorkers(proto.getWorkersNumber());
+    context.setBandwidth(proto.getBandwidth());
+    if (proto.hasJobId()) {
+      context.setJobId(proto.getJobId());
+    }
+    return context;
+  }
+
+  private static Map<TableName, BackupStatus> toMap(List<TableBackupStatus> list) {
+    HashMap<TableName, BackupStatus> map = new HashMap<>();
+    for (TableBackupStatus tbs : list) {
+      map.put(ProtobufUtil.toTableName(tbs.getTable()), BackupStatus.convert(tbs));
+    }
+    return map;
+  }
+
+  public String getShortDescription() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("ID             : " + backupId).append("\n");
+    sb.append("Type           : " + getType()).append("\n");
+    sb.append("Tables         : " + getTableListAsString()).append("\n");
+    sb.append("State          : " + getState()).append("\n");
+    Date date = null;
+    Calendar cal = Calendar.getInstance();
+    cal.setTimeInMillis(getStartTs());
+    date = cal.getTime();
+    sb.append("Start time     : " + date).append("\n");
+    if (state == BackupState.FAILED) {
+      sb.append("Failed message : " + getFailedMsg()).append("\n");
+    } else if (state == BackupState.RUNNING) {
+      sb.append("Phase          : " + getPhase()).append("\n");
+    } else if (state == BackupState.COMPLETE) {
+      cal = Calendar.getInstance();
+      cal.setTimeInMillis(getEndTs());
+      date = cal.getTime();
+      sb.append("End time       : " + date).append("\n");
+    }
+    sb.append("Progress       : " + getProgress()).append("\n");
+    return sb.toString();
+  }
+
+  public String getStatusAndProgressAsString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("id: ").append(getBackupId()).append(" state: ").append(getState())
+        .append(" progress: ").append(getProgress());
+    return sb.toString();
+  }
+
+  public String getTableListAsString() {
+    return StringUtils.join(backupStatusMap.keySet(), ",");
+  }
+
+  @Override
+  public int compareTo(BackupInfo o) {
+    Long thisTS = new Long(this.getBackupId().substring(this.getBackupId().lastIndexOf("_") + 1));
+    Long otherTS = new Long(o.getBackupId().substring(o.getBackupId().lastIndexOf("_") + 1));
+    return thisTS.compareTo(otherTS);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
new file mode 100644
index 0000000..d141239
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRequest.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * POJO class for backup request
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class BackupRequest {
+  private BackupType type;
+  private List<TableName> tableList;
+  private String targetRootDir;
+  private int workers = -1;
+  private long bandwidth = -1L;
+  private String backupSetName;
+
+  public BackupRequest() {
+  }
+
+  public BackupRequest setBackupType(BackupType type) {
+    this.type = type;
+    return this;
+  }
+  public BackupType getBackupType() {
+    return this.type;
+  }
+
+  public BackupRequest setTableList(List<TableName> tableList) {
+    this.tableList = tableList;
+    return this;
+  }
+  public List<TableName> getTableList() {
+    return this.tableList;
+  }
+
+  public BackupRequest setTargetRootDir(String targetRootDir) {
+    this.targetRootDir = targetRootDir;
+    return this;
+  }
+  public String getTargetRootDir() {
+    return this.targetRootDir;
+  }
+
+  public BackupRequest setWorkers(int workers) {
+    this.workers = workers;
+    return this;
+  }
+  public int getWorkers() {
+    return this.workers;
+  }
+
+  public BackupRequest setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+    return this;
+  }
+  public long getBandwidth() {
+    return this.bandwidth;
+  }
+
+  public String getBackupSetName() {
+    return backupSetName;
+  }
+
+  public void setBackupSetName(String backupSetName) {
+    this.backupSetName = backupSetName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
new file mode 100644
index 0000000..c82e05a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+
+/**
+ * Backup status and related information encapsulated for a table.
+ * At this moment only TargetDir and SnapshotName is encapsulated here.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupStatus implements Serializable {
+
+  private static final long serialVersionUID = -5968397963548535982L;
+
+  // table name for backup
+  private TableName table;
+
+  // target directory of the backup image for this table
+  private String targetDir;
+
+  // snapshot name for offline/online snapshot
+  private String snapshotName = null;
+
+  public BackupStatus() {
+
+  }
+
+  public BackupStatus(TableName table, String targetRootDir, String backupId) {
+    this.table = table;
+    this.targetDir = BackupClientUtil.getTableBackupDir(targetRootDir, backupId, table);
+  }
+
+  public String getSnapshotName() {
+    return snapshotName;
+  }
+
+  public void setSnapshotName(String snapshotName) {
+    this.snapshotName = snapshotName;
+  }
+
+  public String getTargetDir() {
+    return targetDir;
+  }
+
+  public TableName getTable() {
+    return table;
+  }
+
+  public void setTable(TableName table) {
+    this.table = table;
+  }
+
+  public void setTargetDir(String targetDir) {
+    this.targetDir = targetDir;
+  }
+
+  public static BackupStatus convert(BackupProtos.TableBackupStatus proto)
+  {
+    BackupStatus bs = new BackupStatus();
+    bs.setTable(ProtobufUtil.toTableName(proto.getTable()));
+    bs.setTargetDir(proto.getTargetDir());
+    if(proto.hasSnapshot()){
+      bs.setSnapshotName(proto.getSnapshot());
+    }
+    return bs;
+  }
+
+  public BackupProtos.TableBackupStatus toProto() {
+    BackupProtos.TableBackupStatus.Builder builder =
+        BackupProtos.TableBackupStatus.newBuilder();
+    if(snapshotName != null) {
+      builder.setSnapshot(snapshotName);
+    }
+    builder.setTable(ProtobufUtil.toProtoTableName(table));
+    builder.setTargetDir(targetDir);
+    return builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
index d3237f7..ce3bb65 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
@@ -29,9 +29,9 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.impl.HBaseBackupAdmin;
 import org.apache.hadoop.hbase.backup.util.BackupServerUtil;
 import org.apache.hadoop.hbase.backup.util.RestoreServerUtil;
-import org.apache.hadoop.hbase.client.BackupAdmin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
@@ -124,7 +124,7 @@ public class RestoreDriver extends AbstractHBaseTool {
     String tables = null;
     String tableMapping = null;
     try (final Connection conn = ConnectionFactory.createConnection(conf);
-        BackupAdmin client = conn.getAdmin().getBackupAdmin();) {
+        BackupAdmin client = new HBaseBackupAdmin(conn);) {
       // Check backup set
       if (cmd.hasOption(OPTION_SET)) {
         String setName = cmd.getOptionValue(OPTION_SET);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
new file mode 100644
index 0000000..7490d20
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * POJO class for restore request
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RestoreRequest {
+
+  private String backupRootDir;
+  private String backupId;
+  private boolean check = false;
+  private TableName[] fromTables;
+  private TableName[] toTables;
+  private boolean overwrite = false;
+
+  public RestoreRequest() {
+  }
+
+  public String getBackupRootDir() {
+    return backupRootDir;
+  }
+
+  public RestoreRequest setBackupRootDir(String backupRootDir) {
+    this.backupRootDir = backupRootDir;
+    return this;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  public RestoreRequest setBackupId(String backupId) {
+    this.backupId = backupId;
+    return this;
+  }
+
+  public boolean isCheck() {
+    return check;
+  }
+
+  public RestoreRequest setCheck(boolean check) {
+    this.check = check;
+    return this;
+  }
+
+  public TableName[] getFromTables() {
+    return fromTables;
+  }
+
+  public RestoreRequest setFromTables(TableName[] fromTables) {
+    this.fromTables = fromTables;
+    return this;
+  }
+
+  public TableName[] getToTables() {
+    return toTables;
+  }
+
+  public RestoreRequest setToTables(TableName[] toTables) {
+    this.toTables = toTables;
+    return this;
+  }
+
+  public boolean isOverwrite() {
+    return overwrite;
+  }
+
+  public RestoreRequest setOverwrite(boolean overwrite) {
+    this.overwrite = overwrite;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
new file mode 100644
index 0000000..478d62d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
@@ -0,0 +1,720 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * General backup commands, options and usage messages
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class BackupCommands {
+  
+  public final static String INCORRECT_USAGE = "Incorrect usage";
+
+  public static final String USAGE = "Usage: hbase backup COMMAND [command-specific arguments]\n"
+      + "where COMMAND is one of:\n" 
+      + "  create     create a new backup image\n"
+      + "  delete     delete an existing backup image\n"
+      + "  describe   show the detailed information of a backup image\n"
+      + "  history    show history of all successful backups\n"
+      + "  progress   show the progress of the latest backup request\n"
+      + "  set        backup set management\n"
+      + "Run \'hbase backup COMMAND -h\' to see help message for each command\n";
+
+  public static final String CREATE_CMD_USAGE =
+      "Usage: hbase backup create <type> <BACKUP_ROOT> [tables] [-set name] "
+          + "[-w workers][-b bandwith]\n" 
+          + " type           \"full\" to create a full backup image\n"
+          + "                \"incremental\" to create an incremental backup image\n"
+          + " BACKUP_ROOT     The full root path to store the backup image,\n"
+          + "                 the prefix can be hdfs, webhdfs or gpfs\n" 
+          + "Options:\n"
+          + " tables          If no tables (\"\") are specified, all tables are backed up.\n"
+          + "                 Otherwise it is a comma separated list of tables.\n"
+          + " -w              number of parallel workers (MapReduce tasks).\n" 
+          + " -b              bandwith per one worker (MapReduce task) in MBs per sec\n" 
+          + " -set            name of backup set to use (mutually exclusive with [tables])" ;
+
+  public static final String PROGRESS_CMD_USAGE = "Usage: hbase backup progress <backupId>\n"
+          + " backupId        backup image id\n";
+  public static final String NO_INFO_FOUND = "No info was found for backup id: ";
+
+  public static final String DESCRIBE_CMD_USAGE = "Usage: hbase backup decsribe <backupId>\n"
+          + " backupId        backup image id\n";
+
+  public static final String HISTORY_CMD_USAGE = 
+      "Usage: hbase backup history [-path BACKUP_ROOT] [-n N] [-t table]\n"
+       + " -n N            show up to N last backup sessions, default - 10\n"
+       + " -path           backup root path\n"
+       + " -t table        table name. If specified, only backup images which contain this table\n"
+       + "                 will be listed."  ;
+  
+
+  public static final String DELETE_CMD_USAGE = "Usage: hbase backup delete <backupId>\n"
+          + " backupId        backup image id\n";
+
+  public static final String CANCEL_CMD_USAGE = "Usage: hbase backup cancel <backupId>\n"
+          + " backupId        backup image id\n";
+
+  public static final String SET_CMD_USAGE = "Usage: hbase backup set COMMAND [name] [tables]\n"
+         + " name            Backup set name\n"
+         + " tables          If no tables (\"\") are specified, all tables will belong to the set.\n"
+         + "                 Otherwise it is a comma separated list of tables.\n"
+         + "COMMAND is one of:\n" 
+         + " add             add tables to a set, create a set if needed\n"
+         + " remove          remove tables from a set\n"
+         + " list            list all backup sets in the system\n"
+         + " describe        describe set\n"
+         + " delete          delete backup set\n";
+
+  public static abstract class Command extends Configured {
+    CommandLine cmdline;
+    
+    Command(Configuration conf) {
+      super(conf);
+    }
+    
+    public void execute() throws IOException
+    {
+      if (cmdline.hasOption("h") || cmdline.hasOption("help")) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+    
+    protected abstract void printUsage();
+  }
+
+  private BackupCommands() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  public static Command createCommand(Configuration conf, BackupCommand type, CommandLine cmdline) {
+    Command cmd = null;
+    switch (type) {
+    case CREATE:
+      cmd = new CreateCommand(conf, cmdline);
+      break;
+    case DESCRIBE:
+      cmd = new DescribeCommand(conf, cmdline);
+      break;
+    case PROGRESS:
+      cmd = new ProgressCommand(conf, cmdline);
+      break;
+    case DELETE:
+      cmd = new DeleteCommand(conf, cmdline);
+      break;
+    case CANCEL:
+      cmd = new CancelCommand(conf, cmdline);
+      break;
+    case HISTORY:
+      cmd = new HistoryCommand(conf, cmdline);
+      break;
+    case SET:
+      cmd = new BackupSetCommand(conf, cmdline);
+      break;
+    case HELP:
+    default:
+      cmd = new HelpCommand(conf, cmdline);
+      break;
+    }
+    return cmd;
+  }
+
+  static int numOfArgs(String[] args) {
+    if (args == null) return 0;
+    return args.length;
+  }
+
+  public static class CreateCommand extends Command {
+
+    CreateCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+    
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null || cmdline.getArgs() == null) {
+        System.err.println("ERROR: missing arguments");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length < 3 || args.length > 4) {
+        System.err.println("ERROR: wrong number of arguments: "+ args.length);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      if (!BackupType.FULL.toString().equalsIgnoreCase(args[1])
+          && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[1])) {
+        System.err.println("ERROR: invalid backup type: "+ args[1]);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+            
+      String tables = null;
+      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+
+      // Check backup set
+      String setName = null;
+      if (cmdline.hasOption("set")) {
+        setName = cmdline.getOptionValue("set");
+        tables = getTablesForSet(setName, conf);
+
+        if (tables == null) {
+          System.err.println("ERROR: Backup set '" + setName+ "' is either empty or does not exist");
+          printUsage();
+          throw new IOException(INCORRECT_USAGE);
+        }
+      } else {
+        tables = (args.length == 4) ? args[3] : null;
+      }
+      int bandwidth = cmdline.hasOption('b') ? Integer.parseInt(cmdline.getOptionValue('b')) : -1;
+      int workers = cmdline.hasOption('w') ? Integer.parseInt(cmdline.getOptionValue('w')) : -1;
+
+      try (Connection conn = ConnectionFactory.createConnection(getConf());
+          HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);) {
+        BackupRequest request = new BackupRequest();
+        request.setBackupType(BackupType.valueOf(args[1].toUpperCase()))
+        .setTableList(tables != null?Lists.newArrayList(BackupClientUtil.parseTableNames(tables)): null)
+        .setTargetRootDir(args[2]).setWorkers(workers).setBandwidth(bandwidth)
+        .setBackupSetName(setName);
+        
+        String backupId = admin.backupTables(request);
+        System.out.println("Backup session "+ backupId+" finished. Status: SUCCESS");
+      } catch (IOException e) {
+        System.err.println("Backup session finished. Status: FAILURE");
+        throw e;
+      }
+    }
+    
+
+
+    private String getTablesForSet(String name, Configuration conf)
+        throws IOException {
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupSystemTable table = new BackupSystemTable(conn)) {
+        List<TableName> tables = table.describeBackupSet(name);
+        if (tables == null) return null;
+        return StringUtils.join(tables, BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);        
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.err.println(CREATE_CMD_USAGE);      
+    }
+  }
+
+  private static class HelpCommand extends Command {
+
+    HelpCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String[] args = cmdline.getArgs();
+      if (args == null || args.length == 0) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      if (args.length != 2) {
+        System.err.println("Only supports help message of a single command type");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String type = args[1];
+
+      if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) {
+        System.out.println(CREATE_CMD_USAGE);
+      } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(type)) {
+        System.out.println(DESCRIBE_CMD_USAGE);
+      } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(type)) {
+        System.out.println(HISTORY_CMD_USAGE);
+      } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(type)) {
+        System.out.println(PROGRESS_CMD_USAGE);
+      } else if (BackupCommand.DELETE.name().equalsIgnoreCase(type)) {
+        System.out.println(DELETE_CMD_USAGE);
+      } else if (BackupCommand.CANCEL.name().equalsIgnoreCase(type)) {
+        System.out.println(CANCEL_CMD_USAGE);
+      } else if (BackupCommand.SET.name().equalsIgnoreCase(type)) {
+        System.out.println(SET_CMD_USAGE);
+      } else {
+        System.out.println("Unknown command : " + type);
+        printUsage();
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.err.println(USAGE);      
+    }
+  }
+
+  private static class DescribeCommand extends Command {
+
+    DescribeCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null || cmdline.getArgs() == null) {
+        System.err.println("ERROR: missing arguments");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length != 2) {
+        System.err.println("ERROR: wrong number of arguments");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+            
+      String backupId = args[1];
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+        BackupInfo info = sysTable.readBackupInfo(backupId);
+        if (info == null) {
+          System.err.println("ERROR: " + backupId + " does not exist");
+          printUsage();
+          throw new IOException(INCORRECT_USAGE);
+        }
+        System.out.println(info.getShortDescription());
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.err.println(DESCRIBE_CMD_USAGE);
+    }
+  }
+
+  private static class ProgressCommand extends Command {
+
+    ProgressCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      
+      if (cmdline == null || cmdline.getArgs() == null ||
+          cmdline.getArgs().length == 1) {
+        System.err.println("No backup id was specified, "
+            + "will retrieve the most recent (ongoing) sessions");
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length > 2) {
+        System.err.println("ERROR: wrong number of arguments: " + args.length);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String backupId = (args == null || args.length <= 1) ? null : args[1];
+      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+      try(final Connection conn = ConnectionFactory.createConnection(conf); 
+          final BackupSystemTable sysTable = new BackupSystemTable(conn);){
+        BackupInfo info = sysTable.readBackupInfo(backupId);
+        int progress = info == null? -1: info.getProgress();
+        if(progress < 0){
+          System.err.println(NO_INFO_FOUND + backupId);
+        } else{
+          System.out.println(backupId+" progress=" + progress+"%");
+        }
+      } 
+    }
+
+    @Override
+    protected void printUsage() {
+      System.err.println(PROGRESS_CMD_USAGE);      
+    }
+  }
+
+  private static class DeleteCommand extends Command {
+    
+    DeleteCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        System.err.println("No backup id(s) was specified");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+            
+      String[] args = cmdline.getArgs();
+
+      String[] backupIds = new String[args.length - 1];
+      System.arraycopy(args, 1, backupIds, 0, backupIds.length);
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);) {
+        int deleted = admin.deleteBackups(args);
+        System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
+      }
+
+    }
+
+    @Override
+    protected void printUsage() {
+      System.err.println(DELETE_CMD_USAGE);      
+    }
+  }
+
+// TODO Cancel command  
+  
+  private static class CancelCommand extends Command {
+
+    CancelCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        System.out.println("No backup id(s) was specified, will use the most recent one");
+      }
+      String[] args = cmdline.getArgs();
+      String backupId = args == null || args.length == 0 ? null : args[1];
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);) {
+        // TODO cancel backup
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+    }
+  }
+
+  private static class HistoryCommand extends Command {
+    
+    private final static int DEFAULT_HISTORY_LENGTH = 10;
+    
+    HistoryCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+
+      super.execute();
+
+      int n = parseHistoryLength();
+      final TableName tableName = getTableName();
+      final String setName = getTableSetName();
+      BackupInfo.Filter tableNameFilter = new BackupInfo.Filter() {
+        @Override
+        public boolean apply(BackupInfo info) {
+          if (tableName == null) return true;
+          List<TableName> names = info.getTableNames();
+          return names.contains(tableName);
+        }
+      };
+      BackupInfo.Filter tableSetFilter = new BackupInfo.Filter() {
+        @Override
+        public boolean apply(BackupInfo info) {
+          if (setName == null) return true;
+          String backupId = info.getBackupId();
+          return backupId.startsWith(setName);
+        }
+      };                
+      Path backupRootPath = getBackupRootPath();
+      List<BackupInfo> history = null;
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      if (backupRootPath == null) {
+        // Load from hbase:backup
+        try (final Connection conn = ConnectionFactory.createConnection(conf);
+             final BackupSystemTable sysTable = new BackupSystemTable(conn);) {
+ 
+          history = sysTable.getBackupHistory(n, tableNameFilter, tableSetFilter);
+        }
+      } else {
+        // load from backup FS
+        history = BackupClientUtil.getHistory(conf, n, backupRootPath, 
+          tableNameFilter, tableSetFilter);
+      }
+      for (BackupInfo info : history) {
+        System.out.println(info.getShortDescription());
+      }
+    }
+    
+    private Path getBackupRootPath() throws IOException {
+      String value = null;
+      try{
+        value = cmdline.getOptionValue("path");
+        if (value == null) return null;
+        return new Path(value);
+      } catch (IllegalArgumentException e) {
+        System.err.println("ERROR: Illegal argument for backup root path: "+ value);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    private TableName getTableName() throws IOException {
+      String value = cmdline.getOptionValue("t"); 
+      if (value == null) return null;
+      try{
+        return TableName.valueOf(value);
+      } catch (IllegalArgumentException e){
+        System.err.println("Illegal argument for table name: "+ value);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    private String getTableSetName() throws IOException {
+      String value = cmdline.getOptionValue("set"); 
+      return value;
+    }
+    
+    private int parseHistoryLength() throws IOException {
+      String value = cmdline.getOptionValue("n");
+      try{
+        if (value == null) return DEFAULT_HISTORY_LENGTH;
+        return Integer.parseInt(value);
+      } catch(NumberFormatException e) {
+        System.err.println("Illegal argument for history length: "+ value);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.err.println(HISTORY_CMD_USAGE);      
+    }
+  }
+
+  private static class BackupSetCommand extends Command {
+    private final static String SET_ADD_CMD = "add";
+    private final static String SET_REMOVE_CMD = "remove";
+    private final static String SET_DELETE_CMD = "delete";
+    private final static String SET_DESCRIBE_CMD = "describe";
+    private final static String SET_LIST_CMD = "list";
+
+    BackupSetCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();      
+      // Command-line must have at least one element
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        System.err.println("ERROR: Command line format");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+            
+      String[] args = cmdline.getArgs();
+      String cmdStr = args[1];
+      BackupCommand cmd = getCommand(cmdStr);
+
+      switch (cmd) {
+      case SET_ADD:
+        processSetAdd(args);
+        break;
+      case SET_REMOVE:
+        processSetRemove(args);
+        break;
+      case SET_DELETE:
+        processSetDelete(args);
+        break;
+      case SET_DESCRIBE:
+        processSetDescribe(args);
+        break;
+      case SET_LIST:
+        processSetList(args);
+        break;
+      default:
+        break;
+
+      }
+    }
+
+    private void processSetList(String[] args) throws IOException {
+      // List all backup set names
+      // does not expect any args
+      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+      try(final Connection conn = ConnectionFactory.createConnection(conf); 
+          HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){
+        List<BackupSet> list = admin.listBackupSets();
+        for(BackupSet bs: list){
+          System.out.println(bs);
+        }
+      }
+    }
+
+    private void processSetDescribe(String[] args) throws IOException {
+      if (args == null || args.length != 3) {
+        System.err.println("ERROR: Wrong number of args for 'set describe' command: "
+            + numOfArgs(args));
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String setName = args[2];
+      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+      try(final Connection conn = ConnectionFactory.createConnection(conf); 
+          final BackupSystemTable sysTable = new BackupSystemTable(conn);){
+        List<TableName> tables = sysTable.describeBackupSet(setName);
+        BackupSet set = tables == null? null : new BackupSet(setName, tables);
+        if(set == null) {
+          System.out.println("Set '"+setName+"' does not exist.");
+        } else{
+          System.out.println(set);
+        }
+      }
+    }
+
+    private void processSetDelete(String[] args) throws IOException {
+      if (args == null || args.length != 3) {
+        System.err.println("ERROR: Wrong number of args for 'set delete' command: "
+            + numOfArgs(args));
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String setName = args[2];
+      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+      try(final Connection conn = ConnectionFactory.createConnection(conf); 
+          final HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){
+        boolean result = admin.deleteBackupSet(setName);
+        if(result){
+          System.out.println("Delete set "+setName+" OK.");
+        } else{
+          System.out.println("Set "+setName+" does not exist");
+        }
+      }
+    }
+
+    private void processSetRemove(String[] args) throws IOException {
+      if (args == null || args.length != 4) {
+        System.err.println("ERROR: Wrong number of args for 'set remove' command: "
+            + numOfArgs(args));
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      
+      String setName = args[2];
+      String[] tables = args[3].split(",");
+      Configuration conf = getConf() != null? getConf(): HBaseConfiguration.create();
+      try(final Connection conn = ConnectionFactory.createConnection(conf); 
+          final HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){
+        admin.removeFromBackupSet(setName, tables);
+      }
+    }
+
+    private void processSetAdd(String[] args) throws IOException {
+      if (args == null || args.length != 4) {
+        System.err.println("ERROR: Wrong number of args for 'set add' command: "
+            + numOfArgs(args));
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String setName = args[2];
+      String[] tables = args[3].split(",");
+      TableName[] tableNames = new TableName[tables.length];
+      for(int i=0; i < tables.length; i++){
+        tableNames[i] = TableName.valueOf(tables[i]);
+      }
+      Configuration conf = getConf() != null? getConf():HBaseConfiguration.create();
+      try(final Connection conn = ConnectionFactory.createConnection(conf); 
+          final HBaseBackupAdmin admin = new HBaseBackupAdmin(conn);){
+        admin.addToBackupSet(setName, tableNames);
+      }
+      
+    }
+
+    private BackupCommand getCommand(String cmdStr) throws IOException {
+      if (cmdStr.equals(SET_ADD_CMD)) {
+        return BackupCommand.SET_ADD;
+      } else if (cmdStr.equals(SET_REMOVE_CMD)) {
+        return BackupCommand.SET_REMOVE;
+      } else if (cmdStr.equals(SET_DELETE_CMD)) {
+        return BackupCommand.SET_DELETE;
+      } else if (cmdStr.equals(SET_DESCRIBE_CMD)) {
+        return BackupCommand.SET_DESCRIBE;
+      } else if (cmdStr.equals(SET_LIST_CMD)) {
+        return BackupCommand.SET_LIST;
+      } else {
+        System.err.println("ERROR: Unknown command for 'set' :" + cmdStr);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.err.println(SET_CMD_USAGE);
+    }
+
+  }  
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b14e2ab1/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
new file mode 100644
index 0000000..ca204b4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupException.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.impl;
+
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Backup exception
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupException extends HBaseIOException {
+  private BackupInfo description;
+
+  /**
+   * Some exception happened for a backup and don't even know the backup that it was about
+   * @param msg Full description of the failure
+   */
+  public BackupException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * Some exception happened for a backup with a cause
+   * @param cause the cause
+   */
+  public BackupException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * Exception for the given backup that has no previous root cause
+   * @param msg reason why the backup failed
+   * @param desc description of the backup that is being failed
+   */
+  public BackupException(String msg, BackupInfo desc) {
+    super(msg);
+    this.description = desc;
+  }
+
+  /**
+   * Exception for the given backup due to another exception
+   * @param msg reason why the backup failed
+   * @param cause root cause of the failure
+   * @param desc description of the backup that is being failed
+   */
+  public BackupException(String msg, Throwable cause, BackupInfo desc) {
+    super(msg, cause);
+    this.description = desc;
+  }
+
+  /**
+   * Exception when the description of the backup cannot be determined, due to some other root
+   * cause
+   * @param message description of what caused the failure
+   * @param e root cause
+   */
+  public BackupException(String message, Exception e) {
+    super(message, e);
+  }
+
+  public BackupInfo getBackupContext() {
+    return this.description;
+  }
+
+}