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 2017/03/13 22:53:00 UTC

[07/12] hbase git commit: HBASE-14123 HBase Backup/Restore Phase 2 (Vladimir Rodionov)

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
new file mode 100644
index 0000000..c1d5258
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupAdminImpl.java
@@ -0,0 +1,524 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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 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.BackupSet;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import com.google.common.collect.Lists;
+
+@InterfaceAudience.Private
+public class BackupAdminImpl implements BackupAdmin {
+  public final static String CHECK_OK = "Checking backup images: OK";
+  public final static String CHECK_FAILED =
+      "Checking backup images: Failed. Some dependencies are missing for restore";
+  private static final Log LOG = LogFactory.getLog(BackupAdminImpl.class);
+
+  private final Connection conn;
+
+  public BackupAdminImpl(Connection conn) {
+    this.conn = conn;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (conn != null) {
+      conn.close();
+    }
+  }
+
+
+  @Override
+  public BackupInfo getBackupInfo(String backupId) throws IOException {
+    BackupInfo backupInfo = null;
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      if (backupId == null) {
+        ArrayList<BackupInfo> recentSessions = table.getBackupInfos(BackupState.RUNNING);
+        if (recentSessions.isEmpty()) {
+          LOG.warn("No ongoing sessions found.");
+          return null;
+        }
+        // else show status for ongoing session
+        // must be one maximum
+        return recentSessions.get(0);
+      } else {
+        backupInfo = table.readBackupInfo(backupId);
+        return backupInfo;
+      }
+    }
+  }
+
+  @Override
+  public int deleteBackups(String[] backupIds) throws IOException {
+    // TODO: requires Fault tolerance support, failure will leave system
+    // in a 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.getBackupRootDir();
+          HashSet<TableName> allTables = allTablesMap.get(rootDir);
+          if (allTables == null) {
+            allTables = new HashSet<TableName>();
+            allTablesMap.put(rootDir, allTables);
+          }
+          allTables.addAll(info.getTableNames());
+          totalDeleted += deleteBackup(backupIds[i], sysTable);
+        }
+      }
+      finalizeDelete(allTablesMap, sysTable);
+    }
+    return totalDeleted;
+  }
+
+  /**
+   * Updates incremental backup set for every backupRoot
+   * @param tablesMap map [backupRoot: Set<TableName>]
+   * @param table backup system table
+   * @throws IOException
+   */
+
+  private void finalizeDelete(Map<String, HashSet<TableName>> tablesMap, BackupSystemTable table)
+      throws IOException {
+    for (String backupRoot : tablesMap.keySet()) {
+      Set<TableName> incrTableSet = table.getIncrementalBackupTableSet(backupRoot);
+      Map<TableName, ArrayList<BackupInfo>> tableMap =
+          table.getBackupHistoryForTableSet(incrTableSet, backupRoot);
+      for (Map.Entry<TableName, ArrayList<BackupInfo>> entry : tableMap.entrySet()) {
+        if (entry.getValue() == null) {
+          // No more backups for a table
+          incrTableSet.remove(entry.getKey());
+        }
+      }
+      if (!incrTableSet.isEmpty()) {
+        table.addIncrementalBackupTableSet(incrTableSet, backupRoot);
+      } else { // empty
+        table.deleteIncrementalBackupTableSet(backupRoot);
+      }
+    }
+  }
+
+  /**
+   * Delete single backup and all related backups <br>
+   * Algorithm:<br>
+   *  Backup type: FULL or INCREMENTAL <br>
+   *  Is this last backup session for table T: YES or NO <br>
+   *  For every table T from table list 'tables':<br>
+   *  if(FULL, YES) deletes only physical data (PD) <br>
+   *  if(FULL, NO), deletes PD, scans all newer backups and removes T from backupInfo,<br>
+   *  until we either reach the most recent backup for T in the system or FULL backup<br>
+   *  which includes T<br>
+   *  if(INCREMENTAL, YES) deletes only physical data (PD)
+   *  if(INCREMENTAL, NO) deletes physical data and for table T scans all backup images between last<br>
+   *  FULL backup, which is older than the backup being deleted and the next FULL backup (if exists) <br>
+   *  or last one for a particular table T and removes T from list of backup tables.
+   * @param backupId backup id
+   * @param sysTable backup system table
+   * @return total number of deleted backup images
+   * @throws IOException
+   */
+  private int deleteBackup(String backupId, BackupSystemTable sysTable) throws IOException {
+
+    BackupInfo backupInfo = sysTable.readBackupInfo(backupId);
+
+    int totalDeleted = 0;
+    if (backupInfo != null) {
+      LOG.info("Deleting backup " + backupInfo.getBackupId() + " ...");
+      BackupUtils.cleanupBackupData(backupInfo, conn.getConfiguration());
+      // List of tables in this backup;
+      List<TableName> tables = backupInfo.getTableNames();
+      long startTime = backupInfo.getStartTs();
+      for (TableName tn : tables) {
+        boolean isLastBackupSession = isLastBackupSession(sysTable, tn, startTime);
+        if (isLastBackupSession) {
+          continue;
+        }
+        // else
+        List<BackupInfo> affectedBackups = 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());
+        BackupUtils.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.getBackupRootDir());
+    // Scan from most recent to backupInfo
+    // break when backupInfo reached
+    for (BackupInfo info : history) {
+      if (info.getStartTs() == ts) {
+        break;
+      }
+      List<TableName> tables = info.getTableNames();
+      if (tables.contains(tn)) {
+        BackupType bt = info.getType();
+        if (bt == BackupType.FULL) {
+          // Clear list if we encounter FULL backup
+          list.clear();
+        } else {
+          LOG.debug("GetAffectedBackupInfos for: " + backupInfo.getBackupId() + " table=" + tn
+              + " added " + info.getBackupId() + " tables=" + info.getTableListAsString());
+          list.add(info);
+        }
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Clean up the data at target directory
+   * @throws IOException
+   */
+  private void cleanupBackupDir(BackupInfo backupInfo, TableName table, Configuration conf)
+      throws IOException {
+    try {
+      // clean up the data at target directory
+      String targetDir = backupInfo.getBackupRootDir();
+      if (targetDir == null) {
+        LOG.warn("No target directory specified for " + backupInfo.getBackupId());
+        return;
+      }
+
+      FileSystem outputFs = FileSystem.get(new Path(backupInfo.getBackupRootDir()).toUri(), conf);
+
+      Path targetDirPath =
+          new Path(BackupUtils.getTableBackupDir(backupInfo.getBackupRootDir(),
+            backupInfo.getBackupId(), table));
+      if (outputFs.delete(targetDirPath, true)) {
+        LOG.info("Cleaning up backup data at " + targetDirPath.toString() + " done.");
+      } else {
+        LOG.info("No data has been found in " + targetDirPath.toString() + ".");
+      }
+
+    } catch (IOException e1) {
+      LOG.error("Cleaning up backup data of " + backupInfo.getBackupId() + " for table " + table
+          + "at " + backupInfo.getBackupRootDir() + " failed due to " + e1.getMessage() + ".");
+      throw e1;
+    }
+  }
+
+  private boolean isLastBackupSession(BackupSystemTable table, TableName tn, long startTime)
+      throws IOException {
+    List<BackupInfo> history = table.getBackupHistory();
+    for (BackupInfo info : history) {
+      List<TableName> tables = info.getTableNames();
+      if (!tables.contains(tn)) {
+        continue;
+      }
+      if (info.getStartTs() <= startTime) {
+        return true;
+      } else {
+        return false;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public List<BackupInfo> getHistory(int n) throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<BackupInfo> history = table.getBackupHistory();
+      if (history.size() <= n) return history;
+      List<BackupInfo> list = new ArrayList<BackupInfo>();
+      for (int i = 0; i < n; i++) {
+        list.add(history.get(i));
+      }
+      return list;
+    }
+  }
+
+  @Override
+  public List<BackupInfo> getHistory(int n, BackupInfo.Filter... filters) throws IOException {
+    if (filters.length == 0) return getHistory(n);
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<BackupInfo> history = table.getBackupHistory();
+      List<BackupInfo> result = new ArrayList<BackupInfo>();
+      for (BackupInfo bi : history) {
+        if (result.size() == n) break;
+        boolean passed = true;
+        for (int i = 0; i < filters.length; i++) {
+          if (!filters[i].apply(bi)) {
+            passed = false;
+            break;
+          }
+        }
+        if (passed) {
+          result.add(bi);
+        }
+      }
+      return result;
+    }
+  }
+
+  @Override
+  public List<BackupSet> listBackupSets() throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<String> list = table.listBackupSets();
+      List<BackupSet> bslist = new ArrayList<BackupSet>();
+      for (String s : list) {
+        List<TableName> tables = table.describeBackupSet(s);
+        if (tables != null) {
+          bslist.add(new BackupSet(s, tables));
+        }
+      }
+      return bslist;
+    }
+  }
+
+  @Override
+  public BackupSet getBackupSet(String name) throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<TableName> list = table.describeBackupSet(name);
+      if (list == null) return null;
+      return new BackupSet(name, list);
+    }
+  }
+
+  @Override
+  public boolean deleteBackupSet(String name) throws IOException {
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      if (table.describeBackupSet(name) == null) {
+        return false;
+      }
+      table.deleteBackupSet(name);
+      return true;
+    }
+  }
+
+  @Override
+  public void addToBackupSet(String name, TableName[] tables) throws IOException {
+    String[] tableNames = new String[tables.length];
+    try (final BackupSystemTable table = new BackupSystemTable(conn);
+        final Admin admin = conn.getAdmin();) {
+      for (int i = 0; i < tables.length; i++) {
+        tableNames[i] = tables[i].getNameAsString();
+        if (!admin.tableExists(TableName.valueOf(tableNames[i]))) {
+          throw new IOException("Cannot add " + tableNames[i] + " because it doesn't exist");
+        }
+      }
+      table.addToBackupSet(name, tableNames);
+      LOG.info("Added tables [" + StringUtils.join(tableNames, " ") + "] to '" + name
+          + "' backup set");
+    }
+  }
+
+  @Override
+  public void removeFromBackupSet(String name, TableName[] tables) throws IOException {
+    LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name + "'");
+    try (final BackupSystemTable table = new BackupSystemTable(conn)) {
+      table.removeFromBackupSet(name, toStringArray(tables));
+      LOG.info("Removing tables [" + StringUtils.join(tables, " ") + "] from '" + name
+          + "' completed.");
+    }
+  }
+
+  private String[] toStringArray(TableName[] list) {
+    String[] arr = new String[list.length];
+    for(int i=0; i < list.length; i++) {
+      arr[i] = list[i].toString();
+    }
+    return arr;
+  }
+
+  @Override
+  public void restore(RestoreRequest request) throws IOException {
+    if (request.isCheck()) {
+      HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
+      // check and load backup image manifest for the tables
+      Path rootPath = new Path(request.getBackupRootDir());
+      String backupId = request.getBackupId();
+      TableName[] sTableArray = request.getFromTables();
+      HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray,
+        conn.getConfiguration(), rootPath, backupId);
+
+      // Check and validate the backup image and its dependencies
+
+      if (BackupUtils.validate(backupManifestMap, conn.getConfiguration())) {
+        LOG.info(CHECK_OK);
+      } else {
+        LOG.error(CHECK_FAILED);
+      }
+      return;
+    }
+    // Execute restore request
+    new RestoreTablesClient(conn, request).execute();
+  }
+
+  @Override
+  public String backupTables(BackupRequest request) throws IOException {
+    BackupType type = request.getBackupType();
+    String targetRootDir = request.getTargetRootDir();
+    List<TableName> tableList = request.getTableList();
+
+    String backupId =BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime();
+    if (type == BackupType.INCREMENTAL) {
+      Set<TableName> incrTableSet = null;
+      try (BackupSystemTable table = new BackupSystemTable(conn)) {
+        incrTableSet = table.getIncrementalBackupTableSet(targetRootDir);
+      }
+
+      if (incrTableSet.isEmpty()) {
+        String msg = "Incremental backup table set contains no tables. "
+            + "You need to run full backup first " +
+            (tableList != null ? "on "+StringUtils.join(tableList, ","): "");
+
+        throw new IOException(msg);
+      }
+      if(tableList != null) {
+        tableList.removeAll(incrTableSet);
+        if (!tableList.isEmpty()) {
+          String extraTables = StringUtils.join(tableList, ",");
+          String msg = "Some tables (" + extraTables + ") haven't gone through full backup. "+
+            "Perform full backup on " + extraTables + " first, "
+            + "then retry the command";
+          throw new IOException(msg);
+        }
+      }
+      tableList = Lists.newArrayList(incrTableSet);
+    }
+    if (tableList != null && !tableList.isEmpty()) {
+      for (TableName table : tableList) {
+        String targetTableBackupDir =
+            HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
+        Path targetTableBackupDirPath = new Path(targetTableBackupDir);
+        FileSystem outputFs =
+            FileSystem.get(targetTableBackupDirPath.toUri(), conn.getConfiguration());
+        if (outputFs.exists(targetTableBackupDirPath)) {
+          throw new IOException("Target backup directory " + targetTableBackupDir
+              + " exists already.");
+        }
+      }
+      ArrayList<TableName> nonExistingTableList = null;
+      try (Admin admin = conn.getAdmin();) {
+        for (TableName tableName : tableList) {
+          if (!admin.tableExists(tableName)) {
+            if (nonExistingTableList == null) {
+              nonExistingTableList = new ArrayList<>();
+            }
+            nonExistingTableList.add(tableName);
+          }
+        }
+      }
+      if (nonExistingTableList != null) {
+        if (type == BackupType.INCREMENTAL) {
+          // Update incremental backup set
+          tableList = excludeNonExistingTables(tableList, nonExistingTableList);
+        } else {
+          // Throw exception only in full mode - we try to backup non-existing table
+          throw new IOException("Non-existing tables found in the table list: "
+              + nonExistingTableList);
+        }
+      }
+    }
+
+    // update table list
+    BackupRequest.Builder builder = new BackupRequest.Builder();
+    request = builder.withBackupType(request.getBackupType()).
+                      withTableList(tableList).
+                      withTargetRootDir(request.getTargetRootDir()).
+                      withBackupSetName(request.getBackupSetName()).
+                      withTotalTasks(request.getTotalTasks()).
+                      withBandwidthPerTasks((int)request.getBandwidth()).build();
+
+    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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/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..75e0ab7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCommands.java
@@ -0,0 +1,780 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup.impl;
+
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_BANDWIDTH_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_PATH_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_RECORD_NUMBER_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_BACKUP_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRequest;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+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
+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_path> [options]\n"
+          + "  type           \"full\" to create a full backup image\n"
+          + "                 \"incremental\" to create an incremental backup image\n"
+          + "  backup_path     Full path to store the backup image\n";
+
+  public static final String PROGRESS_CMD_USAGE = "Usage: hbase backup progress <backup_id>\n"
+      + "  backup_id       Backup image id (optional). If no id specified, the command will show\n"
+      + "                  progress for currently running backup session.";
+  public static final String NO_INFO_FOUND = "No info was found for backup id: ";
+  public static final String NO_ACTIVE_SESSION_FOUND = "No active backup sessions found.";
+
+  public static final String DESCRIBE_CMD_USAGE = "Usage: hbase backup describe <backup_id>\n"
+      + "  backup_id       Backup image id\n";
+
+  public static final String HISTORY_CMD_USAGE = "Usage: hbase backup history [options]";
+
+  public static final String DELETE_CMD_USAGE = "Usage: hbase backup delete <backup_id>\n"
+      + "  backup_id       Backup image id\n";
+
+  public static final String CANCEL_CMD_USAGE = "Usage: hbase backup cancel <backup_id>\n"
+      + "  backup_id       Backup image id\n";
+
+  public static final String SET_CMD_USAGE = "Usage: hbase backup set COMMAND [name] [tables]\n"
+      + "  name            Backup set name\n"
+      + "  tables          Comma separated list of tables.\n"
+      + "COMMAND is one of:\n" + "  add             add tables to a set, create a set if needed\n"
+      + "  remove          remove tables from a set\n"
+      + "  list            list all backup sets in the system\n"
+      + "  describe        describe set\n" + "  delete          delete backup set\n";
+
+  public static final String USAGE_FOOTER = "";
+
+  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) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length !=3) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      if (!BackupType.FULL.toString().equalsIgnoreCase(args[1])
+          && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[1])) {
+        System.out.println("ERROR: invalid backup type: " + args[1]);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      if (!verifyPath(args[2])) {
+        System.out.println("ERROR: invalid backup destination: " + args[2]);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String tables = null;
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+
+      // Check if we have both: backup set and list of tables
+      if (cmdline.hasOption(OPTION_TABLE) && cmdline.hasOption(OPTION_SET)) {
+        System.out.println("ERROR: You can specify either backup set or list"
+            + " of tables, but not both");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      // Check backup set
+      String setName = null;
+      if (cmdline.hasOption(OPTION_SET)) {
+        setName = cmdline.getOptionValue(OPTION_SET);
+        tables = getTablesForSet(setName, conf);
+
+        if (tables == null) {
+          System.out.println("ERROR: Backup set '" + setName
+              + "' is either empty or does not exist");
+          printUsage();
+          throw new IOException(INCORRECT_USAGE);
+        }
+      } else {
+        tables = cmdline.getOptionValue(OPTION_TABLE);
+      }
+      int bandwidth =
+          cmdline.hasOption(OPTION_BANDWIDTH) ? Integer.parseInt(cmdline
+              .getOptionValue(OPTION_BANDWIDTH)) : -1;
+      int workers =
+          cmdline.hasOption(OPTION_WORKERS) ? Integer.parseInt(cmdline
+              .getOptionValue(OPTION_WORKERS)) : -1;
+
+      try (Connection conn = ConnectionFactory.createConnection(getConf());
+          BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+
+       BackupRequest.Builder builder = new BackupRequest.Builder();
+       BackupRequest request = builder.withBackupType(BackupType.valueOf(args[1].toUpperCase()))
+            .withTableList(tables != null ?
+                          Lists.newArrayList(BackupUtils.parseTableNames(tables)) : null)
+            .withTargetRootDir(args[2])
+            .withTotalTasks(workers)
+            .withBandwidthPerTasks(bandwidth)
+            .withBackupSetName(setName).build();
+        String backupId = admin.backupTables(request);
+        System.out.println("Backup session " + backupId + " finished. Status: SUCCESS");
+      } catch (IOException e) {
+        System.out.println("Backup session finished. Status: FAILURE");
+        throw e;
+      }
+    }
+
+    private boolean verifyPath(String path) {
+      try {
+        Path p = new Path(path);
+        Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+        URI uri = p.toUri();
+        if (uri.getScheme() == null) return false;
+        FileSystem.get(uri, conf);
+        return true;
+      } catch (Exception e) {
+        return false;
+      }
+    }
+
+    private String getTablesForSet(String name, Configuration conf) throws IOException {
+      try (final 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.out.println(CREATE_CMD_USAGE);
+      Options options = new Options();
+      options.addOption(OPTION_WORKERS, true, OPTION_WORKERS_DESC);
+      options.addOption(OPTION_BANDWIDTH, true, OPTION_BANDWIDTH_DESC);
+      options.addOption(OPTION_SET, true, OPTION_SET_BACKUP_DESC);
+      options.addOption(OPTION_TABLE, true, OPTION_TABLE_LIST_DESC);
+
+      HelpFormatter helpFormatter = new HelpFormatter();
+      helpFormatter.setLeftPadding(2);
+      helpFormatter.setDescPadding(8);
+      helpFormatter.setWidth(100);
+      helpFormatter.setSyntaxPrefix("Options:");
+      helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
+
+    }
+  }
+
+  private static class HelpCommand extends Command {
+
+    HelpCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      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.out.println("ERROR: Only supports help message of a single command type");
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String type = args[1];
+
+      if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) {
+        System.out.println(CREATE_CMD_USAGE);
+      } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(type)) {
+        System.out.println(DESCRIBE_CMD_USAGE);
+      } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(type)) {
+        System.out.println(HISTORY_CMD_USAGE);
+      } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(type)) {
+        System.out.println(PROGRESS_CMD_USAGE);
+      } else if (BackupCommand.DELETE.name().equalsIgnoreCase(type)) {
+        System.out.println(DELETE_CMD_USAGE);
+      } else if (BackupCommand.CANCEL.name().equalsIgnoreCase(type)) {
+        System.out.println(CANCEL_CMD_USAGE);
+      } else if (BackupCommand.SET.name().equalsIgnoreCase(type)) {
+        System.out.println(SET_CMD_USAGE);
+      } else {
+        System.out.println("Unknown command : " + type);
+        printUsage();
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(USAGE);
+    }
+  }
+
+  private static class DescribeCommand extends Command {
+
+    DescribeCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null || cmdline.getArgs() == null) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length != 2) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      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.out.println("ERROR: " + backupId + " does not exist");
+          printUsage();
+          throw new IOException(INCORRECT_USAGE);
+        }
+        System.out.println(info.getShortDescription());
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(DESCRIBE_CMD_USAGE);
+    }
+  }
+
+  private static class ProgressCommand extends Command {
+
+    ProgressCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length == 1) {
+        System.out.println("No backup id was specified, "
+            + "will retrieve the most recent (ongoing) session");
+      }
+      String[] args = cmdline == null ? null : cmdline.getArgs();
+      if (args != null && args.length > 2) {
+        System.err.println("ERROR: wrong number of arguments: " + args.length);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      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 = null;
+
+        if (backupId != null) {
+          info = sysTable.readBackupInfo(backupId);
+        } else {
+          List<BackupInfo> infos = sysTable.getBackupInfos(BackupState.RUNNING);
+          if (infos != null && infos.size() > 0) {
+            info = infos.get(0);
+            backupId = info.getBackupId();
+            System.out.println("Found ongoing session with backupId=" + backupId);
+          } else {
+          }
+        }
+        int progress = info == null ? -1 : info.getProgress();
+        if (progress < 0) {
+          if (backupId != null) {
+            System.out.println(NO_INFO_FOUND + backupId);
+          } else {
+            System.err.println(NO_ACTIVE_SESSION_FOUND);
+          }
+        } else {
+          System.out.println(backupId + " progress=" + progress + "%");
+        }
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(PROGRESS_CMD_USAGE);
+    }
+  }
+
+  private static class DeleteCommand extends Command {
+
+    DeleteCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        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);
+          BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        int deleted = admin.deleteBackups(backupIds);
+        System.out.println("Deleted " + deleted + " backups. Total requested: " + args.length);
+      }
+
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.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 {
+      throw new UnsupportedOperationException("Cancel command is not supported yet.");
+    }
+
+    @Override
+    protected void printUsage() {
+    }
+  }
+
+  private static class HistoryCommand extends Command {
+
+    private final static int DEFAULT_HISTORY_LENGTH = 10;
+
+    HistoryCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+
+      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 backup system table
+        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 =
+            BackupUtils.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(OPTION_PATH);
+        if (value == null) return null;
+        return new Path(value);
+      } catch (IllegalArgumentException e) {
+        System.out.println("ERROR: Illegal argument for backup root path: " + value);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    private TableName getTableName() throws IOException {
+      String value = cmdline.getOptionValue(OPTION_TABLE);
+      if (value == null) return null;
+      try {
+        return TableName.valueOf(value);
+      } catch (IllegalArgumentException e) {
+        System.out.println("Illegal argument for table name: " + value);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    private String getTableSetName() throws IOException {
+      String value = cmdline.getOptionValue(OPTION_SET);
+      return value;
+    }
+
+    private int parseHistoryLength() throws IOException {
+      String value = cmdline.getOptionValue(OPTION_RECORD_NUMBER);
+      try {
+        if (value == null) return DEFAULT_HISTORY_LENGTH;
+        return Integer.parseInt(value);
+      } catch (NumberFormatException e) {
+        System.out.println("Illegal argument for history length: " + value);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(HISTORY_CMD_USAGE);
+      Options options = new Options();
+      options.addOption(OPTION_RECORD_NUMBER, true, OPTION_RECORD_NUMBER_DESC);
+      options.addOption(OPTION_PATH, true, OPTION_PATH_DESC);
+      options.addOption(OPTION_TABLE, true, OPTION_TABLE_DESC);
+      options.addOption(OPTION_SET, true, OPTION_SET_DESC);
+
+      HelpFormatter helpFormatter = new HelpFormatter();
+      helpFormatter.setLeftPadding(2);
+      helpFormatter.setDescPadding(8);
+      helpFormatter.setWidth(100);
+      helpFormatter.setSyntaxPrefix("Options:");
+      helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
+    }
+  }
+
+  private static class BackupSetCommand extends Command {
+    private final static String SET_ADD_CMD = "add";
+    private final static String SET_REMOVE_CMD = "remove";
+    private final static String SET_DELETE_CMD = "delete";
+    private final static String SET_DESCRIBE_CMD = "describe";
+    private final static String SET_LIST_CMD = "list";
+
+    BackupSetCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      super.execute();
+      // Command-line must have at least one element
+      if (cmdline == null || cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String[] args = cmdline.getArgs();
+      String cmdStr = args[1];
+      BackupCommand cmd = getCommand(cmdStr);
+
+      switch (cmd) {
+      case SET_ADD:
+        processSetAdd(args);
+        break;
+      case SET_REMOVE:
+        processSetRemove(args);
+        break;
+      case SET_DELETE:
+        processSetDelete(args);
+        break;
+      case SET_DESCRIBE:
+        processSetDescribe(args);
+        break;
+      case SET_LIST:
+        processSetList(args);
+        break;
+      default:
+        break;
+
+      }
+    }
+
+    private void processSetList(String[] args) throws IOException {
+      // List all backup set names
+      // does not expect any args
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        List<BackupSet> list = admin.listBackupSets();
+        for (BackupSet bs : list) {
+          System.out.println(bs);
+        }
+      }
+    }
+
+    private void processSetDescribe(String[] args) throws IOException {
+      if (args == null || args.length != 3) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      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) {
+        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 BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        boolean result = admin.deleteBackupSet(setName);
+        if (result) {
+          System.out.println("Delete set " + setName + " OK.");
+        } else {
+          System.out.println("Set " + setName + " does not exist");
+        }
+      }
+    }
+
+    private void processSetRemove(String[] args) throws IOException {
+      if (args == null || args.length != 4) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+
+      String setName = args[2];
+      String[] tables = args[3].split(",");
+      TableName[] tableNames = toTableNames(tables);
+      Configuration conf = getConf() != null ? getConf() : HBaseConfiguration.create();
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        admin.removeFromBackupSet(setName, tableNames);
+      }
+    }
+
+    private TableName[] toTableNames(String[] tables) {
+      TableName[] arr = new TableName[tables.length];
+      for (int i=0; i < tables.length; i++) {
+        arr[i] = TableName.valueOf(tables[i]);
+      }
+      return arr;
+    }
+
+    private void processSetAdd(String[] args) throws IOException {
+      if (args == null || args.length != 4) {
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+      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 BackupAdminImpl admin = new BackupAdminImpl(conn);) {
+        admin.addToBackupSet(setName, tableNames);
+      }
+
+    }
+
+    private BackupCommand getCommand(String cmdStr) throws IOException {
+      if (cmdStr.equals(SET_ADD_CMD)) {
+        return BackupCommand.SET_ADD;
+      } else if (cmdStr.equals(SET_REMOVE_CMD)) {
+        return BackupCommand.SET_REMOVE;
+      } else if (cmdStr.equals(SET_DELETE_CMD)) {
+        return BackupCommand.SET_DELETE;
+      } else if (cmdStr.equals(SET_DESCRIBE_CMD)) {
+        return BackupCommand.SET_DESCRIBE;
+      } else if (cmdStr.equals(SET_LIST_CMD)) {
+        return BackupCommand.SET_LIST;
+      } else {
+        System.out.println("ERROR: Unknown command for 'set' :" + cmdStr);
+        printUsage();
+        throw new IOException(INCORRECT_USAGE);
+      }
+    }
+
+    @Override
+    protected void printUsage() {
+      System.out.println(SET_CMD_USAGE);
+    }
+
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/3aaea8e0/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
new file mode 100644
index 0000000..1d27e79
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupManager.java
@@ -0,0 +1,472 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants;
+import org.apache.hadoop.hbase.backup.BackupType;
+import org.apache.hadoop.hbase.backup.HBackupFileSystem;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
+import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
+import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
+import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.procedure.ProcedureManagerHost;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Handles backup requests, creates backup info records in backup system table to
+ * keep track of backup sessions, dispatches backup request.
+ */
+@InterfaceAudience.Private
+public class BackupManager implements Closeable {
+  private static final Log LOG = LogFactory.getLog(BackupManager.class);
+
+  protected Configuration conf = null;
+  protected BackupInfo backupInfo = null;
+  protected BackupSystemTable systemTable;
+  protected final Connection conn;
+
+  /**
+   * Backup manager constructor.
+   * @param conn connection
+   * @param conf configuration
+   * @throws IOException exception
+   */
+  public BackupManager(Connection conn, Configuration conf) throws IOException {
+    if (!conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY,
+      BackupRestoreConstants.BACKUP_ENABLE_DEFAULT)) {
+      throw new BackupException("HBase backup is not enabled. Check your "
+          + BackupRestoreConstants.BACKUP_ENABLE_KEY + " setting.");
+    }
+    this.conf = conf;
+    this.conn = conn;
+    this.systemTable = new BackupSystemTable(conn);
+
+  }
+
+  /**
+   * Returns backup info
+   */
+  protected BackupInfo getBackupInfo() {
+    return backupInfo;
+  }
+
+  /**
+   * This method modifies the master's configuration in order to inject backup-related features
+   * (TESTs only)
+   * @param conf configuration
+   */
+  @VisibleForTesting
+  public static void decorateMasterConfiguration(Configuration conf) {
+    if (!isBackupEnabled(conf)) {
+      return;
+    }
+    // Add WAL archive cleaner plug-in
+    String plugins = conf.get(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS);
+    String cleanerClass = BackupLogCleaner.class.getCanonicalName();
+    if (!plugins.contains(cleanerClass)) {
+      conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
+    }
+
+    String classes = conf.get(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY);
+    String masterProcedureClass = LogRollMasterProcedureManager.class.getName();
+    if (classes == null) {
+      conf.set(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY, masterProcedureClass);
+    } else if (!classes.contains(masterProcedureClass)) {
+      conf.set(ProcedureManagerHost.MASTER_PROCEDURE_CONF_KEY, classes + "," + masterProcedureClass);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Added log cleaner: " + cleanerClass +"\n" +
+                "Added master procedure manager: " + masterProcedureClass);
+    }
+
+  }
+
+  /**
+   * This method modifies the Region Server configuration in order to inject backup-related features
+   * TESTs only.
+   * @param conf configuration
+   */
+  @VisibleForTesting
+  public static void decorateRegionServerConfiguration(Configuration conf) {
+    if (!isBackupEnabled(conf)) {
+      return;
+    }
+
+    String classes = conf.get(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY);
+    String regionProcedureClass = LogRollRegionServerProcedureManager.class.getName();
+    if (classes == null) {
+      conf.set(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY, regionProcedureClass);
+    } else if (!classes.contains(regionProcedureClass)) {
+      conf.set(ProcedureManagerHost.REGIONSERVER_PROCEDURE_CONF_KEY, classes + ","
+          + regionProcedureClass);
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Added region procedure manager: " + regionProcedureClass);
+    }
+
+  }
+
+  public static boolean isBackupEnabled(Configuration conf) {
+    return conf.getBoolean(BackupRestoreConstants.BACKUP_ENABLE_KEY,
+      BackupRestoreConstants.BACKUP_ENABLE_DEFAULT);
+  }
+
+  /**
+   * Get configuration
+   * @return configuration
+   */
+  Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * Stop all the work of backup.
+   */
+  @Override
+  public void close() {
+
+    if (systemTable != null) {
+      try {
+        systemTable.close();
+      } catch (Exception e) {
+        LOG.error(e);
+      }
+    }
+  }
+
+  /**
+   * Creates a backup info based on input backup request.
+   * @param backupId backup id
+   * @param type type
+   * @param tableList table list
+   * @param targetRootDir root dir
+   * @param workers number of parallel workers
+   * @param bandwidth bandwidth per worker in MB per sec
+   * @return BackupInfo
+   * @throws BackupException exception
+   */
+  public BackupInfo createBackupInfo(String backupId, BackupType type,
+      List<TableName> tableList, String targetRootDir, int workers, long bandwidth)
+      throws BackupException {
+    if (targetRootDir == null) {
+      throw new BackupException("Wrong backup request parameter: target backup root directory");
+    }
+
+    if (type == BackupType.FULL && (tableList == null || tableList.isEmpty())) {
+      // If table list is null for full backup, which means backup all tables. Then fill the table
+      // list with all user tables from meta. It no table available, throw the request exception.
+
+      HTableDescriptor[] htds = null;
+      try (Admin admin = conn.getAdmin()) {
+        htds = admin.listTables();
+      } catch (Exception e) {
+        throw new BackupException(e);
+      }
+
+      if (htds == null) {
+        throw new BackupException("No table exists for full backup of all tables.");
+      } else {
+        tableList = new ArrayList<>();
+        for (HTableDescriptor hTableDescriptor : htds) {
+          TableName tn = hTableDescriptor.getTableName();
+          if (tn.equals(BackupSystemTable.getTableName(conf))) {
+            // skip backup system table
+            continue;
+          }
+          tableList.add(hTableDescriptor.getTableName());
+        }
+
+        LOG.info("Full backup all the tables available in the cluster: " + tableList);
+      }
+    }
+
+    // there are one or more tables in the table list
+    backupInfo =
+        new BackupInfo(backupId, type, tableList.toArray(new TableName[tableList.size()]),
+            targetRootDir);
+    backupInfo.setBandwidth(bandwidth);
+    backupInfo.setWorkers(workers);
+    return backupInfo;
+  }
+
+  /**
+   * Check if any ongoing backup. Currently, we only reply on checking status in backup system
+   * table. We need to consider to handle the case of orphan records in the future. Otherwise, all
+   * the coming request will fail.
+   * @return the ongoing backup id if on going backup exists, otherwise null
+   * @throws IOException exception
+   */
+  private String getOngoingBackupId() throws IOException {
+
+    ArrayList<BackupInfo> sessions = systemTable.getBackupInfos(BackupState.RUNNING);
+    if (sessions.size() == 0) {
+      return null;
+    }
+    return sessions.get(0).getBackupId();
+  }
+
+  /**
+   * Start the backup manager service.
+   * @throws IOException exception
+   */
+  public void initialize() throws IOException {
+    String ongoingBackupId = this.getOngoingBackupId();
+    if (ongoingBackupId != null) {
+      LOG.info("There is a ongoing backup " + ongoingBackupId
+          + ". Can not launch new backup until no ongoing backup remains.");
+      throw new BackupException("There is ongoing backup.");
+    }
+  }
+
+  public void setBackupInfo(BackupInfo backupInfo) {
+    this.backupInfo = backupInfo;
+  }
+
+  /**
+   * Get direct ancestors of the current backup.
+   * @param backupInfo The backup info for the current backup
+   * @return The ancestors for the current backup
+   * @throws IOException exception
+   * @throws BackupException exception
+   */
+  public ArrayList<BackupImage> getAncestors(BackupInfo backupInfo) throws IOException,
+      BackupException {
+    LOG.debug("Getting the direct ancestors of the current backup " + backupInfo.getBackupId());
+
+    ArrayList<BackupImage> ancestors = new ArrayList<BackupImage>();
+
+    // full backup does not have ancestor
+    if (backupInfo.getType() == BackupType.FULL) {
+      LOG.debug("Current backup is a full backup, no direct ancestor for it.");
+      return ancestors;
+    }
+
+    // get all backup history list in descending order
+
+    ArrayList<BackupInfo> allHistoryList = getBackupHistory(true);
+    for (BackupInfo backup : allHistoryList) {
+
+      BackupImage.Builder builder = BackupImage.newBuilder();
+
+      BackupImage image =
+          builder.withBackupId(backup.getBackupId()).withType(backup.getType())
+              .withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames())
+              .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build();
+
+      // add the full backup image as an ancestor until the last incremental backup
+      if (backup.getType().equals(BackupType.FULL)) {
+        // check the backup image coverage, if previous image could be covered by the newer ones,
+        // then no need to add
+        if (!BackupManifest.canCoverImage(ancestors, image)) {
+          ancestors.add(image);
+        }
+      } else {
+        // found last incremental backup, if previously added full backup ancestor images can cover
+        // it, then this incremental ancestor is not the dependent of the current incremental
+        // backup, that is to say, this is the backup scope boundary of current table set.
+        // Otherwise, this incremental backup ancestor is the dependent ancestor of the ongoing
+        // incremental backup
+        if (BackupManifest.canCoverImage(ancestors, image)) {
+          LOG.debug("Met the backup boundary of the current table set:");
+          for (BackupImage image1 : ancestors) {
+            LOG.debug("  BackupID=" + image1.getBackupId() + ", BackupDir=" + image1.getRootDir());
+          }
+        } else {
+          Path logBackupPath =
+              HBackupFileSystem.getLogBackupPath(backup.getBackupRootDir(), backup.getBackupId());
+          LOG.debug("Current backup has an incremental backup ancestor, "
+              + "touching its image manifest in " + logBackupPath.toString()
+              + " to construct the dependency.");
+          BackupManifest lastIncrImgManifest = new BackupManifest(conf, logBackupPath);
+          BackupImage lastIncrImage = lastIncrImgManifest.getBackupImage();
+          ancestors.add(lastIncrImage);
+
+          LOG.debug("Last dependent incremental backup image: " + "{BackupID="
+              + lastIncrImage.getBackupId() + "," + "BackupDir=" + lastIncrImage.getRootDir() + "}");
+        }
+      }
+    }
+    LOG.debug("Got " + ancestors.size() + " ancestors for the current backup.");
+    return ancestors;
+  }
+
+  /**
+   * Get the direct ancestors of this backup for one table involved.
+   * @param backupInfo backup info
+   * @param table table
+   * @return backupImages on the dependency list
+   * @throws BackupException exception
+   * @throws IOException exception
+   */
+  public ArrayList<BackupImage> getAncestors(BackupInfo backupInfo, TableName table)
+      throws BackupException, IOException {
+    ArrayList<BackupImage> ancestors = getAncestors(backupInfo);
+    ArrayList<BackupImage> tableAncestors = new ArrayList<BackupImage>();
+    for (BackupImage image : ancestors) {
+      if (image.hasTable(table)) {
+        tableAncestors.add(image);
+        if (image.getType() == BackupType.FULL) {
+          break;
+        }
+      }
+    }
+    return tableAncestors;
+  }
+
+  /*
+   * backup system table operations
+   */
+
+  /**
+   * Updates status (state) of a backup session in a persistent store
+   * @param context context
+   * @throws IOException exception
+   */
+  public void updateBackupInfo(BackupInfo context) throws IOException {
+    systemTable.updateBackupInfo(context);
+  }
+
+  /**
+   * Read the last backup start code (timestamp) of last successful backup. Will return null if
+   * there is no startcode stored in backup system table or the value is of length 0. These two
+   * cases indicate there is no successful backup completed so far.
+   * @return the timestamp of a last successful backup
+   * @throws IOException exception
+   */
+  public String readBackupStartCode() throws IOException {
+    return systemTable.readBackupStartCode(backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Write the start code (timestamp) to backup system table. If passed in null, then write 0 byte.
+   * @param startCode start code
+   * @throws IOException exception
+   */
+  public void writeBackupStartCode(Long startCode) throws IOException {
+    systemTable.writeBackupStartCode(startCode, backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Get the RS log information after the last log roll from backup system table.
+   * @return RS log info
+   * @throws IOException exception
+   */
+  public HashMap<String, Long> readRegionServerLastLogRollResult() throws IOException {
+    return systemTable.readRegionServerLastLogRollResult(backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Get all completed backup information (in desc order by time)
+   * @return history info of BackupCompleteData
+   * @throws IOException exception
+   */
+  public List<BackupInfo> getBackupHistory() throws IOException {
+    return systemTable.getBackupHistory();
+  }
+
+  public ArrayList<BackupInfo> getBackupHistory(boolean completed) throws IOException {
+    return systemTable.getBackupHistory(completed);
+  }
+
+  /**
+   * Write the current timestamps for each regionserver to backup system table after a successful
+   * full or incremental backup. Each table may have a different set of log timestamps. The saved
+   * timestamp is of the last log file that was backed up already.
+   * @param tables tables
+   * @throws IOException exception
+   */
+  public void writeRegionServerLogTimestamp(Set<TableName> tables,
+      HashMap<String, Long> newTimestamps) throws IOException {
+    systemTable.writeRegionServerLogTimestamp(tables, newTimestamps,
+      backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Read the timestamp for each region server log after the last successful backup. Each table has
+   * its own set of the timestamps.
+   * @return the timestamp for each region server. key: tableName value:
+   *         RegionServer,PreviousTimeStamp
+   * @throws IOException exception
+   */
+  public HashMap<TableName, HashMap<String, Long>> readLogTimestampMap() throws IOException {
+    return systemTable.readLogTimestampMap(backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Return the current tables covered by incremental backup.
+   * @return set of tableNames
+   * @throws IOException exception
+   */
+  public Set<TableName> getIncrementalBackupTableSet() throws IOException {
+    return systemTable.getIncrementalBackupTableSet(backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Adds set of tables to overall incremental backup table set
+   * @param tables tables
+   * @throws IOException exception
+   */
+  public void addIncrementalBackupTableSet(Set<TableName> tables) throws IOException {
+    systemTable.addIncrementalBackupTableSet(tables, backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Saves list of WAL files after incremental backup operation. These files will be stored until
+   * TTL expiration and are used by Backup Log Cleaner plugin to determine which WAL files can be
+   * safely purged.
+   */
+  public void recordWALFiles(List<String> files) throws IOException {
+    systemTable.addWALFiles(files, backupInfo.getBackupId(), backupInfo.getBackupRootDir());
+  }
+
+  /**
+   * Get WAL files iterator
+   * @return WAL files iterator from backup system table
+   * @throws IOException
+   */
+  public Iterator<BackupSystemTable.WALItem> getWALFilesFromBackupSystem() throws IOException {
+    return systemTable.getWALFilesIterator(backupInfo.getBackupRootDir());
+  }
+
+  public Connection getConnection() {
+    return conn;
+  }
+}