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/12/07 21:08:13 UTC

[2/3] hbase git commit: HBASE-14123 patch v40 (Vladimir)

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/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
index ab3f0f6..381cc0f 100644
--- 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
@@ -32,6 +32,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -53,7 +54,6 @@ 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.shaded.protobuf.generated.BackupProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -90,7 +90,8 @@ public final class BackupSystemTable implements Closeable {
 
     @Override
     public String toString() {
-      return "/" + backupRoot + "/" + backupId + "/" + walFile;
+      return Path.SEPARATOR + backupRoot +
+          Path.SEPARATOR + backupId + Path.SEPARATOR + walFile;
     }
 
   }
@@ -121,8 +122,8 @@ public final class BackupSystemTable implements Closeable {
    */
   public void updateBackupInfo(BackupInfo context) throws IOException {
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("update backup status in hbase:backup for: " + context.getBackupId()
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("update backup status in hbase:backup for: " + context.getBackupId()
           + " set status=" + context.getState());
     }
     try (Table table = connection.getTable(tableName)) {
@@ -139,8 +140,8 @@ public final class BackupSystemTable implements Closeable {
 
   public void deleteBackupInfo(String backupId) throws IOException {
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("delete backup status in hbase:backup for " + backupId);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("delete backup status in hbase:backup for " + backupId);
     }
     try (Table table = connection.getTable(tableName)) {
       Delete del = BackupSystemTableHelper.createDeleteForBackupInfo(backupId);
@@ -155,8 +156,8 @@ public final class BackupSystemTable implements Closeable {
    */
 
   public BackupInfo readBackupInfo(String backupId) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read backup status from hbase:backup for: " + backupId);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read backup status from hbase:backup for: " + backupId);
     }
 
     try (Table table = connection.getTable(tableName)) {
@@ -178,8 +179,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException exception
    */
   public String readBackupStartCode(String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read backup start code from hbase:backup");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read backup start code from hbase:backup");
     }
     try (Table table = connection.getTable(tableName)) {
       Get get = BackupSystemTableHelper.createGetForStartCode(backupRoot);
@@ -203,8 +204,8 @@ public final class BackupSystemTable implements Closeable {
    * @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);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("write backup start code to hbase:backup " + startCode);
     }
     try (Table table = connection.getTable(tableName)) {
       Put put = BackupSystemTableHelper.createPutForStartCode(startCode.toString(), backupRoot);
@@ -220,8 +221,8 @@ public final class BackupSystemTable implements Closeable {
    */
   public HashMap<String, Long> readRegionServerLastLogRollResult(String backupRoot)
       throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read region server last roll log result to hbase:backup");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read region server last roll log result to hbase:backup");
     }
 
     Scan scan = BackupSystemTableHelper.createScanForReadRegionServerLastLogRollResult(backupRoot);
@@ -252,8 +253,8 @@ public final class BackupSystemTable implements Closeable {
    */
   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");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("write region server last roll log result to hbase:backup");
     }
     try (Table table = connection.getTable(tableName)) {
       Put put =
@@ -269,8 +270,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException exception
    */
   public ArrayList<BackupInfo> getBackupHistory(boolean onlyCompleted) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get backup history from hbase:backup");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("get backup history from hbase:backup");
     }
     ArrayList<BackupInfo> list;
     BackupState state = onlyCompleted ? BackupState.COMPLETE : BackupState.ANY;
@@ -402,8 +403,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException exception
    */
   public ArrayList<BackupInfo> getBackupContexts(BackupState status) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get backup contexts from hbase:backup");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("get backup contexts from hbase:backup");
     }
 
     Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
@@ -434,8 +435,8 @@ public final class BackupSystemTable implements Closeable {
    */
   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 ["
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("write RS log time stamps to hbase:backup for tables ["
           + StringUtils.join(tables, ",") + "]");
     }
     List<Put> puts = new ArrayList<Put>();
@@ -462,8 +463,8 @@ public final class BackupSystemTable implements Closeable {
    */
   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);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read RS log ts from hbase:backup for root=" + backupRoot);
     }
 
     HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
@@ -498,7 +499,8 @@ public final class BackupSystemTable implements Closeable {
       Map<String, Long> map) {
     BackupProtos.TableServerTimestamp.Builder tstBuilder =
         BackupProtos.TableServerTimestamp.newBuilder();
-    tstBuilder.setTable(ProtobufUtil.toProtoTableNameShaded(table));
+    tstBuilder.setTable(
+      org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toProtoTableName(table));
 
     for (Entry<String, Long> entry : map.entrySet()) {
       BackupProtos.ServerTimestamp.Builder builder = BackupProtos.ServerTimestamp.newBuilder();
@@ -519,7 +521,8 @@ public final class BackupSystemTable implements Closeable {
     HashMap<String, Long> map = new HashMap<String, Long>();
     List<BackupProtos.ServerTimestamp> list = proto.getServerTimestampList();
     for (BackupProtos.ServerTimestamp st : list) {
-      ServerName sn = ProtobufUtil.toServerNameShaded(st.getServer());
+      ServerName sn =
+          org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toServerName(st.getServer());
       map.put(sn.getHostname()+":"+sn.getPort(), st.getTimestamp());
     }
     return map;
@@ -532,8 +535,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException exception
    */
   public Set<TableName> getIncrementalBackupTableSet(String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get incr backup table set from hbase:backup");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("get incremental backup table set from hbase:backup");
     }
     TreeSet<TableName> set = new TreeSet<>();
 
@@ -560,8 +563,8 @@ public final class BackupSystemTable implements Closeable {
    */
   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
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Add incremental backup table set to hbase:backup. ROOT=" + backupRoot
           + " tables [" + StringUtils.join(tables, " ") + "]");
       for (TableName table : tables) {
         LOG.debug(table);
@@ -579,8 +582,8 @@ public final class BackupSystemTable implements Closeable {
    */
 
   public void deleteIncrementalBackupTableSet(String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Delete incremental backup table set to hbase:backup. ROOT=" + backupRoot);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Delete incremental backup table set to hbase:backup. ROOT=" + backupRoot);
     }
     try (Table table = connection.getTable(tableName)) {
       Delete delete = BackupSystemTableHelper.createDeleteForIncrBackupTableSet(backupRoot);
@@ -597,8 +600,8 @@ public final class BackupSystemTable implements Closeable {
    */
   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 ["
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("add WAL files to hbase:backup: " + backupId + " " + backupRoot + " files ["
           + StringUtils.join(files, ",") + "]");
       for (String f : files) {
         LOG.debug("add :" + f);
@@ -617,8 +620,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException exception
    */
   public Iterator<WALItem> getWALFilesIterator(String backupRoot) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get WAL files from hbase:backup");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("get WAL files from hbase:backup");
     }
     final Table table = connection.getTable(tableName);
     Scan scan = BackupSystemTableHelper.createScanForGetWALs(backupRoot);
@@ -676,8 +679,8 @@ public final class BackupSystemTable implements Closeable {
    * @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);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Check if WAL file has been already backed up in hbase:backup " + file);
     }
     try (Table table = connection.getTable(tableName)) {
       Get get = BackupSystemTableHelper.createGetForCheckWALFile(file);
@@ -696,8 +699,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException exception
    */
   public boolean hasBackupSessions() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Has backup sessions from hbase:backup");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Has backup sessions from hbase:backup");
     }
     boolean result = false;
     Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
@@ -721,8 +724,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException
    */
   public List<String> listBackupSets() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(" Backup set list");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(" Backup set list");
     }
     List<String> list = new ArrayList<String>();
     Table table = null;
@@ -755,8 +758,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException
    */
   public List<TableName> describeBackupSet(String name) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(" Backup set describe: " + name);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(" Backup set describe: " + name);
     }
     Table table = null;
     try {
@@ -789,8 +792,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException
    */
   public void addToBackupSet(String name, String[] newTables) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Backup set add: " + name + " tables [" + StringUtils.join(newTables, " ") + "]");
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Backup set add: " + name + " tables [" + StringUtils.join(newTables, " ") + "]");
     }
     Table table = null;
     String[] union = null;
@@ -836,8 +839,8 @@ public final class BackupSystemTable implements Closeable {
    * @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, " ")
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(" Backup set remove from : " + name + " tables [" + StringUtils.join(toRemove, " ")
           + "]");
     }
     Table table = null;
@@ -892,8 +895,8 @@ public final class BackupSystemTable implements Closeable {
    * @throws IOException
    */
   public void deleteBackupSet(String name) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(" Backup set delete: " + name);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(" Backup set delete: " + name);
     }
     Table table = null;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/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
index ddf631d..33ff3ab 100644
--- 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
@@ -45,7 +45,8 @@ import org.apache.hadoop.hbase.util.Bytes;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public final class BackupSystemTableHelper {
+
+final class BackupSystemTableHelper {
 
   /**
    * hbase:backup schema:

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/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
index 01c004f..f743e75 100644
--- 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
@@ -19,236 +19,37 @@
 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.BackupCopyTask;
 import org.apache.hadoop.hbase.backup.BackupInfo;
 import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
 import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
 import org.apache.hadoop.hbase.backup.BackupRequest;
-import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
 import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
 import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.HBackupFileSystem;
-import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
 import org.apache.hadoop.hbase.backup.master.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 {
+public class FullTableBackupClient extends TableBackupClient{
   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.");
+    super(conn, backupId, request);
   }
 
   /**
@@ -296,162 +97,10 @@ public class FullTableBackupClient {
   }
 
   /**
-   * 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
    */
+  @Override
   public void execute() throws IOException {
 
     try (Admin admin = conn.getAdmin();) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/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
deleted file mode 100644
index 57596c8..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/HBaseBackupAdmin.java
+++ /dev/null
@@ -1,555 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.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.BackupAdmin;
-import org.apache.hadoop.hbase.backup.BackupInfo;
-import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
-import org.apache.hadoop.hbase.backup.BackupRequest;
-import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
-import org.apache.hadoop.hbase.backup.BackupType;
-import org.apache.hadoop.hbase.backup.HBackupFileSystem;
-import org.apache.hadoop.hbase.backup.RestoreRequest;
-import org.apache.hadoop.hbase.backup.util.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.util.EnvironmentEdgeManager;
-
-import com.google.common.collect.Lists;
-
-/**
- * The administrative API implementation for HBase Backup . Create an instance from
- * {@link HBaseBackupAdmin#HBaseBackupAdmin(Connection)} 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/7c1eb653/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 6fad17a..7f41c43 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
@@ -53,21 +53,16 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class IncrementalBackupManager {
+public class IncrementalBackupManager extends BackupManager{
   public static final Log LOG = LogFactory.getLog(IncrementalBackupManager.class);
 
-  // parent manager
-  private final BackupManager backupManager;
-  private final Configuration conf;
-
-  public IncrementalBackupManager(BackupManager bm) {
-    this.backupManager = bm;
-    this.conf = bm.getConf();
+  public IncrementalBackupManager(Connection conn, Configuration conf) throws IOException {
+    super(conn, conf);
   }
 
   /**
-   * Obtain the list of logs that need to be copied out for this incremental backup.
-   * The list is set in BackupContext.
+   * Obtain the list of logs that need to be copied out for this incremental backup. The list is set
+   * in BackupContext.
    * @param conn the Connection
    * @param backupContext backup context
    * @return The new HashMap of RS log timestamps after the log roll for this incremental backup.
@@ -79,12 +74,11 @@ public class IncrementalBackupManager {
     HashMap<String, Long> newTimestamps;
     HashMap<String, Long> previousTimestampMins;
 
-    String savedStartCode = backupManager.readBackupStartCode();
+    String savedStartCode = readBackupStartCode();
 
     // key: tableName
     // value: <RegionServer,PreviousTimeStamp>
-    HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
-        backupManager.readLogTimestampMap();
+    HashMap<TableName, HashMap<String, Long>> previousTimestampMap = readLogTimestampMap();
 
     previousTimestampMins = BackupServerUtil.getRSLogTimestampMins(previousTimestampMap);
 
@@ -109,12 +103,12 @@ public class IncrementalBackupManager {
         LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props);
 
     }
-    newTimestamps = backupManager.readRegionServerLastLogRollResult();
+    newTimestamps = readRegionServerLastLogRollResult();
 
     logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
     List<WALItem> logFromSystemTable =
         getLogFilesFromBackupSystem(previousTimestampMins,
-      newTimestamps, backupManager.getBackupContext().getTargetRootDir());
+      newTimestamps, getBackupContext().getTargetRootDir());
     addLogsFromBackupSystemToContext(logFromSystemTable);
 
     logList = excludeAlreadyBackedUpWALs(logList, logFromSystemTable);
@@ -164,7 +158,7 @@ public class IncrementalBackupManager {
   private List<WALItem> getLogFilesFromBackupSystem(HashMap<String, Long> olderTimestamps,
       HashMap<String, Long> newestTimestamps, String backupRoot) throws IOException {
     List<WALItem> logFiles = new ArrayList<WALItem>();
-    Iterator<WALItem> it = backupManager.getWALFilesFromBackupSystem();
+    Iterator<WALItem> it = getWALFilesFromBackupSystem();
     while (it.hasNext()) {
       WALItem item = it.next();
       String rootDir = item.getBackupRoot();
@@ -299,7 +293,7 @@ public class IncrementalBackupManager {
        * last backup.
        */
       if (oldTimeStamp == null) {
-        if (currentLogTS < Long.parseLong(savedStartCode)) {
+        if (currentLogTS < Long.valueOf(savedStartCode)) {
           // This log file is really old, its region server was before our last backup.
           continue;
         } else {
@@ -345,7 +339,7 @@ public class IncrementalBackupManager {
       Long timestamp = null;
       try {
         timestamp = BackupClientUtil.getCreationTime(path);
-        return timestamp > lastBackupTS;
+        return timestamp > Long.valueOf(lastBackupTS);
       } catch (Exception e) {
         LOG.warn("Cannot read timestamp of log file " + path);
         return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/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
index 55be02c..c56afbd 100644
--- 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
@@ -26,7 +26,6 @@ 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;
@@ -44,33 +43,15 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 
 @InterfaceAudience.Private
-public class IncrementalTableBackupClient {
+public class IncrementalTableBackupClient extends TableBackupClient {
   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());
+      throws IOException
+  {
+    super(conn, backupId, request);
   }
 
   private List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
@@ -174,19 +155,19 @@ public class IncrementalTableBackupClient {
     return list;
   }
 
+  @Override
   public void execute() throws IOException {
 
     // case PREPARE_INCREMENTAL:
-    FullTableBackupClient.beginBackup(backupManager, backupContext);
+    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);
+      newTimestamps = ((IncrementalBackupManager)backupManager).
+          getIncrBackupLogFileList(conn, backupContext);
     } catch (Exception e) {
       // fail the overall backup and return
-      FullTableBackupClient.failBackup(conn, backupContext, backupManager, e,
+      failBackup(conn, backupContext, backupManager, e,
         "Unexpected Exception : ", BackupType.INCREMENTAL, conf);
     }
 
@@ -200,7 +181,7 @@ public class IncrementalTableBackupClient {
     } 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,
+      failBackup(conn, backupContext, backupManager, e, msg,
         BackupType.INCREMENTAL, conf);
     }
     // case INCR_BACKUP_COMPLETE:
@@ -225,11 +206,11 @@ public class IncrementalTableBackupClient {
               .getRSLogTimestampMins(newTableSetTimestampMap));
       backupManager.writeBackupStartCode(newStartCode);
       // backup complete
-      FullTableBackupClient.completeBackup(conn, backupContext, backupManager,
+      completeBackup(conn, backupContext, backupManager,
         BackupType.INCREMENTAL, conf);
 
     } catch (IOException e) {
-      FullTableBackupClient.failBackup(conn, backupContext, backupManager, e,
+      failBackup(conn, backupContext, backupManager, e,
         "Unexpected Exception : ", BackupType.INCREMENTAL, conf);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/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
index 768910f..46fa46f 100644
--- 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
@@ -51,10 +51,6 @@ public class RestoreTablesClient {
   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();

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
new file mode 100644
index 0000000..ce1ed8f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/TableBackupClient.java
@@ -0,0 +1,386 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+@InterfaceAudience.Private
+public abstract class TableBackupClient {
+  private static final Log LOG = LogFactory.getLog(TableBackupClient.class);
+
+  protected Configuration conf;
+  protected Connection conn;
+  protected String backupId;
+  protected List<TableName> tableList;
+  protected HashMap<String, Long> newTimestamps = null;
+
+  protected BackupManager backupManager;
+  protected BackupInfo backupContext;
+
+  public TableBackupClient(final Connection conn, final String backupId,
+      BackupRequest request)
+      throws IOException {
+    if (request.getBackupType() == BackupType.FULL) {
+      backupManager = new BackupManager(conn, conn.getConfiguration());
+    } else {
+      backupManager = new IncrementalBackupManager(conn, conn.getConfiguration());
+    }
+    this.backupId = backupId;
+    this.tableList = request.getTableList();
+    this.conn = conn;
+    this.conf = conn.getConfiguration();
+    backupContext =
+        backupManager.createBackupContext(backupId, request.getBackupType(), 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
+   */
+  protected 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 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 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 void cleanupExportSnapshotLog(Configuration conf) throws IOException {
+    FileSystem fs = FSUtils.getCurrentFileSystem(conf);
+    Path stagingDir =
+        new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory()
+            .toString()));
+    FileStatus[] files = FSUtils.listStatus(fs, stagingDir);
+    if (files == null) {
+      return;
+    }
+    for (FileStatus file : files) {
+      if (file.getPath().getName().startsWith("exportSnapshot-")) {
+        LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName());
+        if (FSUtils.delete(fs, file.getPath(), true) == false) {
+          LOG.warn("Can not delete " + file.getPath());
+        }
+      }
+    }
+  }
+
+  /**
+   * Clean up the uncompleted data at target directory if the ongoing backup has already entered the
+   * copy phase.
+   */
+   private void cleanupTargetDir(BackupInfo 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
+   */
+  protected 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.");
+  }
+
+
+  /**
+   * 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 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 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 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
+   */
+  protected 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.");
+  }
+
+  /**
+   * Backup request execution
+   * @throws IOException
+   */
+  public abstract void execute() throws IOException;
+
+
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyTask.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyTask.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyTask.java
index b942446..e116057 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyTask.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyTask.java
@@ -46,14 +46,13 @@ import org.apache.hadoop.tools.DistCp;
 import org.apache.hadoop.tools.DistCpConstants;
 import org.apache.hadoop.tools.DistCpOptions;
 import org.apache.zookeeper.KeeperException.NoNodeException;
+
 /**
  * Copier for backup operation. Basically, there are 2 types of copy. One is copying from snapshot,
- * which bases on extending ExportSnapshot's function with copy progress reporting to ZooKeeper
- * implementation. The other is copying for incremental log files, which bases on extending
- * DistCp's function with copy progress reporting to ZooKeeper implementation.
+ * which bases on extending ExportSnapshot's function. The other is copying for incremental
+ * log files, which bases on extending DistCp's function.
  *
  */
-
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class MapReduceBackupCopyTask implements BackupCopyTask {
@@ -215,7 +214,8 @@ public class MapReduceBackupCopyTask implements BackupCopyTask {
 
         // Update the copy progress to ZK every 0.5s if progress value changed
         int progressReportFreq =
-            this.getConf().getInt("hbase.backup.progressreport.frequency", 500);
+            MapReduceBackupCopyTask.this.getConf().
+              getInt("hbase.backup.progressreport.frequency", 500);
         float lastProgress = progressDone;
         while (!job.isComplete()) {
           float newProgress =

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
index dc19f9b..537fed7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
@@ -88,7 +88,7 @@ public class BackupLogCleaner extends BaseLogCleanerDelegate {
       // If we do not have recorded backup sessions
       try {
         if (!table.hasBackupSessions()) {
-          LOG.debug("BackupLogCleaner has no backup sessions");
+          LOG.trace("BackupLogCleaner has no backup sessions");
           return files;
         }
       } catch (TableNotFoundException tnfe) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
index 99d9d91..d55965c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
@@ -38,10 +38,17 @@ import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
 import org.apache.hadoop.hbase.procedure.Procedure;
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.zookeeper.KeeperException;
 
+/**
+ *  Master procedure manager for coordinated cluster-wide
+ *  WAL roll operation, which is run during backup operation,
+ *  see {@link MasterProcedureManager} and and {@link RegionServerProcedureManager}
+ *
+ */
 public class LogRollMasterProcedureManager extends MasterProcedureManager {
 
   public static final String ROLLLOG_PROCEDURE_SIGNATURE = "rolllog-proc";

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
index e2c0d69..59bf51c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
@@ -89,9 +89,9 @@ public class LogRollBackupSubprocedure extends Subprocedure {
 
       LOG.info("Trying to roll log in backup subprocedure, current log number: " + filenum
           + " highest: " + highest + " on " + rss.getServerName());
-      ((HRegionServer)rss).walRoller.requestRollAll();
+      ((HRegionServer)rss).getWalRoller().requestRollAll();
       long start = EnvironmentEdgeManager.currentTime();
-      while (!((HRegionServer)rss).walRoller.walRollFinished()) {
+      while (!((HRegionServer)rss).getWalRoller().walRollFinished()) {
         Thread.sleep(20);
       }
       LOG.debug("log roll took " + (EnvironmentEdgeManager.currentTime()-start));

http://git-wip-us.apache.org/repos/asf/hbase/blob/7c1eb653/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
index 23270fd..838926a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
@@ -36,12 +36,11 @@ import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
 import org.apache.hadoop.hbase.procedure.Subprocedure;
 import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.zookeeper.KeeperException;
 
 /**
- * This manager class handles the work dealing with backup for a {@link HRegionServer}.
+ * This manager class handles the work dealing with distributed WAL roll request.
  * <p>
  * This provides the mechanism necessary to kick off a backup specific {@link Subprocedure} that is
  * responsible by this region server. If any failures occur with the subprocedure, the manager's