http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-protocol/src/main/protobuf/Backup.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Backup.proto 
b/hbase-protocol/src/main/protobuf/Backup.proto
index c17ad06..7d1ec4b 100644
--- a/hbase-protocol/src/main/protobuf/Backup.proto
+++ b/hbase-protocol/src/main/protobuf/Backup.proto
@@ -77,11 +77,8 @@ message BackupManifest {
   repeated TableName table_list = 4;
   required uint64 start_ts = 5;
   required uint64 complete_ts = 6;
-  required int64 total_bytes = 7;
-  optional int64 log_bytes  = 8;
-  repeated TableServerTimestamp tst_map = 9;
-  repeated BackupImage dependent_backup_image = 10;
-  required bool compacted = 11; 
+  repeated TableServerTimestamp tst_map = 7;
+  repeated BackupImage dependent_backup_image = 8;
 }
 
 message TableBackupStatus {
@@ -90,7 +87,7 @@ message TableBackupStatus {
   optional string snapshot = 3;        
 }
 
-message BackupContext {
+message BackupInfo {
   required string backup_id = 1;
   required BackupType type = 2;
   required string target_root_dir = 3;
@@ -100,9 +97,10 @@ message BackupContext {
   repeated TableBackupStatus table_backup_status = 7;
   optional uint64  start_ts = 8;
   optional uint64  end_ts = 9;
-  optional int64  total_bytes_copied = 10;
-  optional string hlog_target_dir = 11;
-  optional uint32 progress = 12; 
+  optional uint32 progress = 10; 
+  optional string job_id = 11;
+  required uint32 workers_number = 12;
+  required uint64 bandwidth = 13;
   
   enum BackupState {
     WAITING = 0;
@@ -123,7 +121,7 @@ message BackupContext {
 }
 
 message BackupProcContext {
-  required BackupContext ctx = 1;
+  required BackupInfo ctx = 1;
   repeated ServerTimestamp server_timestamp = 2;
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
new file mode 100644
index 0000000..7b0b454
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
@@ -0,0 +1,108 @@
+/**
+ * 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.List;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+
+public interface BackupClient extends Configurable{
+
+  /**
+   * Describe backup image command
+   * @param backupId - backup id
+   * @return backup info
+   * @throws IOException
+   */
+  public BackupInfo getBackupInfo(String backupId) throws IOException;
+
+  /**
+   * Show backup progress command
+   * @param backupId - backup id (may be null)
+   * @return backup progress (0-100%), -1 if no active sessions
+   *  or session not found
+   * @throws IOException
+   */
+  public int getProgress(String backupId) throws IOException;
+
+  /**
+   * Delete backup image command
+   * @param backupIds - backup id
+   * @return total number of deleted sessions
+   * @throws IOException
+   */
+  public int deleteBackups(String[] backupIds) throws IOException;
+
+//  /**
+//  TODO: Phase 3
+//   * Cancel current active backup command
+//   * @param backupId - backup id
+//   * @throws IOException
+//   */
+//  public void cancelBackup(String backupId) throws IOException;
+
+  /**
+   * Show backup history command
+   * @param n - last n backup sessions
+   * @throws IOException
+   */
+  public List<BackupInfo> getHistory(int n) throws IOException;
+
+  /**
+   * Backup sets list command - list all backup sets. Backup set is 
+   * a named group of tables. 
+   * @throws IOException
+   */
+  public List<BackupSet> listBackupSets() throws IOException;
+
+  /**
+   * Backup set describe command. Shows list of tables in
+   * this particular backup set.
+   * @param name set name
+   * @return backup set description or null
+   * @throws IOException
+   */
+  public BackupSet getBackupSet(String name) throws IOException;
+
+  /**
+   * Delete backup set command
+   * @param name - backup set name
+   * @return true, if success, false - otherwise 
+   * @throws IOException
+   */
+  public boolean deleteBackupSet(String name) throws IOException;
+
+  /**
+   * Add tables to backup set command
+   * @param name - name of backup set.
+   * @param tables - list of tables to be added to this set.
+   * @throws IOException
+   */
+  public void addToBackupSet(String name, String[] tablesOrNamespaces) 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
+   */
+  public void removeFromBackupSet(String name, String[] tablesOrNamepsaces) 
throws IOException;
+
+ }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index 015c80b..182c402 100644
--- 
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
@@ -20,9 +20,6 @@ package org.apache.hadoop.hbase.backup;
 import java.io.IOException;
 
 import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.cli.PosixParser;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -39,13 +36,22 @@ import org.apache.log4j.Logger;
 public class BackupDriver extends AbstractHBaseTool {
 
   private static final Log LOG = LogFactory.getLog(BackupDriver.class);
-  private Options opt;
   private CommandLine cmd;
-
+    
+  public BackupDriver() throws IOException
+  {
+    init();
+  }
+  
   protected void init() throws IOException {
     // define supported options
-    opt = new Options();
-    opt.addOption("debug", false, "Enable debug loggings");
+    addOptNoArg("debug", "Enable debug loggings");
+    addOptNoArg("all", "All tables");
+    addOptWithArg("t", "Table name");
+    addOptWithArg("b", "Bandwidth (MB/s)");
+    addOptWithArg("w", "Number of workers");
+    addOptWithArg("n", "History length");
+    addOptWithArg("set", "Backup set name");
 
     // disable irrelevant loggers to avoid it mess up command output
     LogUtils.disableUselessLoggers(LOG);
@@ -64,19 +70,22 @@ public class BackupDriver extends AbstractHBaseTool {
         System.arraycopy(args, 1, remainArgs, 0, args.length - 1);
       }
     }
-    CommandLine cmdline = null;
-    try {
-      cmdline = new PosixParser().parse(opt, remainArgs);
-    } catch (ParseException e) {
-      LOG.error("Could not parse command", e);
-      return -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);
       return -1;
@@ -84,14 +93,18 @@ public class BackupDriver extends AbstractHBaseTool {
 
     // enable debug logging
     Logger backupClientLogger = 
Logger.getLogger("org.apache.hadoop.hbase.backup");
-    if (cmdline.hasOption("debug")) {
+    if (this.cmd.hasOption("debug")) {
       backupClientLogger.setLevel(Level.DEBUG);
     } else {
       backupClientLogger.setLevel(Level.INFO);
     }
 
     // TODO: get rid of Command altogether?
-    BackupCommands.createCommand(getConf(), type, cmdline).execute();
+    BackupCommands.Command command = BackupCommands.createCommand(getConf(), 
type, this.cmd);
+    if( type == BackupCommand.CREATE && conf != null) {
+      ((BackupCommands.CreateCommand) command).setConf(conf);
+    }   
+    command.execute();
     return 0;
   }
 
@@ -106,14 +119,14 @@ public class BackupDriver extends AbstractHBaseTool {
 
   @Override
   protected int doWork() throws Exception {
-    init();
     return parseAndRun(cmd.getArgs());
   }
 
   public static void main(String[] args) throws Exception {
     Configuration conf = HBaseConfiguration.create();
-    int ret = ToolRunner.run(conf, new BackupDriver(), args);
-    System.exit(ret);
+    int ret = ToolRunner.run(conf, new BackupDriver(), args);    
+    System.exit(ret);    
   }
+  
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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..1ed95f4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupInfo.java
@@ -0,0 +1,483 @@
+/**
+ * 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.BackupType;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
+import 
org.apache.hadoop.hbase.protobuf.generated.BackupProtos.BackupInfo.Builder;
+import 
org.apache.hadoop.hbase.protobuf.generated.BackupProtos.TableBackupStatus;
+
+
+/**
+ * An object to encapsulate the information for each backup request
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupInfo implements Comparable<BackupInfo> {
+  private static final Log LOG = LogFactory.getLog(BackupInfo.class);
+  // backup status flag
+  public static enum BackupState {
+    WAITING, RUNNING, COMPLETE, FAILED, CANCELLED;
+  }
+  // backup phase    
+  public static enum BackupPhase {
+    SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
+  }
+
+  // backup id: a timestamp when we request the backup
+  private String backupId;
+
+  // backup type, full or incremental
+  private BackupType type;
+
+  // target root directory for storing the backup files
+  private String targetRootDir;
+
+  // overall backup state
+  private BackupState state;
+
+  // overall backup phase
+  private BackupPhase phase;
+
+  // overall backup failure message
+  private String failedMsg;
+
+  // backup status map for all tables
+  private Map<TableName, BackupStatus> backupStatusMap;
+
+  // actual start timestamp of the backup process
+  private long startTs;
+
+  // actual end timestamp of the backup process, could be fail or complete
+  private long endTs;
+
+  // the total bytes of incremental logs copied
+  private long totalBytesCopied;
+
+  // for incremental backup, the location of the backed-up hlogs
+  private String hlogTargetDir = null;
+
+  // incremental backup file list
+  transient private List<String> incrBackupFileList;
+
+  // new region server log timestamps for table set after distributed log roll
+  // key - table name, value - map of RegionServer hostname -> last log rolled 
timestamp
+  transient private HashMap<TableName, HashMap<String, Long>> 
tableSetTimestampMap;
+
+  // backup progress in %% (0-100)
+  private int progress;
+  
+  // distributed job id
+  private String jobId;
+  
+  // Number of parallel workers. -1 - system defined
+  private int workers = -1;
+ 
+  // Bandwidth per worker in MB per sec. -1 - unlimited
+  private long bandwidth = -1;  
+  
+  public BackupInfo() {
+  }
+
+  public BackupInfo(String backupId, BackupType type, TableName[] tables, 
String targetRootDir) {
+    backupStatusMap = new HashMap<TableName, BackupStatus>();
+
+    this.backupId = backupId;
+    this.type = type;
+    this.targetRootDir = targetRootDir;
+    if(LOG.isDebugEnabled()){
+      LOG.debug("CreateBackupContext: " + tables.length+" "+tables[0] );
+    }
+    this.addTables(tables);
+
+    if (type == BackupType.INCREMENTAL) {
+      setHlogTargetDir(HBackupFileSystem.getLogBackupDir(targetRootDir, 
backupId));
+    }
+
+    this.startTs = 0;
+    this.endTs = 0;
+  }
+
+  public String getJobId() {
+    return jobId;
+  }
+
+  public void setJobId(String jobId) {
+    this.jobId = jobId;
+  }
+
+  public int getWorkers() {
+    return workers;
+  }
+
+  public void setWorkers(int workers) {
+    this.workers = workers;
+  }
+
+  public long getBandwidth() {
+    return bandwidth;
+  }
+
+  public void setBandwidth(long bandwidth) {
+    this.bandwidth = bandwidth;
+  }
+
+  public void setBackupStatusMap(Map<TableName, BackupStatus> backupStatusMap) 
{
+    this.backupStatusMap = backupStatusMap;
+  }
+
+  public HashMap<TableName, HashMap<String, Long>> getTableSetTimestampMap() {
+    return tableSetTimestampMap;
+  }
+  
+  public void setTableSetTimestampMap(HashMap<TableName, HashMap<String, 
Long>> tableSetTimestampMap) {
+    this.tableSetTimestampMap = tableSetTimestampMap;
+  }
+
+  public String getHlogTargetDir() {
+    return hlogTargetDir;
+  }
+
+  public void setType(BackupType type) {
+    this.type = type;
+  }
+
+  public void setTargetRootDir(String targetRootDir) {
+    this.targetRootDir = targetRootDir;
+  }
+
+  public void setTotalBytesCopied(long totalBytesCopied) {
+    this.totalBytesCopied = totalBytesCopied;
+  }
+
+  public void setCancelled(boolean cancelled) {
+    this.state = BackupState.CANCELLED;;
+  }
+  
+  /**
+   * Set progress (0-100%)
+   * @param msg progress value
+   */
+
+  public void setProgress(int p) {
+    this.progress = p;
+  }
+
+  /**
+   * Get current progress
+   */
+  public int getProgress() {
+    return progress;
+  }
+
+
+  /**
+   * Has been marked as cancelled or not.
+   * @return True if marked as cancelled
+   */
+  public boolean isCancelled() {
+    return this.state == BackupState.CANCELLED;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  public void setBackupId(String backupId) {
+    this.backupId = backupId;
+  }
+
+  public BackupStatus getBackupStatus(TableName table) {
+    return this.backupStatusMap.get(table);
+  }
+
+  public String getFailedMsg() {
+    return failedMsg;
+  }
+
+  public void setFailedMsg(String failedMsg) {
+    this.failedMsg = failedMsg;
+  }
+
+  public long getStartTs() {
+    return startTs;
+  }
+
+  public void setStartTs(long startTs) {
+    this.startTs = startTs;
+  }
+
+  public long getEndTs() {
+    return endTs;
+  }
+
+  public void setEndTs(long endTs) {
+    this.endTs = endTs;
+  }
+
+  public long getTotalBytesCopied() {
+    return totalBytesCopied;
+  }
+
+  public BackupState getState() {
+    return state;
+  }
+
+  public void setState(BackupState flag) {
+    this.state = flag;
+  }
+
+  public BackupPhase getPhase() {
+    return phase;
+  }
+
+  public void setPhase(BackupPhase phase) {
+    this.phase = phase;
+  }
+
+  public BackupType getType() {
+    return type;
+  }
+
+  public void setSnapshotName(TableName table, String snapshotName) {
+    this.backupStatusMap.get(table).setSnapshotName(snapshotName);
+  }
+
+  public String getSnapshotName(TableName table) {
+    return this.backupStatusMap.get(table).getSnapshotName();
+  }
+
+  public List<String> getSnapshotNames() {
+    List<String> snapshotNames = new ArrayList<String>();
+    for (BackupStatus backupStatus : this.backupStatusMap.values()) {
+      snapshotNames.add(backupStatus.getSnapshotName());
+    }
+    return snapshotNames;
+  }
+
+  public Set<TableName> getTables() {
+    return this.backupStatusMap.keySet();
+  }
+
+  public List<TableName> getTableNames() {
+    return new ArrayList<TableName>(backupStatusMap.keySet());
+  }
+
+  public void addTables(TableName[] tables) {
+    for (TableName table : tables) {
+      BackupStatus backupStatus = new BackupStatus(table, this.targetRootDir, 
this.backupId);
+      this.backupStatusMap.put(table, backupStatus);
+    }
+  }
+
+  public String getTargetRootDir() {
+    return targetRootDir;
+  }
+
+  public void setHlogTargetDir(String hlogTagetDir) {
+    this.hlogTargetDir = hlogTagetDir;
+  }
+
+  public String getHLogTargetDir() {
+    return hlogTargetDir;
+  }
+
+  public List<String> getIncrBackupFileList() {
+    return incrBackupFileList;
+  }
+
+  public void setIncrBackupFileList(List<String> incrBackupFileList) {
+    this.incrBackupFileList = incrBackupFileList;
+  }
+
+  /**
+   * Set the new region server log timestamps after distributed log roll
+   * @param newTableSetTimestampMap table timestamp map
+   */
+  public void setIncrTimestampMap(HashMap<TableName,
+      HashMap<String, Long>> newTableSetTimestampMap) {
+    this.tableSetTimestampMap = newTableSetTimestampMap;
+  }
+
+  /**
+   * Get new region server log timestamps after distributed log roll
+   * @return new region server log timestamps
+   */
+  public HashMap<TableName, HashMap<String, Long>> getIncrTimestampMap() {
+    return this.tableSetTimestampMap;
+  }
+
+  public TableName getTableBySnapshot(String snapshotName) {
+    for (Entry<TableName, BackupStatus> entry : 
this.backupStatusMap.entrySet()) {
+      if (snapshotName.equals(entry.getValue().getSnapshotName())) {
+        return entry.getKey();
+      }
+    }
+    return null;
+  }
+
+  public BackupProtos.BackupInfo toProtosBackupInfo() {
+    BackupProtos.BackupInfo.Builder builder = 
BackupProtos.BackupInfo.newBuilder();
+    builder.setBackupId(getBackupId());
+    setBackupStatusMap(builder);
+    builder.setEndTs(getEndTs());
+    if (getFailedMsg() != null) {
+      builder.setFailedMessage(getFailedMsg());
+    }
+    if (getState() != null) {
+      
builder.setState(BackupProtos.BackupInfo.BackupState.valueOf(getState().name()));
+    }
+    if (getPhase() != null) {
+      
builder.setPhase(BackupProtos.BackupInfo.BackupPhase.valueOf(getPhase().name()));
+    }
+
+    builder.setProgress(getProgress());
+    builder.setStartTs(getStartTs());
+    builder.setTargetRootDir(getTargetRootDir());
+    builder.setType(BackupProtos.BackupType.valueOf(getType().name()));
+    builder.setWorkersNumber(workers);
+    builder.setBandwidth(bandwidth);
+    if (jobId != null) {
+      builder.setJobId(jobId);
+    }
+    return builder.build();
+  }
+
+  public byte[] toByteArray() throws IOException {
+    return toProtosBackupInfo().toByteArray();
+  }
+
+  private void setBackupStatusMap(Builder builder) {
+    for (Entry<TableName, BackupStatus> entry: backupStatusMap.entrySet()) {
+      builder.addTableBackupStatus(entry.getValue().toProto());
+    }
+  }
+
+  public static BackupInfo fromByteArray(byte[] data) throws IOException {
+    return fromProto(BackupProtos.BackupInfo.parseFrom(data));
+  }
+  
+  public static BackupInfo fromStream(final InputStream stream) throws 
IOException {
+    return fromProto(BackupProtos.BackupInfo.parseDelimitedFrom(stream));
+  }
+
+  public static BackupInfo fromProto(BackupProtos.BackupInfo proto) {
+    BackupInfo context = new BackupInfo();
+    context.setBackupId(proto.getBackupId());
+    context.setBackupStatusMap(toMap(proto.getTableBackupStatusList()));
+    context.setEndTs(proto.getEndTs());
+    if (proto.hasFailedMessage()) {
+      context.setFailedMsg(proto.getFailedMessage());
+    }
+    if (proto.hasState()) {
+      
context.setState(BackupInfo.BackupState.valueOf(proto.getState().name()));
+    }
+
+    
context.setHlogTargetDir(HBackupFileSystem.getLogBackupDir(proto.getTargetRootDir(),
+      proto.getBackupId()));
+
+    if (proto.hasPhase()) {
+      context.setPhase(BackupPhase.valueOf(proto.getPhase().name()));
+    }
+    if (proto.hasProgress()) {
+      context.setProgress(proto.getProgress());
+    }
+    context.setStartTs(proto.getStartTs());
+    context.setTargetRootDir(proto.getTargetRootDir());
+    context.setType(BackupType.valueOf(proto.getType().name()));
+    context.setWorkers(proto.getWorkersNumber());
+    context.setBandwidth(proto.getBandwidth());
+    if (proto.hasJobId()) {
+      context.setJobId(proto.getJobId());
+    }
+    return context;
+  }
+
+  private static Map<TableName, BackupStatus> toMap(List<TableBackupStatus> 
list) {
+    HashMap<TableName, BackupStatus> map = new HashMap<>();
+    for (TableBackupStatus tbs : list){
+      map.put(ProtobufUtil.toTableName(tbs.getTable()), 
BackupStatus.convert(tbs));
+    }
+    return map;
+  }
+
+  public String getShortDescription() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("ID             : " + backupId).append("\n");
+    sb.append("Tables         : " + getTableListAsString()).append("\n");
+    sb.append("State          : " + getState()).append("\n");
+    Date date = null;
+    Calendar cal = Calendar.getInstance();
+    cal.setTimeInMillis(getStartTs());
+    date = cal.getTime();
+    sb.append("Start time     : " + date).append("\n");
+    if (state == BackupState.FAILED) {
+      sb.append("Failed message : " + getFailedMsg()).append("\n");
+    } else if (state == BackupState.RUNNING) {
+      sb.append("Phase          : " + getPhase()).append("\n");
+    } else if (state == BackupState.COMPLETE) {
+      cal = Calendar.getInstance();
+      cal.setTimeInMillis(getEndTs());
+      date = cal.getTime();
+      sb.append("End time       : " + date).append("\n");
+    }
+    sb.append("Progress       : " + getProgress()).append("\n");
+    return sb.toString();
+  }
+
+  public String getStatusAndProgressAsString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("id: ").append(getBackupId()).append(" state: 
").append(getState())
+        .append(" progress: ").append(getProgress());
+    return sb.toString();
+  }
+  
+  public String getTableListAsString() {
+    return StringUtils.join(backupStatusMap.keySet(), ";");
+  }
+
+  @Override
+  public int compareTo(BackupInfo o) {
+      Long thisTS =
+          new 
Long(this.getBackupId().substring(this.getBackupId().lastIndexOf("_") + 1));
+      Long otherTS =
+          new Long(o.getBackupId().substring(o.getBackupId().lastIndexOf("_") 
+ 1));
+      return thisTS.compareTo(otherTS);
+  }
+   
+  
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index e0c6483..30882bd 100644
--- 
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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.backup;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.backup.impl.BackupClientImpl;
 import org.apache.hadoop.hbase.backup.impl.BackupCopyService;
 import org.apache.hadoop.hbase.backup.impl.IncrementalRestoreService;
 import org.apache.hadoop.hbase.backup.impl.RestoreClientImpl;
@@ -33,6 +34,7 @@ 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";
+  public final static String HBASE_BACKUP_CLIENT_IMPL_CLASS = 
"hbase.backup.client.class";
   public final static String HBASE_RESTORE_CLIENT_IMPL_CLASS = 
"hbase.restore.client.class";
 
   private BackupRestoreFactory(){
@@ -48,7 +50,9 @@ public final class BackupRestoreFactory {
     Class<? extends IncrementalRestoreService> cls =
         conf.getClass(HBASE_INCR_RESTORE_IMPL_CLASS, 
MapReduceRestoreService.class,
           IncrementalRestoreService.class);
-    return ReflectionUtils.newInstance(cls, conf);
+    IncrementalRestoreService service =  ReflectionUtils.newInstance(cls, 
conf);
+    service.setConf(conf);
+    return service;
   }
   
   /**
@@ -60,7 +64,22 @@ public final class BackupRestoreFactory {
     Class<? extends BackupCopyService> cls =
         conf.getClass(HBASE_BACKUP_COPY_IMPL_CLASS, 
MapReduceBackupCopyService.class,
           BackupCopyService.class);
-    return ReflectionUtils.newInstance(cls, conf);
+    BackupCopyService service = ReflectionUtils.newInstance(cls, conf);;
+    service.setConf(conf);
+    return service;
+  }
+  /**
+   * Gets backup client implementation
+   * @param conf - configuration
+   * @return backup client
+   */
+  public static BackupClient getBackupClient(Configuration conf) {
+    Class<? extends BackupClient> cls =
+        conf.getClass(HBASE_BACKUP_CLIENT_IMPL_CLASS, BackupClientImpl.class,
+          BackupClient.class);
+    BackupClient client = ReflectionUtils.newInstance(cls, conf);
+    client.setConf(conf);
+    return client;
   }
   
   /**

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index 4e88125..e9c607b 100644
--- 
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
@@ -51,7 +51,7 @@ public class 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/backup1/default/t1_dn/backup_1396650096738"
+   * 
"hdfs://backup.hbase.org:9000/user/biadmin/backup1/backup_1396650096738/default/t1_dn/"
    * @param backupRootDir backup root directory
    * @param backupId  backup id
    * @param table table name
@@ -59,22 +59,22 @@ public class HBackupFileSystem {
    */
   public static String getTableBackupDir(String backupRootDir, String backupId,
       TableName tableName) {
-    return backupRootDir + Path.SEPARATOR + tableName.getNamespaceAsString() + 
Path.SEPARATOR
-        + tableName.getQualifierAsString() + Path.SEPARATOR + backupId;
+    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/backup1/default/t1_dn/backup_1396650096738"
+   * 
"hdfs://backup.hbase.org:9000/user/biadmin/backup_1396650096738/backup1/default/t1_dn/"
    * @param backupRootPath backup root path
    * @param tableName table name
    * @param backupId backup Id
    * @return backupPath for the particular table
    */
-  public static Path getTableBackupPath(Path backupRootPath, TableName 
tableName, String backupId) {
-    return new Path(backupRootPath, tableName.getNamespaceAsString() + 
Path.SEPARATOR
-      + tableName.getQualifierAsString() + Path.SEPARATOR + backupId);
+  public static Path getTableBackupPath(TableName tableName, Path 
backupRootPath, String backupId) {
+    return new Path(getTableBackupDir(backupRootPath.toString(), backupId, 
tableName));
   }
 
   /**
@@ -85,8 +85,8 @@ public class HBackupFileSystem {
    * @return logBackupDir: ".../user/biadmin/backup1/WALs/backup_1396650096738"
    */
   public static String getLogBackupDir(String backupRootDir, String backupId) {
-    return backupRootDir + Path.SEPARATOR + HConstants.HREGION_LOGDIR_NAME + 
Path.SEPARATOR
-        + backupId;
+    return backupRootDir + Path.SEPARATOR + backupId+ Path.SEPARATOR
+        + HConstants.HREGION_LOGDIR_NAME;
   }
 
   public static Path getLogBackupPath(String backupRootDir, String backupId) {
@@ -95,7 +95,7 @@ public class HBackupFileSystem {
 
   private static Path getManifestPath(TableName tableName, Configuration conf,
       Path backupRootPath, String backupId) throws IOException {
-    Path manifestPath = new Path(getTableBackupPath(backupRootPath, tableName, 
backupId),
+    Path manifestPath = new Path(getTableBackupPath(tableName, backupRootPath, 
backupId),
       BackupManifest.MANIFEST_FILE_NAME);
     FileSystem fs = backupRootPath.getFileSystem(conf);
     if (!fs.exists(manifestPath)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
index 541882a..6739b5c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
@@ -20,13 +20,9 @@ package org.apache.hadoop.hbase.backup;
 import java.io.IOException;
 
 import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.cli.PosixParser;
 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.BackupUtil;
@@ -38,8 +34,7 @@ import org.apache.log4j.Logger;
 
 public class RestoreDriver extends AbstractHBaseTool {
 
-  private static final Log LOG = LogFactory.getLog(BackupDriver.class);
-  private Options opt;
+  private static final Log LOG = LogFactory.getLog(RestoreDriver.class);
   private CommandLine cmd;
 
   private static final String OPTION_OVERWRITE = "overwrite";
@@ -68,27 +63,25 @@ public class RestoreDriver extends AbstractHBaseTool {
           + "                   or using \"hbase backup describe\" command. 
Without this option, "
           + "only\n" + "                   this backup image is restored\n";
 
+    
+  protected RestoreDriver() throws IOException
+  {
+    init();
+  }
+  
   protected void init() throws IOException {
     // define supported options
-    opt = new Options();
-    opt.addOption(OPTION_OVERWRITE, false,
+    addOptNoArg(OPTION_OVERWRITE,
         "Overwrite the data if any of the restore target tables exists");
-    opt.addOption(OPTION_CHECK, false, "Check restore sequence and 
dependencies");
-    opt.addOption(OPTION_AUTOMATIC, false, "Restore all dependencies");
-    opt.addOption("debug", false, "Enable debug logging");
+    addOptNoArg(OPTION_CHECK, "Check restore sequence and dependencies");
+    addOptNoArg(OPTION_AUTOMATIC, "Restore all dependencies");
+    addOptNoArg("debug",  "Enable debug logging");
 
     // disable irrelevant loggers to avoid it mess up command output
     LogUtils.disableUselessLoggers(LOG);
   }
 
   private int parseAndRun(String[] args) {
-    CommandLine cmd = null;
-    try {
-      cmd = new PosixParser().parse(opt, args);
-    } catch (ParseException e) {
-      LOG.error("Could not parse command", e);
-      return -1;
-    }
 
     // enable debug logging
     Logger backupClientLogger = 
Logger.getLogger("org.apache.hadoop.hbase.backup");
@@ -119,34 +112,34 @@ public class RestoreDriver extends AbstractHBaseTool {
 
     // parse main restore command options
     String[] remainArgs = cmd.getArgs();
-    if (remainArgs.length < 3) {
-      System.out.println("ERROR: missing arguments");
+    if (remainArgs.length < 4) {
       System.out.println(USAGE);
       return -1;
     }
 
-    String backupRootDir = remainArgs[0];
-    String backupId = remainArgs[1];
-    String tables = remainArgs[2];
-
-    String tableMapping = (remainArgs.length > 3) ? remainArgs[3] : null;
+    String backupRootDir = remainArgs[1];
+    String backupId = remainArgs[2];
+    String tables = remainArgs[3];
+    
+    String tableMapping = (remainArgs.length > 4) ? remainArgs[4] : null;
 
     TableName[] sTableArray = BackupUtil.parseTableNames(tables);
     TableName[] tTableArray = BackupUtil.parseTableNames(tableMapping);
 
     if (sTableArray != null && tTableArray != null && (sTableArray.length != 
tTableArray.length)) {
-      System.err.println("ERROR: table mapping mismatch: " + tables + " : " + 
tableMapping);
+      System.out.println("ERROR: table mapping mismatch: " + tables + " : " + 
tableMapping);
       System.out.println(USAGE);
-      return -1;
+      return -2;
     }
 
-    try {
-      RestoreClient client = BackupRestoreFactory.getRestoreClient(conf);
+    
+    RestoreClient client = BackupRestoreFactory.getRestoreClient(getConf());
+    try{
       client.restore(backupRootDir, backupId, check, autoRestore, sTableArray,
         tTableArray, isOverwrite);
-    } catch (IOException e) {
-      System.err.println("ERROR: " + e.getMessage());
-      return -1;
+    } catch (Exception e){
+      e.printStackTrace();
+      return -3;
     }
     return 0;
   }
@@ -162,13 +155,12 @@ public class RestoreDriver extends AbstractHBaseTool {
 
   @Override
   protected int doWork() throws Exception {
-    init();
     return parseAndRun(cmd.getArgs());
   }
 
   public static void main(String[] args) throws Exception {
     Configuration conf = HBaseConfiguration.create();
-    int ret = ToolRunner.run(conf, new BackupDriver(), args);
+    int ret = ToolRunner.run(conf, new RestoreDriver(), args);
     System.exit(ret);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
new file mode 100644
index 0000000..7b2db3d
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupClientImpl.java
@@ -0,0 +1,231 @@
+/**
+ * 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.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupClient;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+
+/**
+ * Backup HBase tables locally or on a remote cluster Serve as client entry 
point for the following
+ * features: - Full Backup provide local and remote back/restore for a list of 
tables - Incremental
+ * backup to build on top of full backup as daily/weekly backup - Convert 
incremental backup WAL
+ * files into hfiles - Merge several backup images into one(like merge weekly 
into monthly) - Add
+ * and remove table to and from Backup image - Cancel a backup process - Full 
backup based on
+ * existing snapshot - Describe information of a backup image
+ */
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class BackupClientImpl implements BackupClient{
+  private static final Log LOG = LogFactory.getLog(BackupClientImpl.class);
+  private Configuration conf;
+
+  public BackupClientImpl() {
+  }
+   
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+  
+
+  @Override
+  public BackupInfo getBackupInfo(String backupId) throws IOException {
+    BackupInfo backupInfo = null;
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        final BackupSystemTable table = new BackupSystemTable(conn)) {
+      backupInfo = table.readBackupInfo(backupId);
+      return backupInfo;
+    } 
+  }
+
+  @Override  
+  public int getProgress(String backupId) throws IOException {
+    BackupInfo backupInfo = null;
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        final BackupSystemTable table = new BackupSystemTable(conn)) {
+      if (backupId == null) {
+        ArrayList<BackupInfo> recentSessions =
+            table.getBackupContexts(BackupState.RUNNING);
+        if (recentSessions.isEmpty()) {
+          LOG.warn("No ongonig sessions found.");
+          return -1;
+        }
+        // else show status for all ongoing sessions
+        // must be one maximum
+        return recentSessions.get(0).getProgress();
+      } else {
+
+        backupInfo = table.readBackupInfo(backupId);
+        if (backupInfo != null) {
+          return backupInfo.getProgress();
+        } else {
+          LOG.warn("No information found for backupID=" + backupId);
+          return -1;
+        }
+      }
+    }
+  }
+  
+  @Override
+  public int deleteBackups(String[] backupIds) throws IOException {
+    BackupInfo backupInfo = null;
+    String backupId = null;
+    int totalDeleted = 0;
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        final BackupSystemTable table = new BackupSystemTable(conn)) {
+      for (int i = 0; i < backupIds.length; i++) {
+        backupId = backupIds[i];
+        backupInfo = table.readBackupInfo(backupId);
+        if (backupInfo != null) {
+          BackupUtil.cleanupBackupData(backupInfo, conf);
+          table.deleteBackupInfo(backupInfo.getBackupId());
+          System.out.println("Delete backup for backupID=" + backupId + " 
completed.");
+          totalDeleted++;
+        } else {
+          System.out.println("Delete backup failed: no information found for 
backupID=" + backupId);
+        }
+      }
+    }
+    return totalDeleted;
+  }
+
+//TODO: Cancel backup?  
+  
+//  @Override
+//  public void cancelBackup(String backupId) throws IOException {
+//    // Kill distributed job if active
+//    // Backup MUST not be in COMPLETE state
+//    try (final BackupSystemTable table = new BackupSystemTable(conf)) {
+//      BackupContext backupContext = table.readBackupStatus(backupId);
+//      String errMessage = null;
+//      if (backupContext != null && backupContext.getState() != 
BackupState.COMPLETE) {
+//        BackupUtil.cleanupBackupData(backupContext, conf);
+//        table.deleteBackupStatus(backupContext.getBackupId());
+//        byte[] jobId = backupContext.getJobId();
+//        if(jobId != null) {
+//          BackupCopyService service = 
BackupRestoreFactory.getBackupCopyService(conf);
+//          service.cancelCopyJob(jobId);
+//        } else{
+//          errMessage = "Distributed Job ID is null for backup "+backupId +
+//              " in "+ backupContext.getState() + " state.";
+//        }
+//      } else if( backupContext == null){  
+//        errMessage = "No information found for backupID=" + backupId;
+//      } else {
+//        errMessage = "Can not cancel "+ backupId + " in " + 
backupContext.getState()+" state";
+//      }
+//      
+//      if( errMessage != null) {
+//        throw new IOException(errMessage);
+//      }
+//    }
+//    // then clean backup image
+//    deleteBackups(new String[] { backupId });
+//  }
+  
+  @Override
+  public List<BackupInfo> getHistory(int n) throws IOException {
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        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<BackupSet> listBackupSets() throws IOException{
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        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);
+        bslist.add( new BackupSet(s, tables));
+      }
+      return bslist;
+    }
+  }
+  
+  
+  @Override
+  public BackupSet getBackupSet(String name) throws IOException{
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        final BackupSystemTable table = new BackupSystemTable(conn)) {
+      List<TableName> list = table.describeBackupSet(name);
+      return new BackupSet(name, list);
+    }
+  }
+
+  @Override
+  public boolean deleteBackupSet(String name) throws IOException {
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        final BackupSystemTable table = new BackupSystemTable(conn)) {
+      if(table.describeBackupSet(name) == null) {
+        return false;
+      }
+      table.deleteBackupSet(name);
+      return true;
+    }
+  }
+
+  @Override
+  public void addToBackupSet(String name, String[] tablesOrNamespaces) throws 
IOException {
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        final BackupSystemTable table = new BackupSystemTable(conn)) {
+      table.addToBackupSet(name, tablesOrNamespaces);
+      System.out.println("Added tables to '" + name + "'");
+    }
+  }
+  
+  @Override
+  public void removeFromBackupSet(String name, String[] tablesOrNamepsaces) 
throws IOException {
+    try (final Connection conn = ConnectionFactory.createConnection(conf);
+        final BackupSystemTable table = new BackupSystemTable(conn)) {
+      table.removeFromBackupSet(name, tablesOrNamepsaces);
+      System.out.println("Removed tables from '" + name + "'");
+    } 
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index 1789cdf..bd03605 100644
--- 
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
@@ -19,12 +19,21 @@
 package org.apache.hadoop.hbase.backup.impl;
 
 import java.io.IOException;
+import java.util.List;
+
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupClient;
+import org.apache.hadoop.hbase.backup.BackupInfo;
+import org.apache.hadoop.hbase.backup.BackupRestoreFactory;
 import org.apache.hadoop.hbase.backup.BackupRequest;
 import org.apache.hadoop.hbase.backup.BackupType;
 import 
org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants.BackupCommand;
+import org.apache.hadoop.hbase.backup.util.BackupSet;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Admin;
@@ -41,18 +50,54 @@ import com.google.common.collect.Lists;
 public final class BackupCommands {
 
   private static final String USAGE = "Usage: hbase backup COMMAND\n"
-      + "where COMMAND is one of:\n" + "  create     create a new backup 
image\n"
+      + "where COMMAND is one of:\n" 
+      + "  create     create a new backup image\n"
+      + "  cancel     cancel an ongoing backup\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"
       + "Enter \'help COMMAND\' to see help message for each command\n";
 
   private static final String CREATE_CMD_USAGE =
-      "Usage: hbase backup create <type> <backup_root_path> [tables] 
[-convert] "
-          + "\n" + " type          \"full\" to create a full backup image;\n"
+      "Usage: hbase backup create <type> <backup_root_path> [tables] [-s name] 
[-convert] "
+          + "[-silent] [-w workers][-b bandwith]\n" + " type          \"full\" 
to create a full backup image;\n"
           + "               \"incremental\" to create an incremental backup 
image\n"
-          + " backup_root_path   The full root path to store the backup 
image,\n"
-          + "                    the prefix can be hdfs, webhdfs, gpfs, etc\n" 
+ " Options:\n"
-          + "   tables      If no tables (\"\") are specified, all tables are 
backed up. "
+          + "  backup_root_path   The full root path to store the backup 
image,\n"
+          + "                    the prefix can be hdfs, webhdfs or gpfs\n" + 
" Options:\n"
+          + "  tables      If no tables (\"\") are specified, all tables are 
backed up. "
           + "Otherwise it is a\n" + "               comma separated list of 
tables.\n"
-          + "   -convert    For an incremental backup, convert WAL files to 
HFiles\n";
+          + " -s name     Use the specified snapshot for full backup\n"
+          + " -convert    For an incremental backup, convert WAL files to 
HFiles\n"
+          + " -w          number of parallel workers.\n" 
+          + " -b          bandwith per one worker (in MB sec)" ;
+
+  private static final String PROGRESS_CMD_USAGE = "Usage: hbase backup 
progress <backupId>\n"
+      + " backupId      backup image id;\n";
+
+  private static final String DESCRIBE_CMD_USAGE = "Usage: hbase backup 
decsribe <backupId>\n"
+      + " backupId      backup image id\n";
+
+  private static final String HISTORY_CMD_USAGE = "Usage: hbase backup history 
[-n N]\n"
+      + " -n N     show up to N last backup sessions, default - 10;\n";
+
+  private static final String DELETE_CMD_USAGE = "Usage: hbase backup delete 
<backupId>\n"
+      + " backupId      backup image id;\n";
+
+  private static final String CANCEL_CMD_USAGE = "Usage: hbase backup progress 
<backupId>\n"
+      + " backupId      backup image id;\n";
+
+  private static final String SET_CMD_USAGE = "Usage: hbase set COMMAND [name] 
[tables]\n"
+      + " name       Backup set name\n"
+      + " tables      If no tables (\"\") are specified, all tables will 
belong to the set. "
+      + "Otherwise it is a\n" + "               comma separated list of 
tables.\n"
+      + "where COMMAND is one of:\n" 
+      + "  add      add tables to a set, crete set if needed\n"
+      + "  remove   remove tables from set\n"
+      + "  list     list all sets\n"
+      + "  describe describes set\n"
+      + "  delete   delete backup set\n";
 
   public static abstract class Command extends Configured {
     Command(Configuration conf) {
@@ -68,25 +113,44 @@ public final class BackupCommands {
   public static Command createCommand(Configuration conf, BackupCommand type, 
CommandLine cmdline) {
     Command cmd = null;
     switch (type) {
-      case CREATE:
-        cmd = new CreateCommand(conf, cmdline);
-        break;
-      case HELP:
-      default:
-        cmd = new HelpCommand(conf, cmdline);
-        break;
+    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;
   }
 
-  private static class CreateCommand extends Command {
+
+  public static class CreateCommand extends Command {
     CommandLine cmdline;
 
     CreateCommand(Configuration conf, CommandLine cmdline) {
       super(conf);
       this.cmdline = cmdline;
     }
-
+    
     @Override
     public void execute() throws IOException {
       if (cmdline == null || cmdline.getArgs() == null) {
@@ -95,31 +159,53 @@ public final class BackupCommands {
         System.exit(-1);
       }
       String[] args = cmdline.getArgs();
-      if (args.length < 2 || args.length > 3) {
+      if (args.length < 3 || args.length > 4) {
         System.out.println("ERROR: wrong number of arguments");
         System.out.println(CREATE_CMD_USAGE);
         System.exit(-1);
       }
 
-      if (!BackupType.FULL.toString().equalsIgnoreCase(args[0])
-          && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[0])) {
+      if (!BackupType.FULL.toString().equalsIgnoreCase(args[1])
+          && !BackupType.INCREMENTAL.toString().equalsIgnoreCase(args[1])) {
         System.out.println("ERROR: invalid backup type");
         System.out.println(CREATE_CMD_USAGE);
         System.exit(-1);
       }
 
-      String tables = (args.length == 3) ? args[2] : null;
+      String tables = null;
+      Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+
+      // Check backup set
+      if (cmdline.hasOption("set")) {
+        String setName = cmdline.getOptionValue("set");
+        tables = getTablesForSet(setName, conf);
+
+        if (tables == null) throw new IOException("Backup set '" + setName
+          + "' is either empty or does not exist");
+      } else {
+        tables = (args.length == 4) ? args[3] : null;
+      }
+      int bandwidth = cmdline.hasOption('b') ? 
Integer.parseInt(cmdline.getOptionValue('b')) : -1;
+      int workers = cmdline.hasOption('w') ? 
Integer.parseInt(cmdline.getOptionValue('w')) : -1;
 
       try (Connection conn = ConnectionFactory.createConnection(getConf());
           Admin admin = conn.getAdmin();) {
         BackupRequest request = new BackupRequest();
-        request.setBackupType(BackupType.valueOf(args[0].toUpperCase()))
-        .setTableList(Lists.newArrayList(BackupUtil.parseTableNames(tables)))
-        .setTargetRootDir(args[1]);
+        request.setBackupType(BackupType.valueOf(args[1].toUpperCase()))
+        .setTableList(tables != 
null?Lists.newArrayList(BackupUtil.parseTableNames(tables)): null)
+        .setTargetRootDir(args[2]).setWorkers(workers).setBandwidth(bandwidth);
         admin.backupTables(request);
       } catch (IOException e) {
-        System.err.println("ERROR: " + e.getMessage());
-        System.exit(-1);
+        throw e;
+      }
+    }
+    private String getTablesForSet(String name, Configuration conf)
+        throws IOException {
+      try (final Connection conn = ConnectionFactory.createConnection(conf);
+          final BackupSystemTable table = new BackupSystemTable(conn)) {
+        List<TableName> tables = table.describeBackupSet(name);
+        if (tables == null) return null;
+        return StringUtils.join(tables, 
BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);        
       }
     }
   }
@@ -155,9 +241,321 @@ public final class BackupCommands {
 
       if (BackupCommand.CREATE.name().equalsIgnoreCase(type)) {
         System.out.println(CREATE_CMD_USAGE);
-      } // other commands will be supported in future jira
+      } 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);
+      }
+      if (BackupCommand.CANCEL.name().equalsIgnoreCase(type)) {
+        System.out.println(CANCEL_CMD_USAGE);
+      }
+      if (BackupCommand.SET.name().equalsIgnoreCase(type)) {
+        System.out.println(SET_CMD_USAGE);
+      } else {
+        System.out.println("Unknown command : " + type);
+        System.out.println(USAGE);
+      }
       System.exit(0);
     }
   }
 
+  private static class DescribeCommand extends Command {
+    CommandLine cmdline;
+
+    DescribeCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null || cmdline.getArgs() == null) {
+        System.out.println("ERROR: missing arguments");
+        System.out.println(DESCRIBE_CMD_USAGE);
+        System.exit(-1);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length != 2) {
+        System.out.println("ERROR: wrong number of arguments");
+        System.out.println(DESCRIBE_CMD_USAGE);
+        System.exit(-1);
+      }
+
+      String backupId = args[1];
+      try {
+        Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+        BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+        BackupInfo info = client.getBackupInfo(backupId);
+        System.out.println(info.getShortDescription());
+      } catch (RuntimeException e) {
+        System.out.println("ERROR: " + e.getMessage());
+        System.exit(-1);
+      }
+    }
+  }
+
+  private static class ProgressCommand extends Command {
+    CommandLine cmdline;
+
+    ProgressCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null || cmdline.getArgs() == null ||
+          cmdline.getArgs().length != 2) {
+        System.out.println("No backup id was specified, "
+            + "will retrieve the most recent (ongoing) sessions");
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length > 2) {
+        System.out.println("ERROR: wrong number of arguments: " + args.length);
+        System.out.println(PROGRESS_CMD_USAGE);
+        System.exit(-1);
+      }
+
+      String backupId = args == null ? null : args[1];
+      try {
+        Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+        BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+        int progress = client.getProgress(backupId);
+        if(progress < 0){
+          System.out.println("No info was found for backup id: "+backupId);
+        } else{
+          System.out.println(backupId+" progress=" + progress+"%");
+        }
+      } catch (RuntimeException e) {
+        System.out.println("ERROR: " + e.getMessage());
+        System.exit(-1);
+      }
+    }
+  }
+
+  private static class DeleteCommand extends Command {
+    
+    CommandLine cmdline;
+    DeleteCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null || cmdline.getArgs() == null || 
+          cmdline.getArgs().length < 2) {
+        System.out.println("No backup id(s) was specified");
+        System.out.println(PROGRESS_CMD_USAGE);
+        System.exit(-1);
+      }
+      String[] args = cmdline.getArgs();
+      
+      String[] backupIds = new String[args.length-1];
+      System.arraycopy(args, 1, backupIds, 0, backupIds.length);
+      try {
+        Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+        BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+        client.deleteBackups(args);
+      } catch (RuntimeException e) {
+        System.out.println("ERROR: " + e.getMessage());
+        System.exit(-1);
+      }
+    }
+  }
+
+// TODO Cancel command  
+  
+  private static class CancelCommand extends Command {
+    CommandLine cmdline;
+
+    CancelCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null || 
+          cmdline.getArgs() == null || cmdline.getArgs().length < 2) {
+        System.out.println("No backup id(s) was specified, will use the most 
recent one");
+      }
+      String[] args = cmdline.getArgs();
+      String backupId = args == null || args.length == 0 ? null : args[1];
+      try {
+        Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+        BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+//TODO
+//        client.cancelBackup(backupId);
+      } catch (RuntimeException e) {
+        System.out.println("ERROR: " + e.getMessage());
+        System.exit(-1);
+      }
+    }
+  }
+
+  private static class HistoryCommand extends Command {
+    CommandLine cmdline;
+    private final static int DEFAULT_HISTORY_LENGTH = 10;
+    
+    HistoryCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+
+      int n = parseHistoryLength();
+      try {
+        Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+        BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+        List<BackupInfo> history = client.getHistory(n);
+        for(BackupInfo info: history){
+          System.out.println(info.getShortDescription());
+        }
+      } catch (RuntimeException e) {
+        System.out.println("ERROR: " + e.getMessage());
+        System.exit(-1);
+      }
+    }
+
+    private int parseHistoryLength() {
+      String value = cmdline.getOptionValue("n");
+      if (value == null) return DEFAULT_HISTORY_LENGTH;
+      return Integer.parseInt(value);
+    }
+  }
+
+  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";
+
+    CommandLine cmdline;
+
+    BackupSetCommand(Configuration conf, CommandLine cmdline) {
+      super(conf);
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+
+      // Command-line must have at least one element
+      if (cmdline == null || cmdline.getArgs() == null || 
cmdline.getArgs().length < 2) {
+        throw new IOException("command line format");
+      }
+      String[] args = cmdline.getArgs();
+      String cmdStr = args[1];
+      BackupCommand cmd = getCommand(cmdStr);
+
+      try {
+
+        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;
+
+        }
+      } catch (RuntimeException e) {
+        System.out.println("ERROR: " + e.getMessage());
+        System.exit(-1);
+      }
+    }
+
+    private void processSetList(String[] args) throws IOException {
+      // List all backup set names
+      // does not expect any args
+      Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+      BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+      client.listBackupSets();
+    }
+
+    private void processSetDescribe(String[] args) throws IOException {
+      if (args == null || args.length != 3) {
+        throw new RuntimeException("Wrong number of args: "+args.length);
+      }
+      String setName = args[2];
+      Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+      BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+      BackupSet set = client.getBackupSet(setName);
+      System.out.println(set);
+    }
+
+    private void processSetDelete(String[] args) throws IOException {
+      if (args == null || args.length != 3) {
+        throw new RuntimeException("Wrong number of args");
+      }
+      String setName = args[2];
+      Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+      BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+      boolean result = client.deleteBackupSet(setName);
+      if(result){
+        System.out.println("Delete set "+setName+" OK.");
+      } else{
+        System.out.println("Set "+setName+" does not exists");
+      }
+    }
+
+    private void processSetRemove(String[] args) throws IOException {
+      if (args == null || args.length != 4) {
+        throw new RuntimeException("Wrong args");
+      }
+      String setName = args[2];
+      String[] tables = args[3].split(",");
+      Configuration conf = getConf() != null? getConf(): 
HBaseConfiguration.create();
+      BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+      client.removeFromBackupSet(setName, tables);
+    }
+
+    private void processSetAdd(String[] args) throws IOException {
+      if (args == null || args.length != 4) {
+        throw new RuntimeException("Wrong args");
+      }
+      String setName = args[2];
+      String[] tables = args[3].split(",");
+      Configuration conf = getConf() != null? 
getConf():HBaseConfiguration.create();
+      BackupClient client = BackupRestoreFactory.getBackupClient(conf);
+      client.addToBackupSet(setName, tables);
+    }
+
+    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 {
+        throw new IOException("Unknown command for 'set' :" + cmdStr);
+      }
+    }
+
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java
index 1e8da63..a738b5d 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupCopyService.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.backup.BackupInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 
@@ -32,6 +33,24 @@ public interface BackupCopyService extends Configurable {
     FULL, INCREMENTAL
   }
 
-  public int copy(BackupContext backupContext, BackupManager backupManager, 
Configuration conf,
+  /**
+   * Copy backup data
+   * @param backupContext - context
+   * @param backupManager  - manager
+   * @param conf - configuration
+   * @param copyType - copy type
+   * @param options - list of options
+   * @return result (0 - success)
+   * @throws IOException
+   */
+  public int copy(BackupInfo backupContext, BackupManager backupManager, 
Configuration conf,
       BackupCopyService.Type copyType, String[] options) throws IOException;
+  
+
+   /**
+    * Cancel copy job
+    * @param jobHandler - copy job handler
+    * @throws IOException
+    */
+   public void cancelCopyJob(String jobHandler) throws IOException;  
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c61f7c34/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
index af70cc8..ca204b4 100644
--- 
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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.backup.impl;
 
 import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.backup.BackupInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 
@@ -29,7 +30,7 @@ import 
org.apache.hadoop.hbase.classification.InterfaceStability;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class BackupException extends HBaseIOException {
-  private BackupContext description;
+  private BackupInfo description;
 
   /**
    * Some exception happened for a backup and don't even know the backup that 
it was about
@@ -52,7 +53,7 @@ public class BackupException extends HBaseIOException {
    * @param msg reason why the backup failed
    * @param desc description of the backup that is being failed
    */
-  public BackupException(String msg, BackupContext desc) {
+  public BackupException(String msg, BackupInfo desc) {
     super(msg);
     this.description = desc;
   }
@@ -63,7 +64,7 @@ public class BackupException extends HBaseIOException {
    * @param cause root cause of the failure
    * @param desc description of the backup that is being failed
    */
-  public BackupException(String msg, Throwable cause, BackupContext desc) {
+  public BackupException(String msg, Throwable cause, BackupInfo desc) {
     super(msg, cause);
     this.description = desc;
   }
@@ -78,7 +79,7 @@ public class BackupException extends HBaseIOException {
     super(message, e);
   }
 
-  public BackupContext getBackupContext() {
+  public BackupInfo getBackupContext() {
     return this.description;
   }
 

Reply via email to