You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2017/03/20 17:41:53 UTC

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 52b0ce5..bb6b40e 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -42223,7 +42223,7 @@ public final class MasterProtos {
        * <code>required .hbase.pb.SnapshotDescription snapshot = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder>
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescriptionOrBuilder> 
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
           snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -70510,7 +70510,7 @@ public final class MasterProtos {
 
       /**
        * <pre>
-       ** Get a run of the CleanerChore
+       ** Get a run of the CleanerChore 
        * </pre>
        *
        * <code>rpc RunCleanerChore(.hbase.pb.RunCleanerChoreRequest) returns (.hbase.pb.RunCleanerChoreResponse);</code>
@@ -72424,7 +72424,7 @@ public final class MasterProtos {
 
     /**
      * <pre>
-     ** Get a run of the CleanerChore
+     ** Get a run of the CleanerChore 
      * </pre>
      *
      * <code>rpc RunCleanerChore(.hbase.pb.RunCleanerChoreRequest) returns (.hbase.pb.RunCleanerChoreResponse);</code>
@@ -76186,32 +76186,32 @@ public final class MasterProtos {
       internal_static_hbase_pb_IsCatalogJanitorEnabledResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RunCleanerChoreRequest_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RunCleanerChoreRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RunCleanerChoreResponse_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RunCleanerChoreResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetCleanerChoreRunningRequest_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetCleanerChoreRunningRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SetCleanerChoreRunningResponse_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SetCleanerChoreRunningResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsCleanerChoreEnabledRequest_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsCleanerChoreEnabledRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_IsCleanerChoreEnabledResponse_descriptor;
-  private static final
+  private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_IsCleanerChoreEnabledResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-protocol-shaded/src/main/protobuf/Backup.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Backup.proto b/hbase-protocol-shaded/src/main/protobuf/Backup.proto
new file mode 100644
index 0000000..241d42a
--- /dev/null
+++ b/hbase-protocol-shaded/src/main/protobuf/Backup.proto
@@ -0,0 +1,117 @@
+/**
+ * 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 optional 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.
+ */
+
+// This file contains Backup manifest
+package hbase.pb;
+
+option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
+option java_outer_classname = "BackupProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+
+/**
+ * Backup type enum: FULL or INCREMENTAL
+ */
+
+enum BackupType {
+  FULL = 0;
+  INCREMENTAL = 1;
+}
+/**
+ * ServerTimestamp keeps last WAL roll time per Region Server
+ */
+message ServerTimestamp {
+  optional ServerName server_name = 1;
+  optional uint64 timestamp = 2;
+}
+
+/**
+ *  TableServerTimestamp keeps last WAL roll time per Region Server & Table
+ *  Each table have different last WAL roll time stamps across cluster, on every RS
+ */
+message TableServerTimestamp {
+  optional TableName table_name = 1;
+  repeated ServerTimestamp server_timestamp = 2;
+}
+
+/**
+ * Structure keeps relevant info for backup restore session
+ */
+message BackupImage {
+  optional string backup_id = 1;
+  optional BackupType backup_type = 2;
+  optional string backup_root_dir = 3;
+  repeated TableName table_list = 4;
+  optional uint64 start_ts = 5;
+  optional uint64 complete_ts = 6;
+  repeated BackupImage ancestors = 7;
+  repeated TableServerTimestamp tst_map = 8;
+
+}
+
+/**
+ * Internal structure used during backup
+ */
+message BackupTableInfo {
+  optional TableName table_name = 1;
+  optional string snapshot_name = 2;
+}
+
+/**
+ * Backup session information
+ */
+message BackupInfo {
+  optional string backup_id = 1;
+  optional BackupType backup_type = 2;
+  optional string backup_root_dir = 3;
+  optional BackupState backup_state = 4;
+  optional BackupPhase backup_phase = 5;
+  optional string failed_message = 6;
+  repeated BackupTableInfo backup_table_info = 7;
+  optional uint64  start_ts = 8;
+  optional uint64  complete_ts = 9;
+  optional uint32 progress = 10;
+  optional uint32 workers_number = 11;
+  optional uint64 bandwidth = 12;
+
+  /**
+   * Backup session states
+   */
+  enum BackupState {
+    RUNNING = 0;
+    COMPLETE = 1;
+    FAILED = 2;
+    CANCELLED = 3;
+  }
+
+  /**
+   * Phases of a backup session in RUNNING state
+   *
+   */
+  enum BackupPhase {
+    REQUEST = 0;
+    SNAPSHOT = 1;
+    PREPARE_INCREMENTAL = 2;
+    SNAPSHOTCOPY = 3;
+    INCREMENTAL_COPY = 4;
+    STORE_MANIFEST = 5;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 09a4512..1eba784 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -674,6 +674,11 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-distcp</artifactId>
+          <version>${hadoop-two.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
         </dependency>
         <dependency>
@@ -759,6 +764,11 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-distcp</artifactId>
+          <version>${hadoop-three.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
new file mode 100644
index 0000000..6f642a4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupAdmin.java
@@ -0,0 +1,128 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * The administrative API for HBase Backup. Construct an instance and call {@link #close()}
+ * afterwards.
+ * <p>
+ * BackupAdmin can be used to create backups, restore data from backups and for other
+ * backup-related operations.
+ * @since 2.0
+ */
+@InterfaceAudience.Private
+public interface BackupAdmin extends Closeable {
+
+  /**
+   * Backup given list of tables fully. This is a synchronous operation.
+   * It returns backup id on success or throw exception on failure.
+   * @param userRequest BackupRequest instance
+   * @return the backup Id
+   */
+
+  String backupTables(final BackupRequest userRequest) throws IOException;
+
+  /**
+   * Restore backup
+   * @param request restore request
+   * @throws IOException exception
+   */
+  void restore(RestoreRequest request) throws IOException;
+
+  /**
+   * Describe backup image command
+   * @param backupId backup id
+   * @return backup info
+   * @throws IOException exception
+   */
+  BackupInfo getBackupInfo(String backupId) throws IOException;
+
+
+  /**
+   * Delete backup image command
+   * @param backupIds backup id list
+   * @return total number of deleted sessions
+   * @throws IOException exception
+   */
+  int deleteBackups(String[] backupIds) throws IOException;
+
+  /**
+   * Show backup history command
+   * @param n last n backup sessions
+   * @return list of backup info objects
+   * @throws IOException exception
+   */
+  List<BackupInfo> getHistory(int n) throws IOException;
+
+  /**
+   * Show backup history command with filters
+   * @param n last n backup sessions
+   * @param f list of filters
+   * @return list of backup info objects
+   * @throws IOException exception
+   */
+  List<BackupInfo> getHistory(int n, BackupInfo.Filter... f) throws IOException;
+
+  /**
+   * Backup sets list command - list all backup sets. Backup set is a named group of tables.
+   * @return all registered backup sets
+   * @throws IOException exception
+   */
+  List<BackupSet> listBackupSets() throws IOException;
+
+  /**
+   * Backup set describe command. Shows list of tables in this particular backup set.
+   * @param name set name
+   * @return backup set description or null
+   * @throws IOException exception
+   */
+  BackupSet getBackupSet(String name) throws IOException;
+
+  /**
+   * Delete backup set command
+   * @param name backup set name
+   * @return true, if success, false - otherwise
+   * @throws IOException exception
+   */
+  boolean deleteBackupSet(String name) throws IOException;
+
+  /**
+   * Add tables to backup set command
+   * @param name name of backup set.
+   * @param tables list of tables to be added to this set.
+   * @throws IOException exception
+   */
+  void addToBackupSet(String name, TableName[] tables) throws IOException;
+
+  /**
+   * Remove tables from backup set
+   * @param name name of backup set.
+   * @param tables list of tables to be removed from this set.
+   * @throws IOException exception
+   */
+  void removeFromBackupSet(String name, TableName[] tables) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
new file mode 100644
index 0000000..007e4c1
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyJob.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Backup copy job is a part of a backup process. The concrete implementation is responsible for
+ * copying data from a cluster to backup destination. Concrete implementation is provided by backup
+ * provider, see {@link BackupRestoreFactory}
+ */
+@InterfaceAudience.Private
+public interface BackupCopyJob extends Configurable {
+
+  /**
+   * Copy backup data to destination
+   * @param backupInfo context object
+   * @param backupManager backup manager
+   * @param conf configuration
+   * @param backupType backup type (FULL or INCREMENTAL)
+   * @param options array of options (implementation-specific)
+   * @return result (0 - success, -1 failure )
+   * @throws IOException exception
+   */
+  int copy(BackupInfo backupInfo, BackupManager backupManager, Configuration conf,
+      BackupType backupType, String[] options) throws IOException;
+
+  /**
+   * Cancel copy job
+   * @param jobHandler backup copy job handler
+   * @throws IOException
+   */
+  void cancel(String jobHandler) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
new file mode 100644
index 0000000..5794fce
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupDriver.java
@@ -0,0 +1,204 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import static org.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_DEBUG;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_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_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_WORKERS;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_WORKERS_DESC;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.commons.cli.CommandLine;
+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.HBaseConfiguration;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.backup.impl.BackupCommands;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ *
+ * Command-line entry point for backup operation
+ *
+ */
+@InterfaceAudience.Private
+public class BackupDriver extends AbstractHBaseTool {
+
+  private static final Log LOG = LogFactory.getLog(BackupDriver.class);
+  private CommandLine cmd;
+
+  public BackupDriver() throws IOException {
+    init();
+  }
+
+  protected void init() throws IOException {
+    // disable irrelevant loggers to avoid it mess up command output
+    LogUtils.disableZkAndClientLoggers(LOG);
+  }
+
+  private int parseAndRun(String[] args) throws IOException {
+
+    // Check if backup is enabled
+    if (!BackupManager.isBackupEnabled(getConf())) {
+      System.err.println(BackupRestoreConstants.ENABLE_BACKUP);
+      return -1;
+    }
+
+    System.out.println(BackupRestoreConstants.VERIFY_BACKUP);
+
+    String cmd = null;
+    String[] remainArgs = null;
+    if (args == null || args.length == 0) {
+      printToolUsage();
+      return -1;
+    } else {
+      cmd = args[0];
+      remainArgs = new String[args.length - 1];
+      if (args.length > 1) {
+        System.arraycopy(args, 1, remainArgs, 0, args.length - 1);
+      }
+    }
+
+    BackupCommand type = BackupCommand.HELP;
+    if (BackupCommand.CREATE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.CREATE;
+    } else if (BackupCommand.HELP.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.HELP;
+    } else if (BackupCommand.DELETE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.DELETE;
+    } else if (BackupCommand.DESCRIBE.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.DESCRIBE;
+    } else if (BackupCommand.HISTORY.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.HISTORY;
+    } else if (BackupCommand.PROGRESS.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.PROGRESS;
+    } else if (BackupCommand.SET.name().equalsIgnoreCase(cmd)) {
+      type = BackupCommand.SET;
+    } else {
+      System.out.println("Unsupported command for backup: " + cmd);
+      printToolUsage();
+      return -1;
+    }
+
+    // enable debug logging
+    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+    if (this.cmd.hasOption(OPTION_DEBUG)) {
+      backupClientLogger.setLevel(Level.DEBUG);
+    } else {
+      backupClientLogger.setLevel(Level.INFO);
+    }
+
+    BackupCommands.Command command = BackupCommands.createCommand(getConf(), type, this.cmd);
+    if (type == BackupCommand.CREATE && conf != null) {
+      ((BackupCommands.CreateCommand) command).setConf(conf);
+    }
+    try {
+      command.execute();
+    } catch (IOException e) {
+      if (e.getMessage().equals(BackupCommands.INCORRECT_USAGE)) {
+        return -1;
+      }
+      throw e;
+    }
+    return 0;
+  }
+
+  @Override
+  protected void addOptions() {
+    // define supported options
+    addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC);
+    addOptWithArg(OPTION_TABLE, OPTION_TABLE_DESC);
+    addOptWithArg(OPTION_BANDWIDTH, OPTION_BANDWIDTH_DESC);
+    addOptWithArg(OPTION_WORKERS, OPTION_WORKERS_DESC);
+    addOptWithArg(OPTION_RECORD_NUMBER, OPTION_RECORD_NUMBER_DESC);
+    addOptWithArg(OPTION_SET, OPTION_SET_DESC);
+    addOptWithArg(OPTION_PATH, OPTION_PATH_DESC);
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    this.cmd = cmd;
+  }
+
+  @Override
+  protected int doWork() throws Exception {
+    return parseAndRun(cmd.getArgs());
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    Path hbasedir = FSUtils.getRootDir(conf);
+    URI defaultFs = hbasedir.getFileSystem(conf).getUri();
+    FSUtils.setFsDefault(conf, new Path(defaultFs));
+    int ret = ToolRunner.run(conf, new BackupDriver(), args);
+    System.exit(ret);
+  }
+
+  @Override
+  public int run(String[] args) throws IOException {
+    if (conf == null) {
+      LOG.error("Tool configuration is not initialized");
+      throw new NullPointerException("conf");
+    }
+
+    CommandLine cmd;
+    try {
+      // parse the command line arguments
+      cmd = parseArgs(args);
+      cmdLineArgs = args;
+    } catch (Exception e) {
+      System.err.println("Error when parsing command-line arguments: " + e.getMessage());
+      printToolUsage();
+      return EXIT_FAILURE;
+    }
+    processOptions(cmd);
+
+    int ret = EXIT_FAILURE;
+    try {
+      ret = doWork();
+    } catch (Exception e) {
+      LOG.error("Error running command-line tool", e);
+      return EXIT_FAILURE;
+    }
+    return ret;
+  }
+
+  protected void printToolUsage() throws IOException {
+    System.out.println(BackupCommands.USAGE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
new file mode 100644
index 0000000..c7d2848
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
@@ -0,0 +1,545 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos.BackupInfo.Builder;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * An object to encapsulate the information for each backup session
+ */
+@InterfaceAudience.Private
+public class BackupInfo implements Comparable<BackupInfo> {
+  private static final Log LOG = LogFactory.getLog(BackupInfo.class);
+
+  public static interface Filter {
+
+    /**
+     * Filter interface
+     * @param info backup info
+     * @return true if info passes filter, false otherwise
+     */
+    public boolean apply(BackupInfo info);
+  }
+
+  /**
+   * Backup session states
+   */
+  public static enum BackupState {
+    RUNNING, COMPLETE, FAILED, ANY;
+  }
+
+  /**
+   * BackupPhase - phases of an ACTIVE backup session (running), when state of a backup session is
+   * BackupState.RUNNING
+   */
+  public static enum BackupPhase {
+    REQUEST, SNAPSHOT, PREPARE_INCREMENTAL, SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
+  }
+
+  /**
+   * Backup id
+   */
+  private String backupId;
+
+  /**
+   * Backup type, full or incremental
+   */
+  private BackupType type;
+
+  /**
+   * Target root directory for storing the backup files
+   */
+  private String backupRootDir;
+
+  /**
+   * Backup state
+   */
+  private BackupState state;
+
+  /**
+   * Backup phase
+   */
+  private BackupPhase phase;
+
+  /**
+   * Backup failure message
+   */
+  private String failedMsg;
+
+  /**
+   * Backup status map for all tables
+   */
+  private Map<TableName, BackupTableInfo> backupTableInfoMap;
+
+  /**
+   * Actual start timestamp of a backup process
+   */
+  private long startTs;
+
+  /**
+   * Actual end timestamp of the backup process
+   */
+  private long completeTs;
+
+  /**
+   * Total bytes of incremental logs copied
+   */
+  private long totalBytesCopied;
+
+  /**
+   * For incremental backup, a location of a backed-up hlogs
+   */
+  private String hlogTargetDir = null;
+
+  /**
+   * Incremental backup file list
+   */
+  private List<String> incrBackupFileList;
+
+  /**
+   * New region server log timestamps for table set after distributed log roll key - table name,
+   * value - map of RegionServer hostname -> last log rolled timestamp
+   */
+  private HashMap<TableName, HashMap<String, Long>> tableSetTimestampMap;
+
+  /**
+   * Backup progress in %% (0-100)
+   */
+  private int progress;
+
+  /**
+   * Number of parallel workers. -1 - system defined
+   */
+  private int workers = -1;
+
+  /**
+   * Bandwidth per worker in MB per sec. -1 - unlimited
+   */
+  private long bandwidth = -1;
+
+  public BackupInfo() {
+    backupTableInfoMap = new HashMap<TableName, BackupTableInfo>();
+  }
+
+  public BackupInfo(String backupId, BackupType type, TableName[] tables, String targetRootDir) {
+    this();
+    this.backupId = backupId;
+    this.type = type;
+    this.backupRootDir = targetRootDir;
+    this.addTables(tables);
+    if (type == BackupType.INCREMENTAL) {
+      setHLogTargetDir(BackupUtils.getLogBackupDir(targetRootDir, backupId));
+    }
+    this.startTs = 0;
+    this.completeTs = 0;
+  }
+
+  public int getWorkers() {
+    return workers;
+  }
+
+  public void setWorkers(int workers) {
+    this.workers = workers;
+  }
+
+  public long getBandwidth() {
+    return bandwidth;
+  }
+
+  public void setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+  }
+
+  public void setBackupTableInfoMap(Map<TableName, BackupTableInfo> backupTableInfoMap) {
+    this.backupTableInfoMap = backupTableInfoMap;
+  }
+
+  public HashMap<TableName, HashMap<String, Long>> getTableSetTimestampMap() {
+    return tableSetTimestampMap;
+  }
+
+  public void setTableSetTimestampMap(HashMap<TableName,
+                                      HashMap<String, Long>> tableSetTimestampMap) {
+    this.tableSetTimestampMap = tableSetTimestampMap;
+  }
+
+  public void setType(BackupType type) {
+    this.type = type;
+  }
+
+  public void setBackupRootDir(String targetRootDir) {
+    this.backupRootDir = targetRootDir;
+  }
+
+  public void setTotalBytesCopied(long totalBytesCopied) {
+    this.totalBytesCopied = totalBytesCopied;
+  }
+
+  /**
+   * Set progress (0-100%)
+   * @param p progress value
+   */
+
+  public void setProgress(int p) {
+    this.progress = p;
+  }
+
+  /**
+   * Get current progress
+   */
+  public int getProgress() {
+    return progress;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  public void setBackupId(String backupId) {
+    this.backupId = backupId;
+  }
+
+  public BackupTableInfo getBackupTableInfo(TableName table) {
+    return this.backupTableInfoMap.get(table);
+  }
+
+  public String getFailedMsg() {
+    return failedMsg;
+  }
+
+  public void setFailedMsg(String failedMsg) {
+    this.failedMsg = failedMsg;
+  }
+
+  public long getStartTs() {
+    return startTs;
+  }
+
+  public void setStartTs(long startTs) {
+    this.startTs = startTs;
+  }
+
+  public long getCompleteTs() {
+    return completeTs;
+  }
+
+  public void setCompleteTs(long endTs) {
+    this.completeTs = endTs;
+  }
+
+  public long getTotalBytesCopied() {
+    return totalBytesCopied;
+  }
+
+  public BackupState getState() {
+    return state;
+  }
+
+  public void setState(BackupState flag) {
+    this.state = flag;
+  }
+
+  public BackupPhase getPhase() {
+    return phase;
+  }
+
+  public void setPhase(BackupPhase phase) {
+    this.phase = phase;
+  }
+
+  public BackupType getType() {
+    return type;
+  }
+
+  public void setSnapshotName(TableName table, String snapshotName) {
+    this.backupTableInfoMap.get(table).setSnapshotName(snapshotName);
+  }
+
+  public String getSnapshotName(TableName table) {
+    return this.backupTableInfoMap.get(table).getSnapshotName();
+  }
+
+  public List<String> getSnapshotNames() {
+    List<String> snapshotNames = new ArrayList<String>();
+    for (BackupTableInfo backupStatus : this.backupTableInfoMap.values()) {
+      snapshotNames.add(backupStatus.getSnapshotName());
+    }
+    return snapshotNames;
+  }
+
+  public Set<TableName> getTables() {
+    return this.backupTableInfoMap.keySet();
+  }
+
+  public List<TableName> getTableNames() {
+    return new ArrayList<TableName>(backupTableInfoMap.keySet());
+  }
+
+  public void addTables(TableName[] tables) {
+    for (TableName table : tables) {
+      BackupTableInfo backupStatus = new BackupTableInfo(table, this.backupRootDir, this.backupId);
+      this.backupTableInfoMap.put(table, backupStatus);
+    }
+  }
+
+  public void setTables(List<TableName> tables) {
+    this.backupTableInfoMap.clear();
+    for (TableName table : tables) {
+      BackupTableInfo backupStatus = new BackupTableInfo(table, this.backupRootDir, this.backupId);
+      this.backupTableInfoMap.put(table, backupStatus);
+    }
+  }
+
+  public String getBackupRootDir() {
+    return backupRootDir;
+  }
+
+  public String getTableBackupDir(TableName tableName) {
+    return BackupUtils.getTableBackupDir(backupRootDir, backupId, tableName);
+  }
+
+  public void setHLogTargetDir(String hlogTagetDir) {
+    this.hlogTargetDir = hlogTagetDir;
+  }
+
+  public String getHLogTargetDir() {
+    return hlogTargetDir;
+  }
+
+  public List<String> getIncrBackupFileList() {
+    return incrBackupFileList;
+  }
+
+  public void setIncrBackupFileList(List<String> incrBackupFileList) {
+    this.incrBackupFileList = incrBackupFileList;
+  }
+
+  /**
+   * Set the new region server log timestamps after distributed log roll
+   * @param newTableSetTimestampMap table timestamp map
+   */
+  public void
+      setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap) {
+    this.tableSetTimestampMap = newTableSetTimestampMap;
+  }
+
+  /**
+   * Get new region server log timestamps after distributed log roll
+   * @return new region server log timestamps
+   */
+  public HashMap<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+    return this.tableSetTimestampMap;
+  }
+
+  public TableName getTableBySnapshot(String snapshotName) {
+    for (Entry<TableName, BackupTableInfo> entry : this.backupTableInfoMap.entrySet()) {
+      if (snapshotName.equals(entry.getValue().getSnapshotName())) {
+        return entry.getKey();
+      }
+    }
+    return null;
+  }
+
+  public BackupProtos.BackupInfo toProtosBackupInfo() {
+    BackupProtos.BackupInfo.Builder builder = BackupProtos.BackupInfo.newBuilder();
+    builder.setBackupId(getBackupId());
+    setBackupTableInfoMap(builder);
+    builder.setCompleteTs(getCompleteTs());
+    if (getFailedMsg() != null) {
+      builder.setFailedMessage(getFailedMsg());
+    }
+    if (getState() != null) {
+      builder.setBackupState(BackupProtos.BackupInfo.BackupState.valueOf(getState().name()));
+    }
+    if (getPhase() != null) {
+      builder.setBackupPhase(BackupProtos.BackupInfo.BackupPhase.valueOf(getPhase().name()));
+    }
+
+    builder.setProgress(getProgress());
+    builder.setStartTs(getStartTs());
+    builder.setBackupRootDir(getBackupRootDir());
+    builder.setBackupType(BackupProtos.BackupType.valueOf(getType().name()));
+    builder.setWorkersNumber(workers);
+    builder.setBandwidth(bandwidth);
+    return builder.build();
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = 33 * type.hashCode() + backupId != null ? backupId.hashCode() : 0;
+    if (backupRootDir != null) {
+      hash = 33 * hash + backupRootDir.hashCode();
+    }
+    hash = 33 * hash + state.hashCode();
+    hash = 33 * hash + phase.hashCode();
+    hash = 33 * hash + (int) (startTs ^ (startTs >>> 32));
+    hash = 33 * hash + (int) (completeTs ^ (completeTs >>> 32));
+    hash = 33 * hash + (int) (totalBytesCopied ^ (totalBytesCopied >>> 32));
+    if (hlogTargetDir != null) {
+      hash = 33 * hash + hlogTargetDir.hashCode();
+    }
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof BackupInfo) {
+      BackupInfo other = (BackupInfo) obj;
+      try {
+        return Bytes.equals(toByteArray(), other.toByteArray());
+      } catch (IOException e) {
+        LOG.error(e);
+        return false;
+      }
+    } else {
+      return false;
+    }
+  }
+
+  public byte[] toByteArray() throws IOException {
+    return toProtosBackupInfo().toByteArray();
+  }
+
+  private void setBackupTableInfoMap(Builder builder) {
+    for (Entry<TableName, BackupTableInfo> entry : backupTableInfoMap.entrySet()) {
+      builder.addBackupTableInfo(entry.getValue().toProto());
+    }
+  }
+
+  public static BackupInfo fromByteArray(byte[] data) throws IOException {
+    return fromProto(BackupProtos.BackupInfo.parseFrom(data));
+  }
+
+  public static BackupInfo fromStream(final InputStream stream) throws IOException {
+    return fromProto(BackupProtos.BackupInfo.parseDelimitedFrom(stream));
+  }
+
+  public static BackupInfo fromProto(BackupProtos.BackupInfo proto) {
+    BackupInfo context = new BackupInfo();
+    context.setBackupId(proto.getBackupId());
+    context.setBackupTableInfoMap(toMap(proto.getBackupTableInfoList()));
+    context.setCompleteTs(proto.getCompleteTs());
+    if (proto.hasFailedMessage()) {
+      context.setFailedMsg(proto.getFailedMessage());
+    }
+    if (proto.hasBackupState()) {
+      context.setState(BackupInfo.BackupState.valueOf(proto.getBackupState().name()));
+    }
+
+    context.setHLogTargetDir(BackupUtils.getLogBackupDir(proto.getBackupRootDir(),
+      proto.getBackupId()));
+
+    if (proto.hasBackupPhase()) {
+      context.setPhase(BackupPhase.valueOf(proto.getBackupPhase().name()));
+    }
+    if (proto.hasProgress()) {
+      context.setProgress(proto.getProgress());
+    }
+    context.setStartTs(proto.getStartTs());
+    context.setBackupRootDir(proto.getBackupRootDir());
+    context.setType(BackupType.valueOf(proto.getBackupType().name()));
+    context.setWorkers(proto.getWorkersNumber());
+    context.setBandwidth(proto.getBandwidth());
+    return context;
+  }
+
+  private static Map<TableName, BackupTableInfo> toMap(List<BackupProtos.BackupTableInfo> list) {
+    HashMap<TableName, BackupTableInfo> map = new HashMap<>();
+    for (BackupProtos.BackupTableInfo tbs : list) {
+      map.put(ProtobufUtil.toTableName(tbs.getTableName()), BackupTableInfo.convert(tbs));
+    }
+    return map;
+  }
+
+  public String getShortDescription() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("{");
+    sb.append("ID=" + backupId).append(",");
+    sb.append("Type=" + getType()).append(",");
+    sb.append("Tables=" + getTableListAsString()).append(",");
+    sb.append("State=" + getState()).append(",");
+    Date date = null;
+    Calendar cal = Calendar.getInstance();
+    cal.setTimeInMillis(getStartTs());
+    date = cal.getTime();
+    sb.append("Start time=" + date).append(",");
+    if (state == BackupState.FAILED) {
+      sb.append("Failed message=" + getFailedMsg()).append(",");
+    } else if (state == BackupState.RUNNING) {
+      sb.append("Phase=" + getPhase()).append(",");
+    } else if (state == BackupState.COMPLETE) {
+      cal = Calendar.getInstance();
+      cal.setTimeInMillis(getCompleteTs());
+      date = cal.getTime();
+      sb.append("End time=" + date).append(",");
+    }
+    sb.append("Progress=" + getProgress() + "%");
+    sb.append("}");
+
+    return sb.toString();
+  }
+
+  public String getStatusAndProgressAsString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("id: ").append(getBackupId()).append(" state: ").append(getState())
+        .append(" progress: ").append(getProgress());
+    return sb.toString();
+  }
+
+  public String getTableListAsString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("{");
+    sb.append(StringUtils.join(backupTableInfoMap.keySet(), ","));
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * We use only time stamps to compare objects during sort operation
+   */
+  @Override
+  public int compareTo(BackupInfo o) {
+    Long thisTS =
+        Long.valueOf(this.getBackupId().substring(this.getBackupId().lastIndexOf("_") + 1));
+    Long otherTS = Long.valueOf(o.getBackupId().substring(o.getBackupId().lastIndexOf("_") + 1));
+    return thisTS.compareTo(otherTS);
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
new file mode 100644
index 0000000..770ccce
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * BackupRestoreConstants holds a bunch of HBase Backup and Restore constants
+ */
+@InterfaceAudience.Private
+public interface BackupRestoreConstants {
+  /*
+   * Backup/Restore constants
+   */
+  public final static String BACKUP_SYSTEM_TABLE_NAME_KEY = "hbase.backup.system.table.name";
+  public final static String BACKUP_SYSTEM_TABLE_NAME_DEFAULT = "hbase:backup";
+
+  public final static String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl";
+
+  public final static int BACKUP_SYSTEM_TTL_DEFAULT = HConstants.FOREVER;
+  public final static String BACKUP_ENABLE_KEY = "hbase.backup.enable";
+  public final static boolean BACKUP_ENABLE_DEFAULT = false;
+  /*
+   *  Drivers option list
+   */
+  public static final String OPTION_OVERWRITE = "o";
+  public static final String OPTION_OVERWRITE_DESC =
+      "Overwrite data if any of the restore target tables exists";
+
+  public static final String OPTION_CHECK = "c";
+  public static final String OPTION_CHECK_DESC =
+      "Check restore sequence and dependencies only (does not execute the command)";
+
+  public static final String OPTION_SET = "s";
+  public static final String OPTION_SET_DESC = "Backup set name";
+  public static final String OPTION_SET_RESTORE_DESC =
+      "Backup set to restore, mutually exclusive with -t (table list)";
+  public static final String OPTION_SET_BACKUP_DESC =
+      "Backup set to backup, mutually exclusive with -t (table list)";
+  public static final String OPTION_DEBUG = "d";
+  public static final String OPTION_DEBUG_DESC = "Enable debug loggings";
+
+  public static final String OPTION_TABLE = "t";
+  public static final String OPTION_TABLE_DESC = "Table name. If specified, only backup images,"
+      + " which contain this table will be listed.";
+
+  public static final String OPTION_TABLE_LIST = "l";
+  public static final String OPTION_TABLE_LIST_DESC = "Table name list, comma-separated.";
+
+  public static final String OPTION_BANDWIDTH = "b";
+  public static final String OPTION_BANDWIDTH_DESC = "Bandwidth per task (MapReduce task) in MB/s";
+
+  public static final String OPTION_WORKERS = "w";
+  public static final String OPTION_WORKERS_DESC = "Number of parallel MapReduce tasks to execute";
+
+  public static final String OPTION_RECORD_NUMBER = "n";
+  public static final String OPTION_RECORD_NUMBER_DESC =
+      "Number of records of backup history. Default: 10";
+
+  public static final String OPTION_PATH = "p";
+  public static final String OPTION_PATH_DESC = "Backup destination root directory path";
+
+  public static final String OPTION_TABLE_MAPPING = "m";
+  public static final String OPTION_TABLE_MAPPING_DESC =
+      "A comma separated list of target tables. "
+          + "If specified, each table in <tables> must have a mapping";
+
+  public static final String BACKUP_CONFIG_STRING =  BackupRestoreConstants.BACKUP_ENABLE_KEY + "=true\n"
+      + "hbase.master.logcleaner.plugins="
+      +"YOUR_PLUGINS,org.apache.hadoop.hbase.backup.master.BackupLogCleaner\n"
+      + "hbase.procedure.master.classes=YOUR_CLASSES,"
+      +"org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager\n"
+      + "hbase.procedure.regionserver.classes=YOUR_CLASSES,"
+      + "org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager\n"
+      + "and restart the cluster";
+  public static final String ENABLE_BACKUP = "Backup is not enabled. To enable backup, "+
+      "in hbase-site.xml, set:\n "
+      + BACKUP_CONFIG_STRING;
+
+  public static final String VERIFY_BACKUP = "Please make sure that backup is enabled on the cluster. To enable backup, "+
+      "in hbase-site.xml, set:\n "
+      + BACKUP_CONFIG_STRING;
+
+  /*
+   *  Delimiter in table name list in restore command
+   */
+  public static final String TABLENAME_DELIMITER_IN_COMMAND = ",";
+
+  public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root";
+
+  public static final String BACKUPID_PREFIX = "backup_";
+
+  public static enum BackupCommand {
+    CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP, PROGRESS,
+    SET, SET_ADD, SET_REMOVE, SET_DELETE, SET_DESCRIBE, SET_LIST
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
new file mode 100644
index 0000000..6d8967a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreFactory.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceBackupCopyJob;
+import org.apache.hadoop.hbase.backup.mapreduce.MapReduceRestoreJob;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Factory implementation for backup/restore related jobs
+ *
+ */
+@InterfaceAudience.Private
+public final class BackupRestoreFactory {
+
+  public final static String HBASE_INCR_RESTORE_IMPL_CLASS = "hbase.incremental.restore.class";
+  public final static String HBASE_BACKUP_COPY_IMPL_CLASS = "hbase.backup.copy.class";
+
+  private BackupRestoreFactory() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  /**
+   * Gets backup restore job
+   * @param conf configuration
+   * @return backup restore task instance
+   */
+  public static RestoreJob getRestoreJob(Configuration conf) {
+    Class<? extends RestoreJob> cls =
+        conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, MapReduceRestoreJob.class, RestoreJob.class);
+    RestoreJob service = ReflectionUtils.newInstance(cls, conf);
+    service.setConf(conf);
+    return service;
+  }
+
+  /**
+   * Gets backup copy job
+   * @param conf configuration
+   * @return backup copy task
+   */
+  public static BackupCopyJob getBackupCopyJob(Configuration conf) {
+    Class<? extends BackupCopyJob> cls =
+        conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, MapReduceBackupCopyJob.class,
+          BackupCopyJob.class);
+    BackupCopyJob service = ReflectionUtils.newInstance(cls, conf);
+    service.setConf(conf);
+    return service;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
new file mode 100644
index 0000000..46044db
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
@@ -0,0 +1,141 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupManifest;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * View to an on-disk Backup Image FileSytem Provides the set of methods necessary to interact with
+ * the on-disk Backup Image data.
+ */
+@InterfaceAudience.Private
+public class HBackupFileSystem {
+  public static final Log LOG = LogFactory.getLog(HBackupFileSystem.class);
+
+  /**
+   * This is utility class.
+   */
+  private HBackupFileSystem() {
+  }
+
+  /**
+   * Given the backup root dir, backup id and the table name, return the backup image location,
+   * which is also where the backup manifest file is. return value look like:
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup/backup_1396650096738/default/t1_dn/",
+   * where "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
+   * @param backupRootDir backup root directory
+   * @param backupId backup id
+   * @param tableName table name
+   * @return backupPath String for the particular table
+   */
+  public static String
+      getTableBackupDir(String backupRootDir, String backupId, TableName tableName) {
+    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
+        + tableName.getNamespaceAsString() + Path.SEPARATOR + tableName.getQualifierAsString()
+        + Path.SEPARATOR;
+  }
+
+  /**
+   * Given the backup root dir, backup id and the table name, return the backup image location,
+   * which is also where the backup manifest file is. return value look like:
+   * "hdfs://backup.hbase.org:9000/user/biadmin/backup/backup_1396650096738/default/t1_dn/",
+   * where "hdfs://backup.hbase.org:9000/user/biadmin/backup" is a backup root directory
+   * @param backupRootPath backup root path
+   * @param tableName table name
+   * @param backupId backup Id
+   * @return backupPath for the particular table
+   */
+  public static Path getTableBackupPath(TableName tableName,
+      Path backupRootPath, String backupId) {
+    return new Path(getTableBackupDir(backupRootPath.toString(), backupId, tableName));
+  }
+
+  /**
+   * Given the backup root dir and the backup id, return the log file location for an incremental
+   * backup.
+   * @param backupRootDir backup root directory
+   * @param backupId backup id
+   * @return logBackupDir: ".../user/biadmin/backup/WALs/backup_1396650096738"
+   */
+  public static String getLogBackupDir(String backupRootDir, String backupId) {
+    return backupRootDir + Path.SEPARATOR + backupId + Path.SEPARATOR
+        + HConstants.HREGION_LOGDIR_NAME;
+  }
+
+  public static Path getLogBackupPath(String backupRootDir, String backupId) {
+    return new Path(getLogBackupDir(backupRootDir, backupId));
+  }
+
+  private static Path getManifestPath(TableName tableName, Configuration conf, Path backupRootPath,
+      String backupId) throws IOException {
+    Path manifestPath =
+        new Path(getTableBackupPath(tableName, backupRootPath, backupId),
+            BackupManifest.MANIFEST_FILE_NAME);
+
+    FileSystem fs = backupRootPath.getFileSystem(conf);
+    if (!fs.exists(manifestPath)) {
+      // check log dir for incremental backup case
+      manifestPath =
+          new Path(getLogBackupDir(backupRootPath.toString(), backupId) + Path.SEPARATOR
+              + BackupManifest.MANIFEST_FILE_NAME);
+      if (!fs.exists(manifestPath)) {
+        String errorMsg =
+            "Could not find backup manifest " + BackupManifest.MANIFEST_FILE_NAME + " for "
+                + backupId + ". File " + manifestPath + " does not exists. Did " + backupId
+                + " correspond to previously taken backup ?";
+        throw new IOException(errorMsg);
+      }
+    }
+    return manifestPath;
+  }
+
+  public static BackupManifest getManifest(TableName tableName, Configuration conf,
+      Path backupRootPath, String backupId) throws IOException {
+    BackupManifest manifest =
+        new BackupManifest(conf, getManifestPath(tableName, conf, backupRootPath, backupId));
+    return manifest;
+  }
+
+  /**
+   * Check whether the backup image path and there is manifest file in the path.
+   * @param backupManifestMap If all the manifests are found, then they are put into this map
+   * @param tableArray the tables involved
+   * @throws IOException exception
+   */
+  public static void checkImageManifestExist(HashMap<TableName, BackupManifest> backupManifestMap,
+      TableName[] tableArray, Configuration conf, Path backupRootPath, String backupId)
+      throws IOException {
+    for (TableName tableName : tableArray) {
+      BackupManifest manifest = getManifest(tableName, conf, backupRootPath, backupId);
+      backupManifestMap.put(tableName, manifest);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
new file mode 100644
index 0000000..1becb75
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/LogUtils.java
@@ -0,0 +1,50 @@
+/**
+  * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ * Utility class for disabling Zk and client logging
+ *
+ */
+@InterfaceAudience.Private
+final class LogUtils {
+
+  private LogUtils() {
+  }
+
+  /**
+   * Disables Zk- and HBase client logging
+   * @param log
+   */
+  static void disableZkAndClientLoggers(Log log) {
+    // disable zookeeper log to avoid it mess up command output
+    Logger zkLogger = Logger.getLogger("org.apache.zookeeper");
+    zkLogger.setLevel(Level.OFF);
+    // disable hbase zookeeper tool log to avoid it mess up command output
+    Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper");
+    hbaseZkLogger.setLevel(Level.OFF);
+    // disable hbase client log to avoid it mess up command output
+    Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client");
+    hbaseClientLogger.setLevel(Level.OFF);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
new file mode 100644
index 0000000..82a1b56
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
@@ -0,0 +1,265 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.backup;
+
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_CHECK_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_DEBUG_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_OVERWRITE_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_SET_RESTORE_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_LIST_DESC;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING;
+import static org.apache.hadoop.hbase.backup.BackupRestoreConstants.OPTION_TABLE_MAPPING_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.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.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
+import org.apache.hadoop.hbase.backup.impl.BackupManager;
+import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
+import org.apache.hadoop.hbase.backup.util.BackupUtils;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ *
+ * Command-line entry point for restore operation
+ *
+ */
+@InterfaceAudience.Private
+public class RestoreDriver extends AbstractHBaseTool {
+
+  private static final Log LOG = LogFactory.getLog(RestoreDriver.class);
+  private CommandLine cmd;
+
+  private static final String USAGE_STRING =
+      "Usage: hbase restore <backup_path> <backup_id> [options]\n"
+          + "  backup_path     Path to a backup destination root\n"
+          + "  backup_id       Backup image ID to restore\n"
+          + "  table(s)        Comma-separated list of tables to restore\n";
+
+  private static final String USAGE_FOOTER = "";
+
+  protected RestoreDriver() throws IOException {
+    init();
+  }
+
+  protected void init() throws IOException {
+    // disable irrelevant loggers to avoid it mess up command output
+    LogUtils.disableZkAndClientLoggers(LOG);
+  }
+
+  private int parseAndRun(String[] args) throws IOException {
+    // Check if backup is enabled
+    if (!BackupManager.isBackupEnabled(getConf())) {
+      System.err.println(BackupRestoreConstants.ENABLE_BACKUP);
+      return -1;
+    }
+
+    System.out.println(BackupRestoreConstants.VERIFY_BACKUP);
+
+    // enable debug logging
+    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+    if (cmd.hasOption(OPTION_DEBUG)) {
+      backupClientLogger.setLevel(Level.DEBUG);
+    }
+
+    // whether to overwrite to existing table if any, false by default
+    boolean overwrite = cmd.hasOption(OPTION_OVERWRITE);
+    if (overwrite) {
+      LOG.debug("Found -overwrite option in restore command, "
+          + "will overwrite to existing table if any in the restore target");
+    }
+
+    // whether to only check the dependencies, false by default
+    boolean check = cmd.hasOption(OPTION_CHECK);
+    if (check) {
+      LOG.debug("Found -check option in restore command, "
+          + "will check and verify the dependencies");
+    }
+
+    if (cmd.hasOption(OPTION_SET) && cmd.hasOption(OPTION_TABLE)) {
+      System.err.println("Options -s and -t are mutaully exclusive,"+
+          " you can not specify both of them.");
+      printToolUsage();
+      return -1;
+    }
+
+    if (!cmd.hasOption(OPTION_SET) && !cmd.hasOption(OPTION_TABLE)) {
+      System.err.println("You have to specify either set name or table list to restore");
+      printToolUsage();
+      return -1;
+    }
+    // parse main restore command options
+    String[] remainArgs = cmd.getArgs();
+    if (remainArgs.length != 2) {
+      printToolUsage();
+      return -1;
+    }
+
+    String backupRootDir = remainArgs[0];
+    String backupId = remainArgs[1];
+    String tables = null;
+    String tableMapping =
+        cmd.hasOption(OPTION_TABLE_MAPPING) ? cmd.getOptionValue(OPTION_TABLE_MAPPING) : null;
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        BackupAdmin client = new BackupAdminImpl(conn);) {
+      // Check backup set
+      if (cmd.hasOption(OPTION_SET)) {
+        String setName = cmd.getOptionValue(OPTION_SET);
+        try {
+          tables = getTablesForSet(conn, setName, conf);
+        } catch (IOException e) {
+          System.out.println("ERROR: " + e.getMessage() + " for setName=" + setName);
+          printToolUsage();
+          return -2;
+        }
+        if (tables == null) {
+          System.out.println("ERROR: Backup set '" + setName
+              + "' is either empty or does not exist");
+          printToolUsage();
+          return -3;
+        }
+      } else {
+        tables = cmd.getOptionValue(OPTION_TABLE);
+      }
+
+      TableName[] sTableArray = BackupUtils.parseTableNames(tables);
+      TableName[] tTableArray = BackupUtils.parseTableNames(tableMapping);
+
+      if (sTableArray != null && tTableArray != null &&
+          (sTableArray.length != tTableArray.length)) {
+        System.out.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping);
+        printToolUsage();
+        return -4;
+      }
+
+      client.restore(BackupUtils.createRestoreRequest(backupRootDir, backupId, check,
+        sTableArray, tTableArray, overwrite));
+    } catch (Exception e) {
+      e.printStackTrace();
+      return -5;
+    }
+    return 0;
+  }
+
+  private String getTablesForSet(Connection conn, String name, Configuration conf)
+      throws IOException {
+    try (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 addOptions() {
+    // define supported options
+    addOptNoArg(OPTION_OVERWRITE, OPTION_OVERWRITE_DESC);
+    addOptNoArg(OPTION_CHECK, OPTION_CHECK_DESC);
+    addOptNoArg(OPTION_DEBUG, OPTION_DEBUG_DESC);
+    addOptWithArg(OPTION_SET, OPTION_SET_RESTORE_DESC);
+    addOptWithArg(OPTION_TABLE, OPTION_TABLE_LIST_DESC);
+
+    addOptWithArg(OPTION_TABLE_MAPPING, OPTION_TABLE_MAPPING_DESC);
+  }
+
+  @Override
+  protected void processOptions(CommandLine cmd) {
+    this.cmd = cmd;
+  }
+
+  @Override
+  protected int doWork() throws Exception {
+    return parseAndRun(cmd.getArgs());
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    Path hbasedir = FSUtils.getRootDir(conf);
+    URI defaultFs = hbasedir.getFileSystem(conf).getUri();
+    FSUtils.setFsDefault(conf, new Path(defaultFs));
+    int ret = ToolRunner.run(conf, new RestoreDriver(), args);
+    System.exit(ret);
+  }
+
+  @Override
+  public int run(String[] args) throws IOException {
+    if (conf == null) {
+      LOG.error("Tool configuration is not initialized");
+      throw new NullPointerException("conf");
+    }
+
+    CommandLine cmd;
+    try {
+      // parse the command line arguments
+      cmd = parseArgs(args);
+      cmdLineArgs = args;
+    } catch (Exception e) {
+      System.out.println("Error when parsing command-line arguments: " + e.getMessage());
+      printToolUsage();
+      return EXIT_FAILURE;
+    }
+
+    if (cmd.hasOption(SHORT_HELP_OPTION) || cmd.hasOption(LONG_HELP_OPTION)) {
+      printToolUsage();
+      return EXIT_FAILURE;
+    }
+
+    processOptions(cmd);
+
+    int ret = EXIT_FAILURE;
+    try {
+      ret = doWork();
+    } catch (Exception e) {
+      LOG.error("Error running command-line tool", e);
+      return EXIT_FAILURE;
+    }
+    return ret;
+  }
+
+  protected void printToolUsage() throws IOException {
+    System.out.println(USAGE_STRING);
+    HelpFormatter helpFormatter = new HelpFormatter();
+    helpFormatter.setLeftPadding(2);
+    helpFormatter.setDescPadding(8);
+    helpFormatter.setWidth(100);
+    helpFormatter.setSyntaxPrefix("Options:");
+    helpFormatter.printHelp(" ", null, options, USAGE_FOOTER);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/75d0f49d/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
new file mode 100644
index 0000000..86fb963
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreJob.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.backup;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Restore operation job interface Concrete implementation is provided by backup provider, see
+ * {@link BackupRestoreFactory}
+ */
+
+@InterfaceAudience.Private
+public interface RestoreJob extends Configurable {
+
+  /**
+   * Run restore operation
+   * @param dirPaths path array of WAL log directories
+   * @param fromTables from tables
+   * @param toTables to tables
+   * @param fullBackupRestore full backup restore
+   * @throws IOException
+   */
+  void run(Path[] dirPaths, TableName[] fromTables, TableName[] toTables,
+      boolean fullBackupRestore) throws IOException;
+}

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