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 2016/01/04 17:56:51 UTC

[06/29] hbase git commit: HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov)

HBASE-14030 HBase Backup/Restore Phase 1 (Vladimir Rodionov)


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

Branch: refs/heads/hbase-12439
Commit: de69f0df34a6cb00a9fd2e370fdf898274871294
Parents: 5eefe13
Author: tedyu <yu...@gmail.com>
Authored: Sun Dec 27 10:02:09 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Sun Dec 27 10:02:09 2015 -0800

----------------------------------------------------------------------
 bin/hbase                                       |   6 +
 .../org/apache/hadoop/hbase/HConstants.java     | 445 +++++-----
 hbase-server/pom.xml                            |  10 +
 .../hadoop/hbase/backup/BackupClient.java       | 338 ++++++++
 .../hadoop/hbase/backup/BackupCommands.java     | 147 ++++
 .../hadoop/hbase/backup/BackupContext.java      | 318 ++++++++
 .../hadoop/hbase/backup/BackupCopyService.java  |  37 +
 .../hadoop/hbase/backup/BackupException.java    |  85 ++
 .../hadoop/hbase/backup/BackupHandler.java      | 744 +++++++++++++++++
 .../hadoop/hbase/backup/BackupManager.java      | 488 +++++++++++
 .../hadoop/hbase/backup/BackupManifest.java     | 814 +++++++++++++++++++
 .../hbase/backup/BackupRestoreConstants.java    |  66 ++
 .../backup/BackupRestoreServiceFactory.java     |  61 ++
 .../hadoop/hbase/backup/BackupStatus.java       |  67 ++
 .../hadoop/hbase/backup/BackupSystemTable.java  | 642 +++++++++++++++
 .../hbase/backup/BackupSystemTableHelper.java   | 314 +++++++
 .../apache/hadoop/hbase/backup/BackupUtil.java  | 564 +++++++++++++
 .../hadoop/hbase/backup/HBackupFileSystem.java  | 511 ++++++++++++
 .../hbase/backup/IncrementalBackupManager.java  | 269 ++++++
 .../hbase/backup/IncrementalRestoreService.java |  33 +
 .../hadoop/hbase/backup/RestoreClient.java      | 496 +++++++++++
 .../apache/hadoop/hbase/backup/RestoreUtil.java | 503 ++++++++++++
 .../mapreduce/MapReduceBackupCopyService.java   | 292 +++++++
 .../mapreduce/MapReduceRestoreService.java      |  72 ++
 .../hbase/backup/master/BackupLogCleaner.java   | 121 +++
 .../master/LogRollMasterProcedureManager.java   | 129 +++
 .../regionserver/LogRollBackupSubprocedure.java | 138 ++++
 .../LogRollBackupSubprocedurePool.java          | 137 ++++
 .../LogRollRegionServerProcedureManager.java    | 168 ++++
 .../BaseCoordinatedStateManager.java            |  17 +
 .../coordination/ZkCoordinatedStateManager.java |  20 +-
 .../hadoop/hbase/mapreduce/WALPlayer.java       |  54 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   2 +
 .../procedure/RegionServerProcedureManager.java |   2 +-
 .../RegionServerProcedureManagerHost.java       |   3 +-
 .../procedure/ZKProcedureCoordinatorRpcs.java   |   2 +-
 .../hbase/procedure/ZKProcedureMemberRpcs.java  |  83 +-
 .../RegionServerFlushTableProcedureManager.java |   2 +-
 .../hbase/regionserver/HRegionServer.java       |   4 +-
 .../snapshot/RegionServerSnapshotManager.java   |   2 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  67 +-
 .../hadoop/hbase/snapshot/SnapshotCopy.java     |  42 +
 .../hadoop/hbase/wal/DefaultWALProvider.java    |   7 +-
 .../hadoop/hbase/backup/TestBackupBase.java     | 194 +++++
 .../hbase/backup/TestBackupBoundaryTests.java   |  99 +++
 .../hbase/backup/TestBackupLogCleaner.java      | 161 ++++
 .../hbase/backup/TestBackupSystemTable.java     | 341 ++++++++
 .../hadoop/hbase/backup/TestFullBackup.java     |  83 ++
 .../hadoop/hbase/backup/TestFullRestore.java    | 166 ++++
 .../hbase/backup/TestIncrementalBackup.java     | 179 ++++
 .../hadoop/hbase/backup/TestRemoteBackup.java   |  44 +
 .../hadoop/hbase/backup/TestRemoteRestore.java  |  54 ++
 .../hbase/backup/TestRestoreBoundaryTests.java  |  85 ++
 .../procedure/SimpleRSProcedureManager.java     |   2 +-
 54 files changed, 9389 insertions(+), 341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index 5064451..9fd903a 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -99,6 +99,8 @@ if [ $# = 0 ]; then
   echo "  pe              Run PerformanceEvaluation"
   echo "  ltt             Run LoadTestTool"
   echo "  version         Print the version"
+  echo "  backup          backup tables for recovery"
+  echo "  restore         restore tables from existing backup image"
   echo "  CLASSNAME       Run the class named CLASSNAME"
   exit 1
 fi
@@ -303,6 +305,10 @@ elif [ "$COMMAND" = "hfile" ] ; then
   CLASS='org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter'
 elif [ "$COMMAND" = "zkcli" ] ; then
   CLASS="org.apache.hadoop.hbase.zookeeper.ZooKeeperMainServer"
+elif [ "$COMMAND" = "backup" ] ; then
+  CLASS='org.apache.hadoop.hbase.backup.BackupClient'
+elif [ "$COMMAND" = "restore" ] ; then
+  CLASS='org.apache.hadoop.hbase.backup.RestoreClient'
 elif [ "$COMMAND" = "upgrade" ] ; then
   echo "This command was used to upgrade to HBase 0.96, it was removed in HBase 2.0.0."
   echo "Please follow the documentation at http://hbase.apache.org/book.html#upgrading."

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 6fafad3..0f572e6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -40,11 +40,11 @@ import org.apache.hadoop.hbase.util.Bytes;
 public final class HConstants {
   // NOTICE!!!! Please do not add a constants here, unless they are referenced by a lot of classes.
 
-  //Bytes.UTF8_ENCODING should be updated if this changed
+  // Bytes.UTF8_ENCODING should be updated if this changed
   /** When we encode strings, we always specify UTF8 encoding */
   public static final String UTF8_ENCODING = "UTF-8";
 
-  //Bytes.UTF8_CHARSET should be updated if this changed
+  // Bytes.UTF8_CHARSET should be updated if this changed
   /** When we encode strings, we always specify UTF8 encoding */
   public static final Charset UTF8_CHARSET = Charset.forName(UTF8_ENCODING);
   /**
@@ -55,9 +55,9 @@ public final class HConstants {
   /** Used as a magic return value while optimized index key feature enabled(HBASE-7845) */
   public final static int INDEX_KEY_MAGIC = -2;
   /*
-     * Name of directory that holds recovered edits written by the wal log
-     * splitting code, one per region
-     */
+   * Name of directory that holds recovered edits written by the wal log splitting code, one per
+   * region
+   */
   public static final String RECOVERED_EDITS_DIR = "recovered.edits";
   /**
    * The first four bytes of Hadoop RPC connections
@@ -70,27 +70,24 @@ public final class HConstants {
   /** The size data structures with minor version is 0 */
   public static final int HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM = MAGIC_LENGTH + 2 * Bytes.SIZEOF_INT
       + Bytes.SIZEOF_LONG;
-  /** The size of a version 2 HFile block header, minor version 1.
-   * There is a 1 byte checksum type, followed by a 4 byte bytesPerChecksum
-   * followed by another 4 byte value to store sizeofDataOnDisk.
+  /**
+   * The size of a version 2 HFile block header, minor version 1. There is a 1 byte checksum type,
+   * followed by a 4 byte bytesPerChecksum followed by another 4 byte value to store
+   * sizeofDataOnDisk.
    */
-  public static final int HFILEBLOCK_HEADER_SIZE = HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM +
-    Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT;
+  public static final int HFILEBLOCK_HEADER_SIZE = HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM
+      + Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT;
   /** Just an array of bytes of the right size. */
   public static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HFILEBLOCK_HEADER_SIZE];
 
-  //End HFileBlockConstants.
+  // End HFileBlockConstants.
 
   /**
    * Status codes used for return values of bulk operations.
    */
   @InterfaceAudience.Private
   public enum OperationStatusCode {
-    NOT_RUN,
-    SUCCESS,
-    BAD_FAMILY,
-    SANITY_CHECK_FAILURE,
-    FAILURE;
+    NOT_RUN, SUCCESS, BAD_FAMILY, SANITY_CHECK_FAILURE, FAILURE;
   }
 
   /** long constant for zero */
@@ -104,19 +101,16 @@ public final class HConstants {
   public static final String VERSION_FILE_NAME = "hbase.version";
 
   /**
-   * Current version of file system.
-   * Version 4 supports only one kind of bloom filter.
-   * Version 5 changes versions in catalog table regions.
-   * Version 6 enables blockcaching on catalog tables.
-   * Version 7 introduces hfile -- hbase 0.19 to 0.20..
-   * Version 8 introduces namespace
+   * Current version of file system. Version 4 supports only one kind of bloom filter. Version 5
+   * changes versions in catalog table regions. Version 6 enables blockcaching on catalog tables.
+   * Version 7 introduces hfile -- hbase 0.19 to 0.20.. Version 8 introduces namespace
    */
   // public static final String FILE_SYSTEM_VERSION = "6";
   public static final String FILE_SYSTEM_VERSION = "8";
 
   // Configuration parameters
 
-  //TODO: Is having HBase homed on port 60k OK?
+  // TODO: Is having HBase homed on port 60k OK?
 
   /** Cluster is in distributed mode or not */
   public static final String CLUSTER_DISTRIBUTED = "hbase.cluster.distributed";
@@ -131,12 +125,10 @@ public final class HConstants {
   public static final String ENSEMBLE_TABLE_NAME = "hbase:ensemble";
 
   /** Config for pluggable region normalizer */
-  public static final String HBASE_MASTER_NORMALIZER_CLASS =
-    "hbase.master.normalizer.class";
+  public static final String HBASE_MASTER_NORMALIZER_CLASS = "hbase.master.normalizer.class";
 
   /** Config for enabling/disabling pluggable region normalizer */
-  public static final String HBASE_NORMALIZER_ENABLED =
-    "hbase.normalizer.enabled";
+  public static final String HBASE_NORMALIZER_ENABLED = "hbase.normalizer.enabled";
 
   /** Cluster is standalone or pseudo-distributed */
   public static final boolean CLUSTER_IS_LOCAL = false;
@@ -174,21 +166,18 @@ public final class HConstants {
   public static final String ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
 
   /** Common prefix of ZooKeeper configuration properties */
-  public static final String ZK_CFG_PROPERTY_PREFIX =
-      "hbase.zookeeper.property.";
+  public static final String ZK_CFG_PROPERTY_PREFIX = "hbase.zookeeper.property.";
 
-  public static final int ZK_CFG_PROPERTY_PREFIX_LEN =
-      ZK_CFG_PROPERTY_PREFIX.length();
+  public static final int ZK_CFG_PROPERTY_PREFIX_LEN = ZK_CFG_PROPERTY_PREFIX.length();
 
   /**
-   * The ZK client port key in the ZK properties map. The name reflects the
-   * fact that this is not an HBase configuration key.
+   * The ZK client port key in the ZK properties map. The name reflects the fact that this is not an
+   * HBase configuration key.
    */
   public static final String CLIENT_PORT_STR = "clientPort";
 
   /** Parameter name for the client port that the zookeeper listens on */
-  public static final String ZOOKEEPER_CLIENT_PORT =
-      ZK_CFG_PROPERTY_PREFIX + CLIENT_PORT_STR;
+  public static final String ZOOKEEPER_CLIENT_PORT = ZK_CFG_PROPERTY_PREFIX + CLIENT_PORT_STR;
 
   /** Default client port that the zookeeper listens on */
   public static final int DEFAULT_ZOOKEPER_CLIENT_PORT = 2181;
@@ -208,19 +197,15 @@ public final class HConstants {
   public static final String DEFAULT_ZOOKEEPER_ZNODE_PARENT = "/hbase";
 
   /**
-   * Parameter name for the limit on concurrent client-side zookeeper
-   * connections
+   * Parameter name for the limit on concurrent client-side zookeeper connections
    */
-  public static final String ZOOKEEPER_MAX_CLIENT_CNXNS =
-      ZK_CFG_PROPERTY_PREFIX + "maxClientCnxns";
+  public static final String ZOOKEEPER_MAX_CLIENT_CNXNS = ZK_CFG_PROPERTY_PREFIX + "maxClientCnxns";
 
   /** Parameter name for the ZK data directory */
-  public static final String ZOOKEEPER_DATA_DIR =
-      ZK_CFG_PROPERTY_PREFIX + "dataDir";
+  public static final String ZOOKEEPER_DATA_DIR = ZK_CFG_PROPERTY_PREFIX + "dataDir";
 
   /** Parameter name for the ZK tick time */
-  public static final String ZOOKEEPER_TICK_TIME =
-      ZK_CFG_PROPERTY_PREFIX + "tickTime";
+  public static final String ZOOKEEPER_TICK_TIME = ZK_CFG_PROPERTY_PREFIX + "tickTime";
 
   /** Default limit on concurrent client-side zookeeper connections */
   public static final int DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS = 300;
@@ -244,21 +229,19 @@ public final class HConstants {
   public static final int DEFAULT_REGIONSERVER_INFOPORT = 16030;
 
   /** A configuration key for regionserver info port */
-  public static final String REGIONSERVER_INFO_PORT =
-    "hbase.regionserver.info.port";
+  public static final String REGIONSERVER_INFO_PORT = "hbase.regionserver.info.port";
 
   /** A flag that enables automatic selection of regionserver info port */
-  public static final String REGIONSERVER_INFO_PORT_AUTO =
-      REGIONSERVER_INFO_PORT + ".auto";
+  public static final String REGIONSERVER_INFO_PORT_AUTO = REGIONSERVER_INFO_PORT + ".auto";
 
   /** Parameter name for what region server implementation to use. */
-  public static final String REGION_SERVER_IMPL= "hbase.regionserver.impl";
+  public static final String REGION_SERVER_IMPL = "hbase.regionserver.impl";
 
   /** Parameter name for what master implementation to use. */
-  public static final String MASTER_IMPL= "hbase.master.impl";
+  public static final String MASTER_IMPL = "hbase.master.impl";
 
   /** Parameter name for what hbase client implementation to use. */
-  public static final String HBASECLIENT_IMPL= "hbase.hbaseclient.impl";
+  public static final String HBASECLIENT_IMPL = "hbase.hbaseclient.impl";
 
   /** Parameter name for how often threads should wake up */
   public static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency";
@@ -293,7 +276,7 @@ public final class HConstants {
 
   /** Parameter name for HBase client operation timeout, which overrides RPC timeout */
   public static final String HBASE_CLIENT_META_OPERATION_TIMEOUT =
-    "hbase.client.meta.operation.timeout";
+      "hbase.client.meta.operation.timeout";
 
   /** Default HBase client operation timeout, which is tantamount to a blocking call */
   public static final int DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT = 1200000;
@@ -316,9 +299,8 @@ public final class HConstants {
   public static final String MIGRATION_NAME = ".migration";
 
   /**
-   * The directory from which co-processor/custom filter jars can be loaded
-   * dynamically by the region servers. This value can be overridden by the
-   * hbase.dynamic.jars.dir config.
+   * The directory from which co-processor/custom filter jars can be loaded dynamically by the
+   * region servers. This value can be overridden by the hbase.dynamic.jars.dir config.
    */
   public static final String LIB_DIR = "lib";
 
@@ -326,8 +308,7 @@ public final class HConstants {
   public static final String HREGION_COMPACTIONDIR_NAME = "compaction.dir";
 
   /** Conf key for the max file size after which we split the region */
-  public static final String HREGION_MAX_FILESIZE =
-      "hbase.hregion.max.filesize";
+  public static final String HREGION_MAX_FILESIZE = "hbase.hregion.max.filesize";
 
   /** Default maximum file size */
   public static final long DEFAULT_MAX_FILE_SIZE = 10 * 1024 * 1024 * 1024L;
@@ -343,25 +324,24 @@ public final class HConstants {
   public static final long TABLE_MAX_ROWSIZE_DEFAULT = 1024 * 1024 * 1024L;
 
   /**
-   * The max number of threads used for opening and closing stores or store
-   * files in parallel
+   * The max number of threads used for opening and closing stores or store files in parallel
    */
   public static final String HSTORE_OPEN_AND_CLOSE_THREADS_MAX =
-    "hbase.hstore.open.and.close.threads.max";
+      "hbase.hstore.open.and.close.threads.max";
 
   /**
-   * The default number for the max number of threads used for opening and
-   * closing stores or store files in parallel
+   * The default number for the max number of threads used for opening and closing stores or store
+   * files in parallel
    */
   public static final int DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX = 1;
 
   /**
-   * Block updates if memstore has hbase.hregion.memstore.block.multiplier
-   * times hbase.hregion.memstore.flush.size bytes.  Useful preventing
-   * runaway memstore during spikes in update traffic.
+   * Block updates if memstore has hbase.hregion.memstore.block.multiplier times
+   * hbase.hregion.memstore.flush.size bytes. Useful preventing runaway memstore during spikes in
+   * update traffic.
    */
   public static final String HREGION_MEMSTORE_BLOCK_MULTIPLIER =
-          "hbase.hregion.memstore.block.multiplier";
+      "hbase.hregion.memstore.block.multiplier";
 
   /**
    * Default value for hbase.hregion.memstore.block.multiplier
@@ -369,14 +349,12 @@ public final class HConstants {
   public static final int DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER = 4;
 
   /** Conf key for the memstore size at which we flush the memstore */
-  public static final String HREGION_MEMSTORE_FLUSH_SIZE =
-      "hbase.hregion.memstore.flush.size";
+  public static final String HREGION_MEMSTORE_FLUSH_SIZE = "hbase.hregion.memstore.flush.size";
 
   public static final String HREGION_EDITS_REPLAY_SKIP_ERRORS =
       "hbase.hregion.edits.replay.skip.errors";
 
-  public static final boolean DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS =
-      false;
+  public static final boolean DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS = false;
 
   /** Maximum value length, enforced on KeyValue construction */
   public static final int MAXIMUM_VALUE_LENGTH = Integer.MAX_VALUE - 1;
@@ -412,12 +390,12 @@ public final class HConstants {
   // be the first to be reassigned if the server(s) they are being served by
   // should go down.
 
-
   /**
    * The hbase:meta table's name.
    * @deprecated For upgrades of 0.94 to 0.96
    */
-  @Deprecated  // for compat from 0.94 -> 0.96.
+  @Deprecated
+  // for compat from 0.94 -> 0.96.
   public static final byte[] META_TABLE_NAME = TableName.META_TABLE_NAME.getName();
 
   public static final String BASE_NAMESPACE_DIR = "data";
@@ -425,52 +403,52 @@ public final class HConstants {
   /** delimiter used between portions of a region name */
   public static final int META_ROW_DELIMITER = ',';
 
-  /** The catalog family as a string*/
+  /** The catalog family as a string */
   public static final String CATALOG_FAMILY_STR = "info";
 
   /** The catalog family */
-  public static final byte [] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR);
+  public static final byte[] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR);
 
   /** The RegionInfo qualifier as a string */
   public static final String REGIONINFO_QUALIFIER_STR = "regioninfo";
 
   /** The regioninfo column qualifier */
-  public static final byte [] REGIONINFO_QUALIFIER = Bytes.toBytes(REGIONINFO_QUALIFIER_STR);
+  public static final byte[] REGIONINFO_QUALIFIER = Bytes.toBytes(REGIONINFO_QUALIFIER_STR);
 
   /** The server column qualifier */
   public static final String SERVER_QUALIFIER_STR = "server";
   /** The server column qualifier */
-  public static final byte [] SERVER_QUALIFIER = Bytes.toBytes(SERVER_QUALIFIER_STR);
+  public static final byte[] SERVER_QUALIFIER = Bytes.toBytes(SERVER_QUALIFIER_STR);
 
   /** The startcode column qualifier */
   public static final String STARTCODE_QUALIFIER_STR = "serverstartcode";
   /** The startcode column qualifier */
-  public static final byte [] STARTCODE_QUALIFIER = Bytes.toBytes(STARTCODE_QUALIFIER_STR);
+  public static final byte[] STARTCODE_QUALIFIER = Bytes.toBytes(STARTCODE_QUALIFIER_STR);
 
   /** The open seqnum column qualifier */
   public static final String SEQNUM_QUALIFIER_STR = "seqnumDuringOpen";
   /** The open seqnum column qualifier */
-  public static final byte [] SEQNUM_QUALIFIER = Bytes.toBytes(SEQNUM_QUALIFIER_STR);
+  public static final byte[] SEQNUM_QUALIFIER = Bytes.toBytes(SEQNUM_QUALIFIER_STR);
 
   /** The state column qualifier */
   public static final String STATE_QUALIFIER_STR = "state";
 
-  public static final byte [] STATE_QUALIFIER = Bytes.toBytes(STATE_QUALIFIER_STR);
+  public static final byte[] STATE_QUALIFIER = Bytes.toBytes(STATE_QUALIFIER_STR);
 
   /**
-   * The serverName column qualifier. Its the server where the region is
-   * transitioning on, while column server is the server where the region is
-   * opened on. They are the same when the region is in state OPEN.
+   * The serverName column qualifier. Its the server where the region is transitioning on, while
+   * column server is the server where the region is opened on. They are the same when the region is
+   * in state OPEN.
    */
   public static final String SERVERNAME_QUALIFIER_STR = "sn";
 
-  public static final byte [] SERVERNAME_QUALIFIER = Bytes.toBytes(SERVERNAME_QUALIFIER_STR);
+  public static final byte[] SERVERNAME_QUALIFIER = Bytes.toBytes(SERVERNAME_QUALIFIER_STR);
 
   /** The lower-half split region column qualifier */
-  public static final byte [] SPLITA_QUALIFIER = Bytes.toBytes("splitA");
+  public static final byte[] SPLITA_QUALIFIER = Bytes.toBytes("splitA");
 
   /** The upper-half split region column qualifier */
-  public static final byte [] SPLITB_QUALIFIER = Bytes.toBytes("splitB");
+  public static final byte[] SPLITB_QUALIFIER = Bytes.toBytes("splitB");
 
   /** The lower-half merge region column qualifier */
   public static final byte[] MERGEA_QUALIFIER = Bytes.toBytes("mergeA");
@@ -478,32 +456,28 @@ public final class HConstants {
   /** The upper-half merge region column qualifier */
   public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes("mergeB");
 
-  /** The catalog family as a string*/
+  /** The catalog family as a string */
   public static final String TABLE_FAMILY_STR = "table";
 
   /** The catalog family */
-  public static final byte [] TABLE_FAMILY = Bytes.toBytes(TABLE_FAMILY_STR);
+  public static final byte[] TABLE_FAMILY = Bytes.toBytes(TABLE_FAMILY_STR);
 
   /** The serialized table state qualifier */
   public static final byte[] TABLE_STATE_QUALIFIER = Bytes.toBytes("state");
 
-
   /**
-   * The meta table version column qualifier.
-   * We keep current version of the meta table in this column in <code>-ROOT-</code>
-   * table: i.e. in the 'info:v' column.
+   * The meta table version column qualifier. We keep current version of the meta table in this
+   * column in <code>-ROOT-</code> table: i.e. in the 'info:v' column.
    */
-  public static final byte [] META_VERSION_QUALIFIER = Bytes.toBytes("v");
+  public static final byte[] META_VERSION_QUALIFIER = Bytes.toBytes("v");
 
   /**
-   * The current version of the meta table.
-   * - pre-hbase 0.92.  There is no META_VERSION column in the root table
-   * in this case. The meta has HTableDescriptor serialized into the HRegionInfo;
-   * - version 0 is 0.92 and 0.94. Meta data has serialized HRegionInfo's using
-   * Writable serialization, and HRegionInfo's does not contain HTableDescriptors.
-   * - version 1 for 0.96+ keeps HRegionInfo data structures, but changes the
-   * byte[] serialization from Writables to Protobuf.
-   * See HRegionInfo.VERSION
+   * The current version of the meta table. - pre-hbase 0.92. There is no META_VERSION column in the
+   * root table in this case. The meta has HTableDescriptor serialized into the HRegionInfo; -
+   * version 0 is 0.92 and 0.94. Meta data has serialized HRegionInfo's using Writable
+   * serialization, and HRegionInfo's does not contain HTableDescriptors. - version 1 for 0.96+
+   * keeps HRegionInfo data structures, but changes the byte[] serialization from Writables to
+   * Protobuf. See HRegionInfo.VERSION
    */
   public static final short META_VERSION = 1;
 
@@ -512,25 +486,24 @@ public final class HConstants {
   /**
    * An empty instance.
    */
-  public static final byte [] EMPTY_BYTE_ARRAY = new byte [0];
+  public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
 
   public static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.wrap(EMPTY_BYTE_ARRAY);
 
   /**
    * Used by scanners, etc when they want to start at the beginning of a region
    */
-  public static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY;
+  public static final byte[] EMPTY_START_ROW = EMPTY_BYTE_ARRAY;
 
   /**
    * Last row in a table.
    */
-  public static final byte [] EMPTY_END_ROW = EMPTY_START_ROW;
+  public static final byte[] EMPTY_END_ROW = EMPTY_START_ROW;
 
   /**
-    * Used by scanners and others when they're trying to detect the end of a
-    * table
-    */
-  public static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY;
+   * Used by scanners and others when they're trying to detect the end of a table
+   */
+  public static final byte[] LAST_ROW = EMPTY_BYTE_ARRAY;
 
   /**
    * Max length a row can have because of the limitation in TFile.
@@ -538,9 +511,8 @@ public final class HConstants {
   public static final int MAX_ROW_LENGTH = Short.MAX_VALUE;
 
   /**
-   * Timestamp to use when we want to refer to the latest cell.
-   * This is the timestamp sent by clients when no timestamp is specified on
-   * commit.
+   * Timestamp to use when we want to refer to the latest cell. This is the timestamp sent by
+   * clients when no timestamp is specified on commit.
    */
   public static final long LATEST_TIMESTAMP = Long.MAX_VALUE;
 
@@ -552,17 +524,12 @@ public final class HConstants {
   /**
    * LATEST_TIMESTAMP in bytes form
    */
-  public static final byte [] LATEST_TIMESTAMP_BYTES = {
+  public static final byte[] LATEST_TIMESTAMP_BYTES = {
     // big-endian
-    (byte) (LATEST_TIMESTAMP >>> 56),
-    (byte) (LATEST_TIMESTAMP >>> 48),
-    (byte) (LATEST_TIMESTAMP >>> 40),
-    (byte) (LATEST_TIMESTAMP >>> 32),
-    (byte) (LATEST_TIMESTAMP >>> 24),
-    (byte) (LATEST_TIMESTAMP >>> 16),
-    (byte) (LATEST_TIMESTAMP >>> 8),
-    (byte) LATEST_TIMESTAMP,
-  };
+    (byte) (LATEST_TIMESTAMP >>> 56), (byte) (LATEST_TIMESTAMP >>> 48),
+    (byte) (LATEST_TIMESTAMP >>> 40), (byte) (LATEST_TIMESTAMP >>> 32),
+    (byte) (LATEST_TIMESTAMP >>> 24), (byte) (LATEST_TIMESTAMP >>> 16),
+    (byte) (LATEST_TIMESTAMP >>> 8), (byte) LATEST_TIMESTAMP };
 
   /**
    * Define for 'return-all-versions'.
@@ -572,7 +539,7 @@ public final class HConstants {
   /**
    * Unlimited time-to-live.
    */
-//  public static final int FOREVER = -1;
+  // public static final int FOREVER = -1;
   public static final int FOREVER = Integer.MAX_VALUE;
 
   /**
@@ -587,10 +554,10 @@ public final class HConstants {
   public static final int HOUR_IN_SECONDS = 60 * 60;
   public static final int MINUTE_IN_SECONDS = 60;
 
-  //TODO: although the following are referenced widely to format strings for
-  //      the shell. They really aren't a part of the public API. It would be
-  //      nice if we could put them somewhere where they did not need to be
-  //      public. They could have package visibility
+  // TODO: although the following are referenced widely to format strings for
+  // the shell. They really aren't a part of the public API. It would be
+  // nice if we could put them somewhere where they did not need to be
+  // public. They could have package visibility
   public static final String NAME = "NAME";
   public static final String VERSIONS = "VERSIONS";
   public static final String IN_MEMORY = "IN_MEMORY";
@@ -598,45 +565,38 @@ public final class HConstants {
   public static final String CONFIGURATION = "CONFIGURATION";
 
   /**
-   * Retrying we multiply hbase.client.pause setting by what we have in this array until we
-   * run out of array items.  Retries beyond this use the last number in the array.  So, for
-   * example, if hbase.client.pause is 1 second, and maximum retries count
-   * hbase.client.retries.number is 10, we will retry at the following intervals:
-   * 1, 2, 3, 5, 10, 20, 40, 100, 100, 100.
-   * With 100ms, a back-off of 200 means 20s
+   * Retrying we multiply hbase.client.pause setting by what we have in this array until we run out
+   * of array items. Retries beyond this use the last number in the array. So, for example, if
+   * hbase.client.pause is 1 second, and maximum retries count hbase.client.retries.number is 10, we
+   * will retry at the following intervals: 1, 2, 3, 5, 10, 20, 40, 100, 100, 100. With 100ms, a
+   * back-off of 200 means 20s
    */
-  public static final int [] RETRY_BACKOFF = {1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200};
+  public static final int[] RETRY_BACKOFF =
+    { 1, 2, 3, 5, 10, 20, 40, 100, 100, 100, 100, 200, 200 };
 
   public static final String REGION_IMPL = "hbase.hregion.impl";
 
   /** modifyTable op for replacing the table descriptor */
   @InterfaceAudience.Private
   public static enum Modify {
-    CLOSE_REGION,
-    TABLE_COMPACT,
-    TABLE_FLUSH,
-    TABLE_MAJOR_COMPACT,
-    TABLE_SET_HTD,
-    TABLE_SPLIT
+    CLOSE_REGION, TABLE_COMPACT, TABLE_FLUSH, TABLE_MAJOR_COMPACT, TABLE_SET_HTD, TABLE_SPLIT
   }
 
   /**
-   * Scope tag for locally scoped data.
-   * This data will not be replicated.
+   * Scope tag for locally scoped data. This data will not be replicated.
    */
   public static final int REPLICATION_SCOPE_LOCAL = 0;
 
   /**
-   * Scope tag for globally scoped data.
-   * This data will be replicated to all peers.
+   * Scope tag for globally scoped data. This data will be replicated to all peers.
    */
   public static final int REPLICATION_SCOPE_GLOBAL = 1;
 
   /**
-   * Default cluster ID, cannot be used to identify a cluster so a key with
-   * this value means it wasn't meant for replication.
+   * Default cluster ID, cannot be used to identify a cluster so a key with this value means it
+   * wasn't meant for replication.
    */
-  public static final UUID DEFAULT_CLUSTER_ID = new UUID(0L,0L);
+  public static final UUID DEFAULT_CLUSTER_ID = new UUID(0L, 0L);
 
   /**
    * Parameter name for maximum number of bytes returned when calling a scanner's next method.
@@ -653,27 +613,22 @@ public final class HConstants {
       "hbase.server.scanner.max.result.size";
 
   /**
-   * Maximum number of bytes returned when calling a scanner's next method.
-   * Note that when a single row is larger than this limit the row is still
-   * returned completely.
-   *
-   * The default value is 2MB.
+   * Maximum number of bytes returned when calling a scanner's next method. Note that when a single
+   * row is larger than this limit the row is still returned completely. The default value is 2MB.
    */
   public static final long DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE = 2 * 1024 * 1024;
 
   /**
-   * Maximum number of bytes returned when calling a scanner's next method.
-   * Note that when a single row is larger than this limit the row is still
-   * returned completely.
-   * Safety setting to protect the region server.
-   *
-   * The default value is 100MB. (a client would rarely request larger chunks on purpose)
+   * Maximum number of bytes returned when calling a scanner's next method. Note that when a single
+   * row is larger than this limit the row is still returned completely. Safety setting to protect
+   * the region server. The default value is 100MB. (a client would rarely request larger chunks on
+   * purpose)
    */
   public static final long DEFAULT_HBASE_SERVER_SCANNER_MAX_RESULT_SIZE = 100 * 1024 * 1024;
 
   /**
-   * Parameter name for client pause value, used mostly as value to wait
-   * before running a retry of a failed get, region lookup, etc.
+   * Parameter name for client pause value, used mostly as value to wait before running a retry of a
+   * failed get, region lookup, etc.
    */
   public static final String HBASE_CLIENT_PAUSE = "hbase.client.pause";
 
@@ -693,8 +648,7 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS = 100;
 
   /**
-   * The maximum number of concurrent connections the client will maintain to a single
-   * RegionServer.
+   * The maximum number of concurrent connections the client will maintain to a single RegionServer.
    */
   public static final String HBASE_CLIENT_MAX_PERSERVER_TASKS = "hbase.client.max.perserver.tasks";
 
@@ -704,8 +658,7 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS = 2;
 
   /**
-   * The maximum number of concurrent connections the client will maintain to a single
-   * Region.
+   * The maximum number of concurrent connections the client will maintain to a single Region.
    */
   public static final String HBASE_CLIENT_MAX_PERREGION_TASKS = "hbase.client.max.perregion.tasks";
 
@@ -715,8 +668,8 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS = 1;
 
   /**
-   * Parameter name for server pause value, used mostly as value to wait before
-   * running a retry of a failed operation.
+   * Parameter name for server pause value, used mostly as value to wait before running a retry of a
+   * failed operation.
    */
   public static final String HBASE_SERVER_PAUSE = "hbase.server.pause";
 
@@ -726,9 +679,9 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_SERVER_PAUSE = 1000;
 
   /**
-   * Parameter name for maximum retries, used as maximum for all retryable
-   * operations such as fetching of the root region from root region server,
-   * getting a cell's value, starting a row update, etc.
+   * Parameter name for maximum retries, used as maximum for all retryable operations such as
+   * fetching of the root region from root region server, getting a cell's value, starting a row
+   * update, etc.
    */
   public static final String HBASE_CLIENT_RETRIES_NUMBER = "hbase.client.retries.number";
 
@@ -748,10 +701,9 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_CLIENT_SCANNER_CACHING = Integer.MAX_VALUE;
 
   /**
-   * Parameter name for number of rows that will be fetched when calling next on
-   * a scanner if it is not served from memory. Higher caching values will
-   * enable faster scanners but will eat up more memory and some calls of next
-   * may take longer and longer times when the cache is empty.
+   * Parameter name for number of rows that will be fetched when calling next on a scanner if it is
+   * not served from memory. Higher caching values will enable faster scanners but will eat up more
+   * memory and some calls of next may take longer and longer times when the cache is empty.
    */
   public static final String HBASE_META_SCANNER_CACHING = "hbase.meta.scanner.caching";
 
@@ -918,18 +870,13 @@ public final class HConstants {
   public static final String LOCALHOST = "localhost";
 
   /**
-   * If this parameter is set to true, then hbase will read
-   * data and then verify checksums. Checksum verification
-   * inside hdfs will be switched off.  However, if the hbase-checksum
-   * verification fails, then it will switch back to using
-   * hdfs checksums for verifiying data that is being read from storage.
-   *
-   * If this parameter is set to false, then hbase will not
-   * verify any checksums, instead it will depend on checksum verification
-   * being done in the hdfs client.
+   * If this parameter is set to true, then hbase will read data and then verify checksums. Checksum
+   * verification inside hdfs will be switched off. However, if the hbase-checksum verification
+   * fails, then it will switch back to using hdfs checksums for verifiying data that is being read
+   * from storage. If this parameter is set to false, then hbase will not verify any checksums,
+   * instead it will depend on checksum verification being done in the hdfs client.
    */
-  public static final String HBASE_CHECKSUM_VERIFICATION =
-      "hbase.regionserver.checksum.verify";
+  public static final String HBASE_CHECKSUM_VERIFICATION = "hbase.regionserver.checksum.verify";
 
   public static final String LOCALHOST_IP = "127.0.0.1";
 
@@ -944,17 +891,15 @@ public final class HConstants {
   public static final int DEFAULT_REGION_SERVER_HANDLER_COUNT = 30;
 
   /*
-   * REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT:
-   * -1  => Disable aborting
-   * 0   => Abort if even a single handler has died
-   * 0.x => Abort only when this percent of handlers have died
-   * 1   => Abort only all of the handers have died
+   * REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT: -1 => Disable aborting 0 => Abort if even a
+   * single handler has died 0.x => Abort only when this percent of handlers have died 1 => Abort
+   * only all of the handers have died
    */
   public static final String REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT =
       "hbase.regionserver.handler.abort.on.error.percent";
   public static final double DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT = 0.5;
 
-  //High priority handlers to deal with admin requests and system table operation requests
+  // High priority handlers to deal with admin requests and system table operation requests
   public static final String REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT =
       "hbase.regionserver.metahandler.count";
   public static final int DEFAULT_REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT = 20;
@@ -977,19 +922,16 @@ public final class HConstants {
   public static final int DEFAULT_META_REPLICA_NUM = 1;
 
   /**
-   * The name of the configuration parameter that specifies
-   * the number of bytes in a newly created checksum chunk.
+   * The name of the configuration parameter that specifies the number of bytes in a newly created
+   * checksum chunk.
    */
-  public static final String BYTES_PER_CHECKSUM =
-      "hbase.hstore.bytes.per.checksum";
+  public static final String BYTES_PER_CHECKSUM = "hbase.hstore.bytes.per.checksum";
 
   /**
-   * The name of the configuration parameter that specifies
-   * the name of an algorithm that is used to compute checksums
-   * for newly created blocks.
+   * The name of the configuration parameter that specifies the name of an algorithm that is used to
+   * compute checksums for newly created blocks.
    */
-  public static final String CHECKSUM_TYPE_NAME =
-      "hbase.hstore.checksum.algorithm";
+  public static final String CHECKSUM_TYPE_NAME = "hbase.hstore.checksum.algorithm";
 
   /** Enable file permission modification from standard hbase */
   public static final String ENABLE_DATA_FILE_UMASK = "hbase.data.umask.enable";
@@ -997,16 +939,14 @@ public final class HConstants {
   public static final String DATA_FILE_UMASK_KEY = "hbase.data.umask";
 
   /** Configuration name of WAL Compression */
-  public static final String ENABLE_WAL_COMPRESSION =
-    "hbase.regionserver.wal.enablecompression";
-
-  /** Configuration name of WAL storage policy
-   * Valid values are:
-   *  NONE: no preference in destination of block replicas
-   *  ONE_SSD: place only one block replica in SSD and the remaining in default storage
-   *  and ALL_SSD: place all block replicas on SSD
-   *
-   * See http://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html*/
+  public static final String ENABLE_WAL_COMPRESSION = "hbase.regionserver.wal.enablecompression";
+
+  /**
+   * Configuration name of WAL storage policy Valid values are: NONE: no preference in destination
+   * of block replicas ONE_SSD: place only one block replica in SSD and the remaining in default
+   * storage and ALL_SSD: place all block replicas on SSD See
+   * http://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html
+   */
   public static final String WAL_STORAGE_POLICY = "hbase.wal.storage.policy";
   public static final String DEFAULT_WAL_STORAGE_POLICY = "NONE";
 
@@ -1017,17 +957,16 @@ public final class HConstants {
   public static final String LOAD_BALANCER_SLOP_KEY = "hbase.regions.slop";
 
   /**
-   * The byte array represents for NO_NEXT_INDEXED_KEY;
-   * The actual value is irrelevant because this is always compared by reference.
+   * The byte array represents for NO_NEXT_INDEXED_KEY; The actual value is irrelevant because this
+   * is always compared by reference.
    */
   public static final Cell NO_NEXT_INDEXED_KEY = new KeyValue();
   /** delimiter used between portions of a region name */
   public static final int DELIMITER = ',';
 
   /**
-   * QOS attributes: these attributes are used to demarcate RPC call processing
-   * by different set of handlers. For example, HIGH_QOS tagged methods are
-   * handled by high priority handlers.
+   * QOS attributes: these attributes are used to demarcate RPC call processing by different set of
+   * handlers. For example, HIGH_QOS tagged methods are handled by high priority handlers.
    */
   // normal_QOS < QOS_threshold < replication_QOS < replay_QOS < admin_QOS < high_QOS
   public static final int NORMAL_QOS = 0;
@@ -1042,8 +981,8 @@ public final class HConstants {
   public static final String HFILE_ARCHIVE_DIRECTORY = "archive";
 
   /**
-   * Name of the directory to store all snapshots. See SnapshotDescriptionUtils for
-   * remaining snapshot constants; this is here to keep HConstants dependencies at a minimum and
+   * Name of the directory to store all snapshots. See SnapshotDescriptionUtils for remaining
+   * snapshot constants; this is here to keep HConstants dependencies at a minimum and
    * uni-directional.
    */
   public static final String SNAPSHOT_DIR_NAME = ".hbase-snapshot";
@@ -1059,49 +998,42 @@ public final class HConstants {
   public static final String REGIONSERVER_METRICS_PERIOD = "hbase.regionserver.metrics.period";
   public static final long DEFAULT_REGIONSERVER_METRICS_PERIOD = 5000;
   /** Directories that are not HBase table directories */
-  public static final List<String> HBASE_NON_TABLE_DIRS =
-    Collections.unmodifiableList(Arrays.asList(new String[] {
-      HBCK_SIDELINEDIR_NAME, HBASE_TEMP_DIRECTORY, MIGRATION_NAME
-    }));
+  public static final List<String> HBASE_NON_TABLE_DIRS = Collections.unmodifiableList(Arrays
+      .asList(new String[] { HBCK_SIDELINEDIR_NAME, HBASE_TEMP_DIRECTORY, MIGRATION_NAME }));
 
   /** Directories that are not HBase user table directories */
-  public static final List<String> HBASE_NON_USER_TABLE_DIRS =
-    Collections.unmodifiableList(Arrays.asList((String[])ArrayUtils.addAll(
-      new String[] { TableName.META_TABLE_NAME.getNameAsString() },
-      HBASE_NON_TABLE_DIRS.toArray())));
+  public static final List<String> HBASE_NON_USER_TABLE_DIRS = Collections.unmodifiableList(Arrays
+      .asList((String[]) ArrayUtils.addAll(
+        new String[] { TableName.META_TABLE_NAME.getNameAsString() },
+        HBASE_NON_TABLE_DIRS.toArray())));
 
   /** Health script related settings. */
   public static final String HEALTH_SCRIPT_LOC = "hbase.node.health.script.location";
   public static final String HEALTH_SCRIPT_TIMEOUT = "hbase.node.health.script.timeout";
-  public static final String HEALTH_CHORE_WAKE_FREQ =
-      "hbase.node.health.script.frequency";
+  public static final String HEALTH_CHORE_WAKE_FREQ = "hbase.node.health.script.frequency";
   public static final long DEFAULT_HEALTH_SCRIPT_TIMEOUT = 60000;
   /**
    * The maximum number of health check failures a server can encounter consecutively.
    */
-  public static final String HEALTH_FAILURE_THRESHOLD =
-      "hbase.node.health.failure.threshold";
+  public static final String HEALTH_FAILURE_THRESHOLD = "hbase.node.health.failure.threshold";
   public static final int DEFAULT_HEALTH_FAILURE_THRESHOLD = 3;
 
-
   /**
-   * Setting to activate, or not, the publication of the status by the master. Default
-   *  notification is by a multicast message.
+   * Setting to activate, or not, the publication of the status by the master. Default notification
+   * is by a multicast message.
    */
   public static final String STATUS_PUBLISHED = "hbase.status.published";
   public static final boolean STATUS_PUBLISHED_DEFAULT = false;
 
   /**
-   * IP to use for the multicast status messages between the master and the clients.
-   * The default address is chosen as one among others within the ones suitable for multicast
-   * messages.
+   * IP to use for the multicast status messages between the master and the clients. The default
+   * address is chosen as one among others within the ones suitable for multicast messages.
    */
   public static final String STATUS_MULTICAST_ADDRESS = "hbase.status.multicast.address.ip";
   public static final String DEFAULT_STATUS_MULTICAST_ADDRESS = "226.1.1.3";
 
   /**
-   * The address to use for binding the local socket for receiving multicast. Defaults to
-   * 0.0.0.0.
+   * The address to use for binding the local socket for receiving multicast. Defaults to 0.0.0.0.
    * @see <a href="https://issues.apache.org/jira/browse/HBASE-9961">HBASE-9961</a>
    */
   public static final String STATUS_MULTICAST_BIND_ADDRESS =
@@ -1134,7 +1066,7 @@ public final class HConstants {
 
   /** Configuration key for the name of the alternate master key for the cluster, a string */
   public static final String CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY =
-    "hbase.crypto.master.alternate.key.name";
+      "hbase.crypto.master.alternate.key.name";
 
   /** Configuration key for the algorithm to use when encrypting the WAL, a string */
   public static final String CRYPTO_WAL_ALGORITHM_CONF_KEY = "hbase.crypto.wal.algorithm";
@@ -1166,7 +1098,7 @@ public final class HConstants {
 
   /** Config for pluggable consensus provider */
   public static final String HBASE_COORDINATED_STATE_MANAGER_CLASS =
-    "hbase.coordinated.state.manager.class";
+      "hbase.coordinated.state.manager.class";
 
   /** Configuration key for SplitLog manager timeout */
   public static final String HBASE_SPLITLOG_MANAGER_TIMEOUT = "hbase.splitlog.manager.timeout";
@@ -1180,17 +1112,18 @@ public final class HConstants {
   // hbase-common?
 
   /**
-   * Current ioengine options in include: heap, offheap and file:PATH (where PATH is the path
-   * to the file that will host the file-based cache.  See BucketCache#getIOEngineFromName() for
-   * list of supported ioengine options.
-   * <p>Set this option and a non-zero {@link #BUCKET_CACHE_SIZE_KEY} to enable bucket cache.
+   * Current ioengine options in include: heap, offheap and file:PATH (where PATH is the path to the
+   * file that will host the file-based cache. See BucketCache#getIOEngineFromName() for list of
+   * supported ioengine options.
+   * <p>
+   * Set this option and a non-zero {@link #BUCKET_CACHE_SIZE_KEY} to enable bucket cache.
    */
   public static final String BUCKET_CACHE_IOENGINE_KEY = "hbase.bucketcache.ioengine";
 
   /**
    * When using bucket cache, this is a float that EITHER represents a percentage of total heap
-   * memory size to give to the cache (if &lt; 1.0) OR, it is the capacity in
-   * megabytes of the cache.
+   * memory size to give to the cache (if &lt; 1.0) OR, it is the capacity in megabytes of the
+   * cache.
    */
   public static final String BUCKET_CACHE_SIZE_KEY = "hbase.bucketcache.size";
 
@@ -1203,26 +1136,25 @@ public final class HConstants {
   public static final String HBASE_CLIENT_FAST_FAIL_MODE_ENABLED =
       "hbase.client.fast.fail.mode.enabled";
 
-  public static final boolean HBASE_CLIENT_ENABLE_FAST_FAIL_MODE_DEFAULT =
-      false;
+  public static final boolean HBASE_CLIENT_ENABLE_FAST_FAIL_MODE_DEFAULT = false;
 
   public static final String HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS =
       "hbase.client.fastfail.threshold";
 
-  public static final long HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS_DEFAULT =
-      60000;
+  public static final long HBASE_CLIENT_FAST_FAIL_THREASHOLD_MS_DEFAULT = 60000;
 
   public static final String HBASE_CLIENT_FAST_FAIL_CLEANUP_MS_DURATION_MS =
       "hbase.client.fast.fail.cleanup.duration";
 
-  public static final long HBASE_CLIENT_FAST_FAIL_CLEANUP_DURATION_MS_DEFAULT =
-      600000;
+  public static final long HBASE_CLIENT_FAST_FAIL_CLEANUP_DURATION_MS_DEFAULT = 600000;
 
   public static final String HBASE_CLIENT_FAST_FAIL_INTERCEPTOR_IMPL =
       "hbase.client.fast.fail.interceptor.impl";
 
-  /** Config key for if the server should send backpressure and if the client should listen to
-   * that backpressure from the server */
+  /**
+   * Config key for if the server should send backpressure and if the client should listen to that
+   * backpressure from the server
+   */
   public static final String ENABLE_CLIENT_BACKPRESSURE = "hbase.client.backpressure.enabled";
   public static final boolean DEFAULT_ENABLE_CLIENT_BACKPRESSURE = false;
 
@@ -1234,11 +1166,11 @@ public final class HConstants {
   public static final float DEFAULT_HEAP_OCCUPANCY_HIGH_WATERMARK = 0.98f;
 
   /**
-   * The max number of threads used for splitting storefiles in parallel during
-   * the region split process.
+   * The max number of threads used for splitting storefiles in parallel during the region split
+   * process.
    */
   public static final String REGION_SPLIT_THREADS_MAX =
-    "hbase.regionserver.region.split.threads.max";
+      "hbase.regionserver.region.split.threads.max";
 
   /** Canary config keys */
   public static final String HBASE_CANARY_WRITE_DATA_TTL_KEY = "hbase.canary.write.data.ttl";
@@ -1263,6 +1195,15 @@ public final class HConstants {
   public static final String ZK_SERVER_KEYTAB_FILE = "hbase.zookeeper.server.keytab.file";
   public static final String ZK_SERVER_KERBEROS_PRINCIPAL =
       "hbase.zookeeper.server.kerberos.principal";
+  /**
+   * Backup/Restore constants
+   */
+
+  public final static String BACKUP_ENABLE_KEY = "hbase.backup.enable";
+  public final static boolean BACKUP_ENABLE_DEFAULT = true;
+  public final static String BACKUP_SYSTEM_TTL_KEY = "hbase.backup.system.ttl";
+  // Default TTL = 1 year
+  public final static int BACKUP_SYSTEM_TTL_DEFAULT = 365 * 24 * 3600;
 
   private HConstants() {
     // Can't be instantiated with this ctor.

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 26aad71..807b021 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -394,6 +394,11 @@
        <version>${project.version}</version>
        <optional>true</optional>
     </dependency>
+     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <version>${hadoop-two.version}</version>
+    </dependency>
     <dependency>
       <groupId>commons-httpclient</groupId>
       <artifactId>commons-httpclient</artifactId>
@@ -407,6 +412,11 @@
       <artifactId>commons-collections</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-distcp</artifactId>
+      <version>${hadoop-two.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/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..28cf627
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
@@ -0,0 +1,338 @@
+/**
+ * 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.ArrayList;
+import java.util.Set;
+
+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.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_COMMAND;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * 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 BackupClient {
+  private static final Log LOG = LogFactory.getLog(BackupClient.class);
+  private static Options opt;
+  private static Configuration conf = null;
+
+  private BackupClient() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  protected static void init() throws IOException {
+    // define supported options
+    opt = new Options();
+
+    opt.addOption("all", false, "All tables");
+    opt.addOption("debug", false, "Enable debug loggings");
+    opt.addOption("t", true, "Table name");
+
+    // create configuration instance
+    conf = getConf();
+
+    // disable irrelevant loggers to avoid it mess up command output
+    disableUselessLoggers();
+
+  }
+
+
+  public static void main(String[] args) throws IOException {
+    init();
+    parseAndRun(args);
+    System.exit(0);
+  }
+
+  /**
+   * Set the configuration from a given one.
+   * @param newConf A new given configuration
+   */
+  public synchronized static void setConf(Configuration newConf) {
+    conf = newConf;
+    BackupUtil.setConf(newConf);
+  }
+
+  public static Configuration getConf() {
+    if (conf == null) {
+      conf = BackupUtil.getConf();
+    }
+    return conf;
+  }
+
+  private static void disableUselessLoggers() {
+    // disable zookeeper log to avoid it mess up command output
+    Logger zkLogger = Logger.getLogger("org.apache.zookeeper");
+    LOG.debug("Zookeeper log level before set: " + zkLogger.getLevel());
+    zkLogger.setLevel(Level.OFF);
+    LOG.debug("Zookeeper log level after set: " + zkLogger.getLevel());
+
+    // disable hbase zookeeper tool log to avoid it mess up command output
+    Logger hbaseZkLogger = Logger.getLogger("org.apache.hadoop.hbase.zookeeper");
+    LOG.debug("HBase zookeeper log level before set: " + hbaseZkLogger.getLevel());
+    hbaseZkLogger.setLevel(Level.OFF);
+    LOG.debug("HBase Zookeeper log level after set: " + hbaseZkLogger.getLevel());
+
+    // disable hbase client log to avoid it mess up command output
+    Logger hbaseClientLogger = Logger.getLogger("org.apache.hadoop.hbase.client");
+    LOG.debug("HBase client log level before set: " + hbaseClientLogger.getLevel());
+    hbaseClientLogger.setLevel(Level.OFF);
+    LOG.debug("HBase client log level after set: " + hbaseClientLogger.getLevel());
+  }
+
+  private static void parseAndRun(String[] args) throws IOException {
+
+    String cmd = null;
+    String[] remainArgs = null;
+    if (args == null || args.length == 0) {
+      BackupCommands.createCommand(BackupRestoreConstants.BACKUP_COMMAND.HELP, null).execute();
+    } else {
+      cmd = args[0];
+      remainArgs = new String[args.length - 1];
+      if (args.length > 1) {
+        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);
+      System.exit(-1);
+    }
+
+    BACKUP_COMMAND type = BACKUP_COMMAND.HELP;
+    if (BACKUP_COMMAND.CREATE.name().equalsIgnoreCase(cmd)) {
+      type = BACKUP_COMMAND.CREATE;
+    } else if (BACKUP_COMMAND.HELP.name().equalsIgnoreCase(cmd)) {
+      type = BACKUP_COMMAND.HELP;
+    } else {
+      System.out.println("Unsupported command for backup: " + cmd);
+    }
+
+    // enable debug logging
+    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
+    if (cmdline.hasOption("debug")) {
+      backupClientLogger.setLevel(Level.DEBUG);
+    } else {
+      backupClientLogger.setLevel(Level.INFO);
+    }
+
+    BackupCommands.createCommand(type, cmdline).execute();
+  }
+
+  /**
+   * Send backup request to server, and monitor the progress if necessary
+   * @param backupType : full or incremental
+   * @param backupRootPath : the rooPath specified by user
+   * @param tableListStr : the table list specified by user
+   * @param snapshot : using existing snapshot if specified by user (in future jira)
+   * @return backupId backup id
+   * @throws IOException exception
+   * @throws KeeperException excpetion
+   */
+  public static String create(String backupType, String backupRootPath, String tableListStr,
+      String snapshot) throws IOException {
+
+    String backupId = BackupRestoreConstants.BACKUPID_PREFIX + EnvironmentEdgeManager.currentTime();
+
+    // check target path first, confirm it doesn't exist before backup
+    boolean isTargetExist = false;
+    try {
+      isTargetExist = HBackupFileSystem.checkPathExist(backupRootPath, conf);
+    } catch (IOException e) {
+      String expMsg = e.getMessage();
+      String newMsg = null;
+      if (expMsg.contains("No FileSystem for scheme")) {
+        newMsg =
+            "Unsupported filesystem scheme found in the backup target url. Error Message: "
+                + newMsg;
+        LOG.error(newMsg);
+        throw new IOException(newMsg);
+      } else {
+        throw e;
+      }
+    } catch (RuntimeException e) {
+      LOG.error(e.getMessage());
+      throw e;
+    }
+
+    if (isTargetExist) {
+      LOG.info("Using existing backup root dir: " + backupRootPath);
+    } else {
+      LOG.info("Backup root dir " + backupRootPath + " does not exist. Will be created.");
+    }
+
+    // table list specified for backup, trigger backup on specified tables
+    String tableList = tableListStr;
+    // (tableListStr == null) ? null : tableListStr.replaceAll(
+    // BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND,
+    // BackupRestoreConstants.TABLENAME_DELIMITER_IN_ZNODE);
+    try {
+      requestBackup(backupId, backupType, tableList, backupRootPath, snapshot);
+    } catch (RuntimeException e) {
+      String errMsg = e.getMessage();
+      if (errMsg != null
+          && (errMsg.startsWith("Non-existing tables found") || errMsg
+              .startsWith("Snapshot is not found"))) {
+        LOG.error(errMsg + ", please check your command");
+        throw e;
+      } else {
+        throw e;
+      }
+    }
+    return backupId;
+  }
+
+  /**
+   * Prepare and submit Backup request
+   * @param backupId : backup_timestame (something like backup_1398729212626)
+   * @param backupType : full or incremental
+   * @param tableList : tables to be backuped
+   * @param targetRootDir : specified by user
+   * @param snapshot : use existing snapshot if specified by user (for future jira)
+   * @throws IOException exception
+   */
+  protected static void requestBackup(String backupId, String backupType, String tableList,
+      String targetRootDir, String snapshot) throws IOException {
+
+    Configuration conf = getConf();
+    BackupManager backupManager = null;
+    BackupContext backupContext = null;
+    if (snapshot != null) {
+      LOG.warn("Snapshot option specified, backup type and table option will be ignored,\n"
+          + "full backup will be taken based on the given snapshot.");
+      throw new IOException("backup using existing Snapshot will be implemented in future jira");
+    }
+
+    HBaseAdmin hbadmin = null;
+    Connection conn = null;
+    try {
+
+      backupManager = new BackupManager(conf);
+      String tables = tableList;
+      if (backupType.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+        Set<String> incrTableSet = backupManager.getIncrementalBackupTableSet();
+        if (incrTableSet.isEmpty()) {
+          LOG.warn("Incremental backup table set contains no table.\n"
+              + "Use 'backup create full' or 'backup stop' to \n "
+              + "change the tables covered by incremental backup.");
+          throw new RuntimeException("No table covered by incremental backup.");
+        }
+        StringBuilder sb = new StringBuilder();
+        for (String tableName : incrTableSet) {
+          sb.append(tableName + " ");
+        }
+        LOG.info("Incremental backup for the following table set: " + sb.toString());
+        tables =
+            sb.toString().trim()
+            .replaceAll(" ", BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+      }
+
+      // check whether table exists first before starting real request
+      if (tables != null) {
+        String[] tableNames = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+        ArrayList<String> noneExistingTableList = null;
+        conn = ConnectionFactory.createConnection(conf);
+        hbadmin = (HBaseAdmin) conn.getAdmin();
+        for (String tableName : tableNames) {
+          if (!hbadmin.tableExists(TableName.valueOf(tableName))) {
+            if (noneExistingTableList == null) {
+              noneExistingTableList = new ArrayList<String>();
+            }
+            noneExistingTableList.add(tableName);
+          }
+        }
+        if (noneExistingTableList != null) {
+          if (backupType.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+            LOG.warn("Incremental backup table set contains no-exising table: "
+                + noneExistingTableList);
+          } else {
+            // Throw exception only in full mode - we try to backup non-existing table
+            throw new RuntimeException("Non-existing tables found in the table list: "
+                + noneExistingTableList);
+          }
+        }
+      }
+
+      // if any target table backup dir already exist, then no backup action taken
+      String[] tableNames = null;
+      if (tables != null && !tables.equals("")) {
+        tableNames = tables.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
+      }
+      if (tableNames != null && tableNames.length > 0) {
+        for (String table : tableNames) {
+          String targetTableBackupDir =
+              HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
+          Path targetTableBackupDirPath = new Path(targetTableBackupDir);
+          FileSystem outputFs = FileSystem.get(targetTableBackupDirPath.toUri(), conf);
+          if (outputFs.exists(targetTableBackupDirPath)) {
+            throw new IOException("Target backup directory " + targetTableBackupDir
+              + " exists already.");
+          }
+        }
+      }
+      backupContext =
+          backupManager.createBackupContext(backupId, backupType, tables, targetRootDir, snapshot);
+      backupManager.initialize();
+      backupManager.dispatchRequest(backupContext);
+    } catch (BackupException e) {
+      // suppress the backup exception wrapped within #initialize or #dispatchRequest, backup
+      // exception has already been handled normally
+      StackTraceElement[] stes = e.getStackTrace();
+      for (StackTraceElement ste : stes) {
+        LOG.info(ste);
+      }
+      LOG.error("Backup Exception " + e.getMessage());
+    } finally {
+      if (hbadmin != null) {
+        hbadmin.close();
+      }
+      if (conn != null) {
+        conn.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java
new file mode 100644
index 0000000..c7aff15
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java
@@ -0,0 +1,147 @@
+/**
+ * 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.commons.cli.CommandLine;
+import org.apache.hadoop.hbase.backup.BackupRestoreConstants.BACKUP_COMMAND;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * General backup commands, options and usage messages
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+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"
+      + "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] [-s name] [-convert] "
+          + "[-silent]\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 gpfs, hdfs or webhdfs\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"
+          + "   -s name     Use the specified snapshot for full backup\n"
+          + "   -convert    For an incremental backup, convert WAL files to HFiles\n";
+
+  interface Command {
+    void execute() throws IOException;
+  }
+
+  private BackupCommands() {
+    throw new AssertionError("Instantiating utility class...");
+  }
+
+  static Command createCommand(BACKUP_COMMAND type, CommandLine cmdline) {
+    Command cmd = null;
+    switch (type) {
+      case CREATE:
+        cmd = new CreateCommand(cmdline);
+        break;
+      case HELP:
+      default:
+        cmd = new HelpCommand(cmdline);
+        break;
+    }
+    return cmd;
+  }
+
+  private static class CreateCommand implements Command {
+    CommandLine cmdline;
+
+    CreateCommand(CommandLine cmdline) {
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null || cmdline.getArgs() == null) {
+        System.out.println("ERROR: missing arguments");
+        System.out.println(CREATE_CMD_USAGE);
+        System.exit(-1);
+      }
+      String[] args = cmdline.getArgs();
+      if (args.length < 2 || args.length > 3) {
+        System.out.println("ERROR: wrong number of arguments");
+        System.out.println(CREATE_CMD_USAGE);
+        System.exit(-1);
+      }
+
+      if (!BackupRestoreConstants.BACKUP_TYPE_FULL.equalsIgnoreCase(args[0])
+          && !BackupRestoreConstants.BACKUP_TYPE_INCR.equalsIgnoreCase(args[0])) {
+        System.out.println("ERROR: invalid backup type");
+        System.out.println(CREATE_CMD_USAGE);
+        System.exit(-1);
+      }
+
+      String snapshot = cmdline.hasOption('s') ? cmdline.getOptionValue('s') : null;
+      String tables = (args.length == 3) ? args[2] : null;
+
+      try {
+        BackupClient.create(args[0], args[1], tables, snapshot);
+      } catch (RuntimeException e) {
+        System.out.println("ERROR: " + e.getMessage());
+        System.exit(-1);
+      }
+    }
+  }
+
+  private static class HelpCommand implements Command {
+    CommandLine cmdline;
+
+    HelpCommand(CommandLine cmdline) {
+      this.cmdline = cmdline;
+    }
+
+    @Override
+    public void execute() throws IOException {
+      if (cmdline == null) {
+        System.out.println(USAGE);
+        System.exit(0);
+      }
+
+      String[] args = cmdline.getArgs();
+      if (args == null || args.length == 0) {
+        System.out.println(USAGE);
+        System.exit(0);
+      }
+
+      if (args.length != 1) {
+        System.out.println("Only support check help message of a single command type");
+        System.out.println(USAGE);
+        System.exit(0);
+      }
+
+      String type = args[0];
+
+      if (BACKUP_COMMAND.CREATE.name().equalsIgnoreCase(type)) {
+        System.out.println(CREATE_CMD_USAGE);
+      } // other commands will be supported in future jira
+      System.exit(0);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java
new file mode 100644
index 0000000..e8d6444
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java
@@ -0,0 +1,318 @@
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * An object to encapsulate the information for each backup request
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupContext implements Serializable {
+
+  private static final long serialVersionUID = 2401435114454300992L;
+
+  // backup id: a timestamp when we request the backup
+  private String backupId;
+
+  // backup type, full or incremental
+  private String type;
+
+  // target root directory for storing the backup files
+  private String targetRootDir;
+
+  // overall backup status
+  private BackupHandler.BACKUPSTATUS flag;
+
+  // overall backup phase
+  private BackupHandler.BACKUPPHASE phase;
+
+  // overall backup failure message
+  private String failedMsg;
+
+  // backup status map for all tables
+  private Map<String, 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<String, HashMap<String, String>> tableSetTimestampMap;
+
+  // cancel flag
+  private boolean cancelled = false;
+  // backup progress string
+
+  private String progress;
+
+  public BackupContext() {
+  }
+
+  public BackupContext(String backupId, String type, String[] tables, String targetRootDir,
+      String snapshot) {
+    super();
+
+    if (backupStatusMap == null) {
+      backupStatusMap = new HashMap<String, BackupStatus>();
+    }
+
+    this.backupId = backupId;
+    this.type = type;
+    this.targetRootDir = targetRootDir;
+
+    this.addTables(tables);
+
+    if (type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
+      setHlogTargetDir(HBackupFileSystem.getLogBackupDir(targetRootDir, backupId));
+    }
+
+    this.startTs = 0;
+    this.endTs = 0;
+
+  }
+
+  /**
+   * Set progress string
+   * @param msg progress message
+   */
+
+  public void setProgress(String msg) {
+    this.progress = msg;
+  }
+
+  /**
+   * Get current progress msg
+   */
+  public String getProgress() {
+    return progress;
+  }
+
+  /**
+   * Mark cancel flag.
+   */
+  public void markCancel() {
+    this.cancelled = true;
+  }
+
+  /**
+   * Has been marked as cancelled or not.
+   * @return True if marked as cancelled
+   */
+  public boolean isCancelled() {
+    return this.cancelled;
+  }
+
+  public String getBackupId() {
+    return backupId;
+  }
+
+  public void setBackupId(String backupId) {
+    this.backupId = backupId;
+  }
+
+  public BackupStatus getBackupStatus(String 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 BackupHandler.BACKUPSTATUS getFlag() {
+    return flag;
+  }
+
+  public void setFlag(BackupHandler.BACKUPSTATUS flag) {
+    this.flag = flag;
+  }
+
+  public BackupHandler.BACKUPPHASE getPhase() {
+    return phase;
+  }
+
+  public void setPhase(BackupHandler.BACKUPPHASE phase) {
+    this.phase = phase;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setSnapshotName(String table, String snapshotName) {
+    this.backupStatusMap.get(table).setSnapshotName(snapshotName);
+  }
+
+  public String getSnapshotName(String 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<String> getTables() {
+    return this.backupStatusMap.keySet();
+  }
+
+  public String getTableListAsString() {
+    return BackupUtil.concat(backupStatusMap.keySet(), ";");
+  }
+
+  public void addTables(String[] tables) {
+    for (String 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 List<String> setIncrBackupFileList(List<String> incrBackupFileList) {
+    this.incrBackupFileList = incrBackupFileList;
+    return this.incrBackupFileList;
+  }
+
+  /**
+   * Set the new region server log timestamps after distributed log roll
+   * @param newTableSetTimestampMap table timestamp map
+   */
+  public void setIncrTimestampMap(HashMap<String, 
+      HashMap<String, String>> newTableSetTimestampMap) {
+    this.tableSetTimestampMap = newTableSetTimestampMap;
+  }
+
+  /**
+   * Get new region server log timestamps after distributed log roll
+   * @return new region server log timestamps
+   */
+  public HashMap<String, HashMap<String, String>> getIncrTimestampMap() {
+    return this.tableSetTimestampMap;
+  }
+
+  /**
+   * Get existing snapshot if backing up from existing snapshot.
+   * @return The existing snapshot, null if not backing up from existing snapshot
+   */
+  public String getExistingSnapshot() {
+    // this feature will be supported in another Jira
+    return null;
+  }
+
+  /**
+   * Check whether this backup context are for backing up from existing snapshot or not.
+   * @return true if it is for backing up from existing snapshot, otherwise false
+   */
+  public boolean fromExistingSnapshot() {
+    // this feature will be supported in later jiras
+    return false;
+  }
+
+  public String getTableBySnapshot(String snapshotName) {
+    for (Entry<String, BackupStatus> entry : this.backupStatusMap.entrySet()) {
+      if (snapshotName.equals(entry.getValue().getSnapshotName())) {
+        return entry.getKey();
+      }
+    }
+    return null;
+  }
+
+  public byte[] toByteArray() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    ObjectOutputStream oos = new ObjectOutputStream(baos);
+    oos.writeObject(this);
+    return baos.toByteArray();
+  }
+
+  public static BackupContext fromByteArray(byte[] data) 
+      throws IOException, ClassNotFoundException {
+    ByteArrayInputStream bais = new ByteArrayInputStream(data);
+    ObjectInputStream ois = new ObjectInputStream(bais);
+    return (BackupContext) ois.readObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java
new file mode 100644
index 0000000..223746d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java
@@ -0,0 +1,37 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface BackupCopyService extends Configurable {
+  static enum Type {
+    FULL, INCREMENTAL
+  }
+
+  public int copy(BackupHandler backupHandler, Configuration conf, BackupCopyService.Type copyType,
+      String[] options) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de69f0df/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java
new file mode 100644
index 0000000..fe0d0e2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java
@@ -0,0 +1,85 @@
+/**
+ * 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.HBaseIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Backup exception
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BackupException extends HBaseIOException {
+  private BackupContext description;
+
+  /**
+   * Some exception happened for a backup and don't even know the backup that it was about
+   * @param msg Full description of the failure
+   */
+  public BackupException(String msg) {
+    super(msg);
+  }
+
+  /**
+   * Some exception happened for a backup with a cause
+   * @param cause the cause
+   */
+  public BackupException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * Exception for the given backup that has no previous root cause
+   * @param msg reason why the backup failed
+   * @param desc description of the backup that is being failed
+   */
+  public BackupException(String msg, BackupContext desc) {
+    super(msg);
+    this.description = desc;
+  }
+
+  /**
+   * Exception for the given backup due to another exception
+   * @param msg reason why the backup failed
+   * @param cause root cause of the failure
+   * @param desc description of the backup that is being failed
+   */
+  public BackupException(String msg, Throwable cause, BackupContext desc) {
+    super(msg, cause);
+    this.description = desc;
+  }
+
+  /**
+   * Exception when the description of the backup cannot be determined, due to some other root
+   * cause
+   * @param message description of what caused the failure
+   * @param e root cause
+   */
+  public BackupException(String message, Exception e) {
+    super(message, e);
+  }
+
+  public BackupContext getBackupContext() {
+    return this.description;
+  }
+
+}