You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2015/12/28 10:08:11 UTC

[1/6] hbase git commit: HBASE-14030 Revert due to pending review comments

Repository: hbase
Updated Branches:
  refs/heads/master de69f0df3 -> 449fb8128


http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
deleted file mode 100644
index ad23f9b..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRestoreBoundaryTests.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class TestRestoreBoundaryTests extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestRestoreBoundaryTests.class);
-
-  /**
-   * Verify that a single empty table is restored to a new table
-   * @throws Exception
-   */
-  @Test
-  public void testFullRestoreSingleEmpty() throws Exception {
-
-    LOG.info("test full restore on a single table empty table");
-    String backupId =
- BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
-    LOG.info("backup complete");
-    assertTrue(checkSucceeded(backupId));
-    String[] tableset = new String[] { table1.getNameAsString() };
-    String[] tablemap = new String[] { table1_restore };
-    Path path = new Path(BACKUP_ROOT_DIR);
-    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
-      false);
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
-    assertTrue(hba.tableExists(TableName.valueOf(table1_restore)));
-    TEST_UTIL.deleteTable(TableName.valueOf(table1_restore));
-  }
-
-  /**
-   * Verify that multiple tables are restored to new tables.
-   * @throws Exception
-   */
-  @Test
-  public void testFullRestoreMultipleEmpty() throws Exception {
-    LOG.info("create full backup image on multiple tables");
-    String tableset =
-        table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table3.getNameAsString();
-    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
-    assertTrue(checkSucceeded(backupId));
-    String[] restore_tableset = new String[] { table2.getNameAsString(), table3.getNameAsString() };
-    String[] tablemap = new String[] { table2_restore, table3_restore };
-    Path path = new Path(BACKUP_ROOT_DIR);
-    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, restore_tableset,
-      tablemap,
-      false);
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
-    assertTrue(hba.tableExists(TableName.valueOf(table2_restore)));
-    assertTrue(hba.tableExists(TableName.valueOf(table3_restore)));
-    TEST_UTIL.deleteTable(TableName.valueOf(table2_restore));
-    TEST_UTIL.deleteTable(TableName.valueOf(table3_restore));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
index cd2efad..7620bbb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleRSProcedureManager.java
@@ -49,7 +49,7 @@ public class SimpleRSProcedureManager extends RegionServerProcedureManager {
   private ProcedureMember member;
 
   @Override
-  public void initialize(RegionServerServices rss) throws IOException {
+  public void initialize(RegionServerServices rss) throws KeeperException {
     this.rss = rss;
     ZooKeeperWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw, getProcedureSignature());


[2/6] hbase git commit: HBASE-14030 Revert due to pending review comments

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index 2ceeda5..9d9cee0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -85,9 +85,6 @@ public class WALPlayer extends Configured implements Tool {
 
   private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
 
-  public WALPlayer(){
-  }
-
   protected WALPlayer(final Configuration c) {
     super(c);
   }
@@ -97,7 +94,7 @@ public class WALPlayer extends Configured implements Tool {
    * This one can be used together with {@link KeyValueSortReducer}
    */
   static class WALKeyValueMapper
-    extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
+  extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
     private byte[] table;
 
     @Override
@@ -109,9 +106,7 @@ public class WALPlayer extends Configured implements Tool {
         if (Bytes.equals(table, key.getTablename().getName())) {
           for (Cell cell : value.getCells()) {
             KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-            if (WALEdit.isMetaEditFamily(kv)) {
-              continue;
-            }
+            if (WALEdit.isMetaEditFamily(kv)) continue;
             context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)), kv);
           }
         }
@@ -137,7 +132,7 @@ public class WALPlayer extends Configured implements Tool {
    * a running HBase instance.
    */
   protected static class WALMapper
-    extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
+  extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
     private Map<TableName, TableName> tables = new TreeMap<TableName, TableName>();
 
     @Override
@@ -154,9 +149,7 @@ public class WALPlayer extends Configured implements Tool {
           Cell lastCell = null;
           for (Cell cell : value.getCells()) {
             // filtering WAL meta entries
-            if (WALEdit.isMetaEditFamily(cell)) {
-              continue;
-            }
+            if (WALEdit.isMetaEditFamily(cell)) continue;
 
             // Allow a subclass filter out this cell.
             if (filter(context, cell)) {
@@ -167,12 +160,8 @@ public class WALPlayer extends Configured implements Tool {
               if (lastCell == null || lastCell.getTypeByte() != cell.getTypeByte()
                   || !CellUtil.matchingRow(lastCell, cell)) {
                 // row or type changed, write out aggregate KVs.
-                if (put != null) {
-                  context.write(tableOut, put);
-                }
-                if (del != null) {
-                  context.write(tableOut, del);
-                }
+                if (put != null) context.write(tableOut, put);
+                if (del != null) context.write(tableOut, del);
                 if (CellUtil.isDelete(cell)) {
                   del = new Delete(CellUtil.cloneRow(cell));
                 } else {
@@ -188,12 +177,8 @@ public class WALPlayer extends Configured implements Tool {
             lastCell = cell;
           }
           // write residual KVs
-          if (put != null) {
-            context.write(tableOut, put);
-          }
-          if (del != null) {
-            context.write(tableOut, del);
-          }
+          if (put != null) context.write(tableOut, put);
+          if (del != null) context.write(tableOut, del);
         }
       } catch (InterruptedException e) {
         e.printStackTrace();
@@ -201,8 +186,7 @@ public class WALPlayer extends Configured implements Tool {
     }
 
     /**
-     * Filter cell
-     * @param cell cell
+     * @param cell
      * @return Return true if we are to emit this cell.
      */
     protected boolean filter(Context context, final Cell cell) {
@@ -213,7 +197,9 @@ public class WALPlayer extends Configured implements Tool {
     public void setup(Context context) throws IOException {
       String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY);
       String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY);
-      if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
+      if (tablesToUse == null && tableMap == null) {
+        // Then user wants all tables.
+      } else if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
         // this can only happen when WALMapper is used directly by a class other than WALPlayer
         throw new IOException("No tables or incorrect table mapping specified.");
       }
@@ -229,9 +215,7 @@ public class WALPlayer extends Configured implements Tool {
 
   void setupTime(Configuration conf, String option) throws IOException {
     String val = conf.get(option);
-    if (null == val) {
-      return;
-    }
+    if (null == val) return;
     long ms;
     try {
       // first try to parse in user friendly form
@@ -311,8 +295,7 @@ public class WALPlayer extends Configured implements Tool {
     return job;
   }
 
-  /**
-   * Print usage
+  /*
    * @param errorMsg Error message.  Can be null.
    */
   private void usage(final String errorMsg) {
@@ -322,8 +305,7 @@ public class WALPlayer extends Configured implements Tool {
     System.err.println("Usage: " + NAME + " [options] <wal inputdir> <tables> [<tableMappings>]");
     System.err.println("Read all WAL entries for <tables>.");
     System.err.println("If no tables (\"\") are specific, all tables are imported.");
-    System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported"+
-      " in that case.)");
+    System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported in that case.)");
     System.err.println("Otherwise <tables> is a comma separated list of tables.\n");
     System.err.println("The WAL entries can be mapped to new set of tables via <tableMapping>.");
     System.err.println("<tableMapping> is a command separated list of targettables.");
@@ -336,10 +318,10 @@ public class WALPlayer extends Configured implements Tool {
     System.err.println("  -D" + WALInputFormat.START_TIME_KEY + "=[date|ms]");
     System.err.println("  -D" + WALInputFormat.END_TIME_KEY + "=[date|ms]");
     System.err.println("   -D " + JOB_NAME_CONF_KEY
-      + "=jobName - use the specified mapreduce job name for the wal player");
+        + "=jobName - use the specified mapreduce job name for the wal player");
     System.err.println("For performance also consider the following options:\n"
-      + "  -Dmapreduce.map.speculative=false\n"
-      + "  -Dmapreduce.reduce.speculative=false");
+        + "  -Dmapreduce.map.speculative=false\n"
+        + "  -Dmapreduce.reduce.speculative=false");
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 5cd38b5..bdb19f4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -75,7 +75,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.hadoop.hbase.backup.BackupManager;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
@@ -385,7 +384,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
 
     Replication.decorateMasterConfiguration(this.conf);
-    BackupManager.decorateMasterConfiguration(this.conf);
 
     // Hack! Maps DFSClient => Master for logs.  HDFS made this
     // config param for task trackers, but we can piggyback off of it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
index b6e11ea..95c3ffe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManager.java
@@ -37,7 +37,7 @@ public abstract class RegionServerProcedureManager extends ProcedureManager {
    * @param rss Region Server service interface
    * @throws KeeperException
    */
-  public abstract void initialize(RegionServerServices rss) throws IOException;
+  public abstract void initialize(RegionServerServices rss) throws KeeperException;
 
   /**
    * Start accepting procedure requests.

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
index adb3604..0f4ea64 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/RegionServerProcedureManagerHost.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.procedure.flush.RegionServerFlushTableProcedureManager;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
+import org.apache.zookeeper.KeeperException;
 
 /**
  * Provides the globally barriered procedure framework and environment
@@ -38,7 +39,7 @@ public class RegionServerProcedureManagerHost extends
   private static final Log LOG = LogFactory
       .getLog(RegionServerProcedureManagerHost.class);
 
-  public void initialize(RegionServerServices rss) throws IOException {
+  public void initialize(RegionServerServices rss) throws KeeperException {
     for (RegionServerProcedureManager proc : procedures) {
       LOG.debug("Procedure " + proc.getProcedureSignature() + " is initializing");
       proc.initialize(rss);

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
index 3865ba9..085d642 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinatorRpcs.java
@@ -54,7 +54,7 @@ public class ZKProcedureCoordinatorRpcs implements ProcedureCoordinatorRpcs {
    * @throws KeeperException if an unexpected zk error occurs
    */
   public ZKProcedureCoordinatorRpcs(ZooKeeperWatcher watcher,
-      String procedureClass, String coordName) throws IOException {
+      String procedureClass, String coordName) throws KeeperException {
     this.watcher = watcher;
     this.procedureType = procedureClass;
     this.coordName = coordName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
index fff75a7..2e03a60 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureMemberRpcs.java
@@ -68,54 +68,49 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
    * @throws KeeperException if we can't reach zookeeper
    */
   public ZKProcedureMemberRpcs(final ZooKeeperWatcher watcher, final String procType)
-      throws IOException {
-    try {
-      this.zkController = new ZKProcedureUtil(watcher, procType) {
-        @Override
-        public void nodeCreated(String path) {
-          if (!isInProcedurePath(path)) {
-            return;
-          }
+      throws KeeperException {
+    this.zkController = new ZKProcedureUtil(watcher, procType) {
+      @Override
+      public void nodeCreated(String path) {
+        if (!isInProcedurePath(path)) {
+          return;
+        }
 
-          LOG.info("Received created event:" + path);
-          // if it is a simple start/end/abort then we just rewatch the node
-          if (isAcquiredNode(path)) {
-            waitForNewProcedures();
-            return;
-          } else if (isAbortNode(path)) {
-            watchForAbortedProcedures();
-            return;
-          }
-          String parent = ZKUtil.getParent(path);
-          // if its the end barrier, the procedure can be completed
-          if (isReachedNode(parent)) {
-            receivedReachedGlobalBarrier(path);
-            return;
-          } else if (isAbortNode(parent)) {
-            abort(path);
-            return;
-          } else if (isAcquiredNode(parent)) {
-            startNewSubprocedure(path);
-          } else {
-            LOG.debug("Ignoring created notification for node:" + path);
-          }
+        LOG.info("Received created event:" + path);
+        // if it is a simple start/end/abort then we just rewatch the node
+        if (isAcquiredNode(path)) {
+          waitForNewProcedures();
+          return;
+        } else if (isAbortNode(path)) {
+          watchForAbortedProcedures();
+          return;
         }
+        String parent = ZKUtil.getParent(path);
+        // if its the end barrier, the procedure can be completed
+        if (isReachedNode(parent)) {
+          receivedReachedGlobalBarrier(path);
+          return;
+        } else if (isAbortNode(parent)) {
+          abort(path);
+          return;
+        } else if (isAcquiredNode(parent)) {
+          startNewSubprocedure(path);
+        } else {
+          LOG.debug("Ignoring created notification for node:" + path);
+        }
+      }
 
-        @Override
-        public void nodeChildrenChanged(String path) {
-          if (path.equals(this.acquiredZnode)) {
-            LOG.info("Received procedure start children changed event: " + path);
-            waitForNewProcedures();
-          } else if (path.equals(this.abortZnode)) {
-            LOG.info("Received procedure abort children changed event: " + path);
-            watchForAbortedProcedures();
-          }
+      @Override
+      public void nodeChildrenChanged(String path) {
+        if (path.equals(this.acquiredZnode)) {
+          LOG.info("Received procedure start children changed event: " + path);
+          waitForNewProcedures();
+        } else if (path.equals(this.abortZnode)) {
+          LOG.info("Received procedure abort children changed event: " + path);
+          watchForAbortedProcedures();
         }
-      };
-    } catch (KeeperException e) {
-      // TODO Auto-generated catch block
-      throw new IOException(e);
-    }
+      }
+    };
   }
 
   public ZKProcedureUtil getZkController() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
index bd65cc7..1aa959c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/RegionServerFlushTableProcedureManager.java
@@ -317,7 +317,7 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur
    * @throws KeeperException if the zookeeper cannot be reached
    */
   @Override
-  public void initialize(RegionServerServices rss) throws IOException {
+  public void initialize(RegionServerServices rss) throws KeeperException {
     this.rss = rss;
     ZooKeeperWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw,

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 1cd54fa..211fed5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -807,8 +807,8 @@ public class HRegionServer extends HasThread implements
       rspmHost = new RegionServerProcedureManagerHost();
       rspmHost.loadProcedures(conf);
       rspmHost.initialize(this);
-    } catch (IOException e) {
-      this.abort("Failed to reach coordination cluster when creating procedure handler.", e);
+    } catch (KeeperException e) {
+      this.abort("Failed to reach zk cluster when creating procedure handler.", e);
     }
     // register watcher for recovering regions
     this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
index e56dd28..537329a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/snapshot/RegionServerSnapshotManager.java
@@ -390,7 +390,7 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
    * @throws KeeperException if the zookeeper cluster cannot be reached
    */
   @Override
-  public void initialize(RegionServerServices rss) throws IOException {
+  public void initialize(RegionServerServices rss) throws KeeperException {
     this.rss = rss;
     ZooKeeperWatcher zkw = rss.getZooKeeper();
     this.memberRpcs = new ZKProcedureMemberRpcs(zkw,

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index acde21e..9ae72e6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -97,8 +97,6 @@ import com.lmax.disruptor.TimeoutException;
 import com.lmax.disruptor.dsl.Disruptor;
 import com.lmax.disruptor.dsl.ProducerType;
 
-
-
 /**
  * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS.
  * Only one WAL is ever being written at a time.  When a WAL hits a configured maximum size,
@@ -361,9 +359,7 @@ public class FSHLog implements WAL {
     public int compare(Path o1, Path o2) {
       long t1 = getFileNumFromFileName(o1);
       long t2 = getFileNumFromFileName(o2);
-      if (t1 == t2) {
-        return 0;
-      }
+      if (t1 == t2) return 0;
       return (t1 > t2) ? 1 : -1;
     }
   };
@@ -406,7 +402,7 @@ public class FSHLog implements WAL {
    * @param root path for stored and archived wals
    * @param logDir dir where wals are stored
    * @param conf configuration to use
-   * @throws IOException exception
+   * @throws IOException
    */
   public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf)
       throws IOException {
@@ -414,7 +410,7 @@ public class FSHLog implements WAL {
   }
 
   /**
-   * Create an edit log at the given directory location.
+   * Create an edit log at the given <code>dir</code> location.
    *
    * You should never have to load an existing log. If there is a log at
    * startup, it should have already been processed and deleted by the time the
@@ -429,13 +425,13 @@ public class FSHLog implements WAL {
    * be registered before we do anything else; e.g. the
    * Constructor {@link #rollWriter()}.
    * @param failIfWALExists If true IOException will be thrown if files related to this wal
-   *     already exist.
+   *        already exist.
    * @param prefix should always be hostname and port in distributed env and
-   *     it will be URL encoded before being used.
-   *     If prefix is null, "wal" will be used
+   *        it will be URL encoded before being used.
+   *        If prefix is null, "wal" will be used
    * @param suffix will be url encoded. null is treated as empty. non-empty must start with
-   *     {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
-   * @throws IOException exception
+   *        {@link DefaultWALProvider#WAL_FILE_NAME_DELIMITER}
+   * @throws IOException
    */
   public FSHLog(final FileSystem fs, final Path rootDir, final String logDir,
       final String archiveDir, final Configuration conf,
@@ -597,9 +593,7 @@ public class FSHLog implements WAL {
   @VisibleForTesting
   OutputStream getOutputStream() {
     FSDataOutputStream fsdos = this.hdfs_out;
-    if (fsdos == null) {
-      return null;
-    }
+    if (fsdos == null) return null;
     return fsdos.getWrappedStream();
   }
 
@@ -634,7 +628,7 @@ public class FSHLog implements WAL {
 
   /**
    * Tell listeners about pre log roll.
-   * @throws IOException exception
+   * @throws IOException
    */
   private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath)
   throws IOException {
@@ -647,7 +641,7 @@ public class FSHLog implements WAL {
 
   /**
    * Tell listeners about post log roll.
-   * @throws IOException exception
+   * @throws IOException
    */
   private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
   throws IOException {
@@ -660,7 +654,8 @@ public class FSHLog implements WAL {
 
   /**
    * Run a sync after opening to set up the pipeline.
-   * @param nextWriter next writer
+   * @param nextWriter
+   * @param startTimeNanos
    */
   private void preemptiveSync(final ProtobufLogWriter nextWriter) {
     long startTimeNanos = System.nanoTime();
@@ -678,9 +673,7 @@ public class FSHLog implements WAL {
     rollWriterLock.lock();
     try {
       // Return if nothing to flush.
-      if (!force && (this.writer != null && this.numEntries.get() <= 0)) {
-        return null;
-      }
+      if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null;
       byte [][] regionsToFlush = null;
       if (this.closed) {
         LOG.debug("WAL closed. Skipping rolling of writer");
@@ -735,7 +728,7 @@ public class FSHLog implements WAL {
 
   /**
    * Archive old logs. A WAL is eligible for archiving if all its WALEdits have been flushed.
-   * @throws IOException exception
+   * @throws IOException
    */
   private void cleanOldLogs() throws IOException {
     List<Path> logsToArchive = null;
@@ -745,13 +738,9 @@ public class FSHLog implements WAL {
       Path log = e.getKey();
       Map<byte[], Long> sequenceNums = e.getValue();
       if (this.sequenceIdAccounting.areAllLower(sequenceNums)) {
-        if (logsToArchive == null) {
-          logsToArchive = new ArrayList<Path>();
-        }
+        if (logsToArchive == null) logsToArchive = new ArrayList<Path>();
         logsToArchive.add(log);
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("WAL file ready for archiving " + log);
-        }
+        if (LOG.isTraceEnabled()) LOG.trace("WAL file ready for archiving " + log);
       }
     }
     if (logsToArchive != null) {
@@ -781,9 +770,7 @@ public class FSHLog implements WAL {
     if (regions != null) {
       StringBuilder sb = new StringBuilder();
       for (int i = 0; i < regions.length; i++) {
-        if (i > 0) {
-          sb.append(", ");
-        }
+        if (i > 0) sb.append(", ");
         sb.append(Bytes.toStringBinary(regions[i]));
       }
       LOG.info("Too many WALs; count=" + logCount + ", max=" + this.maxLogs +
@@ -849,9 +836,7 @@ public class FSHLog implements WAL {
         }
       } catch (FailedSyncBeforeLogCloseException e) {
         // If unflushed/unsynced entries on close, it is reason to abort.
-        if (isUnflushedEntries()) {
-          throw e;
-        }
+        if (isUnflushedEntries()) throw e;
         LOG.warn("Failed sync-before-close but no outstanding appends; closing WAL: " +
           e.getMessage());
       }
@@ -912,9 +897,7 @@ public class FSHLog implements WAL {
             try {
               blockOnSync(syncFuture);
             } catch (IOException ioe) {
-              if (LOG.isTraceEnabled()) {
-                LOG.trace("Stale sync exception", ioe);
-              }
+              if (LOG.isTraceEnabled()) LOG.trace("Stale sync exception", ioe);
             }
           }
         }
@@ -985,15 +968,7 @@ public class FSHLog implements WAL {
   public Path getCurrentFileName() {
     return computeFilename(this.filenum.get());
   }
-  
-  /**
-   * To support old API compatibility
-   * @return current file number (timestamp)
-   */
-  public long getFilenum() {
-    return filenum.get();
-  }
-  
+
   @Override
   public String toString() {
     return "FSHLog " + logFilePrefix + ":" + logFileSuffix + "(num " + filenum + ")";

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java
deleted file mode 100644
index 0360000..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCopy.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.snapshot;
-
-import org.apache.hadoop.hbase.backup.BackupHandler;
-
-/* this class will be extended in future jira to support progress report */
-public class SnapshotCopy extends ExportSnapshot {
-  private BackupHandler backupHandler;
-  private String table;
-
-  public SnapshotCopy(BackupHandler backupHandler, String table) {
-    super();
-    this.backupHandler = backupHandler;
-    this.table = table;
-  }
-
-  public BackupHandler getBackupHandler() {
-    return this.backupHandler;
-  }
-
-  public String getTable() {
-    return this.table;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
index dd4d337..027e7a2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DefaultWALProvider.java
@@ -209,18 +209,13 @@ public class DefaultWALProvider implements WALProvider {
   @VisibleForTesting
   public static long extractFileNumFromWAL(final WAL wal) {
     final Path walName = ((FSHLog)wal).getCurrentFileName();
-    return extractFileNumFromWAL(walName);
-  }
-
-  @VisibleForTesting
-  public static long extractFileNumFromWAL(final Path walName) {
     if (walName == null) {
       throw new IllegalArgumentException("The WAL path couldn't be null");
     }
     final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER);
     return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2:1)]);
   }
-  
+
   /**
    * Pattern used to validate a WAL file name
    * see {@link #validateWALFilename(String)} for description.

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
deleted file mode 100644
index bc0c848..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS;
-import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
-import org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager;
-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.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-/**
- * This class is only a base for other integration-level backup tests.
- * Do not add tests here.
- * TestBackupSmallTests is where tests that don't require bring machines up/down should go
- * All other tests should have their own classes and extend this one
- */
-public class TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestBackupBase.class);
-
-  protected static Configuration conf1;
-  protected static Configuration conf2;
-
-  protected static HBaseTestingUtility TEST_UTIL;
-  protected static HBaseTestingUtility TEST_UTIL2;
-
-  protected static TableName table1;
-  protected static TableName table2;
-  protected static TableName table3;
-  protected static TableName table4;
-
-  protected static String table1_restore = "table1_restore";
-  protected static String table2_restore = "table2_restore";
-  protected static String table3_restore = "table3_restore";
-  protected static String table4_restore = "table4_restore";
-
-  protected static final int NB_ROWS_IN_BATCH = 100;
-  protected static final byte[] qualName = Bytes.toBytes("q1");
-  protected static final byte[] famName = Bytes.toBytes("f");
-
-  protected static String BACKUP_ROOT_DIR = "/backupUT";
-  protected static String BACKUP_REMOTE_ROOT_DIR = "/backupUT";
-
-  protected static final String BACKUP_ZNODE = "/backup/hbase";
-  protected static final String BACKUP_SUCCEED_NODE = "complete";
-  protected static final String BACKUP_FAILED_NODE = "failed";
-
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-
-    TEST_UTIL = new HBaseTestingUtility();
-    TEST_UTIL.getConfiguration().set("hbase.procedure.regionserver.classes",
-      LogRollRegionServerProcedureManager.class.getName());
-    TEST_UTIL.getConfiguration().set("hbase.procedure.master.classes",
-      LogRollMasterProcedureManager.class.getName());
-    TEST_UTIL.getConfiguration().set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
-    TEST_UTIL.startMiniZKCluster();
-    MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
-
-    conf1 = TEST_UTIL.getConfiguration();
-    conf2 = HBaseConfiguration.create(conf1);
-    conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
-    TEST_UTIL2 = new HBaseTestingUtility(conf2);
-    TEST_UTIL2.setZkCluster(miniZK);
-    TEST_UTIL.startMiniCluster();
-    TEST_UTIL2.startMiniCluster();
-    conf1 = TEST_UTIL.getConfiguration();
-
-    TEST_UTIL.startMiniMapReduceCluster();
-    BACKUP_ROOT_DIR = TEST_UTIL.getConfiguration().get("fs.defaultFS") + "/backupUT";
-    LOG.info("ROOTDIR " + BACKUP_ROOT_DIR);
-    BACKUP_REMOTE_ROOT_DIR = TEST_UTIL2.getConfiguration().get("fs.defaultFS") + "/backupUT";
-    LOG.info("REMOTE ROOTDIR " + BACKUP_REMOTE_ROOT_DIR);
-
-    BackupClient.setConf(conf1);
-    RestoreClient.setConf(conf1);
-    createTables();
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
-    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
-    //zkw1.close();
-    TEST_UTIL2.shutdownMiniCluster();
-    TEST_UTIL.shutdownMiniCluster();
-    TEST_UTIL.shutdownMiniMapReduceCluster();
-  }
-
-  protected static void loadTable(HTable table) throws Exception {
-
-    Put p; // 100 + 1 row to t1_syncup
-    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
-      p = new Put(Bytes.toBytes("row" + i));
-      p.addColumn(famName, qualName, Bytes.toBytes("val" + i));
-      table.put(p);
-    }
-  }
-
-  protected static void createTables() throws Exception {
-
-    long tid = System.currentTimeMillis();
-    table1 = TableName.valueOf("test-" + tid);
-    HBaseAdmin ha = TEST_UTIL.getHBaseAdmin();
-    HTableDescriptor desc = new HTableDescriptor(table1);
-    HColumnDescriptor fam = new HColumnDescriptor(famName);
-    desc.addFamily(fam);
-    ha.createTable(desc);
-    Connection conn = ConnectionFactory.createConnection(conf1);    
-    HTable table = (HTable) conn.getTable(table1);
-    loadTable(table);
-    table.close();
-    table2 = TableName.valueOf("test-" + tid + 1);
-    desc = new HTableDescriptor(table2);
-    desc.addFamily(fam);
-    ha.createTable(desc);
-    table = (HTable) conn.getTable(table2);
-    loadTable(table);
-    table.close();
-    table3 = TableName.valueOf("test-" + tid + 2);
-    table = TEST_UTIL.createTable(table3, famName);
-    table.close();
-    table4 = TableName.valueOf("test-" + tid + 3);
-    table = TEST_UTIL.createTable(table4, famName);
-    table.close();
-    ha.close();
-    conn.close();
-  }
-
-  protected boolean checkSucceeded(String backupId) throws IOException
-  {
-    BackupContext status = getBackupContext(backupId);
-    if(status == null) return false;
-    return status.getFlag() == BACKUPSTATUS.COMPLETE;
-  }
-
-  protected boolean checkFailed(String backupId) throws IOException
-  {
-    BackupContext status = getBackupContext(backupId);
-    if(status == null) return false;
-    return status.getFlag() == BACKUPSTATUS.FAILED;
-  }
-
-  private BackupContext getBackupContext(String backupId) throws IOException
-  {
-    Configuration conf = BackupClient.getConf();
-    BackupSystemTable table = BackupSystemTable.getTable(conf);
-    BackupContext status =  table.readBackupStatus(backupId);
-    return status;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
deleted file mode 100644
index 8be07bc..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBoundaryTests.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class TestBackupBoundaryTests extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestBackupBoundaryTests.class);
-
-  /**
-   * Verify that full backup is created on a single empty table correctly.
-   * @throws Exception
-   */
-  @Test
-  public void testFullBackupSingleEmpty() throws Exception {
-
-    LOG.info("create full backup image on single table");
-
-    String backupId =
-        BackupClient.create("full", BACKUP_ROOT_DIR, table3.getNameAsString(), null);
-    LOG.info("Finished Backup");
-    assertTrue(checkSucceeded(backupId));
-  }
-
-  /**
-   * Verify that full backup is created on multiple empty tables correctly.
-   * @throws Exception
-   */
-  @Test
-  public void testFullBackupMultipleEmpty() throws Exception {
-    LOG.info("create full backup image on mulitple empty tables");
-    String tableset =
-        table3.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table4.getNameAsString();
-    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
-    assertTrue(checkSucceeded(backupId));
-
-  }
-
-  /**
-   * Verify that full backup fails on a single table that does not exist.
-   * @throws Exception
-   */
-  @Test(expected = RuntimeException.class)
-  public void testFullBackupSingleDNE() throws Exception {
-
-    LOG.info("test full backup fails on a single table that does not exist");
-    BackupClient.create("full", BACKUP_ROOT_DIR, "tabledne", null);
-  }
-
-  /**
-   * Verify that full backup fails on multiple tables that do not exist.
-   * @throws Exception
-   */
-  @Test(expected = RuntimeException.class)
-  public void testFullBackupMultipleDNE() throws Exception {
-
-    LOG.info("test full backup fails on multiple tables that do not exist");
-    BackupClient.create("full", BACKUP_ROOT_DIR, "table1dne,table2dne", null);
-  }
-
-  /**
-   * Verify that full backup fails on tableset containing real and fake tables.
-   * @throws Exception
-   */
-  @Test(expected = RuntimeException.class)
-  public void testFullBackupMixExistAndDNE() throws Exception {
-    LOG.info("create full backup fails on tableset containing real and fake table");
-    String tableset =
-        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + "tabledne";
-    BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
deleted file mode 100644
index 158479b..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupLogCleaner.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-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.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.google.common.collect.Iterables;
-
-@Category(LargeTests.class)
-public class TestBackupLogCleaner extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestBackupLogCleaner.class);
-
-  // implements all test cases in 1 test since incremental full backup/
-  // incremental backup has dependencies
-  @Test
-  public void testBackupLogCleaner() throws Exception {
-
-    // #1 - create full backup for all tables
-    LOG.info("create full backup image for all tables");
-    String tablesetFull =
-        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table3.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table4.getNameAsString();
-
-    BackupSystemTable systemTable = BackupSystemTable.getTable(TEST_UTIL.getConfiguration());
-    // Verify that we have no backup sessions yet
-    assertFalse(systemTable.hasBackupSessions());
-
-    List<FileStatus> walFiles = getListOfWALFiles(TEST_UTIL.getConfiguration());
-    List<String> swalFiles = convert(walFiles);
-    BackupLogCleaner cleaner = new BackupLogCleaner();
-    cleaner.setConf(TEST_UTIL.getConfiguration());
-
-    Iterable<FileStatus> deletable = cleaner.getDeletableFiles(walFiles);
-    // We can delete all files because we do not have yet recorded backup sessions
-    assertTrue(Iterables.size(deletable) == walFiles.size());
-
-    systemTable.addWALFiles(swalFiles, "backup");
-    String backupIdFull = BackupClient.create("full", BACKUP_ROOT_DIR, tablesetFull, null);
-    assertTrue(checkSucceeded(backupIdFull));
-    // Check one more time
-    deletable = cleaner.getDeletableFiles(walFiles);
-    // We can delete wal files because they were saved into hbase:backup table
-    int size = Iterables.size(deletable);
-    assertTrue(size == walFiles.size());
-
-    List<FileStatus> newWalFiles = getListOfWALFiles(TEST_UTIL.getConfiguration());
-    LOG.debug("WAL list after full backup");
-    convert(newWalFiles);
-
-    // New list of wal files is greater than the previous one,
-    // because new wal per RS have been opened after full backup
-    assertTrue(walFiles.size() < newWalFiles.size());
-    // TODO : verify that result files are not walFiles collection
-    Connection conn = ConnectionFactory.createConnection(conf1);
-    // #2 - insert some data to table
-    HTable t1 = (HTable) conn.getTable(table1);
-    Put p1;
-    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
-      p1 = new Put(Bytes.toBytes("row-t1" + i));
-      p1.addColumn(famName, qualName, Bytes.toBytes("val" + i));
-      t1.put(p1);
-    }
-
-    t1.close();
-
-    HTable t2 = (HTable) conn.getTable(table2);
-    Put p2;
-    for (int i = 0; i < 5; i++) {
-      p2 = new Put(Bytes.toBytes("row-t2" + i));
-      p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
-      t2.put(p2);
-    }
-
-    t2.close();
-
-    // #3 - incremental backup for multiple tables
-    String tablesetIncMultiple =
-        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table3.getNameAsString();
-
-    String backupIdIncMultiple =
-        BackupClient.create("incremental", BACKUP_ROOT_DIR, tablesetIncMultiple, null);
-    assertTrue(checkSucceeded(backupIdIncMultiple));
-    deletable = cleaner.getDeletableFiles(newWalFiles);
-
-    assertTrue(Iterables.size(deletable) == newWalFiles.size());
-
-    conn.close();
-
-  }
-
-  private List<String> convert(List<FileStatus> walFiles) {
-    List<String> result = new ArrayList<String>();
-    for (FileStatus fs : walFiles) {
-      LOG.debug("+++WAL: " + fs.getPath().toString());
-      result.add(fs.getPath().toString());
-    }
-    return result;
-  }
-
-  private List<FileStatus> getListOfWALFiles(Configuration c) throws IOException {
-    Path logRoot = new Path(FSUtils.getRootDir(c), HConstants.HREGION_LOGDIR_NAME);
-    FileSystem fs = FileSystem.get(c);
-    RemoteIterator<LocatedFileStatus> it = fs.listFiles(logRoot, true);
-    List<FileStatus> logFiles = new ArrayList<FileStatus>();
-    while (it.hasNext()) {
-      LocatedFileStatus lfs = it.next();
-      if (lfs.isFile() && !DefaultWALProvider.isMetaFile(lfs.getPath())) {
-        logFiles.add(lfs);
-        LOG.info(lfs);
-      }
-    }
-    return logFiles;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
deleted file mode 100644
index 7421707..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupSystemTable.java
+++ /dev/null
@@ -1,341 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS;
-import org.apache.hadoop.hbase.backup.BackupUtil.BackupCompleteData;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test cases for hbase:backup API 
- *
- */
-@Category(MediumTests.class)
-public class TestBackupSystemTable {
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  protected static Configuration conf = UTIL.getConfiguration();
-  protected static MiniHBaseCluster cluster;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    cluster = UTIL.startMiniCluster();
-
-  }
-
-  @Test
-  public void testUpdateReadDeleteBackupStatus() throws IOException {
-    BackupSystemTable table = BackupSystemTable.getTable(conf);
-    BackupContext ctx = createBackupContext();
-    table.updateBackupStatus(ctx);
-    BackupContext readCtx = table.readBackupStatus(ctx.getBackupId());
-    assertTrue(compare(ctx, readCtx));
-
-    // try fake backup id
-    readCtx = table.readBackupStatus("fake");
-
-    assertNull(readCtx);
-    // delete backup context
-    table.deleteBackupStatus(ctx.getBackupId());
-    readCtx = table.readBackupStatus(ctx.getBackupId());
-    assertNull(readCtx);
-    cleanBackupTable();
-  }
-
-  @Test
-  public void testWriteReadBackupStartCode() throws IOException {
-    BackupSystemTable table = BackupSystemTable.getTable(conf);
-    String code = "100";
-    table.writeBackupStartCode(code);
-    String readCode = table.readBackupStartCode();
-    assertEquals(code, readCode);
-    cleanBackupTable();
-  }
-
-  private void cleanBackupTable() throws IOException {
-    Admin admin = UTIL.getHBaseAdmin();
-    admin.disableTable(BackupSystemTable.getTableName());
-    admin.truncateTable(BackupSystemTable.getTableName(), true);
-    if (admin.isTableDisabled(BackupSystemTable.getTableName())) {
-      admin.enableTable(BackupSystemTable.getTableName());
-    }
-  }
-
-  @Test
-  public void testBackupHistory() throws IOException {
-    BackupSystemTable table = BackupSystemTable.getTable(conf);
-    int n = 10;
-    List<BackupContext> list = createBackupContextList(n);
-
-    // Load data
-    for (BackupContext bc : list) {
-      // Make sure we set right status
-      bc.setFlag(BACKUPSTATUS.COMPLETE);
-      table.updateBackupStatus(bc);
-    }
-
-    // Reverse list for comparison
-    Collections.reverse(list);
-    ArrayList<BackupCompleteData> history = table.getBackupHistory();
-    assertTrue(history.size() == n);
-
-    for (int i = 0; i < n; i++) {
-      BackupContext ctx = list.get(i);
-      BackupCompleteData data = history.get(i);
-      assertTrue(compare(ctx, data));
-    }
-
-    cleanBackupTable();
-
-  }
-
-  @Test
-  public void testRegionServerLastLogRollResults() throws IOException {
-    BackupSystemTable table = BackupSystemTable.getTable(conf);
-
-    String[] servers = new String[] { "server1", "server2", "server3" };
-    String[] timestamps = new String[] { "100", "102", "107" };
-
-    for (int i = 0; i < servers.length; i++) {
-      table.writeRegionServerLastLogRollResult(servers[i], timestamps[i]);
-    }
-
-    HashMap<String, String> result = table.readRegionServerLastLogRollResult();
-    assertTrue(servers.length == result.size());
-    Set<String> keys = result.keySet();
-    String[] keysAsArray = new String[keys.size()];
-    keys.toArray(keysAsArray);
-    Arrays.sort(keysAsArray);
-
-    for (int i = 0; i < keysAsArray.length; i++) {
-      assertEquals(keysAsArray[i], servers[i]);
-      String ts1 = timestamps[i];
-      String ts2 = result.get(keysAsArray[i]);
-      assertEquals(ts1, ts2);
-    }
-
-    cleanBackupTable();
-
-  }
-
-  @Test
-  public void testIncrementalBackupTableSet() throws IOException {
-    BackupSystemTable table = BackupSystemTable.getTable(conf);
-
-    TreeSet<String> tables1 = new TreeSet<String>();
-
-    tables1.add("t1");
-    tables1.add("t2");
-    tables1.add("t3");
-
-    TreeSet<String> tables2 = new TreeSet<String>();
-
-    tables2.add("t3");
-    tables2.add("t4");
-    tables2.add("t5");
-
-    table.addIncrementalBackupTableSet(tables1);
-    TreeSet<String> res1 = (TreeSet<String>) table.getIncrementalBackupTableSet();
-    assertTrue(tables1.size() == res1.size());
-    Iterator<String> desc1 = tables1.descendingIterator();
-    Iterator<String> desc2 = res1.descendingIterator();
-    while (desc1.hasNext()) {
-      assertEquals(desc1.next(), desc2.next());
-    }
-
-    table.addIncrementalBackupTableSet(tables2);
-    TreeSet<String> res2 = (TreeSet<String>) table.getIncrementalBackupTableSet();
-    assertTrue((tables2.size() + tables1.size() - 1) == res2.size());
-
-    tables1.addAll(tables2);
-
-    desc1 = tables1.descendingIterator();
-    desc2 = res2.descendingIterator();
-
-    while (desc1.hasNext()) {
-      assertEquals(desc1.next(), desc2.next());
-    }
-    cleanBackupTable();
-
-  }
-
-  @Test
-  public void testRegionServerLogTimestampMap() throws IOException {
-    BackupSystemTable table = BackupSystemTable.getTable(conf);
-
-    TreeSet<String> tables = new TreeSet<String>();
-
-    tables.add("t1");
-    tables.add("t2");
-    tables.add("t3");
-
-    HashMap<String, String> rsTimestampMap = new HashMap<String, String>();
-
-    rsTimestampMap.put("rs1", "100");
-    rsTimestampMap.put("rs2", "101");
-    rsTimestampMap.put("rs3", "103");
-
-    table.writeRegionServerLogTimestamp(tables, rsTimestampMap);
-
-    HashMap<String, HashMap<String, String>> result = table.readLogTimestampMap();
-
-    assertTrue(tables.size() == result.size());
-
-    for (String t : tables) {
-      HashMap<String, String> rstm = result.get(t);
-      assertNotNull(rstm);
-      assertEquals(rstm.get("rs1"), "100");
-      assertEquals(rstm.get("rs2"), "101");
-      assertEquals(rstm.get("rs3"), "103");
-    }
-
-    Set<String> tables1 = new TreeSet<String>();
-
-    tables1.add("t3");
-    tables1.add("t4");
-    tables1.add("t5");
-
-    HashMap<String, String> rsTimestampMap1 = new HashMap<String, String>();
-
-    rsTimestampMap1.put("rs1", "200");
-    rsTimestampMap1.put("rs2", "201");
-    rsTimestampMap1.put("rs3", "203");
-
-    table.writeRegionServerLogTimestamp(tables1, rsTimestampMap1);
-
-    result = table.readLogTimestampMap();
-
-    assertTrue(5 == result.size());
-
-    for (String t : tables) {
-      HashMap<String, String> rstm = result.get(t);
-      assertNotNull(rstm);
-      if (t.equals("t3") == false) {
-        assertEquals(rstm.get("rs1"), "100");
-        assertEquals(rstm.get("rs2"), "101");
-        assertEquals(rstm.get("rs3"), "103");
-      } else {
-        assertEquals(rstm.get("rs1"), "200");
-        assertEquals(rstm.get("rs2"), "201");
-        assertEquals(rstm.get("rs3"), "203");
-      }
-    }
-
-    for (String t : tables1) {
-      HashMap<String, String> rstm = result.get(t);
-      assertNotNull(rstm);
-      assertEquals(rstm.get("rs1"), "200");
-      assertEquals(rstm.get("rs2"), "201");
-      assertEquals(rstm.get("rs3"), "203");
-    }
-
-    cleanBackupTable();
-
-  }
-
-  @Test
-  public void testAddWALFiles() throws IOException {
-    BackupSystemTable table = BackupSystemTable.getTable(conf);
-    FileSystem fs = FileSystem.get(conf);
-    List<String> files =
-        Arrays.asList("hdfs://server/WALs/srv1,101,15555/srv1,101,15555.default.1",
-          "hdfs://server/WALs/srv2,102,16666/srv2,102,16666.default.2",
-            "hdfs://server/WALs/srv3,103,17777/srv3,103,17777.default.3");
-    String newFile = "hdfs://server/WALs/srv1,101,15555/srv1,101,15555.default.5";
-
-    table.addWALFiles(files, "backup");
-
-    assertTrue(table.checkWALFile(files.get(0)));
-    assertTrue(table.checkWALFile(files.get(1)));
-    assertTrue(table.checkWALFile(files.get(2)));
-    assertFalse(table.checkWALFile(newFile));
-
-    cleanBackupTable();
-  }
-
-  private boolean compare(BackupContext ctx, BackupCompleteData data) {
-
-    return ctx.getBackupId().equals(data.getBackupToken())
-        && ctx.getTargetRootDir().equals(data.getBackupRootPath())
-        && ctx.getType().equals(data.getType())
-        && ctx.getStartTs() == Long.parseLong(data.getStartTime())
-        && ctx.getEndTs() == Long.parseLong(data.getEndTime());
-
-  }
-
-  private boolean compare(BackupContext one, BackupContext two) {
-    return one.getBackupId().equals(two.getBackupId()) && one.getType().equals(two.getType())
-        && one.getTargetRootDir().equals(two.getTargetRootDir())
-        && one.getStartTs() == two.getStartTs() && one.getEndTs() == two.getEndTs();
-  }
-
-  private BackupContext createBackupContext() {
-
-    BackupContext ctxt =
-        new BackupContext("backup_" + System.nanoTime(), "full", new String[] { "t1", "t2", "t3" },
-          "/hbase/backup", null);
-    ctxt.setStartTs(System.currentTimeMillis());
-    ctxt.setEndTs(System.currentTimeMillis() + 1);
-    return ctxt;
-  }
-
-  private List<BackupContext> createBackupContextList(int size) {
-    List<BackupContext> list = new ArrayList<BackupContext>();
-    for (int i = 0; i < size; i++) {
-      list.add(createBackupContext());
-      try {
-        Thread.sleep(10);
-      } catch (InterruptedException e) {
-        e.printStackTrace();
-      }
-    }
-    return list;
-  }
-
-  @AfterClass
-  public static void tearDown() throws IOException {
-    if (cluster != null) cluster.shutdown();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
deleted file mode 100644
index 19caf37..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullBackup.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class TestFullBackup extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestFullBackup.class);
-
-  /**
-   * Verify that full backup is created on a single table with data correctly.
-   * @throws Exception
-   */
-  @Test
-  public void testFullBackupSingle() throws Exception {
-
-    LOG.info("test full backup on a single table with data");
-    String backupId =
-        BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
-    LOG.info("backup complete");
-    assertTrue(checkSucceeded(backupId));
-  }
-
-  /**
-   * Verify that full backup is created on multiple tables correctly.
-   * @throws Exception
-   */
-  @Test
-  public void testFullBackupMultiple() throws Exception {
-    LOG.info("create full backup image on multiple tables with data");
-    String tableset =
-        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table2.getNameAsString();
-    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
-    assertTrue(checkSucceeded(backupId));
-
-  }
-
-  /**
-   * Verify that full backup is created on all tables correctly.
-   * @throws Exception
-   */
-  @Test
-  public void testFullBackupAll() throws Exception {
-    LOG.info("create full backup image on all tables");
-    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, null, null);
-    assertTrue(checkSucceeded(backupId));
-
-  }
-
-  /**
-   * Verify that full backup is created on a table correctly using a snapshot.
-   * @throws Exception
-   */
-  //@Test
-  //public void testFullBackupUsingSnapshot() throws Exception {
-   // HBaseAdmin hba = new HBaseAdmin(conf1);
-    //String snapshot = "snapshot";
-    //hba.snapshot(snapshot, table1);
-    //LOG.info("create full backup image on a table using snapshot");
-    //String backupId =
-    //    BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(),
-    //      snapshot);
-  // }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
deleted file mode 100644
index 1262c54..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestFullRestore.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class TestFullRestore extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestFullRestore.class);
-
-  /**
-   * Verify that a single table is restored to a new table
-   * @throws Exception
-   */
-  @Test
-  public void testFullRestoreSingle() throws Exception {
-
-    LOG.info("test full restore on a single table empty table");
-    String backupId =
-        BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
-    LOG.info("backup complete");
-    assertTrue(checkSucceeded(backupId));
-
-    String[] tableset = new String[] { table1.getNameAsString() };
-    String[] tablemap = new String[] { table1_restore };
-    Path path = new Path(BACKUP_ROOT_DIR);
-    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
-      false);
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
-    assertTrue(hba.tableExists(TableName.valueOf(table1_restore)));
-    TEST_UTIL.deleteTable(TableName.valueOf(table1_restore));
-    hba.close();
-  }
-
-  /**
-   * Verify that multiple tables are restored to new tables.
-   * @throws Exception
-   */
-  @Test
-  public void testFullRestoreMultiple() throws Exception {
-    LOG.info("create full backup image on multiple tables");
-    String tableset =
-        table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table3.getNameAsString();
-    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
-    assertTrue(checkSucceeded(backupId));
-
-    String[] restore_tableset = new String[] { table2.getNameAsString(), table3.getNameAsString() };
-    String[] tablemap = new String[] { table2_restore, table3_restore };
-    Path path = new Path(BACKUP_ROOT_DIR);
-    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, 
-      restore_tableset, tablemap, false);
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
-    assertTrue(hba.tableExists(TableName.valueOf(table2_restore)));
-    assertTrue(hba.tableExists(TableName.valueOf(table3_restore)));
-    TEST_UTIL.deleteTable(TableName.valueOf(table2_restore));
-    TEST_UTIL.deleteTable(TableName.valueOf(table3_restore));
-    hba.close();
-  }
-
-  /**
-   * Verify that a single table is restored using overwrite
-   * @throws Exception
-   */
-  @Test
-  public void testFullRestoreSingleOverwrite() throws Exception {
-
-    LOG.info("test full restore on a single table empty table");
-    String backupId =
-        BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
-    LOG.info("backup complete");
-    assertTrue(checkSucceeded(backupId));
-
-    String[] tableset = new String[] { table1.getNameAsString() };
-    Path path = new Path(BACKUP_ROOT_DIR);
-    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, null,
-      true);
-  }
-
-  /**
-   * Verify that multiple tables are restored to new tables using overwrite.
-   * @throws Exception
-   */
-  @Test
-  public void testFullRestoreMultipleOverwrite() throws Exception {
-    LOG.info("create full backup image on multiple tables");
-    String tableset =
-        table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table3.getNameAsString();
-    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
-    assertTrue(checkSucceeded(backupId));
-
-    String[] restore_tableset = new String[] { table2.getNameAsString(), table3.getNameAsString() };
-    Path path = new Path(BACKUP_ROOT_DIR);
-    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, 
-      false, restore_tableset, null, true);
-  }
-
-  /**
-   * Verify that restore fails on a single table that does not exist.
-   * @throws Exception
-   */
-  @Test(expected = IOException.class)
-  public void testFullRestoreSingleDNE() throws Exception {
-
-    LOG.info("test restore fails on a single table that does not exist");
-    String backupId =
-        BackupClient.create("full", BACKUP_ROOT_DIR, table1.getNameAsString(), null);
-    LOG.info("backup complete");
-    assertTrue(checkSucceeded(backupId));
-
-    String[] tableset = new String[] { "faketable" };
-    String[] tablemap = new String[] { table1_restore };
-    Path path = new Path(BACKUP_ROOT_DIR);
-    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
-      false);
-  }
-
-  /**
-   * Verify that restore fails on multiple tables that do not exist.
-   * @throws Exception
-   */
-  @Test(expected = IOException.class)
-  public void testFullRestoreMultipleDNE() throws Exception {
-
-    LOG.info("test restore fails on multiple tables that do not exist");
-    String tableset =
-        table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table3.getNameAsString();
-    String backupId = BackupClient.create("full", BACKUP_ROOT_DIR, tableset, null);
-    assertTrue(checkSucceeded(backupId));
-
-    String[] restore_tableset = new String[] { "faketable1", "faketable2" };
-    String[] tablemap = new String[] { table2_restore, table3_restore };
-    Path path = new Path(BACKUP_ROOT_DIR);
-    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupId, false, 
-      false, restore_tableset, tablemap, false);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
deleted file mode 100644
index 5437b84..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.TableName;
-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.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.backup.HBackupFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class TestIncrementalBackup extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestIncrementalBackup.class);
-  //implement all testcases in 1 test since incremental backup/restore has dependencies
-  @Test
-  public void TestIncBackupRestore() throws Exception {
-    HBackupFileSystem hbfs;
-
-    // #1 - create full backup for all tables
-    LOG.info("create full backup image for all tables");
-    String tablesetFull =
-        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table3.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table4.getNameAsString();
-
-    String backupIdFull =
-        BackupClient.create("full", BACKUP_ROOT_DIR, tablesetFull, null);
-    assertTrue(checkSucceeded(backupIdFull));
-
-    Connection conn = ConnectionFactory.createConnection(conf1); 
-    // #2 - insert some data to table
-    HTable t1 = (HTable) conn.getTable(table1);
-    Put p1;
-    for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
-      p1 = new Put(Bytes.toBytes("row-t1" + i));
-      p1.addColumn(famName, qualName, Bytes.toBytes("val" + i));
-      t1.put(p1);
-    }
-
-    Assert.assertThat(TEST_UTIL.countRows(t1), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
-    t1.close();
-
-    HTable t2 =  (HTable) conn.getTable(table2);
-    Put p2;
-    for (int i = 0; i < 5; i++) {
-      p2 = new Put(Bytes.toBytes("row-t2" + i));
-      p2.addColumn(famName, qualName, Bytes.toBytes("val" + i));
-      t2.put(p2);
-    }
-
-    Assert.assertThat(TEST_UTIL.countRows(t2), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
-    t2.close();
-
-    // #3 - incremental backup for multiple tables
-    String tablesetIncMultiple =
-        table1.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table2.getNameAsString() + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND
-            + table3.getNameAsString();
-
-    String backupIdIncMultiple = BackupClient.create("incremental", BACKUP_ROOT_DIR, 
-      tablesetIncMultiple, null);
-    assertTrue(checkSucceeded(backupIdIncMultiple));
-
-
-    // #4 - restore full backup for all tables, without overwrite
-    String[] tablesRestoreFull =
-        new String[] { table1.getNameAsString(), table2.getNameAsString(),
-            table3.getNameAsString(), table4.getNameAsString() };
-
-    String[] tablesMapFull =
-        new String[] { table1_restore, table2_restore, table3_restore, table4_restore };
-
-    hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdFull);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupIdFull, false, false,
-      tablesRestoreFull,
-      tablesMapFull, false);
-
-    // #5.1 - check tables for full restore
-    HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
-    assertTrue(hAdmin.tableExists(TableName.valueOf(table1_restore)));
-    assertTrue(hAdmin.tableExists(TableName.valueOf(table2_restore)));
-    assertTrue(hAdmin.tableExists(TableName.valueOf(table3_restore)));
-    assertTrue(hAdmin.tableExists(TableName.valueOf(table4_restore)));
-
-    hAdmin.close();
-
-    // #5.2 - checking row count of tables for full restore
-    HTable hTable = (HTable) conn.getTable(TableName.valueOf(table1_restore));
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(TableName.valueOf(table2_restore));
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(TableName.valueOf(table3_restore));
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(TableName.valueOf(table4_restore));
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-
-    // #6 - restore incremental backup for multiple tables, with overwrite
-    String[] tablesRestoreIncMultiple =
-        new String[]
-        { table1.getNameAsString(), table2.getNameAsString(), table3.getNameAsString() };
-    String[] tablesMapIncMultiple =
-        new String[] { table1_restore, table2_restore, table3_restore };
-    hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdIncMultiple);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupIdIncMultiple, false, false,
-      tablesRestoreIncMultiple, tablesMapIncMultiple, true);
-
-    hTable = (HTable) conn.getTable(TableName.valueOf(table1_restore));
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(TableName.valueOf(table2_restore));
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(TableName.valueOf(table3_restore));
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-
-    // #7 - incremental backup for single, empty table
-
-    String tablesetIncEmpty = table4.getNameAsString();
-    String backupIdIncEmpty =
-        BackupClient.create("incremental", BACKUP_ROOT_DIR, tablesetIncEmpty, null);
-    assertTrue(checkSucceeded(backupIdIncEmpty));
-
-
-    // #8 - restore incremental backup for single empty table, with overwrite
-    String[] tablesRestoreIncEmpty = new String[] { table4.getNameAsString() };
-    String[] tablesMapIncEmpty = new String[] { table4_restore };
-    hbfs = new HBackupFileSystem(conf1, new Path(BACKUP_ROOT_DIR), backupIdIncEmpty);
-    RestoreClient.restore_stage1(hbfs, BACKUP_ROOT_DIR, backupIdIncEmpty, false, false,
-      tablesRestoreIncEmpty,
-      tablesMapIncEmpty, true);
-
-    hTable = (HTable) conn.getTable(TableName.valueOf(table4_restore));
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-    conn.close();
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
deleted file mode 100644
index 03822a2..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class TestRemoteBackup extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestRemoteBackup.class);
-
-  /**
-   * Verify that a remote full backup is created on a single table with data correctly.
-   * @throws Exception
-   */
-  @Test
-  public void testFullBackupRemote() throws Exception {
-
-    LOG.info("test remote full backup on a single table");
-
-    // String rootdir = TEST_UTIL2.getDefaultRootDirPath() + BACKUP_ROOT_DIR;
-    // LOG.info("ROOTDIR " + rootdir);
-    String backupId =
-        BackupClient.create("full", BACKUP_REMOTE_ROOT_DIR, table1.getNameAsString(), null);
-    LOG.info("backup complete");
-    assertTrue(checkSucceeded(backupId));
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
deleted file mode 100644
index e1315c5..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteRestore.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import static org.junit.Assert.assertTrue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(LargeTests.class)
-public class TestRemoteRestore extends TestBackupBase {
-
-  private static final Log LOG = LogFactory.getLog(TestRemoteRestore.class);
-
-  /**
-   * Verify that a remote restore on a single table is successful.
-   * @throws Exception
-   */
-  @Test
-  public void testFullRestoreRemote() throws Exception {
-
-    LOG.info("test remote full backup on a single table");
-    String backupId =
-        BackupClient.create("full", BACKUP_REMOTE_ROOT_DIR, table1.getNameAsString(), null);
-    LOG.info("backup complete");
-    assertTrue(checkSucceeded(backupId));
-    String[] tableset = new String[] { table1.getNameAsString() };
-    String[] tablemap = new String[] { table1_restore };
-    Path path = new Path(BACKUP_REMOTE_ROOT_DIR);
-    HBackupFileSystem hbfs = new HBackupFileSystem(conf1, path, backupId);
-    RestoreClient.restore_stage1(hbfs, BACKUP_REMOTE_ROOT_DIR, backupId, false, false, tableset,
-      tablemap, false);
-    HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
-    assertTrue(hba.tableExists(TableName.valueOf(table1_restore)));
-    TEST_UTIL.deleteTable(TableName.valueOf(table1_restore));
-    hba.close();
-  }
-
-}
\ No newline at end of file


[4/6] hbase git commit: HBASE-14030 Revert due to pending review comments

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java
deleted file mode 100644
index 14769f9..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTable.java
+++ /dev/null
@@ -1,642 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS;
-import org.apache.hadoop.hbase.backup.BackupUtil.BackupCompleteData;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-
-/**
- * This class provides 'hbase:backup' table API
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class BackupSystemTable {
-
-  private static final Log LOG = LogFactory.getLog(BackupSystemTable.class);
-  private final static String TABLE_NAMESPACE = "hbase";
-  private final static String TABLE_NAME = "backup";
-  private final static TableName tableName = TableName.valueOf(TABLE_NAMESPACE, TABLE_NAME);
-  public final static byte[] familyName = "f".getBytes();
-
-  // Connection to HBase cluster
-  private static Connection connection;
-  // Cluster configuration
-  private static Configuration config;
-  // singleton
-  private static BackupSystemTable table;
-
-  /**
-   * Get instance by a given configuration
-   * @param conf - HBase configuration
-   * @return instance of BackupSystemTable
-   * @throws IOException exception
-   */
-  public synchronized static BackupSystemTable getTable(Configuration conf) throws IOException {
-    if (connection == null) {
-      connection = ConnectionFactory.createConnection(conf);
-      config = conf;
-      // Verify hbase:system exists
-      createSystemTableIfNotExists();
-      table = new BackupSystemTable();
-    }
-    return table;
-  }
-
-  /**
-   * TODO: refactor
-   * @throws IOException exception
-   */
-  public static void close() throws IOException {
-    connection.close();
-    table = null;
-  }
-
-  /**
-   * Gets table name
-   * @return table name
-   */
-  public static TableName getTableName() {
-    return tableName;
-  }
-
-  private static void createSystemTableIfNotExists() throws IOException {
-    Admin admin = null;
-    try {
-      admin = connection.getAdmin();
-      if (admin.tableExists(tableName) == false) {
-        HTableDescriptor tableDesc = new HTableDescriptor(tableName);
-        HColumnDescriptor colDesc = new HColumnDescriptor(familyName);
-        colDesc.setMaxVersions(1);
-        int ttl =
-            config.getInt(HConstants.BACKUP_SYSTEM_TTL_KEY, HConstants.BACKUP_SYSTEM_TTL_DEFAULT);
-        colDesc.setTimeToLive(ttl);
-        tableDesc.addFamily(colDesc);
-        admin.createTable(tableDesc);
-      }
-    } catch (IOException e) {
-      LOG.error(e);
-      throw e;
-    } finally {
-      if (admin != null) {
-        admin.close();
-      }
-    }
-  }
-
-  private BackupSystemTable() {
-  }
-
-  /**
-   * Updates status (state) of a backup session in hbase:backup table
-   * @param context context
-   * @throws IOException exception
-   */
-  public void updateBackupStatus(BackupContext context) throws IOException {
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("update backup status in hbase:backup for: " + context.getBackupId()
-        + " set status=" + context.getFlag());
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Put put = BackupSystemTableHelper.createPutForBackupContext(context);
-      table.put(put);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Deletes backup status from hbase:backup table
-   * @param backupId backup id
-   * @throws IOException exception
-   */
-
-  public void deleteBackupStatus(String backupId) throws IOException {
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("delete backup status in hbase:backup for " + backupId);
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Delete del = BackupSystemTableHelper.createDeletForBackupContext(backupId);
-      table.delete(del);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Reads backup status object (instance of BackupContext) from hbase:backup table
-   * @param backupId - backupId
-   * @return Current status of backup session or null
-   */
-
-  public BackupContext readBackupStatus(String backupId) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read backup status from hbase:backup for: " + backupId);
-    }
-
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Get get = BackupSystemTableHelper.createGetForBackupContext(backupId);
-      Result res = table.get(get);
-      if(res.isEmpty()){
-        return null;
-      }
-      return BackupSystemTableHelper.resultToBackupContext(res);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Read the last backup start code (timestamp) of last successful backup. Will return null if
-   * there is no start code stored on hbase or the value is of length 0. These two cases indicate
-   * there is no successful backup completed so far.
-   * @return the timestamp of last successful backup
-   * @throws IOException exception
-   */
-  public String readBackupStartCode() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read backup start code from hbase:backup");
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Get get = BackupSystemTableHelper.createGetForStartCode();
-      Result res = table.get(get);
-      if (res.isEmpty()){
-        return null;
-      }
-      Cell cell = res.listCells().get(0);
-      byte[] val = CellUtil.cloneValue(cell);
-      if (val.length == 0){
-        return null;
-      }
-      return new String(val);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
-   * @param startCode start code
-   * @throws IOException exception
-   */
-  public void writeBackupStartCode(String startCode) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write backup start code to hbase:backup " + startCode);
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Put put = BackupSystemTableHelper.createPutForStartCode(startCode);
-      table.put(put);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Get the Region Servers log information after the last log roll from hbase:backup.
-   * @return RS log info
-   * @throws IOException exception
-   */
-  public HashMap<String, String> readRegionServerLastLogRollResult() 
-      throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read region server last roll log result to hbase:backup");
-    }
-    Table table = null;
-    ResultScanner scanner = null;
-
-    try {
-      table = connection.getTable(tableName);
-      Scan scan = BackupSystemTableHelper.createScanForReadRegionServerLastLogRollResult();
-      scan.setMaxVersions(1);
-      scanner = table.getScanner(scan);
-      Result res = null;
-      HashMap<String, String> rsTimestampMap = new HashMap<String, String>();
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        Cell cell = res.current();
-        byte[] row = CellUtil.cloneRow(cell);
-        String server =
-            BackupSystemTableHelper.getServerNameForReadRegionServerLastLogRollResult(row);
-
-        byte[] data = CellUtil.cloneValue(cell);
-        rsTimestampMap.put(server, new String(data));
-      }
-      return rsTimestampMap;
-    } finally {
-      if (scanner != null) {
-        scanner.close();
-      }
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Writes Region Server last roll log result (timestamp) to hbase:backup table
-   * @param server - Region Server name
-   * @param fileName - last log timestamp
-   * @throws IOException exception
-   */
-  public void writeRegionServerLastLogRollResult(String server, String fileName) 
-      throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write region server last roll log result to hbase:backup");
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Put put = 
-          BackupSystemTableHelper.createPutForRegionServerLastLogRollResult(server, fileName);
-      table.put(put);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Get all completed backup information (in desc order by time)
-   * @return history info of BackupCompleteData
-   * @throws IOException exception
-   */
-  public ArrayList<BackupCompleteData> getBackupHistory() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get backup history from hbase:backup");
-    }
-    Table table = null;
-    ResultScanner scanner = null;
-    ArrayList<BackupCompleteData> list = new ArrayList<BackupCompleteData>();
-    try {
-      table = connection.getTable(tableName);
-      Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
-      scan.setMaxVersions(1);
-      scanner = table.getScanner(scan);
-      Result res = null;
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        BackupContext context = BackupSystemTableHelper.cellToBackupContext(res.current());
-        if (context.getFlag() != BACKUPSTATUS.COMPLETE) {
-          continue;
-        }
-
-        BackupCompleteData history = new BackupCompleteData();
-        history.setBackupToken(context.getBackupId());
-        history.setStartTime(Long.toString(context.getStartTs()));
-        history.setEndTime(Long.toString(context.getEndTs()));
-        history.setBackupRootPath(context.getTargetRootDir());
-        history.setTableList(context.getTableListAsString());
-        history.setType(context.getType());
-        history.setBytesCopied(Long.toString(context.getTotalBytesCopied()));
-
-        if (context.fromExistingSnapshot()) {
-          history.markFromExistingSnapshot();
-        }
-        list.add(history);
-      }
-      return BackupUtil.sortHistoryListDesc(list);
-    } finally {
-      if (scanner != null) {
-        scanner.close();
-      }
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Get all backup session with a given status (in desc order by time)
-   * @param status status
-   * @return history info of backup contexts
-   * @throws IOException exception
-   */
-  public ArrayList<BackupContext> getBackupContexts(BACKUPSTATUS status) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get backup contexts from hbase:backup");
-    }
-    Table table = null;
-    ResultScanner scanner = null;
-    ArrayList<BackupContext> list = new ArrayList<BackupContext>();
-    try {
-      table = connection.getTable(tableName);
-      Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
-      scan.setMaxVersions(1);
-      scanner = table.getScanner(scan);
-      Result res = null;
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        BackupContext context = BackupSystemTableHelper.cellToBackupContext(res.current());
-        if (context.getFlag() != status){
-          continue;
-        }
-        list.add(context);
-      }
-      return list;
-    } finally {
-      if (scanner != null) {
-        scanner.close();
-      }
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Write the current timestamps for each regionserver to hbase:backup after a successful full or
-   * incremental backup. The saved timestamp is of the last log file that was backed up already.
-   * @param tables tables
-   * @param newTimestamps timestamps
-   * @throws IOException exception
-   */
-  public void writeRegionServerLogTimestamp(Set<String> tables,
-      HashMap<String, String> newTimestamps) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("write RS log ts to HBASE_BACKUP");
-    }
-    StringBuilder sb = new StringBuilder();
-    for (Map.Entry<String, String> entry : newTimestamps.entrySet()) {
-      String host = entry.getKey();
-      String timestamp = entry.getValue();
-      sb.append(host).append(BackupUtil.FIELD_SEPARATOR).append(timestamp)
-      .append(BackupUtil.RECORD_SEPARATOR);
-    }
-    String smap = sb.toString();
-    List<Put> puts = new ArrayList<Put>();
-    for (String table : tables) {
-      Put put = BackupSystemTableHelper.createPutForWriteRegionServerLogTimestamp(table, smap);
-      puts.add(put);
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      table.put(puts);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Read the timestamp for each region server log after the last successful backup. Each table has
-   * its own set of the timestamps. The info is stored for each table as a concatenated string of
-   * rs->timestapmp
-   * @return the timestamp for each region server. key: tableName value:
-   *         RegionServer,PreviousTimeStamp
-   * @throws IOException exception
-   */
-  public HashMap<String, HashMap<String, String>> readLogTimestampMap() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("read RS log ts from HBASE_BACKUP");
-    }
-
-    Table table = null;
-    ResultScanner scanner = null;
-    HashMap<String, HashMap<String, String>> tableTimestampMap =
-        new HashMap<String, HashMap<String, String>>();
-
-    try {
-      table = connection.getTable(tableName);
-      Scan scan = BackupSystemTableHelper.createScanForReadLogTimestampMap();
-      scanner = table.getScanner(scan);
-      Result res = null;
-      while ((res = scanner.next()) != null) {
-        res.advance();
-        Cell cell = res.current();
-        byte[] row = CellUtil.cloneRow(cell);
-        String tabName = BackupSystemTableHelper.getTableNameForReadLogTimestampMap(row);
-        HashMap<String, String> lastBackup = new HashMap<String, String>();
-        byte[] data = CellUtil.cloneValue(cell);
-        if (data == null) {
-          // TODO
-          throw new IOException("Data of last backup data from HBASE_BACKUP "
-              + "is empty. Create a backup first.");
-        }
-        if (data != null && data.length > 0) {
-          String s = new String(data);
-          String[] records = s.split(BackupUtil.RECORD_SEPARATOR);
-          for (String record : records) {
-            String[] flds = record.split(BackupUtil.FIELD_SEPARATOR);
-            if (flds.length != 2) {
-              throw new IOException("data from HBASE_BACKUP is corrupted: "
-                  + Arrays.toString(flds));
-            }
-            lastBackup.put(flds[0], flds[1]);
-          }
-          tableTimestampMap.put(tabName, lastBackup);
-        }
-      }
-      return tableTimestampMap;
-    } finally {
-      if (scanner != null) {
-        scanner.close();
-      }
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Return the current tables covered by incremental backup.
-   * @return set of tableNames
-   * @throws IOException exception
-   */
-  public Set<String> getIncrementalBackupTableSet() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("get incr backup table set from hbase:backup");
-    }
-    Table table = null;
-    TreeSet<String> set = new TreeSet<String>();
-
-    try {
-      table = connection.getTable(tableName);
-      Get get = BackupSystemTableHelper.createGetForIncrBackupTableSet();
-      Result res = table.get(get);
-      if (res.isEmpty()) {
-        return set;
-      }
-      List<Cell> cells = res.listCells();
-      for (Cell cell : cells) {
-        // qualifier = table name - we use table names as qualifiers
-        // TODO ns:table as qualifier?
-        set.add(new String(CellUtil.cloneQualifier(cell)));
-      }
-      return set;
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Add tables to global incremental backup set
-   * @param tables - set of tables
-   * @throws IOException exception
-   */
-  public void addIncrementalBackupTableSet(Set<String> tables) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("add incr backup table set to hbase:backup");
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Put put = BackupSystemTableHelper.createPutForIncrBackupTableSet(tables);
-      table.put(put);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Register WAL files as eligible for deletion
-   * @param files files
-   * @throws IOException exception
-   */
-  public void addWALFiles(List<String> files, String backupId) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("add WAL files to hbase:backup");
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      List<Put> puts = BackupSystemTableHelper.createPutsForAddWALFiles(files, backupId);
-      table.put(puts);
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Check if WAL file is eligible for deletion
-   * @param file file
-   * @return true, if - yes.
-   * @throws IOException exception
-   */
-  public boolean checkWALFile(String file) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Check if WAL file has been already backuped in hbase:backup");
-    }
-    Table table = null;
-    try {
-      table = connection.getTable(tableName);
-      Get get = BackupSystemTableHelper.createGetForCheckWALFile(file);
-      Result res = table.get(get);
-      if (res.isEmpty()){
-        return false;
-      }
-      return true;
-    } finally {
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-
-  /**
-   * Checks if we have at least one backup session in hbase:backup This API is used by
-   * BackupLogCleaner
-   * @return true, if - at least one session exists in hbase:backup table
-   * @throws IOException exception
-   */
-  public boolean hasBackupSessions() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("has backup sessions from hbase:backup");
-    }
-    Table table = null;
-    ResultScanner scanner = null;
-    boolean result = false;
-    try {
-      table = connection.getTable(tableName);
-      Scan scan = BackupSystemTableHelper.createScanForBackupHistory();
-      scan.setMaxVersions(1);
-      scan.setCaching(1);
-      scanner = table.getScanner(scan);
-      if (scanner.next() != null) {
-        result = true;
-      }
-      return result;
-    } finally {
-      if (scanner != null) {
-        scanner.close();
-      }
-      if (table != null) {
-        table.close();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java
deleted file mode 100644
index bf62a84..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupSystemTableHelper.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-
-
-/**
- * A collection for methods used by BackupSystemTable.
- */
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class BackupSystemTableHelper {
-
-  /**
-   * hbase:backup schema: 
-   * 1. Backup sessions rowkey= "session." + backupId; value = serialized
-   * BackupContext 
-   * 2. Backup start code rowkey = "startcode"; value = startcode 
-   * 3. Incremental backup set rowkey="incrbackupset"; value=[list of tables] 
-   * 4. Table-RS-timestamp map rowkey="trslm"+ table_name; value = map[RS-> last WAL timestamp] 
-   * 5. RS - WAL ts map rowkey="rslogts."+server; value = last WAL timestamp 
-   * 6. WALs recorded rowkey="wals."+WAL unique file name; value = NULL (value is not used)
-   */
-  private static final Log LOG = LogFactory.getLog(BackupSystemTableHelper.class);
-
-  private final static String BACKUP_CONTEXT_PREFIX = "session.";
-  private final static String START_CODE_ROW = "startcode";
-  private final static String INCR_BACKUP_SET = "incrbackupset";
-  private final static String TABLE_RS_LOG_MAP_PREFIX = "trslm.";
-  private final static String RS_LOG_TS_PREFIX = "rslogts.";
-  private final static String WALS_PREFIX = "wals.";
-
-  private final static byte[] q0 = "0".getBytes();
-  private final static byte[] EMPTY_VALUE = new byte[] {};
-
-  private BackupSystemTableHelper() {
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  /**
-   * Creates Put operation for a given backup context object
-   * @param context backup context
-   * @return put operation
-   * @throws IOException exception
-   */
-  static Put createPutForBackupContext(BackupContext context) throws IOException {
-
-    Put put = new Put((BACKUP_CONTEXT_PREFIX + context.getBackupId()).getBytes());
-    put.addColumn(BackupSystemTable.familyName, q0, context.toByteArray());
-    return put;
-  }
-
-  /**
-   * Creates Get operation for a given backup id
-   * @param backupId - backup's ID
-   * @return get operation
-   * @throws IOException exception
-   */
-  static Get createGetForBackupContext(String backupId) throws IOException {
-    Get get = new Get((BACKUP_CONTEXT_PREFIX + backupId).getBytes());
-    get.addFamily(BackupSystemTable.familyName);
-    get.setMaxVersions(1);
-    return get;
-  }
-
-  /**
-   * Creates Delete operation for a given backup id
-   * @param backupId - backup's ID
-   * @return delete operation
-   * @throws IOException exception
-   */
-  public static Delete createDeletForBackupContext(String backupId) {
-    Delete del = new Delete((BACKUP_CONTEXT_PREFIX + backupId).getBytes());
-    del.addFamily(BackupSystemTable.familyName);
-    return del;
-  }
-
-  /**
-   * Converts Result to BackupContext
-   * @param res - HBase result
-   * @return backup context instance
-   * @throws IOException exception
-   */
-  static BackupContext resultToBackupContext(Result res) throws IOException {
-    res.advance();
-    Cell cell = res.current();
-    return cellToBackupContext(cell);
-  }
-
-  /**
-   * Creates Get operation to retrieve start code from hbase:backup
-   * @return get operation
-   * @throws IOException exception
-   */
-  static Get createGetForStartCode() throws IOException {
-    Get get = new Get(START_CODE_ROW.getBytes());
-    get.addFamily(BackupSystemTable.familyName);
-    get.setMaxVersions(1);
-    return get;
-  }
-
-  /**
-   * Creates Put operation to store start code to hbase:backup
-   * @return put operation
-   * @throws IOException exception
-   */
-  static Put createPutForStartCode(String startCode) {
-    Put put = new Put(START_CODE_ROW.getBytes());
-    put.addColumn(BackupSystemTable.familyName, q0, startCode.getBytes());
-    return put;
-  }
-
-  /**
-   * Creates Get to retrieve incremental backup table set from hbase:backup
-   * @return get operation
-   * @throws IOException exception
-   */
-  static Get createGetForIncrBackupTableSet() throws IOException {
-    Get get = new Get(INCR_BACKUP_SET.getBytes());
-    get.addFamily(BackupSystemTable.familyName);
-    get.setMaxVersions(1);
-    return get;
-  }
-
-  /**
-   * Creates Put to store incremental backup table set
-   * @param tables tables
-   * @return put operation
-   */
-  static Put createPutForIncrBackupTableSet(Set<String> tables) {
-    Put put = new Put(INCR_BACKUP_SET.getBytes());
-    for (String table : tables) {
-      put.addColumn(BackupSystemTable.familyName, table.getBytes(), EMPTY_VALUE);
-    }
-    return put;
-  }
-
-  /**
-   * Creates Scan operation to load backup history
-   * @return scan operation
-   */
-  static Scan createScanForBackupHistory() {
-    Scan scan = new Scan();
-    byte[] startRow = BACKUP_CONTEXT_PREFIX.getBytes();
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.familyName);
-
-    return scan;
-  }
-
-  /**
-   * Converts cell to backup context instance.
-   * @param current - cell
-   * @return backup context instance
-   * @throws IOException exception
-   */
-  static BackupContext cellToBackupContext(Cell current) throws IOException {
-    byte[] data = CellUtil.cloneValue(current);
-    try {
-      BackupContext ctxt = BackupContext.fromByteArray(data);
-      return ctxt;
-    } catch (ClassNotFoundException e) {
-      throw new IOException(e);
-    }
-  }
-
-  /**
-   * Creates Put to write RS last roll log timestamp map
-   * @param table - table
-   * @param smap - map, containing RS:ts
-   * @return put operation
-   */
-  static Put createPutForWriteRegionServerLogTimestamp(String table, String smap) {
-    Put put = new Put((TABLE_RS_LOG_MAP_PREFIX + table).getBytes());
-    put.addColumn(BackupSystemTable.familyName, q0, smap.getBytes());
-    return put;
-  }
-
-  /**
-   * Creates Scan to load table-> { RS -> ts} map of maps
-   * @return scan operation
-   */
-  static Scan createScanForReadLogTimestampMap() {
-    Scan scan = new Scan();
-    byte[] startRow = TABLE_RS_LOG_MAP_PREFIX.getBytes();
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.familyName);
-
-    return scan;
-  }
-
-  /**
-   * Get table name from rowkey
-   * @param cloneRow rowkey
-   * @return table name
-   */
-  static String getTableNameForReadLogTimestampMap(byte[] cloneRow) {
-    int prefixSize = TABLE_RS_LOG_MAP_PREFIX.length();
-    return new String(cloneRow, prefixSize, cloneRow.length - prefixSize);
-  }
-
-  /**
-   * Creates Put to store RS last log result
-   * @param server - server name
-   * @param fileName - log roll result (timestamp)
-   * @return put operation
-   */
-  static Put createPutForRegionServerLastLogRollResult(String server, String fileName) {
-    Put put = new Put((RS_LOG_TS_PREFIX + server).getBytes());
-    put.addColumn(BackupSystemTable.familyName, q0, fileName.getBytes());
-    return put;
-  }
-
-  /**
-   * Creates Scan operation to load last RS log roll results
-   * @return scan operation
-   */
-  static Scan createScanForReadRegionServerLastLogRollResult() {
-    Scan scan = new Scan();
-    byte[] startRow = RS_LOG_TS_PREFIX.getBytes();
-    byte[] stopRow = Arrays.copyOf(startRow, startRow.length);
-    stopRow[stopRow.length - 1] = (byte) (stopRow[stopRow.length - 1] + 1);
-    scan.setStartRow(startRow);
-    scan.setStopRow(stopRow);
-    scan.addFamily(BackupSystemTable.familyName);
-
-    return scan;
-  }
-
-  /**
-   * Get server's name from rowkey
-   * @param row - rowkey
-   * @return server's name
-   */
-  static String getServerNameForReadRegionServerLastLogRollResult(byte[] row) {
-    int prefixSize = RS_LOG_TS_PREFIX.length();
-    return new String(row, prefixSize, row.length - prefixSize);
-  }
-
-  /**
-   * Creates put list for list of WAL files
-   * @param files list of WAL file paths
-   * @param backupId backup id
-   * @return put list
-   * @throws IOException exception
-   */
-  public static List<Put> createPutsForAddWALFiles(List<String> files, String backupId)
-      throws IOException {
-
-    List<Put> puts = new ArrayList<Put>();
-    for (String file : files) {
-      LOG.debug("+++ put: " + BackupUtil.getUniqueWALFileNamePart(file));
-      byte[] row = (WALS_PREFIX + BackupUtil.getUniqueWALFileNamePart(file)).getBytes();
-      Put put = new Put(row);
-      put.addColumn(BackupSystemTable.familyName, q0, backupId.getBytes());
-      puts.add(put);
-    }
-    return puts;
-  }
-
-  /**
-   * Creates Get operation for a given wal file name
-   * @param file file
-   * @return get operation
-   * @throws IOException exception
-   */
-  public static Get createGetForCheckWALFile(String file) throws IOException {
-    byte[] row = (WALS_PREFIX + BackupUtil.getUniqueWALFileNamePart(file)).getBytes();
-    Get get = new Get(row);
-    get.addFamily(BackupSystemTable.familyName);
-    get.setMaxVersions(1);
-    return get;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java
deleted file mode 100644
index ff8bd2e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupUtil.java
+++ /dev/null
@@ -1,564 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableDescriptor;
-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.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-
-/**
- * A collection for methods used by multiple classes to backup HBase tables.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class BackupUtil {
-  protected static final Log LOG = LogFactory.getLog(BackupUtil.class);
-
-  public static final String FIELD_SEPARATOR = "\001";
-  public static final String RECORD_SEPARATOR = "\002";
-  public static final String LOGNAME_SEPARATOR = ".";
-  protected static final String HDFS = "hdfs://";
-  protected static Configuration conf = null;
-
-  private BackupUtil(){
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  /**
-   * Set the configuration from a given one.
-   * @param newConf A new given configuration
-   */
-  public synchronized static void setConf(Configuration newConf) {
-    conf = newConf;
-  }
-
-  /**
-   * Get and merge Hadoop and HBase configuration.
-   * @throws IOException exception
-   */
-  protected static Configuration getConf() {
-    if (conf == null) {
-      conf = new Configuration();
-      HBaseConfiguration.merge(conf, HBaseConfiguration.create());
-    }
-    return conf;
-  }
-
-  /**
-   * Loop through the RS log timestamp map for the tables, for each RS, find the min timestamp value
-   * for the RS among the tables.
-   * @param rsLogTimestampMap timestamp map
-   * @return the min timestamp of each RS
-   */
-  protected static HashMap<String, String> getRSLogTimestampMins(
-    HashMap<String, HashMap<String, String>> rsLogTimestampMap) {
-
-    if (rsLogTimestampMap == null || rsLogTimestampMap.isEmpty()) {
-      return null;
-    }
-
-    HashMap<String, String> rsLogTimestamptMins = new HashMap<String, String>();
-    HashMap<String, HashMap<String, String>> rsLogTimestampMapByRS =
-        new HashMap<String, HashMap<String, String>>();
-
-    for (Entry<String, HashMap<String, String>> tableEntry : rsLogTimestampMap.entrySet()) {
-      String table = tableEntry.getKey();
-      HashMap<String, String> rsLogTimestamp = tableEntry.getValue();
-      for (Entry<String, String> rsEntry : rsLogTimestamp.entrySet()) {
-        String rs = rsEntry.getKey();
-        String ts = rsEntry.getValue();
-        if (!rsLogTimestampMapByRS.containsKey(rs)) {
-          rsLogTimestampMapByRS.put(rs, new HashMap<String, String>());
-          rsLogTimestampMapByRS.get(rs).put(table, ts);
-        } else {
-          rsLogTimestampMapByRS.get(rs).put(table, ts);
-        }
-      }
-    }
-
-    for (String rs : rsLogTimestampMapByRS.keySet()) {
-      rsLogTimestamptMins.put(rs, getMinValue(rsLogTimestampMapByRS.get(rs)));
-    }
-
-    return rsLogTimestamptMins;
-  }
-
-  /**
-   * Get the min value for all the Values a map.
-   * @param map map
-   * @return the min value
-   */
-  protected static String getMinValue(HashMap<String, String> map) {
-    String minTimestamp = null;
-    if (map != null) {
-      ArrayList<String> timestampList = new ArrayList<String>(map.values());
-      Collections.sort(timestampList, new Comparator<String>() {
-        @Override
-        public int compare(String s1, String s2) {
-          long l1 = Long.valueOf(s1);
-          long l2 = Long.valueOf(s2);
-          if (l1 > l2) {
-            return 1;
-          } else if (l1 < l2) {
-            return -1;
-          } else {
-            return 0;
-          }
-        }
-      });
-      // The min among all the RS log timestamps will be kept in ZK.
-      minTimestamp = timestampList.get(0);
-    }
-    return minTimestamp;
-  }
-
-  /**
-   * copy out Table RegionInfo into incremental backup image need to consider move this logic into
-   * HBackupFileSystem
-   * @param backupContext backup context
-   * @param conf configuration
-   * @throws IOException exception
-   * @throws InterruptedException exception
-   */
-  protected static void copyTableRegionInfo(BackupContext backupContext, Configuration conf)
-      throws IOException, InterruptedException {
-
-    Path rootDir = FSUtils.getRootDir(conf);
-    FileSystem fs = rootDir.getFileSystem(conf);
-
-    // for each table in the table set, copy out the table info and region info files in the correct
-    // directory structure
-    for (String table : backupContext.getTables()) {
-
-      LOG.debug("Attempting to copy table info for:" + table);
-      TableDescriptor orig =
-          FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, TableName.valueOf(table));
-
-      // write a copy of descriptor to the target directory
-      Path target = new Path(backupContext.getBackupStatus(table).getTargetDir());
-      FileSystem targetFs = target.getFileSystem(conf);
-      FSTableDescriptors descriptors =
-          new FSTableDescriptors(conf, targetFs, FSUtils.getRootDir(conf));
-      descriptors.createTableDescriptorForTableDirectory(target, orig, false);
-      LOG.debug("Finished copying tableinfo.");
-
-      HBaseAdmin hbadmin = null;
-      // TODO: optimize
-      Connection conn = null;
-      List<HRegionInfo> regions = null;
-      try {
-        conn = ConnectionFactory.createConnection(conf);
-        hbadmin = (HBaseAdmin) conn.getAdmin();
-        regions = hbadmin.getTableRegions(TableName.valueOf(table));
-      } catch (Exception e) {
-        throw new BackupException(e);
-      } finally {
-        if (hbadmin != null) {
-          hbadmin.close();
-        }
-        if(conn != null){
-          conn.close();
-        }
-      }
-
-      // For each region, write the region info to disk
-      LOG.debug("Starting to write region info for table " + table);
-      for (HRegionInfo regionInfo : regions) {
-        Path regionDir =
-            HRegion.getRegionDir(new Path(backupContext.getBackupStatus(table).getTargetDir()),
-              regionInfo);
-        regionDir =
-            new Path(backupContext.getBackupStatus(table).getTargetDir(), regionDir.getName());
-        writeRegioninfoOnFilesystem(conf, targetFs, regionDir, regionInfo);
-      }
-      LOG.debug("Finished writing region info for table " + table);
-    }
-  }
-
-  /**
-   * Write the .regioninfo file on-disk.
-   */
-  public static void writeRegioninfoOnFilesystem(final Configuration conf, final FileSystem fs,
-      final Path regionInfoDir, HRegionInfo regionInfo) throws IOException {
-    final byte[] content = regionInfo.toDelimitedByteArray();
-    Path regionInfoFile = new Path(regionInfoDir, ".regioninfo");
-    // First check to get the permissions
-    FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
-    // Write the RegionInfo file content
-    FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null);
-    try {
-      out.write(content);
-    } finally {
-      out.close();
-    }
-  }
-
-  /**
-   * TODO: verify the code
-   * @param p path
-   * @return host name
-   * @throws IOException exception
-   */
-  protected static String parseHostFromOldLog(Path p) throws IOException {
-    String n = p.getName();
-    int idx = n.lastIndexOf(LOGNAME_SEPARATOR);
-    String s = URLDecoder.decode(n.substring(0, idx), "UTF8");
-    return ServerName.parseHostname(s);
-  }
-
-  public static String parseHostNameFromLogFile(Path p) throws IOException {
-    if (isArchivedLogFile(p)) {
-      return parseHostFromOldLog(p);
-    } else {
-      return DefaultWALProvider.getServerNameFromWALDirectoryName(p).getHostname();
-    }
-  }
-
-  private static boolean isArchivedLogFile(Path p) {
-    String oldLog = Path.SEPARATOR + HConstants.HREGION_OLDLOGDIR_NAME + Path.SEPARATOR;
-    return p.toString().contains(oldLog);
-  }
-
-  /**
-   * Return WAL file name
-   * @param walFileName WAL file name
-   * @return WAL file name
-   * @throws IOException exception
-   * @throws IllegalArgumentException exception
-   */
-  public static String getUniqueWALFileNamePart(String walFileName) throws IOException {
-    return new Path(walFileName).getName();
-  }
-
-  /**
-   * Return WAL file name
-   * @param p - WAL file path
-   * @return WAL file name
-   * @throws IOException exception
-   */
-  public static String getUniqueWALFileNamePart(Path p) throws IOException {
-    return p.getName();
-  }
-
-  /**
-   * Given the log file, parse the timestamp from the file name. The timestamp is the last number.
-   * @param p a path to the log file
-   * @return the timestamp
-   * @throws IOException exception
-   */
-  protected static String getCreationTime(Path p, Configuration conf) throws IOException {
-    int idx = p.getName().lastIndexOf(LOGNAME_SEPARATOR);
-    if (idx < 0) {
-      throw new IOException("Cannot parse timestamp from path " + p);
-    }
-    String ts = p.getName().substring(idx + 1);
-    return ts;
-  }
-
-  /**
-   * Get the total length of files under the given directory recursively.
-   * @param fs The hadoop file system
-   * @param dir The target directory
-   * @return the total length of files
-   * @throws IOException exception
-   */
-  public static long getFilesLength(FileSystem fs, Path dir) throws IOException {
-    long totalLength = 0;
-    FileStatus[] files = FSUtils.listStatus(fs, dir);
-    if (files != null) {
-      for (FileStatus fileStatus : files) {
-        if (fileStatus.isDir()) {
-          totalLength += getFilesLength(fs, fileStatus.getPath());
-        } else {
-          totalLength += fileStatus.getLen();
-        }
-      }
-    }
-    return totalLength;
-  }
-
-  /**
-   * Keep the record for dependency for incremental backup and history info p.s, we may be able to
-   * merge this class into backupImage class later
-   */
-  public static class BackupCompleteData implements Comparable<BackupCompleteData> {
-    private String startTime;
-    private String endTime;
-    private String type;
-    private String backupRootPath;
-    private String tableList;
-    private String backupToken;
-    private String bytesCopied;
-    private List<String> ancestors;
-    private boolean fromExistingSnapshot = false;
-
-    public List<String> getAncestors() {
-      if (fromExistingSnapshot) {
-        return null;
-      }
-      if (this.ancestors == null) {
-        this.ancestors = new ArrayList<String>();
-      }
-      return this.ancestors;
-    }
-
-    public void addAncestor(String backupToken) {
-      this.getAncestors().add(backupToken);
-    }
-
-    public String getBytesCopied() {
-      return bytesCopied;
-    }
-
-    public void setBytesCopied(String bytesCopied) {
-      this.bytesCopied = bytesCopied;
-    }
-
-    public String getBackupToken() {
-      return backupToken;
-    }
-
-    public void setBackupToken(String backupToken) {
-      this.backupToken = backupToken;
-    }
-
-    public String getStartTime() {
-      return startTime;
-    }
-
-    public void setStartTime(String startTime) {
-      this.startTime = startTime;
-    }
-
-    public String getEndTime() {
-      return endTime;
-    }
-
-    public void setEndTime(String endTime) {
-      this.endTime = endTime;
-    }
-
-    public String getType() {
-      return type;
-    }
-
-    public void setType(String type) {
-      this.type = type;
-    }
-
-    public String getBackupRootPath() {
-      return backupRootPath;
-    }
-
-    public void setBackupRootPath(String backupRootPath) {
-      this.backupRootPath = backupRootPath;
-    }
-
-    public String getTableList() {
-      return tableList;
-    }
-
-    public void setTableList(String tableList) {
-      this.tableList = tableList;
-    }
-
-    public boolean fromExistingSnapshot() {
-      return this.fromExistingSnapshot;
-    }
-
-    public void markFromExistingSnapshot() {
-      this.fromExistingSnapshot = true;
-    }
-
-    @Override
-    public int compareTo(BackupCompleteData o) {
-      Long thisTS =
-          new Long(this.getBackupToken().substring(this.getBackupToken().lastIndexOf("_") + 1));
-      Long otherTS =
-          new Long(o.getBackupToken().substring(o.getBackupToken().lastIndexOf("_") + 1));
-      return thisTS.compareTo(otherTS);
-    }
-
-  }
-
-  /**
-   * Sort history list by start time in descending order.
-   * @param historyList history list
-   * @return sorted list of BackupCompleteData
-   */
-  public static ArrayList<BackupCompleteData> sortHistoryListDesc(
-    ArrayList<BackupCompleteData> historyList) {
-    ArrayList<BackupCompleteData> list = new ArrayList<BackupCompleteData>();
-    TreeMap<String, BackupCompleteData> map = new TreeMap<String, BackupCompleteData>();
-    for (BackupCompleteData h : historyList) {
-      map.put(h.getStartTime(), h);
-    }
-    Iterator<String> i = map.descendingKeySet().iterator();
-    while (i.hasNext()) {
-      list.add(map.get(i.next()));
-    }
-    return list;
-  }
-
-  /**
-   * Get list of all WAL files (WALs and archive)
-   * @param c - configuration
-   * @return list of WAL files
-   * @throws IOException exception
-   */
-  public static List<String> getListOfWALFiles(Configuration c) throws IOException {
-    Path rootDir = FSUtils.getRootDir(c);
-    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
-    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
-    List<String> logFiles = new ArrayList<String>();
-
-    FileSystem fs = FileSystem.get(c);
-    logFiles = getFiles(fs, logDir, logFiles, null);
-    logFiles = getFiles(fs, oldLogDir, logFiles, null);
-    return logFiles;
-  }
-
-  /**
-   * Get list of all WAL files (WALs and archive)
-   * @param c - configuration
-   * @return list of WAL files
-   * @throws IOException exception
-   */
-  public static List<String> getListOfWALFiles(Configuration c, PathFilter filter)
-      throws IOException {
-    Path rootDir = FSUtils.getRootDir(c);
-    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
-    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
-    List<String> logFiles = new ArrayList<String>();
-
-    FileSystem fs = FileSystem.get(c);
-    logFiles = getFiles(fs, logDir, logFiles, filter);
-    logFiles = getFiles(fs, oldLogDir, logFiles, filter);
-    return logFiles;
-  }
-
-  /**
-   * Get list of all old WAL files (WALs and archive)
-   * @param c - configuration
-   * @return list of WAL files
-   * @throws IOException exception
-   */
-  public static List<String> getWALFilesOlderThan(final Configuration c,
-    final HashMap<String, String> hostTimestampMap) throws IOException {
-    Path rootDir = FSUtils.getRootDir(c);
-    Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
-    Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
-    List<String> logFiles = new ArrayList<String>();
-
-    PathFilter filter = new PathFilter() {
-
-      @Override
-      public boolean accept(Path p) {
-        try {
-          if (DefaultWALProvider.isMetaFile(p)) {
-            return false;
-          }
-          String host = BackupUtil.parseHostNameFromLogFile(p);
-          String oldTimestamp = hostTimestampMap.get(host);
-          String currentLogTS = getCreationTime(p, c);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("path=" + p);
-            LOG.debug("oldTimestamp=" + oldTimestamp);
-            LOG.debug("currentLogTS=" + currentLogTS);
-          }
-          return Long.parseLong(currentLogTS) <= Long.parseLong(oldTimestamp);
-        } catch (IOException e) {
-          LOG.error(e);
-          return false;
-        }
-      }
-    };
-    FileSystem fs = FileSystem.get(c);
-    logFiles = getFiles(fs, logDir, logFiles, filter);
-    logFiles = getFiles(fs, oldLogDir, logFiles, filter);
-    return logFiles;
-  }
-
-  private static List<String> getFiles(FileSystem fs, Path rootDir, List<String> files,
-    PathFilter filter) throws FileNotFoundException, IOException {
-
-    RemoteIterator<LocatedFileStatus> it = fs.listFiles(rootDir, true);
-
-    while (it.hasNext()) {
-      LocatedFileStatus lfs = it.next();
-      if (lfs.isDirectory()) {
-        continue;
-      }
-      // apply filter
-      if (filter.accept(lfs.getPath())) {
-        files.add(lfs.getPath().toString());
-        LOG.info(lfs.getPath());
-      }
-    }
-    return files;
-  }
-
-  public static String concat(Collection<String> col, String separator) {
-    if (col.size() == 0) {
-      return "";
-    }
-    StringBuilder sb = new StringBuilder();
-    for (String s : col) {
-      sb.append(s + separator);
-    }
-    sb.deleteCharAt(sb.lastIndexOf(";"));
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/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
deleted file mode 100644
index 74411da..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HBackupFileSystem.java
+++ /dev/null
@@ -1,511 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.io.HFileLink;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * View to an on-disk Backup Image FileSytem
- * Provides the set of methods necessary to interact with the on-disk Backup Image data.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class HBackupFileSystem {
-  public static final Log LOG = LogFactory.getLog(HBackupFileSystem.class);
-
-  private final String RESTORE_TMP_PATH = "/tmp/restoreTemp";
-  private final String[] ignoreDirs = { "recovered.edits" };
-
-  private final Configuration conf;
-  private final FileSystem fs;
-  private final Path backupRootPath;
-  private final String backupId;
-
-  /**
-   * Create a view to the on-disk Backup Image. 
-   * @param conf  to use
-   * @param backupPath  to where the backup Image stored
-   * @param backupId represent backup Image
-   */
-  HBackupFileSystem(final Configuration conf, final Path backupRootPath, final String backupId)
-      throws IOException {
-    this.conf = conf;
-    this.fs = backupRootPath.getFileSystem(conf);
-    this.backupRootPath = backupRootPath;
-    this.backupId = backupId; // the backup ID for the lead backup Image
-  }
-
-  /**
-   * @param tableName is the table backuped
-   * @return {@link HTableDescriptor} saved in backup image of the table
-   */
-  protected HTableDescriptor getTableDesc(String tableName) throws FileNotFoundException,
-  IOException {
-
-    Path tableInfoPath = this.getTableInfoPath(tableName);
-    LOG.debug("tableInfoPath = " + tableInfoPath.toString());
-    SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath);
-    LOG.debug("desc = " + desc.getName());
-    SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc);
-    HTableDescriptor tableDescriptor = manifest.getTableDescriptor();
-    /*
-     * for HBase 0.96 or 0.98 HTableDescriptor tableDescriptor =
-     * FSTableDescriptors.getTableDescriptorFromFs(fs, tableInfoPath);
-     */
-    if (!tableDescriptor.getNameAsString().equals(tableName)) {
-      LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: "
-          + tableInfoPath.toString());
-      LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString());
-    }
-    return tableDescriptor;
-  }
-
-  /**
-   * 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"
-   * @param backupRootDir backup root directory
-   * @param backupId  backup id
-   * @param table table name
-   * @return backupPath String for the particular table
-   */
-  protected static String getTableBackupDir(String backupRootDir, String backupId, String table) {
-    TableName tableName = TableName.valueOf(table);
-    return backupRootDir + File.separator + tableName.getNamespaceAsString() + File.separator
-        + tableName.getQualifierAsString() + File.separator + backupId;
-  }
-
-  /**
-   * 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"
-   * @param tableN table name
-   * @return backupPath for the particular table
-   */
-  protected Path getTableBackupPath(String tableN) {
-    TableName tableName = TableName.valueOf(tableN);
-    return new Path(this.backupRootPath, tableName.getNamespaceAsString() + File.separator
-      + tableName.getQualifierAsString() + File.separator + backupId);
-  }
-
-  /**
-   * return value represent path for:
-   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/.hbase-snapshot"
-   * @param tableName table name
-   * @return path for snapshot
-   */
-  protected Path getTableSnapshotPath(String tableName) {
-    return new Path(this.getTableBackupPath(tableName), HConstants.SNAPSHOT_DIR_NAME);
-  }
-
-  /**
-   * return value represent path for:
-   * "..../default/t1_dn/backup_1396650096738/.hbase-snapshot/snapshot_1396650097621_default_t1_dn"
-   * this path contains .snapshotinfo, .tabledesc (0.96 and 0.98) this path contains .snapshotinfo,
-   * .data.manifest (trunk)
-   * @param tableName table name
-   * @return path to table info
-   * @throws FileNotFoundException exception
-   * @throws IOException exception
-   */
-  protected Path getTableInfoPath(String tableName) throws FileNotFoundException, IOException {
-
-    Path tableSnapShotPath = this.getTableSnapshotPath(tableName);
-    Path tableInfoPath = null;
-
-    // can't build the path directly as the timestamp values are different
-    FileStatus[] snapshots = fs.listStatus(tableSnapShotPath);
-    for (FileStatus snapshot : snapshots) {
-      tableInfoPath = snapshot.getPath();
-      // SnapshotManifest.DATA_MANIFEST_NAME = "data.manifest";
-      if (tableInfoPath.getName().endsWith("data.manifest")) {
-        LOG.debug("find Snapshot Manifest");
-        break;
-      }
-    }
-    return tableInfoPath;
-  }
-
-  /**
-   * return value represent path for:
-   * ".../user/biadmin/backup1/default/t1_dn/backup_1396650096738/archive/data/default/t1_dn"
-   * @param tabelName table name
-   * @return path to table archive
-   * @throws IOException exception
-   */
-  protected Path getTableArchivePath(String tableName) throws IOException {
-    Path baseDir = new Path(getTableBackupPath(tableName), HConstants.HFILE_ARCHIVE_DIRECTORY);
-    Path dataDir = new Path(baseDir, HConstants.BASE_NAMESPACE_DIR);
-    Path archivePath = new Path(dataDir, TableName.valueOf(tableName).getNamespaceAsString());
-    Path tableArchivePath =
-        new Path(archivePath, TableName.valueOf(tableName).getQualifierAsString());
-    if (!fs.exists(tableArchivePath) || !fs.getFileStatus(tableArchivePath).isDirectory()) {
-      LOG.debug("Folder tableArchivePath: " + tableArchivePath.toString() + " does not exists");
-      tableArchivePath = null; // empty table has no archive
-    }
-    return tableArchivePath;
-  }
-
-  /**
-   * Given the backup root dir and the backup id, return the log file location for an incremental
-   * backup.
-   * @param backupRootDir backup root directory
-   * @param backupId backup id
-   * @return logBackupDir: ".../user/biadmin/backup1/WALs/backup_1396650096738"
-   */
-  protected static String getLogBackupDir(String backupRootDir, String backupId) {
-    return backupRootDir + File.separator + HConstants.HREGION_LOGDIR_NAME + File.separator
-        + backupId;
-  }
-
-  protected static Path getLogBackupPath(String backupRootDir, String backupId) {
-    return new Path(getLogBackupDir(backupRootDir, backupId));
-  }
-
-  private Path getManifestPath(String tableName) throws IOException {
-    Path manifestPath = new Path(getTableBackupPath(tableName), BackupManifest.FILE_NAME);
-
-    LOG.debug("Looking for " + manifestPath.toString());
-    if (!fs.exists(manifestPath)) {
-      // check log dir for incremental backup case
-      manifestPath =
-          new Path(getLogBackupDir(this.backupRootPath.toString(), this.backupId) + File.separator
-            + BackupManifest.FILE_NAME);
-      LOG.debug("Looking for " + manifestPath.toString());
-      if (!fs.exists(manifestPath)) {
-        String errorMsg =
-            "Could not find backup manifest for " + backupId + " in " + backupRootPath.toString();
-        throw new IOException(errorMsg);
-      }
-    }
-    return manifestPath;
-  }
-
-  protected BackupManifest getManifest(String tableName) throws IOException {
-    BackupManifest manifest = new BackupManifest(conf, this.getManifestPath(tableName));
-    return manifest;
-  }
-
-  /**
-   * Gets region list
-   * @param tableName table name
-   * @return RegionList region list
-   * @throws FileNotFoundException exception
-   * @throws IOException exception
-   */
-
-  protected ArrayList<Path> getRegionList(String tableName) throws FileNotFoundException,
-  IOException {
-    Path tableArchivePath = this.getTableArchivePath(tableName);
-    ArrayList<Path> regionDirList = new ArrayList<Path>();
-    FileStatus[] children = fs.listStatus(tableArchivePath);
-    for (FileStatus childStatus : children) {
-      // here child refer to each region(Name)
-      Path child = childStatus.getPath();
-      regionDirList.add(child);
-    }
-    return regionDirList;
-  }
-
-  /**
-   * Gets region list
-   * @param tableArchivePath table archive path
-   * @return RegionList region list
-   * @throws FileNotFoundException exception
-   * @throws IOException exception
-   */
-  protected ArrayList<Path> getRegionList(Path tableArchivePath) throws FileNotFoundException,
-  IOException {
-    ArrayList<Path> regionDirList = new ArrayList<Path>();
-    FileStatus[] children = fs.listStatus(tableArchivePath);
-    for (FileStatus childStatus : children) {
-      // here child refer to each region(Name)
-      Path child = childStatus.getPath();
-      regionDirList.add(child);
-    }
-    return regionDirList;
-  }
-
-  /**
-   * Counts the number of files in all subdirectories of an HBase tables, i.e. HFiles. And finds the
-   * maximum number of files in one HBase table.
-   * @param tableArchivePath archive path
-   * @return the maximum number of files found in 1 HBase table
-   * @throws IOException exception
-   */
-  protected int getMaxNumberOfFilesInSubDir(Path tableArchivePath) throws IOException {
-    int result = 1;
-    ArrayList<Path> regionPathList = this.getRegionList(tableArchivePath);
-    // tableArchivePath = this.getTableArchivePath(tableName);
-
-    if (regionPathList == null || regionPathList.size() == 0) {
-      throw new IllegalStateException("Cannot restore hbase table because directory '"
-          + tableArchivePath + "' is not a directory.");
-    }
-
-    for (Path regionPath : regionPathList) {
-      result = Math.max(result, getNumberOfFilesInDir(regionPath));
-    }
-    return result;
-  }
-
-  /**
-   * Counts the number of files in all subdirectories of an HBase table, i.e. HFiles.
-   * @param regionPath Path to an HBase table directory
-   * @return the number of files all directories
-   * @throws IOException exception
-   */
-  protected int getNumberOfFilesInDir(Path regionPath) throws IOException {
-    int result = 0;
-
-    if (!fs.exists(regionPath) || !fs.getFileStatus(regionPath).isDirectory()) {
-      throw new IllegalStateException("Cannot restore hbase table because directory '"
-          + regionPath.toString() + "' is not a directory.");
-    }
-
-    FileStatus[] tableDirContent = fs.listStatus(regionPath);
-    for (FileStatus subDirStatus : tableDirContent) {
-      FileStatus[] colFamilies = fs.listStatus(subDirStatus.getPath());
-      for (FileStatus colFamilyStatus : colFamilies) {
-        FileStatus[] colFamilyContent = fs.listStatus(colFamilyStatus.getPath());
-        result += colFamilyContent.length;
-      }
-    }
-    return result;
-  }
-
-  /**
-   * Duplicate the backup image if it's on local cluster
-   * @see HStore#bulkLoadHFile(String, long)
-   * @see HRegionFileSystem#bulkLoadStoreFile(String familyName, Path srcPath, long seqNum)
-   * @param tableArchivePath archive path
-   * @return the new tableArchivePath 
-   * @throws IOException exception
-   */
-  protected Path checkLocalAndBackup(Path tableArchivePath) throws IOException {
-    // Move the file if it's on local cluster
-    boolean isCopyNeeded = false;
-
-    FileSystem srcFs = tableArchivePath.getFileSystem(conf);
-    FileSystem desFs = FileSystem.get(conf);
-    if (tableArchivePath.getName().startsWith("/")) {
-      isCopyNeeded = true;
-    } else {
-      // This should match what is done in @see HRegionFileSystem#bulkLoadStoreFile(String, Path,
-      // long)
-      if (srcFs.getUri().equals(desFs.getUri())) {
-        LOG.debug("cluster hold the backup image: " + srcFs.getUri() + "; local cluster node: "
-            + desFs.getUri());
-        isCopyNeeded = true;
-      }
-    }
-    if (isCopyNeeded) {
-      LOG.debug("File " + tableArchivePath + " on local cluster, back it up before restore");
-      Path tmpPath = new Path(RESTORE_TMP_PATH);
-      if (desFs.exists(tmpPath)) {
-        try {
-          desFs.delete(tmpPath, true);
-        } catch (IOException e) {
-          LOG.debug("Failed to delete path: " + tmpPath
-            + ", need to check whether restore target DFS cluster is healthy");
-        }
-      }
-      FileUtil.copy(srcFs, tableArchivePath, desFs, tmpPath, false, conf);
-      LOG.debug("Copied to temporary path on local cluster: " + tmpPath);
-      tableArchivePath = tmpPath;
-    }
-    return tableArchivePath;
-  }
-
-  /**
-   * Calculate region boundaries and add all the column families to the table descriptor
-   * @param regionDirList region dir list
-   * @return a set of keys to store the boundaries
-   */
-  protected byte[][] generateBoundaryKeys(ArrayList<Path> regionDirList)
-      throws FileNotFoundException, IOException {
-    TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
-    // Build a set of keys to store the boundaries
-    byte[][] keys = null;
-    // calculate region boundaries and add all the column families to the table descriptor
-    for (Path regionDir : regionDirList) {
-      LOG.debug("Parsing region dir: " + regionDir);
-      Path hfofDir = regionDir;
-
-      if (!fs.exists(hfofDir)) {
-        LOG.warn("HFileOutputFormat dir " + hfofDir + " not found");
-      }
-
-      FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
-      if (familyDirStatuses == null) {
-        throw new IOException("No families found in " + hfofDir);
-      }
-
-      for (FileStatus stat : familyDirStatuses) {
-        if (!stat.isDirectory()) {
-          LOG.warn("Skipping non-directory " + stat.getPath());
-          continue;
-        }
-        boolean isIgnore = false;
-        String pathName = stat.getPath().getName();
-        for (String ignore : ignoreDirs) {
-          if (pathName.contains(ignore)) {
-            LOG.warn("Skipping non-family directory" + pathName);
-            isIgnore = true;
-            break;
-          }
-        }
-        if (isIgnore) {
-          continue;
-        }
-        Path familyDir = stat.getPath();
-        LOG.debug("Parsing family dir [" + familyDir.toString() + " in region [" + regionDir + "]");
-        // Skip _logs, etc
-        if (familyDir.getName().startsWith("_") || familyDir.getName().startsWith(".")) {
-          continue;
-        }
-
-        // start to parse hfile inside one family dir
-        Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
-        for (Path hfile : hfiles) {
-          if (hfile.getName().startsWith("_") || hfile.getName().startsWith(".")
-              || StoreFileInfo.isReference(hfile.getName())
-              || HFileLink.isHFileLink(hfile.getName())) {
-            continue;
-          }
-          HFile.Reader reader = HFile.createReader(fs, hfile, new CacheConfig(conf), conf);
-          final byte[] first, last;
-          try {
-            reader.loadFileInfo();
-            first = reader.getFirstRowKey();
-            last = reader.getLastRowKey();
-            LOG.debug("Trying to figure out region boundaries hfile=" + hfile + " first="
-                + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));
-
-            // To eventually infer start key-end key boundaries
-            Integer value = map.containsKey(first) ? (Integer) map.get(first) : 0;
-            map.put(first, value + 1);
-            value = map.containsKey(last) ? (Integer) map.get(last) : 0;
-            map.put(last, value - 1);
-          } finally {
-            reader.close();
-          }
-        }
-      }
-    }
-    keys = LoadIncrementalHFiles.inferBoundaries(map);
-    return keys;
-  }
-
-  /**
-   * Check whether the backup path exist
-   * @param backupStr backup
-   * @param conf configuration
-   * @return Yes if path exists
-   * @throws IOException exception
-   */
-  protected static boolean checkPathExist(String backupStr, Configuration conf) 
-    throws IOException {
-    boolean isExist = false;
-    Path backupPath = new Path(backupStr);
-    FileSystem fileSys = backupPath.getFileSystem(conf);
-    String targetFsScheme = fileSys.getUri().getScheme();
-    LOG.debug("Schema of given url: " + backupStr + " is: " + targetFsScheme);
-    if (fileSys.exists(backupPath)) {
-      isExist = true;
-    }
-    return isExist;
-  }
-
-  /**
-   * Check whether the backup image path and there is manifest file in the path.
-   * @param backupManifestMap If all the manifests are found, then they are put into this map
-   * @param tableArray the tables involved
-   * @throws IOException exception
-   */
-  protected void checkImageManifestExist(HashMap<String, BackupManifest> backupManifestMap,
-      String[] tableArray) throws IOException {
-
-    try {
-      for (String tableName : tableArray) {
-        BackupManifest manifest = this.getManifest(tableName);
-        backupManifestMap.put(tableName, manifest);
-      }
-    } catch (IOException e) {
-      String expMsg = e.getMessage();
-      if (expMsg.contains("No FileSystem for scheme")) {
-        if (expMsg.contains("gpfs")) {
-          LOG.error("Please change to use webhdfs url when "
-              + "the backup image to restore locates on gpfs cluster");
-        } else {
-          LOG.error("Unsupported filesystem scheme found in the backup target url, "
-              + "please check the url to make sure no typo in it");
-        }
-        throw e;
-      } else if (expMsg.contains("no authority supported")) {
-        LOG.error("Please change to use webhdfs url when "
-            + "the backup image to restore locates on gpfs cluster");
-        throw e;
-      } else {
-        LOG.error(expMsg);
-        throw e;
-      }
-    }
-  }
-
-  public static String join(String[] names) {
-    StringBuilder sb = new StringBuilder();
-    String sep = BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND;
-    for (String s : names) {
-      sb.append(sep).append(s);
-    }
-    return sb.toString();
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java
deleted file mode 100644
index e91857f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalBackupManager.java
+++ /dev/null
@@ -1,269 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
-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.FSUtils;
-import org.apache.hadoop.hbase.wal.DefaultWALProvider;
-
-/**
- * After a full backup was created, the incremental backup will only store the changes made
- * after the last full or incremental backup.
- *
- * Creating the backup copies the logfiles in .logs and .oldlogs since the last backup timestamp.
- *
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class IncrementalBackupManager {
-  // parent manager
-  private BackupManager backupManager;
-
-  public static final Log LOG = LogFactory.getLog(IncrementalBackupManager.class);
-
-  public IncrementalBackupManager(BackupManager bm) {
-    this.backupManager = bm;
-  }
-
-  /**
-   * Obtain the list of logs that need to be copied out for this incremental backup. The list is set
-   * in BackupContext.
-   * @param backupContext backup context
-   * @return The new HashMap of RS log timestamps after the log roll for this incremental backup.
-   * @throws IOException exception
-   */
-  public HashMap<String, String> getIncrBackupLogFileList(BackupContext backupContext)
-      throws IOException {
-    List<String> logList;
-    HashMap<String, String> newTimestamps;
-    HashMap<String, String> previousTimestampMins;
-
-    Configuration conf = BackupUtil.getConf();
-    String savedStartCode = backupManager.readBackupStartCode();
-
-    // key: tableName
-    // value: <RegionServer,PreviousTimeStamp>
-    HashMap<String, HashMap<String, String>> previousTimestampMap =
-        backupManager.readLogTimestampMap();
-
-    previousTimestampMins = BackupUtil.getRSLogTimestampMins(previousTimestampMap);
-
-    LOG.debug("StartCode " + savedStartCode + "for backupID " + backupContext.getBackupId());
-    LOG.debug("Timestamps " + previousTimestampMap);
-    // get all new log files from .logs and .oldlogs after last TS and before new timestamp
-    if (savedStartCode == null || 
-        previousTimestampMins == null || 
-          previousTimestampMins.isEmpty()) {
-      throw new IOException("Cannot read any previous back up timestamps from hbase:backup. "
-          + "In order to create an incremental backup, at least one full backup is needed.");
-    }
-
-    HBaseAdmin hbadmin = null;
-    Connection conn = null;
-    try {
-      LOG.info("Execute roll log procedure for incremental backup ...");
-      conn = ConnectionFactory.createConnection(conf);
-      hbadmin = (HBaseAdmin) conn.getAdmin();
-      hbadmin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
-        LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, new HashMap<String, String>());
-    } finally {
-      if (hbadmin != null) {
-        hbadmin.close();
-      }
-      if(conn != null){
-        conn.close();
-      }
-    }
-
-    newTimestamps = backupManager.readRegionServerLastLogRollResult();
-
-    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf, savedStartCode);
-
-    backupContext.setIncrBackupFileList(logList);
-
-    return newTimestamps;
-  }
-
-  /**
-   * For each region server: get all log files newer than the last timestamps but not newer than the
-   * newest timestamps.
-   * @param olderTimestamps the timestamp for each region server of the last backup.
-   * @param newestTimestamps the timestamp for each region server that the backup should lead to.
-   * @param conf the Hadoop and Hbase configuration
-   * @param savedStartCode the startcode (timestamp) of last successful backup.
-   * @return a list of log files to be backed up
-   * @throws IOException exception
-   */
-  private List<String> getLogFilesForNewBackup(HashMap<String, String> olderTimestamps,
-    HashMap<String, String> newestTimestamps, Configuration conf, String savedStartCode)
-        throws IOException {
-    LOG.debug("In getLogFilesForNewBackup()\n" + "olderTimestamps: " + olderTimestamps
-      + "\n newestTimestamps: " + newestTimestamps);
-    Path rootdir = FSUtils.getRootDir(conf);
-    Path logDir = new Path(rootdir, HConstants.HREGION_LOGDIR_NAME);
-    Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
-    FileSystem fs = rootdir.getFileSystem(conf);
-    NewestLogFilter pathFilter = new NewestLogFilter(conf);
-
-    List<String> resultLogFiles = new ArrayList<String>();
-    List<String> newestLogs = new ArrayList<String>();
-
-    /*
-     * The old region servers and timestamps info we kept in hbase:backup may be out of sync if new
-     * region server is added or existing one lost. We'll deal with it here when processing the
-     * logs. If data in hbase:backup has more hosts, just ignore it. If the .logs directory includes
-     * more hosts, the additional hosts will not have old timestamps to compare with. We'll just use
-     * all the logs in that directory. We always write up-to-date region server and timestamp info
-     * to hbase:backup at the end of successful backup.
-     */
-
-    FileStatus[] rss;
-    Path p;
-    String host;
-    String oldTimeStamp;
-    String currentLogFile;
-    String currentLogTS;
-
-    // Get the files in .logs.
-    rss = fs.listStatus(logDir);
-    for (FileStatus rs : rss) {
-      p = rs.getPath();
-      host = DefaultWALProvider.getServerNameFromWALDirectoryName(p).getHostname();
-      FileStatus[] logs;
-      oldTimeStamp = olderTimestamps.get(host);
-      // It is possible that there is no old timestamp in hbase:backup for this host if
-      // this region server is newly added after our last backup.
-      if (oldTimeStamp == null) {
-        logs = fs.listStatus(p);
-      } else {
-        pathFilter.setLastBackupTS(oldTimeStamp);
-        logs = fs.listStatus(p, pathFilter);
-      }
-      for (FileStatus log : logs) {
-        LOG.debug("currentLogFile: " + log.getPath().toString());
-        if (DefaultWALProvider.isMetaFile(log.getPath())) {
-          LOG.debug("Skip hbase:meta log file: " + log.getPath().getName());
-          continue;
-        }
-        currentLogFile = log.getPath().toString();
-        resultLogFiles.add(currentLogFile);
-        currentLogTS = BackupUtil.getCreationTime(log.getPath(), conf);
-        // newestTimestamps is up-to-date with the current list of hosts
-        // so newestTimestamps.get(host) will not be null.
-        if (Long.valueOf(currentLogTS) > Long.valueOf(newestTimestamps.get(host))) {
-          newestLogs.add(currentLogFile);
-        }
-      }
-    }
-
-    // Include the .oldlogs files too.
-    FileStatus[] oldlogs = fs.listStatus(oldLogDir);
-    for (FileStatus oldlog : oldlogs) {
-      p = oldlog.getPath();
-      currentLogFile = p.toString();
-      if (DefaultWALProvider.isMetaFile(p)) {
-        LOG.debug("Skip .meta log file: " + currentLogFile);
-        continue;
-      }
-      host = BackupUtil.parseHostFromOldLog(p);
-      currentLogTS = BackupUtil.getCreationTime(p, conf);
-      oldTimeStamp = olderTimestamps.get(host);
-      /*
-       * It is possible that there is no old timestamp in hbase:backup for this host. At the time of
-       * our last backup operation, this rs did not exist. The reason can be one of the two: 1. The
-       * rs already left/crashed. Its logs were moved to .oldlogs. 2. The rs was added after our
-       * last backup.
-       */
-      if (oldTimeStamp == null) {
-        if (Long.valueOf(currentLogTS) < Long.valueOf(savedStartCode)) {
-          // This log file is really old, its region server was before our last backup.
-          continue;
-        } else {
-          resultLogFiles.add(currentLogFile);
-        }
-      } else if (Long.valueOf(currentLogTS) > Long.valueOf(oldTimeStamp)) {
-        resultLogFiles.add(currentLogFile);
-      }
-
-      LOG.debug("resultLogFiles before removal of newestLogs: " + resultLogFiles);
-      // It is possible that a host in .oldlogs is an obsolete region server
-      // so newestTimestamps.get(host) here can be null.
-      // Even if these logs belong to a obsolete region server, we still need
-      // to include they to avoid loss of edits for backup.
-      String newTimestamp = newestTimestamps.get(host);
-      if (newTimestamp != null && Long.valueOf(currentLogTS) > Long.valueOf(newTimestamp)) {
-        newestLogs.add(currentLogFile);
-      }
-    }
-    LOG.debug("newestLogs: " + newestLogs);
-    // remove newest log per host because they are still in use
-    resultLogFiles.removeAll(newestLogs);
-    LOG.debug("resultLogFiles after removal of newestLogs: " + resultLogFiles);
-    return resultLogFiles;
-  }
-
-  class NewestLogFilter implements PathFilter {
-    private String lastBackupTS = "0";
-    final private Configuration conf;
-
-    public NewestLogFilter(Configuration conf) {
-      this.conf = conf;
-    }
-
-    protected void setLastBackupTS(String ts) {
-      this.lastBackupTS = ts;
-    }
-
-    @Override
-    public boolean accept(Path path) {
-      // skip meta table log -- ts.meta file
-      if (DefaultWALProvider.isMetaFile(path)) {
-        LOG.debug("Skip .meta log file: " + path.getName());
-        return false;
-      }
-      String timestamp;
-      try {
-        timestamp = BackupUtil.getCreationTime(path, conf);
-        return Long.valueOf(timestamp) > Long.valueOf(lastBackupTS);
-      } catch (IOException e) {
-        LOG.warn("Cannot read timestamp of log file " + path);
-        return false;
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java
deleted file mode 100644
index 72e4879..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/IncrementalRestoreService.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public interface IncrementalRestoreService extends Configurable{
-
-  public void run(String logDirectory, String[] fromTables, String[] toTables)
-    throws IOException;
-}


[6/6] hbase git commit: HBASE-14030 Revert due to pending review comments

Posted by te...@apache.org.
HBASE-14030 Revert due to pending review comments


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

Branch: refs/heads/master
Commit: 449fb81286f405cd7b373499c5db65ea61f9ffab
Parents: de69f0d
Author: tedyu <yu...@gmail.com>
Authored: Mon Dec 28 01:08:02 2015 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Mon Dec 28 01:08:02 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, 341 insertions(+), 9389 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/bin/hbase
----------------------------------------------------------------------
diff --git a/bin/hbase b/bin/hbase
index 9fd903a..5064451 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -99,8 +99,6 @@ 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
@@ -305,10 +303,6 @@ 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/449fb812/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 0f572e6..6fafad3 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,24 +70,27 @@ 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 */
@@ -101,16 +104,19 @@ 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";
@@ -125,10 +131,12 @@ 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;
@@ -166,18 +174,21 @@ 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;
@@ -197,15 +208,19 @@ 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;
@@ -229,19 +244,21 @@ 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";
@@ -276,7 +293,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;
@@ -299,8 +316,9 @@ 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";
 
@@ -308,7 +326,8 @@ 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;
@@ -324,24 +343,25 @@ 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
@@ -349,12 +369,14 @@ 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;
@@ -390,12 +412,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";
@@ -403,52 +425,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");
@@ -456,28 +478,32 @@ 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;
 
@@ -486,24 +512,25 @@ 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.
@@ -511,8 +538,9 @@ 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;
 
@@ -524,12 +552,17 @@ 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'.
@@ -539,7 +572,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;
 
   /**
@@ -554,10 +587,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";
@@ -565,38 +598,45 @@ 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.
@@ -613,22 +653,27 @@ 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";
 
@@ -648,7 +693,8 @@ 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";
 
@@ -658,7 +704,8 @@ 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";
 
@@ -668,8 +715,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";
 
@@ -679,9 +726,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";
 
@@ -701,9 +748,10 @@ 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";
 
@@ -870,13 +918,18 @@ 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";
 
@@ -891,15 +944,17 @@ 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;
@@ -922,16 +977,19 @@ 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";
@@ -939,14 +997,16 @@ 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";
 
@@ -957,16 +1017,17 @@ 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;
@@ -981,8 +1042,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";
@@ -998,42 +1059,49 @@ 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 =
@@ -1066,7 +1134,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";
@@ -1098,7 +1166,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";
@@ -1112,18 +1180,17 @@ 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";
 
@@ -1136,25 +1203,26 @@ 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;
 
@@ -1166,11 +1234,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";
@@ -1195,15 +1263,6 @@ 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/449fb812/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 807b021..26aad71 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -394,11 +394,6 @@
        <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>
@@ -412,11 +407,6 @@
       <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/449fb812/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
deleted file mode 100644
index 28cf627..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupClient.java
+++ /dev/null
@@ -1,338 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-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/449fb812/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
deleted file mode 100644
index c7aff15..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCommands.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-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/449fb812/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
deleted file mode 100644
index e8d6444..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupContext.java
+++ /dev/null
@@ -1,318 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-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/449fb812/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
deleted file mode 100644
index 223746d..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupCopyService.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-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/449fb812/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
deleted file mode 100644
index fe0d0e2..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupException.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-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;
-  }
-
-}


[5/6] hbase git commit: HBASE-14030 Revert due to pending review comments

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java
deleted file mode 100644
index f764f18..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupHandler.java
+++ /dev/null
@@ -1,744 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.concurrent.Callable;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.BackupManifest.BackupImage;
-import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-
-/**
- * A Handler to carry the operations of backup progress
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BackupHandler implements Callable<Object> {
-  private static final Log LOG = LogFactory.getLog(BackupHandler.class);
-
-  // backup phase
-  // for overall backup (for table list, some table may go online, while some may go offline)
-  protected static enum BACKUPPHASE {
-    REQUEST, SNAPSHOT, PREPARE_INCREMENTAL, SNAPSHOTCOPY, INCREMENTAL_COPY, STORE_MANIFEST;
-  }
-
-  // backup status flag
-  protected static enum BACKUPSTATUS {
-    WAITING, ONGOING, COMPLETE, FAILED, CANCELLED;
-  }
-
-  protected BackupContext backupContext;
-  private BackupManager backupManager;
-  private Configuration conf;
-
-  public BackupHandler(BackupContext backupContext, 
-      BackupManager backupManager, Configuration conf) {
-    this.backupContext = backupContext;
-    this.backupManager = backupManager;
-    this.conf = conf;
-  }
-
-  public BackupContext getBackupContext() {
-    return backupContext;
-  }
-
-  @Override
-  public Object call() throws Exception {
-    try {
-      // overall backup begin
-      this.beginBackup(backupContext);
-      HashMap<String, String> newTimestamps = null;
-      boolean fromExistingSnapshot = false; // supported by future jira
-      // handle full or incremental backup for table or table list
-      if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
-        String savedStartCode = null;
-        HBaseAdmin hbadmin = null;
-        Connection conn = null;
-        boolean firstBackup = false;
-        // do snapshot for full table backup, if backing up from existing snapshot, then skip the
-        // step of taking snapshot
-        if (fromExistingSnapshot) {
-          LOG.error("Backup from existing snapshot, so skip the snapshot step. ");
-          LOG.error("This feature will be supported by a future jira");
-          throw new RuntimeException("Backup from existing snapshot is not supported");
-        } else {
-          try {
-            savedStartCode = backupManager.readBackupStartCode();
-            firstBackup = savedStartCode == null;
-            if (firstBackup) {
-              // This is our first backup. Let's put some marker on ZK so that we can hold the logs
-              // while we do the backup.
-              backupManager.writeBackupStartCode("0");
-            }
-            // We roll log here before we do the snapshot. It is possible there is duplicate data
-            // in the log that is already in the snapshot. But if we do it after the snapshot, we
-            // could have data loss.
-            // A better approach is to do the roll log on each RS in the same global procedure as
-            // the snapshot.
-            LOG.info("Execute roll log procedure for full backup ...");
-            conn = ConnectionFactory.createConnection(conf);
-            hbadmin = (HBaseAdmin) conn.getAdmin();
-            hbadmin.execProcedure(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
-              LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, new HashMap<String, String>());
-            newTimestamps = backupManager.readRegionServerLastLogRollResult();
-            if (firstBackup) {
-              // Updates registered log files
-              // We record ALL old WAL files as registered, because
-              // this is a first full backup in the system and these
-              // files are not needed for next incremental backup
-              List<String> logFiles = BackupUtil.getWALFilesOlderThan(conf, newTimestamps);
-              backupManager.recordWALFiles(logFiles);
-            }
-            this.snapshotForFullBackup(backupContext);
-          } catch (BackupException e) {
-            // fail the overall backup and return
-            this.failBackup(backupContext, e, "Unexpected BackupException : ");
-            return null;
-          } finally {
-            if (hbadmin != null) {
-              hbadmin.close();
-            }
-            if (conn != null) {
-              conn.close();
-            }
-          }
-        }
-
-        // update the faked progress currently for snapshot done
-        this.updateProgress("10.0%", 0);
-        // do snapshot copy
-        try {
-          this.snapshotCopy(backupContext);
-        } catch (Exception e) {
-          // fail the overall backup and return
-          this.failBackup(backupContext, e, "Unexpected BackupException : ");
-          return null;
-        }
-        // Updates incremental backup table set
-        backupManager.addIncrementalBackupTableSet(backupContext.getTables());
-
-      } else if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-        LOG.info("For incremental backup, current table set is "
-            + backupManager.getIncrementalBackupTableSet());
-        // do incremental table backup preparation
-        backupContext.setPhase(BACKUPPHASE.PREPARE_INCREMENTAL);
-        // avoid action if has been cancelled
-        if (backupContext.isCancelled()) {
-          return null;
-        }
-        try {
-          IncrementalBackupManager incrBackupManager = new IncrementalBackupManager(backupManager);
-
-          newTimestamps = incrBackupManager.getIncrBackupLogFileList(backupContext);
-        } catch (Exception e) {
-          // fail the overall backup and return
-          this.failBackup(backupContext, e, "Unexpected Exception : ");
-          return null;
-        }
-        // update the faked progress currently for incremental preparation done
-        this.updateProgress("10.0%", 0);
-
-        // do incremental copy
-        try {
-          // copy out the table and region info files for each table
-          BackupUtil.copyTableRegionInfo(backupContext, conf);
-          this.incrementalCopy(backupContext);
-          // Save list of WAL files copied
-          backupManager.recordWALFiles(backupContext.getIncrBackupFileList());
-        } catch (Exception e) {
-          // fail the overall backup and return
-          this.failBackup(backupContext, e, "Unexpected exception doing incremental copy : ");
-          return null;
-        }
-
-      } else {
-        LOG.error("Unsupport backup type: " + backupContext.getType());
-      }
-
-      // set overall backup status: complete. Here we make sure to complete the backup. After this
-      // checkpoint, even if entering cancel process, will let the backup finished
-      backupContext.setFlag(BACKUPSTATUS.COMPLETE);
-
-      if (!fromExistingSnapshot) {
-        if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-          // Set the previousTimestampMap which is before this current log roll to the manifest.
-          HashMap<String, HashMap<String, String>> previousTimestampMap =
-              backupManager.readLogTimestampMap();
-          backupContext.setIncrTimestampMap(previousTimestampMap);
-        }
-        // The table list in backupContext is good for both full backup and incremental backup.
-        // For incremental backup, it contains the incremental backup table set.
-
-        backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps);
-
-        HashMap<String, HashMap<String, String>> newTableSetTimestampMap =
-            backupManager.readLogTimestampMap();
-
-        String newStartCode =
-            BackupUtil.getMinValue(BackupUtil.getRSLogTimestampMins(newTableSetTimestampMap));
-        backupManager.writeBackupStartCode(newStartCode);
-      }
-
-      // backup complete
-      this.completeBackup(backupContext);
-    } catch (Exception e) {
-      // even during completing backup (#completeBackup(backupContext)), exception may occur, or
-      // exception occur during other process, fail the backup finally
-      this.failBackup(backupContext, e, "Error caught during backup progress: ");
-    }
-    return null;
-  }
-
-  /**
-   * Begin the overall backup.
-   * @param backupContext backup context
-   * @throws IOException exception
-   */
-  private void beginBackup(BackupContext backupContext) throws IOException {
-
-    // set the start timestamp of the overall backup
-    long startTs = EnvironmentEdgeManager.currentTime();
-    backupContext.setStartTs(startTs);
-    // set overall backup status: ongoing
-    backupContext.setFlag(BACKUPSTATUS.ONGOING);
-    LOG.info("Backup " + backupContext.getBackupId() + " starts at " + startTs + ".");
-
-    backupManager.updateBackupStatus(backupContext);
-    LOG.debug("Backup session " + backupContext.getBackupId() + " has been started.");
-
-  }
-
-  /**
-   * Snapshot for full table backup.
-   * @param backupContext backup context
-   * @throws IOException exception
-   */
-  private void snapshotForFullBackup(BackupContext backupContext) throws IOException {
-
-    LOG.info("HBase snapshot full backup for " + backupContext.getBackupId());
-
-    // avoid action if has been cancelled
-    if (backupContext.isCancelled()) {
-      return;
-    }
-
-    HBaseAdmin hbadmin = null;
-    Connection conn = null;
-
-    // we do HBase snapshot for tables in the table list one by one currently
-    for (String table : backupContext.getTables()) {
-
-      // avoid action if it has been cancelled
-      if (backupContext.isCancelled()) {
-        return;
-      }
-
-      HBaseProtos.SnapshotDescription backupSnapshot;
-      try {
-        // wrap a SnapshotDescription for offline/online snapshot
-        backupSnapshot = this.wrapSnapshotDescription(table);
-
-        // set the snapshot name in BackupStatus of this table
-        backupContext.setSnapshotName(table, backupSnapshot.getName());
-
-        // Kick off snapshot for backup
-        conn = ConnectionFactory.createConnection(conf);
-        hbadmin = (HBaseAdmin) conn.getAdmin();
-        hbadmin.snapshot(backupSnapshot);
-
-        if (LOG.isDebugEnabled() == false) {
-          // In DEBUG mode we log message already.
-          // This is not to duplicate that message.
-          LOG.info("Snapshot has been launched, waiting to finish ...");
-        }
-
-      } catch (Exception e) {
-        LOG.error("Snapshot failed to create " + getMessage(e));
-
-        // currently, we fail the overall backup if any table in the list failed, so throw the
-        // exception out for overall backup failing
-        throw new BackupException("Backup snapshot failed on table " + table, e);
-      } finally {
-        if (hbadmin != null) {
-          hbadmin.close();
-        }
-        if (conn != null) {
-          conn.close();
-        }
-      }
-
-      // set the snapshot name in BackupStatus of this table, only after snapshot success.
-      backupContext.setSnapshotName(table, backupSnapshot.getName());
-
-    } // for each table in the backup table list
-
-  }
-
-  /**
-   * Fail the overall backup.
-   * @param backupContext backup context
-   * @param e exception
-   * @throws Exception exception
-   */
-  private void failBackup(BackupContext backupContext, Exception e, String msg) throws Exception {
-
-    LOG.error(msg + getMessage(e));
-
-    // If this is a cancel exception, then we've already cleaned.
-
-    if (this.backupContext.getFlag().equals(BACKUPSTATUS.CANCELLED)) {
-      return;
-    }
-
-    // set the failure timestamp of the overall backup
-    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
-
-    // set failure message
-    backupContext.setFailedMsg(e.getMessage());
-
-    // set overall backup status: failed
-    backupContext.setFlag(BACKUPSTATUS.FAILED);
-
-    // compose the backup failed data
-    String backupFailedData =
-        "BackupId=" + backupContext.getBackupId() + ",startts=" + backupContext.getStartTs()
-        + ",failedts=" + backupContext.getEndTs() + ",failedphase=" + backupContext.getPhase()
-        + ",failedmessage=" + backupContext.getFailedMsg();
-    LOG.error(backupFailedData);
-
-    backupManager.updateBackupStatus(backupContext);
-
-    // if full backup, then delete HBase snapshots if there already have snapshots taken
-    // and also clean up export snapshot log files if exist
-    if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
-      if (!backupContext.fromExistingSnapshot()) {
-        this.deleteSnapshot(backupContext);
-      }
-      this.cleanupExportSnapshotLog();
-    } /*
-     * else { // support incremental backup code in future jira // TODO. See HBASE-14124 }
-     */
-
-    // clean up the uncompleted data at target directory if the ongoing backup has already entered
-    // the copy phase
-    // For incremental backup, DistCp logs will be cleaned with the targetDir.
-    this.cleanupTargetDir();
-
-    LOG.info("Backup " + backupContext.getBackupId() + " failed.");
-  }
-
-  /**
-   * Update the ongoing back token znode with new progress.
-   * @param newProgress progress
-   * @param bytesCopied bytes copied
-   * @throws NoNodeException exception
-   */
-  public void updateProgress(String newProgress, long bytesCopied) throws IOException {
-
-    // compose the new backup progress data, using fake number for now
-    String backupProgressData = newProgress;
-
-    backupContext.setProgress(newProgress);
-    backupManager.updateBackupStatus(backupContext);
-    LOG.debug("Backup progress data \"" + backupProgressData
-      + "\" has been updated to hbase:backup for " + backupContext.getBackupId());
-  }
-
-  /**
-   * Complete the overall backup.
-   * @param backupContext backup context
-   * @throws Exception exception
-   */
-  private void completeBackup(BackupContext backupContext) throws Exception {
-
-    // set the complete timestamp of the overall backup
-    backupContext.setEndTs(EnvironmentEdgeManager.currentTime());
-    // set overall backup status: complete
-    backupContext.setFlag(BACKUPSTATUS.COMPLETE);
-    // add and store the manifest for the backup
-    this.addManifest(backupContext);
-
-    // after major steps done and manifest persisted, do convert if needed for incremental backup
-    /* in-fly convert code here, provided by future jira */
-    LOG.debug("in-fly convert code here, provided by future jira");
-
-    // compose the backup complete data
-    String backupCompleteData =
-        this.obtainBackupMetaDataStr(backupContext) + ",startts=" + backupContext.getStartTs()
-        + ",completets=" + backupContext.getEndTs() + ",bytescopied="
-        + backupContext.getTotalBytesCopied();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Backup " + backupContext.getBackupId() + " finished: " + backupCompleteData);
-    }
-    backupManager.updateBackupStatus(backupContext);
-
-    // when full backup is done:
-    // - delete HBase snapshot
-    // - clean up directories with prefix "exportSnapshot-", which are generated when exporting
-    // snapshots
-    if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
-      if (!backupContext.fromExistingSnapshot()) {
-        this.deleteSnapshot(backupContext);
-      }
-      this.cleanupExportSnapshotLog();
-    } else if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-      this.cleanupDistCpLog();
-    } else {
-      LOG.error(" other backup types have not been implemented yet");
-    }
-
-    LOG.info("Backup " + backupContext.getBackupId() + " completed.");
-  }
-
-  /**
-   * Get backup request meta data dir as string.
-   * @param backupContext backup context
-   * @return meta data dir
-   */
-  private String obtainBackupMetaDataStr(BackupContext backupContext) {
-    StringBuffer sb = new StringBuffer();
-    sb.append("type=" + backupContext.getType() + ",tablelist=");
-    for (String table : backupContext.getTables()) {
-      sb.append(table + ";");
-    }
-    if (sb.lastIndexOf(";") > 0) {
-      sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1);
-    }
-    sb.append(",targetRootDir=" + backupContext.getTargetRootDir());
-    if (backupContext.fromExistingSnapshot()) {
-      sb.append(",snapshot=" + backupContext.getExistingSnapshot());
-    }
-    return sb.toString();
-  }
-
-  /**
-   * Do snapshot copy.
-   * @param backupContext backup context
-   * @throws Exception exception
-   */
-  private void snapshotCopy(BackupContext backupContext) throws Exception {
-
-    LOG.info("Snapshot copy is starting.");
-
-    // set overall backup phase: snapshot_copy
-    backupContext.setPhase(BACKUPPHASE.SNAPSHOTCOPY);
-
-    // avoid action if has been cancelled
-    if (backupContext.isCancelled()) {
-      return;
-    }
-
-    // call ExportSnapshot to copy files based on hbase snapshot for backup
-    // ExportSnapshot only support single snapshot export, need loop for multiple tables case
-    BackupCopyService copyService = BackupRestoreServiceFactory.getBackupCopyService(conf);
-
-    // number of snapshots matches number of tables
-    float numOfSnapshots = backupContext.getSnapshotNames().size();
-
-    LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied.");
-
-    for (String table : backupContext.getTables()) {
-      // Currently we simply set the sub copy tasks by counting the table snapshot number, we can
-      // calculate the real files' size for the percentage in the future.
-      // TODO this below
-      // backupCopier.setSubTaskPercntgInWholeTask(1f / numOfSnapshots);
-      int res = 0;
-      String[] args = new String[4];
-      args[0] = "-snapshot";
-      args[1] = backupContext.getSnapshotName(table);
-      args[2] = "-copy-to";
-      args[3] = backupContext.getBackupStatus(table).getTargetDir();
-
-      LOG.debug("Copy snapshot " + args[1] + " to " + args[3]);
-      res = copyService.copy(this, conf, BackupCopyService.Type.FULL, args);
-      // if one snapshot export failed, do not continue for remained snapshots
-      if (res != 0) {
-
-        LOG.error("Exporting Snapshot " + args[1] + " failed with return code: " + res + ".");
-
-        throw new IOException("Failed of exporting snapshot " + args[1] + " to " + args[3]
-            + " with reason code " + res);
-      }
-
-      LOG.info("Snapshot copy " + args[1] + " finished.");
-    }
-  }
-
-  /**
-   * Wrap a SnapshotDescription for a target table.
-   * @param table table
-   * @return a SnapshotDescription especially for backup.
-   */
-  private SnapshotDescription wrapSnapshotDescription(String table) {
-    // Mock a SnapshotDescription from backupContext to call SnapshotManager function,
-    // Name it in the format "snapshot_<timestamp>_<table>"
-    HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
-    builder.setTable(table);
-    TableName tableName = TableName.valueOf(table);
-    builder.setName("snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime()) + "_"
-        + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString());
-    HBaseProtos.SnapshotDescription backupSnapshot = builder.build();
-
-    LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName()
-      + " from backupContext to request snapshot for backup.");
-
-    return backupSnapshot;
-  }
-
-  /**
-   * Delete HBase snapshot for backup.
-   * @param backupCtx backup context
-   * @throws Exception exception
-   */
-  private void deleteSnapshot(BackupContext backupCtx) throws IOException {
-
-    LOG.debug("Trying to delete snapshot for full backup.");
-    Connection conn = null;
-    Admin admin = null;
-    try {
-      conn = ConnectionFactory.createConnection(conf);
-      admin = conn.getAdmin();
-      for (String snapshotName : backupCtx.getSnapshotNames()) {
-        if (snapshotName == null) {
-          continue;
-        }
-        LOG.debug("Trying to delete snapshot: " + snapshotName);
-        admin.deleteSnapshot(snapshotName);
-        LOG.debug("Deleting the snapshot " + snapshotName + " for backup "
-            + backupCtx.getBackupId() + " succeeded.");
-      }
-    } finally {
-      if (admin != null) {
-        admin.close();
-      }
-      if (conn != null) {
-        conn.close();
-      }
-    }
-  }
-
-  /**
-   * Clean up directories with prefix "exportSnapshot-", which are generated when exporting
-   * snapshots.
-   * @throws IOException exception
-   */
-  private void cleanupExportSnapshotLog() throws IOException {
-    FileSystem fs = FSUtils.getCurrentFileSystem(conf);
-    Path stagingDir =
-        new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory()
-          .toString()));
-    FileStatus[] files = FSUtils.listStatus(fs, stagingDir);
-    if (files == null) {
-      return;
-    }
-    for (FileStatus file : files) {
-      if (file.getPath().getName().startsWith("exportSnapshot-")) {
-        LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName());
-        if (FSUtils.delete(fs, file.getPath(), true) == false) {
-          LOG.warn("Can not delete " + file.getPath());
-        }
-      }
-    }
-  }
-
-  /**
-   * Clean up directories with prefix "_distcp_logs-", which are generated when DistCp copying
-   * hlogs.
-   * @throws IOException exception
-   */
-  private void cleanupDistCpLog() throws IOException {
-    Path rootPath = new Path(backupContext.getHLogTargetDir()).getParent();
-    FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
-    FileStatus[] files = FSUtils.listStatus(fs, rootPath);
-    if (files == null) {
-      return;
-    }
-    for (FileStatus file : files) {
-      if (file.getPath().getName().startsWith("_distcp_logs")) {
-        LOG.debug("Delete log files of DistCp: " + file.getPath().getName());
-        FSUtils.delete(fs, file.getPath(), true);
-      }
-    }
-  }
-
-  /**
-   * Clean up the uncompleted data at target directory if the ongoing backup has already entered the
-   * copy phase.
-   */
-  private void cleanupTargetDir() {
-    try {
-      // clean up the uncompleted data at target directory if the ongoing backup has already entered
-      // the copy phase
-      LOG.debug("Trying to cleanup up target dir. Current backup phase: "
-          + backupContext.getPhase());
-      if (backupContext.getPhase().equals(BACKUPPHASE.SNAPSHOTCOPY)
-          || backupContext.getPhase().equals(BACKUPPHASE.INCREMENTAL_COPY)
-          || backupContext.getPhase().equals(BACKUPPHASE.STORE_MANIFEST)) {
-        FileSystem outputFs =
-            FileSystem.get(new Path(backupContext.getTargetRootDir()).toUri(), conf);
-
-        // now treat one backup as a transaction, clean up data that has been partially copied at
-        // table level
-        for (String table : backupContext.getTables()) {
-          Path targetDirPath =
-              new Path(HBackupFileSystem.getTableBackupDir(backupContext.getTargetRootDir(),
-                backupContext.getBackupId(), table));
-          if (outputFs.delete(targetDirPath, true)) {
-            LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString()
-              + " done.");
-          } else {
-            LOG.info("No data has been copied to " + targetDirPath.toString() + ".");
-          }
-
-          Path tableDir = targetDirPath.getParent();
-          FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir);
-          if (backups == null || backups.length == 0) {
-            outputFs.delete(tableDir, true);
-            LOG.debug(tableDir.toString() + " is empty, remove it.");
-          }
-        }
-      }
-
-    } catch (IOException e1) {
-      LOG.error("Cleaning up uncompleted backup data of " + backupContext.getBackupId() + " at "
-          + backupContext.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
-    }
-  }
-
-  /**
-   * Add manifest for the current backup. The manifest is stored 
-   * within the table backup directory.  
-   * @param backupContext The current backup context
-   * @throws IOException exception
-   * @throws BackupException exception
-   */
-  private void addManifest(BackupContext backupContext) throws IOException, BackupException {
-    // set the overall backup phase : store manifest
-    backupContext.setPhase(BACKUPPHASE.STORE_MANIFEST);
-
-    // avoid action if has been cancelled
-    if (backupContext.isCancelled()) {
-      return;
-    }
-
-    BackupManifest manifest;
-    boolean fromExistingSnapshot = false; // to be implemented in future jira
-
-    // Since we have each table's backup in its own directory structure,
-    // we'll store its manifest with the table directory.
-    for (String table : backupContext.getTables()) {
-      manifest = new BackupManifest(backupContext, table);
-      if (fromExistingSnapshot) {
-        // mark backing up from existing snapshot in manifest, so that later, dependency analysis
-        // can skip this backup image
-        LOG.debug("backup using existing snapshot will be supported in future jira");
-      } else {
-        ArrayList<BackupImage> ancestors = this.backupManager.getAncestors(backupContext, table);
-        for (BackupImage image : ancestors) {
-          manifest.addDependentImage(image);
-        }
-      }
-      if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-        // We'll store the log timestamps for this table only in its manifest.
-        HashMap<String, HashMap<String, String>> tableTimestampMap =
-            new HashMap<String, HashMap<String, String>>();
-        tableTimestampMap.put(table, backupContext.getIncrTimestampMap().get(table));
-        manifest.setIncrTimestampMap(tableTimestampMap);
-      }
-      manifest.store(conf);
-    }
-
-    // For incremental backup, we store a overall manifest in
-    // <backup-root-dir>/WALs/<backup-id>
-    // This is used when created the next incremental backup
-    if (backupContext.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-      manifest = new BackupManifest(backupContext);
-      // set the table region server start and end timestamps for incremental backup
-      manifest.setIncrTimestampMap(backupContext.getIncrTimestampMap());
-      ArrayList<BackupImage> ancestors = this.backupManager.getAncestors(backupContext);
-      for (BackupImage image : ancestors) {
-        manifest.addDependentImage(image);
-      }
-      manifest.store(conf);
-    }
-  }
-
-  /**
-   * Do incremental copy.
-   * @param backupContext backup context
-   */
-  private void incrementalCopy(BackupContext backupContext) throws Exception {
-
-    LOG.info("Incremental copy is starting.");
-
-    // set overall backup phase: incremental_copy
-    backupContext.setPhase(BACKUPPHASE.INCREMENTAL_COPY);
-
-    // avoid action if has been cancelled
-    if (backupContext.isCancelled()) {
-      return;
-    }
-
-    // get incremental backup file list and prepare parms for DistCp
-    List<String> incrBackupFileList = backupContext.getIncrBackupFileList();
-    String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
-    strArr[strArr.length - 1] = backupContext.getHLogTargetDir();
-
-    BackupCopyService copyService = BackupRestoreServiceFactory.getBackupCopyService(conf);
-    int res = copyService.copy(this, conf, BackupCopyService.Type.INCREMENTAL, strArr);
-
-    if (res != 0) {
-      LOG.error("Copy incremental log files failed with return code: " + res + ".");
-      throw new IOException("Failed of Hadoop Distributed Copy from " + incrBackupFileList + " to "
-          + backupContext.getHLogTargetDir());
-    }
-    LOG.info("Incremental copy from " + incrBackupFileList + " to "
-        + backupContext.getHLogTargetDir() + " finished.");
-
-  }
-
-  private String getMessage(Exception e) {
-    String msg = e.getMessage();
-    if (msg == null || msg.equals("")) {
-      msg = e.getClass().getName();
-    }
-    return msg;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java
deleted file mode 100644
index fdb3c46..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManager.java
+++ /dev/null
@@ -1,488 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.backup.BackupHandler.BACKUPSTATUS;
-import org.apache.hadoop.hbase.backup.BackupManifest.BackupImage;
-import org.apache.hadoop.hbase.backup.BackupUtil.BackupCompleteData;
-import org.apache.hadoop.hbase.backup.master.BackupLogCleaner;
-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;
-
-
-
-/**
- * Handles backup requests on server-side, creates backup context records in hbase:backup 
- * to keep track backup. The timestamps kept in hbase:backup table will be used for future 
- * incremental backup. Creates BackupContext and DispatchRequest.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BackupManager {
-  private static final Log LOG = LogFactory.getLog(BackupManager.class);
-
-  private Configuration conf = null;
-  private BackupContext backupContext = null;
-  private ExecutorService pool = null;
-
-  private boolean backupComplete = false;
-
-  private BackupSystemTable systemTable;
-
-  /**
-   * Backup manager constructor.
-   * @param conf configuration
-   * @throws IOException exception
-   */
-  public BackupManager(Configuration conf) throws IOException {
-    if (!conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) {
-      throw new BackupException("HBase backup is not enabled. Check your " + 
-          HConstants.BACKUP_ENABLE_KEY + " setting.");
-    }
-    this.conf = conf;
-    this.systemTable = BackupSystemTable.getTable(conf);
-    Runtime.getRuntime().addShutdownHook(new ExitHandler());
-  }
-
-  /**
-   * This method modifies the master's configuration in order to inject backup-related features
-   * @param conf configuration
-   */
-  public static void decorateMasterConfiguration(Configuration conf) {
-    if (!isBackupEnabled(conf)) {
-      return;
-    }
-    String plugins = conf.get(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS);
-    String cleanerClass = BackupLogCleaner.class.getCanonicalName();
-    if (!plugins.contains(cleanerClass)) {
-      conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Added log cleaner: " + cleanerClass);
-    }
-  }
-
-  private static boolean isBackupEnabled(Configuration conf) {
-    return conf.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT);
-  }
-
-  private class ExitHandler extends Thread {
-    public ExitHandler() {
-      super("Backup Manager Exit Handler");
-    }
-
-    public void run() {
-
-      if (backupContext != null && !backupComplete) {
-
-        // program exit and backup is not complete, then mark as cancelled to avoid submitted backup
-        // handler's taking further action
-        backupContext.markCancel();
-
-        LOG.debug("Backup is cancelled due to force program exiting.");
-        try {
-          cancelBackup(backupContext.getBackupId());
-        } catch (Exception e) {
-          String msg = e.getMessage();
-          if (msg == null || msg.equals("")) {
-            msg = e.getClass().getName();
-          }
-          LOG.error("Failed to cancel backup " + backupContext.getBackupId() + " due to " + msg);
-        }
-      }
-
-      exit();
-    }
-  }
-
-  /**
-   * Cancel the ongoing backup via backup id.
-   * @param backupId The id of the ongoing backup to be cancelled
-   * @throws Exception exception
-   */
-  private void cancelBackup(String backupId) throws Exception {
-    // TODO: will be implemented in Phase 2: HBASE-14125
-    LOG.debug("Try to cancel the backup " + backupId + ". the feature is NOT implemented yet");
-
-  }
-
-  /**
-   * Stop all the work of backup.
-   */
-  public void exit() {
-
-    // currently, we shutdown now for all ongoing back handlers, we may need to do something like
-    // record the failed list somewhere later
-    if (this.pool != null) {
-      this.pool.shutdownNow();
-    }
-
-  }
-
-  /**
-   * Create a BackupContext based on input backup request.
-   * @param backupId backup id
-   * @param type    type
-   * @param tablelist table list
-   * @param targetRootDir root dir
-   * @param snapshot snapshot name
-   * @return BackupContext context
-   * @throws BackupException exception
-   */
-  protected BackupContext createBackupContext(String backupId, String type, String tablelist,
-      String targetRootDir, String snapshot) throws BackupException {
-
-    if (targetRootDir == null) {
-      throw new BackupException("Wrong backup request parameter: target backup root directory");
-    }
-
-    if (type.equals(BackupRestoreConstants.BACKUP_TYPE_FULL) && tablelist == null) {
-      // If table list is null for full backup, which means backup all tables. Then fill the table
-      // list with all user tables from meta. It no table available, throw the request exception.
-
-      HTableDescriptor[] htds = null;
-      try (Connection conn = ConnectionFactory.createConnection(conf);
-          HBaseAdmin hbadmin = (HBaseAdmin) conn.getAdmin()) {
-
-        htds = hbadmin.listTables();
-      } catch (Exception e) {
-        throw new BackupException(e);
-      }
-
-      if (htds == null) {
-        throw new BackupException("No table exists for full backup of all tables.");
-      } else {
-        StringBuilder sb = new StringBuilder();
-        for (HTableDescriptor hTableDescriptor : htds) {
-          sb.append(hTableDescriptor.getNameAsString()
-              + BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND);
-        }
-        sb.deleteCharAt(sb.lastIndexOf(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND));
-        tablelist = sb.toString();
-
-        LOG.info("Full backup all the tables available in the cluster: " + tablelist);
-      }
-    }
-
-    // there are one or more tables in the table list
-    return new BackupContext(backupId, type,
-        tablelist.split(BackupRestoreConstants.TABLENAME_DELIMITER_IN_COMMAND), targetRootDir,
-        snapshot);
-
-  }
-
-  /**
-   * Check if any ongoing backup. Currently, we only reply on checking status in hbase:backup. We
-   * need to consider to handle the case of orphan records in the future. Otherwise, all the coming
-   * request will fail.
-   * @return the ongoing backup id if on going backup exists, otherwise null
-   * @throws IOException exception
-   */
-  private String getOngoingBackupId() throws IOException {
-
-    ArrayList<BackupContext> sessions = systemTable.getBackupContexts(BACKUPSTATUS.ONGOING);
-    if (sessions.size() == 0) {
-      return null;
-    }
-    return sessions.get(0).getBackupId();
-  }
-
-  /**
-   * Start the backup manager service.
-   * @throws IOException exception
-   */
-  public void initialize() throws IOException {
-    String ongoingBackupId = this.getOngoingBackupId();
-    if (ongoingBackupId != null) {
-      LOG.info("There is a ongoing backup " + ongoingBackupId
-          + ". Can not launch new backup until no ongoing backup remains.");
-      throw new BackupException("There is ongoing backup.");
-    }
-
-    // Initialize thread pools
-    int nrThreads = this.conf.getInt("hbase.backup.threads.max", 1);
-    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
-    builder.setNameFormat("BackupHandler-%1$d");
-    this.pool =
-        new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,
-            new LinkedBlockingQueue<Runnable>(), builder.build());
-    ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
-  }
-
-  /**
-   * Dispatch and handle a backup request.
-   * @param backupContext backup context
-   * @throws BackupException exception
-   */
-  public void dispatchRequest(BackupContext backupContext) throws BackupException {
-
-    this.backupContext = backupContext;
-
-    LOG.info("Got a backup request: " + "Type: " + backupContext.getType() + "; Tables: "
-        + backupContext.getTableListAsString() + "; TargetRootDir: "
-        + backupContext.getTargetRootDir());
-
-    // dispatch the request to a backup handler and put it handler map
-
-    BackupHandler handler = new BackupHandler(this.backupContext, this, conf);
-    Future<Object> future = this.pool.submit(handler);
-    // wait for the execution to complete
-    try {
-      future.get();
-    } catch (InterruptedException e) {
-      throw new BackupException(e);
-    } catch (CancellationException e) {
-      throw new BackupException(e);
-    } catch (ExecutionException e) {
-      throw new BackupException(e);
-    }
-
-    // mark the backup complete for exit handler's processing
-    backupComplete = true;
-
-    LOG.info("Backup request " + backupContext.getBackupId() + " has been executed.");
-  }
-
-  /**
-   * Get direct ancestors of the current backup.
-   * @param backupCtx The backup context for the current backup
-   * @return The ancestors for the current backup
-   * @throws IOException exception
-   * @throws BackupException exception
-   */
-  protected ArrayList<BackupImage> getAncestors(BackupContext backupCtx) throws IOException,
-      BackupException {
-    LOG.debug("Getting the direct ancestors of the current backup ...");
-
-    ArrayList<BackupImage> ancestors = new ArrayList<BackupImage>();
-
-    // full backup does not have ancestor
-    if (backupCtx.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
-      LOG.debug("Current backup is a full backup, no direct ancestor for it.");
-      return ancestors;
-    }
-
-    // get all backup history list in descending order
-
-    ArrayList<BackupCompleteData> allHistoryList = getBackupHistory();
-    for (BackupCompleteData backup : allHistoryList) {
-      BackupImage image =
-          new BackupImage(backup.getBackupToken(), backup.getType(), backup.getBackupRootPath(),
-              backup.getTableList(), Long.parseLong(backup.getStartTime()), Long.parseLong(backup
-                  .getEndTime()));
-
-      // add the full backup image as an ancestor until the last incremental backup
-      if (backup.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
-
-        // backup image from existing snapshot does not involve in dependency
-        if (backup.fromExistingSnapshot()) {
-          continue;
-        }
-        // check the backup image coverage, if previous image could be covered by the newer ones,
-        // then no need to add
-        if (!BackupManifest.canCoverImage(ancestors, image)) {
-          ancestors.add(image);
-        }
-      } else {
-        // found last incremental backup, if previously added full backup ancestor images can cover
-        // it, then this incremental ancestor is not the dependent of the current incremental
-        // backup, that is to say, this is the backup scope boundary of current table set.
-        // Otherwise, this incremental backup ancestor is the dependent ancestor of the ongoing
-        // incremental backup
-        if (BackupManifest.canCoverImage(ancestors, image)) {
-          LOG.debug("Met the backup boundary of the current table set. "
-              + "The root full backup images for the current backup scope:");
-          for (BackupImage image1 : ancestors) {
-            LOG.debug("  BackupId: " + image1.getBackupId() + ", Backup directory: "
-                + image1.getRootDir());
-          }
-        } else {
-          Path logBackupPath =
-              HBackupFileSystem.getLogBackupPath(backup.getBackupRootPath(),
-                backup.getBackupToken());
-          LOG.debug("Current backup has an incremental backup ancestor, "
-              + "touching its image manifest in " + logBackupPath.toString()
-              + " to construct the dependency.");
-
-          BackupManifest lastIncrImgManifest = new BackupManifest(conf, logBackupPath);
-          BackupImage lastIncrImage = lastIncrImgManifest.getBackupImage();
-          ancestors.add(lastIncrImage);
-
-          LOG.debug("Last dependent incremental backup image information:");
-          LOG.debug("  Token: " + lastIncrImage.getBackupId());
-          LOG.debug("  Backup directory: " + lastIncrImage.getRootDir());
-        }
-      }
-    }
-    LOG.debug("Got " + ancestors.size() + " ancestors for the current backup.");
-    return ancestors;
-  }
-
-  /**
-   * Get the direct ancestors of this backup for one table involved.
-   * @param backupContext backup context
-   * @param table table
-   * @return backupImages on the dependency list
-   * @throws BackupException exception
-   * @throws IOException exception
-   */
-  protected ArrayList<BackupImage> getAncestors(BackupContext backupContext, String table)
-      throws BackupException, IOException {
-    ArrayList<BackupImage> ancestors = getAncestors(backupContext);
-    ArrayList<BackupImage> tableAncestors = new ArrayList<BackupImage>();
-    for (BackupImage image : ancestors) {
-      if (image.hasTable(table)) {
-        tableAncestors.add(image);
-        if (image.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
-          break;
-        }
-      }
-    }
-    return tableAncestors;
-  }
-
-  /**
-   * hbase:backup operations
-   */
-
-  /**
-   * Updates status (state) of a backup session in a persistent store
-   * @param context context
-   * @throws IOException exception
-   */
-  public void updateBackupStatus(BackupContext context) throws IOException {
-    systemTable.updateBackupStatus(context);
-  }
-
-  /**
-   * Read the last backup start code (timestamp) of last successful backup. Will return null 
-   * if there is no startcode stored in hbase:backup or the value is of length 0. These two 
-   * cases indicate there is no successful backup completed so far.
-   * @return the timestamp of a last successful backup
-   * @throws IOException exception
-   */
-  public String readBackupStartCode() throws IOException {
-    return systemTable.readBackupStartCode();
-  }
-
-  /**
-   * Write the start code (timestamp) to hbase:backup. If passed in null, then write 0 byte.
-   * @param startCode start code
-   * @throws IOException exception
-   */
-  public void writeBackupStartCode(String startCode) throws IOException {
-    systemTable.writeBackupStartCode(startCode);
-  }
-
-  /**
-   * Get the RS log information after the last log roll from hbase:backup.
-   * @return RS log info
-   * @throws IOException exception
-   */
-  public HashMap<String, String> readRegionServerLastLogRollResult() throws IOException {
-    return systemTable.readRegionServerLastLogRollResult();
-  }
-
-  /**
-   * Get all completed backup information (in desc order by time)
-   * @return history info of BackupCompleteData
-   * @throws IOException exception
-   */
-  public ArrayList<BackupCompleteData> getBackupHistory() throws IOException {
-    return systemTable.getBackupHistory();
-  }
-
-  /**
-   * Write the current timestamps for each regionserver to hbase:backup after a successful full or
-   * incremental backup. Each table may have a different set of log timestamps. The saved timestamp
-   * is of the last log file that was backed up already.
-   * @param tables tables
-   * @throws IOException exception
-   */
-  public void writeRegionServerLogTimestamp(Set<String> tables,
-      HashMap<String, String> newTimestamps) throws IOException {
-    systemTable.writeRegionServerLogTimestamp(tables, newTimestamps);
-  }
-
-  /**
-   * Read the timestamp for each region server log after the last successful backup. Each table has
-   * its own set of the timestamps. The info is stored for each table as a concatinated string on ZK
-   * under //hbase//backup//incr//tablelogtimestamp//table_name
-   * @return the timestamp for each region server. key: tableName value:
-   *         RegionServer,PreviousTimeStamp
-   * @throws IOException exception
-   */
-  public HashMap<String, HashMap<String, String>> readLogTimestampMap() throws IOException {
-    return systemTable.readLogTimestampMap();
-  }
-
-  /**
-   * Return the current tables covered by incremental backup.
-   * @return set of tableNames
-   * @throws IOException exception
-   */
-  public Set<String> getIncrementalBackupTableSet() throws IOException {
-    return systemTable.getIncrementalBackupTableSet();
-  }
-
-  /**
-   * Adds set of tables to overall incremental backup table set
-   * @param tables tables
-   * @throws IOException exception
-   */
-  public void addIncrementalBackupTableSet(Set<String> tables) throws IOException {
-    systemTable.addIncrementalBackupTableSet(tables);
-  }
-  
-  /**
-   * Saves list of WAL files after incremental backup operation. These files will be stored until
-   * TTL expiration and are used by Backup Log Cleaner plugin to determine which WAL files can be
-   * safely purged.
-   */
-
-  public void recordWALFiles(List<String> files) throws IOException {
-    systemTable.addWALFiles(files, backupContext.getBackupId());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java
deleted file mode 100644
index f41540b..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupManifest.java
+++ /dev/null
@@ -1,814 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.codehaus.jackson.JsonGenerationException;
-import org.codehaus.jackson.JsonParseException;
-import org.codehaus.jackson.map.JsonMappingException;
-import org.codehaus.jackson.map.ObjectMapper;
-
-/**
- * Backup manifest Contains all the meta data of a backup image. The manifest info will be bundled
- * as manifest file together with data. So that each backup image will contain all the info needed
- * for restore.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BackupManifest {
-
-  private static final Log LOG = LogFactory.getLog(BackupManifest.class);
-
-  // manifest file name
-  public static final String FILE_NAME = ".backup.manifest";
-
-  // manifest file version, current is 1.0
-  public static final String MANIFEST_VERSION = "1.0";
-
-  // tags of fields for manifest file
-  public static final String TAG_VERSION = "Manifest-Version";
-  public static final String TAG_BACKUPID = "Backup-Id";
-  public static final String TAG_BACKUPTYPE = "Backup-Type";
-  public static final String TAG_TABLESET = "Table-Set";
-  public static final String TAG_STARTTS = "Start-Timestamp";
-  public static final String TAG_COMPLETETS = "Complete-Timestamp";
-  public static final String TAG_TABLEBYTES = "Total-Table-Bytes";
-  public static final String TAG_LOGBYTES = "Total-Log-Bytes";
-  public static final String TAG_INCRTIMERANGE = "Incremental-Time-Range";
-  public static final String TAG_DEPENDENCY = "Dependency";
-  public static final String TAG_IMAGESTATE = "Image-State";
-  public static final String TAG_COMPACTION = "Compaction";
-
-  public static final String ERROR_DEPENDENCY = "DEPENDENCY_ERROR";
-
-  public static final int DELETE_SUCCESS = 0;
-  public static final int DELETE_FAILED = -1;
-
-  // currently only one state, will have CONVERTED, and MERGED in future JIRA
-  public static final String IMAGE_STATE_ORIG = "ORIGINAL";
-  public static final String IMAGE_STATE_CONVERT = "CONVERTED";
-  public static final String IMAGE_STATE_MERGE = "MERGED";
-  public static final String IMAGE_STATE_CONVERT_MERGE = "CONVERTED,MERGED";
-
-  // backup image, the dependency graph is made up by series of backup images
-
-  public static class BackupImage implements Comparable<BackupImage> {
-
-    private String backupId;
-    private String type;
-    private String rootDir;
-    private String tableSet;
-    private long startTs;
-    private long completeTs;
-    private ArrayList<BackupImage> ancestors;
-
-    public BackupImage() {
-      super();
-    }
-
-    public BackupImage(String backupId, String type, String rootDir, String tableSet, long startTs,
-        long completeTs) {
-      this.backupId = backupId;
-      this.type = type;
-      this.rootDir = rootDir;
-      this.tableSet = tableSet;
-      this.startTs = startTs;
-      this.completeTs = completeTs;
-    }
-
-    public String getBackupId() {
-      return backupId;
-    }
-
-    public void setBackupId(String backupId) {
-      this.backupId = backupId;
-    }
-
-    public String getType() {
-      return type;
-    }
-
-    public void setType(String type) {
-      this.type = type;
-    }
-
-    public String getRootDir() {
-      return rootDir;
-    }
-
-    public void setRootDir(String rootDir) {
-      this.rootDir = rootDir;
-    }
-
-    public String getTableSet() {
-      return tableSet;
-    }
-
-    public void setTableSet(String tableSet) {
-      this.tableSet = tableSet;
-    }
-
-    public long getStartTs() {
-      return startTs;
-    }
-
-    public void setStartTs(long startTs) {
-      this.startTs = startTs;
-    }
-
-    public long getCompleteTs() {
-      return completeTs;
-    }
-
-    public void setCompleteTs(long completeTs) {
-      this.completeTs = completeTs;
-    }
-
-    public ArrayList<BackupImage> getAncestors() {
-      if (this.ancestors == null) {
-        this.ancestors = new ArrayList<BackupImage>();
-      }
-      return this.ancestors;
-    }
-
-    public void addAncestor(BackupImage backupImage) {
-      this.getAncestors().add(backupImage);
-    }
-
-    public boolean hasAncestor(String token) {
-      for (BackupImage image : this.getAncestors()) {
-        if (image.getBackupId().equals(token)) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    public boolean hasTable(String table) {
-      String[] tables = this.getTableSet().split(";");
-      for (String t : tables) {
-        if (t.equals(table)) {
-          return true;
-        }
-      }
-      return false;
-    }
-
-    @Override
-    public int compareTo(BackupImage other) {
-      String thisBackupId = this.getBackupId();
-      String otherBackupId = other.getBackupId();
-      Long thisTS = new Long(thisBackupId.substring(thisBackupId.lastIndexOf("_") + 1));
-      Long otherTS = new Long(otherBackupId.substring(otherBackupId.lastIndexOf("_") + 1));
-      return thisTS.compareTo(otherTS);
-    }
-  }
-
-  // manifest version
-  private String version = MANIFEST_VERSION;
-
-  // hadoop hbase configuration
-  protected Configuration config = null;
-
-  // backup root directory
-  private String rootDir = null;
-
-  // backup image directory
-  private String tableBackupDir = null;
-
-  // backup log directory if this is an incremental backup
-  private String logBackupDir = null;
-
-  // backup token
-  private String token;
-
-  // backup type, full or incremental
-  private String type;
-
-  // the table set for the backup
-  private ArrayList<String> tableSet;
-
-  // actual start timestamp of the backup process
-  private long startTs;
-
-  // actual complete timestamp of the backup process
-  private long completeTs;
-
-  // total bytes for table backup image
-  private long tableBytes;
-
-  // total bytes for the backed-up logs for incremental backup
-  private long logBytes;
-
-  // the region server timestamp for tables:
-  // <table, <rs, timestamp>>
-  private Map<String, HashMap<String, String>> incrTimeRanges;
-
-  // dependency of this backup, including all the dependent images to do PIT recovery
-  private Map<String, BackupImage> dependency;
-
-  // the state of backup image
-  private String imageState;
-
-  // the indicator of the image compaction
-  private boolean isCompacted = false;
-
-  // the merge chain of the original backups, null if not a merged backup
-  private LinkedList<String> mergeChain;
-
-  /**
-   * Construct manifest for a ongoing backup.
-   * @param backupCtx The ongoing backup context
-   */
-  public BackupManifest(BackupContext backupCtx) {
-    this.token = backupCtx.getBackupId();
-    this.type = backupCtx.getType();
-    this.rootDir = backupCtx.getTargetRootDir();
-    if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-      this.logBackupDir = backupCtx.getHLogTargetDir();
-      this.logBytes = backupCtx.getTotalBytesCopied();
-    }
-    this.startTs = backupCtx.getStartTs();
-    this.completeTs = backupCtx.getEndTs();
-    this.loadTableSet(backupCtx.getTableListAsString());
-    this.setImageOriginal();
-  }
-
-  /**
-   * Construct a table level manifest for a backup of the named table.
-   * @param backupCtx The ongoing backup context
-   */
-  public BackupManifest(BackupContext backupCtx, String table) {
-    this.token = backupCtx.getBackupId();
-    this.type = backupCtx.getType();
-    this.rootDir = backupCtx.getTargetRootDir();
-    this.tableBackupDir = backupCtx.getBackupStatus(table).getTargetDir();
-    if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-      this.logBackupDir = backupCtx.getHLogTargetDir();
-      this.logBytes = backupCtx.getTotalBytesCopied();
-    }
-    this.startTs = backupCtx.getStartTs();
-    this.completeTs = backupCtx.getEndTs();
-    this.loadTableSet(table);
-    this.setImageOriginal();
-  }
-
-  /**
-   * Construct manifest from a backup directory.
-   * @param conf configuration
-   * @param backupPath backup path
-   * @throws BackupException exception
-   */
-  public BackupManifest(Configuration conf, Path backupPath) throws BackupException {
-
-    LOG.debug("Loading manifest from: " + backupPath.toString());
-    // The input backupDir may not exactly be the backup table dir.
-    // It could be the backup log dir where there is also a manifest file stored.
-    // This variable's purpose is to keep the correct and original location so
-    // that we can store/persist it.
-    this.tableBackupDir = backupPath.toString();
-    this.config = conf;
-    try {
-
-      FileSystem fs = backupPath.getFileSystem(conf);
-      FileStatus[] subFiles = FSUtils.listStatus(fs, backupPath);
-      if (subFiles == null) {
-        String errorMsg = backupPath.toString() + " does not exist";
-        LOG.error(errorMsg);
-        throw new IOException(errorMsg);
-      }
-      for (FileStatus subFile : subFiles) {
-        if (subFile.getPath().getName().equals(FILE_NAME)) {
-
-          // load and set manifest field from file content
-          FSDataInputStream in = fs.open(subFile.getPath());
-          Properties props = new Properties();
-          try {
-            props.load(in);
-          } catch (IOException e) {
-            LOG.error("Error when loading from manifest file!");
-            throw e;
-          } finally {
-            in.close();
-          }
-
-          this.version = props.getProperty(TAG_VERSION);
-          this.token = props.getProperty(TAG_BACKUPID);
-          this.type = props.getProperty(TAG_BACKUPTYPE);
-          // Here the parameter backupDir is where the manifest file is.
-          // There should always be a manifest file under:
-          // backupRootDir/namespace/table/backupId/.backup.manifest
-          this.rootDir = backupPath.getParent().getParent().getParent().toString();
-
-          Path p = backupPath.getParent();
-          if (p.getName().equals(HConstants.HREGION_LOGDIR_NAME)) {
-            this.rootDir = p.getParent().toString();
-          } else {
-            this.rootDir = p.getParent().getParent().toString();
-          }
-
-          this.loadTableSet(props.getProperty(TAG_TABLESET));
-
-          this.startTs = Long.parseLong(props.getProperty(TAG_STARTTS));
-          this.completeTs = Long.parseLong(props.getProperty(TAG_COMPLETETS));
-          this.tableBytes = Long.parseLong(props.getProperty(TAG_TABLEBYTES));
-          if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-            this.logBytes = (Long.parseLong(props.getProperty(TAG_LOGBYTES)));
-            LOG.debug("convert will be implemented by future jira");
-          }
-          this.loadIncrementalTimeRanges(props.getProperty(TAG_INCRTIMERANGE));
-          this.loadDependency(props.getProperty(TAG_DEPENDENCY));
-          this.imageState = props.getProperty(TAG_IMAGESTATE);
-          this.isCompacted =
-              props.getProperty(TAG_COMPACTION).equalsIgnoreCase("TRUE") ? true : false;
-          LOG.debug("merge and from existing snapshot will be implemented by future jira");
-          LOG.debug("Loaded manifest instance from manifest file: "
-              + FSUtils.getPath(subFile.getPath()));
-          return;
-        }
-      }
-      String errorMsg = "No manifest file found in: " + backupPath.toString();
-      LOG.error(errorMsg);
-      throw new IOException(errorMsg);
-
-    } catch (IOException e) {
-      throw new BackupException(e.getMessage());
-    }
-  }
-
-  public String getType() {
-    return type;
-  }
-
-  public void setType(String type) {
-    this.type = type;
-  }
-
-  /**
-   * Load table set from a table set list string (t1;t2;t3;...).
-   * @param tableSetStr Table set list string
-   */
-  private void loadTableSet(String tableSetStr) {
-
-    LOG.debug("Loading table set: " + tableSetStr);
-
-    String[] tableSet = tableSetStr.split(";");
-    this.tableSet = this.getTableSet();
-    if (this.tableSet.size() > 0) {
-      this.tableSet.clear();
-    }
-    for (int i = 0; i < tableSet.length; i++) {
-      this.tableSet.add(tableSet[i]);
-    }
-
-    LOG.debug(tableSet.length + " tables exist in table set.");
-  }
-
-  public void setImageOriginal() {
-    this.imageState = IMAGE_STATE_ORIG;
-  }
-
-  /**
-   * Get the table set of this image.
-   * @return The table set list
-   */
-  public ArrayList<String> getTableSet() {
-    if (this.tableSet == null) {
-      this.tableSet = new ArrayList<String>();
-    }
-    return this.tableSet;
-  }
-
-  /**
-   * Persist the manifest file.
-   * @throws IOException IOException when storing the manifest file.
-   */
-  public void store(Configuration conf) throws BackupException {
-    Properties props = new Properties();
-    props.setProperty(TAG_VERSION, this.version);
-    props.setProperty(TAG_BACKUPID, this.token);
-    props.setProperty(TAG_BACKUPTYPE, this.type);
-    props.setProperty(TAG_TABLESET, this.getTableSetStr());
-    LOG.debug("convert will be supported in future jira");
-    // String convertedTables = this.getConvertedTableSetStr();
-    // if (convertedTables != null )
-    // props.setProperty(TAG_CONVERTEDTABLESET, convertedTables);
-    props.setProperty(TAG_STARTTS, Long.toString(this.startTs));
-    props.setProperty(TAG_COMPLETETS, Long.toString(this.completeTs));
-    props.setProperty(TAG_TABLEBYTES, Long.toString(this.tableBytes));
-    if (this.type.equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-      props.setProperty(TAG_LOGBYTES, Long.toString(this.logBytes));
-    }
-    props.setProperty(TAG_INCRTIMERANGE, this.getIncrTimestampStr());
-    props.setProperty(TAG_DEPENDENCY, this.getDependencyStr());
-    props.setProperty(TAG_IMAGESTATE, this.getImageState());
-    props.setProperty(TAG_COMPACTION, this.isCompacted ? "TRUE" : "FALSE");
-    LOG.debug("merge will be supported in future jira");
-    // props.setProperty(TAG_MERGECHAIN, this.getMergeChainStr());
-    LOG.debug("backup from existing snapshot will be supported in future jira");
-    // props.setProperty(TAG_FROMSNAPSHOT, this.isFromSnapshot() ? "TRUE" : "FALSE");
-
-    // write the file, overwrite if already exist
-    Path manifestFilePath =
-        new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir)
-            + File.separator + FILE_NAME);
-    try {
-      FSDataOutputStream out = manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);
-      props.store(out, "HBase backup manifest.");
-      out.close();
-    } catch (IOException e) {
-      throw new BackupException(e.getMessage());
-    }
-
-    LOG.debug("Manifest file stored to " + this.tableBackupDir != null ? this.tableBackupDir
-        : this.logBackupDir + File.separator + FILE_NAME);
-  }
-
-  /**
-   * Get the table set string in the format of t1;t2;t3...
-   */
-  private String getTableSetStr() {
-    return BackupUtil.concat(getTableSet(), ";");
-  }
-
-  public String getImageState() {
-    return imageState;
-  }
-
-  public String getVersion() {
-    return version;
-  }
-
-  /**
-   * Get this backup image.
-   * @return the backup image.
-   */
-  public BackupImage getBackupImage() {
-    return this.getDependency().get(this.token);
-  }
-
-  /**
-   * Add dependent backup image for this backup.
-   * @param image The direct dependent backup image
-   */
-  public void addDependentImage(BackupImage image) {
-    this.getDependency().get(this.token).addAncestor(image);
-    this.setDependencyMap(this.getDependency(), image);
-  }
-
-  /**
-   * Get the dependency' string in the json format.
-   */
-  private String getDependencyStr() {
-    BackupImage thisImage = this.getDependency().get(this.token);
-    if (thisImage == null) {
-      LOG.warn("There is no dependency set yet.");
-      return null;
-    }
-
-    ObjectMapper mapper = new ObjectMapper();
-    try {
-      return mapper.writeValueAsString(thisImage);
-    } catch (JsonGenerationException e) {
-      LOG.error("Error when generating dependency string from backup image.", e);
-      return ERROR_DEPENDENCY;
-    } catch (JsonMappingException e) {
-      LOG.error("Error when generating dependency string from backup image.", e);
-      return ERROR_DEPENDENCY;
-    } catch (IOException e) {
-      LOG.error("Error when generating dependency string from backup image.", e);
-      return ERROR_DEPENDENCY;
-    }
-  }
-
-  /**
-   * Get all dependent backup images. The image of this backup is also contained.
-   * @return The dependent backup images map
-   */
-  public Map<String, BackupImage> getDependency() {
-    if (this.dependency == null) {
-      this.dependency = new HashMap<String, BackupImage>();
-      LOG.debug(this.rootDir + " " + this.token + " " + this.type);
-      this.dependency.put(this.token,
-        new BackupImage(this.token, this.type, this.rootDir, this.getTableSetStr(), this.startTs,
-            this.completeTs));
-    }
-    return this.dependency;
-  }
-
-  /**
-   * Set the incremental timestamp map directly.
-   * @param incrTimestampMap timestamp map
-   */
-  public void setIncrTimestampMap(HashMap<String, HashMap<String, String>> incrTimestampMap) {
-    this.incrTimeRanges = incrTimestampMap;
-  }
-
-  /**
-   * Get the incremental time range string in the format of:
-   * t1,rs1:ts,rs2:ts,...;t2,rs1:ts,rs2:ts,...;t3,rs1:ts,rs2:ts,...
-   */
-  private String getIncrTimestampStr() {
-    StringBuilder sb = new StringBuilder();
-    for (Entry<String, HashMap<String, String>> tableEntry : this.getIncrTimestamps().entrySet()) {
-      sb.append(tableEntry.getKey() + ","); // table
-      for (Entry<String, String> rsEntry : tableEntry.getValue().entrySet()) {
-        sb.append(rsEntry.getKey() + ":"); // region server
-        sb.append(rsEntry.getValue() + ","); // timestamp
-      }
-      if (sb.length() > 1 && sb.charAt(sb.length() - 1) == ',') {
-        sb.deleteCharAt(sb.length() - 1);
-      }
-      sb.append(";");
-    }
-    if (sb.length() > 1 && sb.charAt(sb.length() - 1) == ';') {
-      sb.deleteCharAt(sb.length() - 1);
-    }
-    return sb.toString();
-  }
-
-  public Map<String, HashMap<String, String>> getIncrTimestamps() {
-    if (this.incrTimeRanges == null) {
-      this.incrTimeRanges = new HashMap<String, HashMap<String, String>>();
-    }
-    return this.incrTimeRanges;
-  }
-
-  /**
-   * Load incremental timestamps from a given string, and store them in the collection. The
-   * timestamps in string is in the format of
-   * t1,rs1:ts,rs2:ts,...;t2,rs1:ts,rs2:ts,...;t3,rs1:ts,rs2:ts,...
-   * @param timeRangesInStr Incremental time ranges in string
-   */
-  private void loadIncrementalTimeRanges(String timeRangesStr) throws IOException {
-
-    LOG.debug("Loading table's incremental time ranges of region servers from string in manifest: "
-        + timeRangesStr);
-
-    Map<String, HashMap<String, String>> timeRangeMap = this.getIncrTimestamps();
-
-    String[] entriesOfTables = timeRangesStr.split(";");
-    for (int i = 0; i < entriesOfTables.length; i++) {
-      String[] itemsForTable = entriesOfTables[i].split(",");
-
-      // validate the incremental timestamps string format for a table:
-      // t1,rs1:ts,rs2:ts,...
-      if (itemsForTable.length < 1) {
-        String errorMsg = "Wrong incremental time range format: " + timeRangesStr;
-        LOG.error(errorMsg);
-        throw new IOException(errorMsg);
-      }
-
-      HashMap<String, String> rsTimestampMap = new HashMap<String, String>();
-      for (int j = 1; j < itemsForTable.length; j++) {
-        String[] rsTsEntry = itemsForTable[j].split(":");
-
-        // validate the incremental timestamps string format for a region server:
-        // rs1:ts
-        if (rsTsEntry.length != 2) {
-          String errorMsg = "Wrong incremental timestamp format: " + itemsForTable[j];
-          LOG.error(errorMsg);
-          throw new IOException(errorMsg);
-        }
-
-        // an entry for timestamp of a region server
-        rsTimestampMap.put(rsTsEntry[0], rsTsEntry[1]);
-      }
-
-      timeRangeMap.put(itemsForTable[0], rsTimestampMap);
-    }
-
-    // all entries have been loaded
-    LOG.debug(entriesOfTables.length + " tables' incremental time ranges have been loaded.");
-  }
-
-  /**
-   * Get the image list of this backup for restore in time order.
-   * @param reverse If true, then output in reverse order, otherwise in time order from old to new
-   * @return the backup image list for restore in time order
-   */
-  public ArrayList<BackupImage> getRestoreDependentList(boolean reverse) {
-    TreeMap<Long, BackupImage> restoreImages = new TreeMap<Long, BackupImage>();
-    for (BackupImage image : this.getDependency().values()) {
-      restoreImages.put(Long.valueOf(image.startTs), image);
-    }
-    return new ArrayList<BackupImage>(reverse ? (restoreImages.descendingMap().values())
-        : (restoreImages.values()));
-  }
-
-  /**
-   * Get the dependent image list for a specific table of this backup in time order from old to new
-   * if want to restore to this backup image level.
-   * @param table table
-   * @return the backup image list for a table in time order
-   */
-  public ArrayList<BackupImage> getDependentListByTable(String table) {
-    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
-    ArrayList<BackupImage> imageList = getRestoreDependentList(true);
-    for (BackupImage image : imageList) {
-      if (image.hasTable(table)) {
-        tableImageList.add(image);
-        if (image.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL)) {
-          break;
-        }
-      }
-    }
-    Collections.reverse(tableImageList);
-    return tableImageList;
-  }
-
-  /**
-   * Get the full dependent image list in the whole dependency scope for a specific table of this
-   * backup in time order from old to new.
-   * @param table table
-   * @return the full backup image list for a table in time order in the whole scope of the
-   *         dependency of this image
-   */
-  public ArrayList<BackupImage> getAllDependentListByTable(String table) {
-    ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
-    ArrayList<BackupImage> imageList = getRestoreDependentList(false);
-    for (BackupImage image : imageList) {
-      if (image.hasTable(table)) {
-        tableImageList.add(image);
-      }
-    }
-    return tableImageList;
-  }
-
-  /**
-   * Load dependency from a dependency json string.
-   * @param dependencyStr The dependency string
-   * @throws IOException exception
-   */
-  private void loadDependency(String dependencyStr) throws IOException {
-
-    LOG.debug("Loading dependency: " + dependencyStr);
-
-    String msg = "Dependency is broken in the manifest.";
-    if (dependencyStr.equals(ERROR_DEPENDENCY)) {
-      throw new IOException(msg);
-    }
-
-    ObjectMapper mapper = new ObjectMapper();
-    BackupImage image = null;
-    try {
-      image = mapper.readValue(dependencyStr, BackupImage.class);
-    } catch (JsonParseException e) {
-      LOG.error(msg);
-      throw new IOException(e.getMessage());
-    } catch (JsonMappingException e) {
-      LOG.error(msg);
-      throw new IOException(e.getMessage());
-    } catch (IOException e) {
-      LOG.error(msg);
-      throw new IOException(e.getMessage());
-    }
-    LOG.debug("Manifest's current backup image information:");
-    LOG.debug("  Token: " + image.getBackupId());
-    LOG.debug("  Backup directory: " + image.getRootDir());
-    this.setDependencyMap(this.getDependency(), image);
-
-    LOG.debug("Dependent images map:");
-    for (Entry<String, BackupImage> entry : this.getDependency().entrySet()) {
-      LOG.debug("  " + entry.getKey() + " : " + entry.getValue().getBackupId() + " -- "
-          + entry.getValue().getRootDir());
-    }
-
-    LOG.debug("Dependency has been loaded.");
-  }
-
-  /**
-   * Recursively set the dependency map of the backup images.
-   * @param map The dependency map
-   * @param image The backup image
-   */
-  private void setDependencyMap(Map<String, BackupImage> map, BackupImage image) {
-    if (image == null) {
-      return;
-    } else {
-      map.put(image.getBackupId(), image);
-      for (BackupImage img : image.getAncestors()) {
-        setDependencyMap(map, img);
-      }
-    }
-  }
-
-  /**
-   * Check whether backup image1 could cover backup image2 or not.
-   * @param image1 backup image 1
-   * @param image2 backup image 2
-   * @return true if image1 can cover image2, otherwise false
-   */
-  public static boolean canCoverImage(BackupImage image1, BackupImage image2) {
-    // image1 can cover image2 only when the following conditions are satisfied:
-    // - image1 must not be an incremental image;
-    // - image1 must be taken after image2 has been taken;
-    // - table set of image1 must cover the table set of image2.
-    if (image1.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-      return false;
-    }
-    if (image1.getStartTs() < image2.getStartTs()) {
-      return false;
-    }
-    String[] image1TableSet = image1.getTableSet().split(";");
-    String[] image2TableSet = image2.getTableSet().split(";");
-    boolean found = false;
-    for (int i = 0; i < image2TableSet.length; i++) {
-      found = false;
-      for (int j = 0; j < image1TableSet.length; j++) {
-        if (image2TableSet[i].equals(image1TableSet[j])) {
-          found = true;
-          break;
-        }
-      }
-      if (!found) {
-        return false;
-      }
-    }
-
-    LOG.debug("Backup image " + image1.getBackupId() + " can cover " + image2.getBackupId());
-    return true;
-  }
-
-  /**
-   * Check whether backup image set could cover a backup image or not.
-   * @param fullImages The backup image set
-   * @param image The target backup image
-   * @return true if fullImages can cover image, otherwise false
-   */
-  public static boolean canCoverImage(ArrayList<BackupImage> fullImages, BackupImage image) {
-    // fullImages can cover image only when the following conditions are satisfied:
-    // - each image of fullImages must not be an incremental image;
-    // - each image of fullImages must be taken after image has been taken;
-    // - sum table set of fullImages must cover the table set of image.
-    for (BackupImage image1 : fullImages) {
-      if (image1.getType().equals(BackupRestoreConstants.BACKUP_TYPE_INCR)) {
-        return false;
-      }
-      if (image1.getStartTs() < image.getStartTs()) {
-        return false;
-      }
-    }
-
-    ArrayList<String> image1TableSet = new ArrayList<String>();
-    for (BackupImage image1 : fullImages) {
-      String[] tableSet = image1.getTableSet().split(";");
-      for (String table : tableSet) {
-        image1TableSet.add(table);
-      }
-    }
-    ArrayList<String> image2TableSet = new ArrayList<String>();
-    String[] tableSet = image.getTableSet().split(";");
-    for (String table : tableSet) {
-      image2TableSet.add(table);
-    }
-    
-    for (int i = 0; i < image2TableSet.size(); i++) {
-      if (image1TableSet.contains(image2TableSet.get(i)) == false) {
-        return false;
-      }
-    }
-
-    LOG.debug("Full image set can cover image " + image.getBackupId());
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
deleted file mode 100644
index 9610b27..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupRestoreConstants.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * HConstants holds a bunch of HBase Backup and Restore constants
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public final class BackupRestoreConstants {
-
-  // constants for znode data keys in backup znode
-  public static final String BACKUP_PROGRESS = "progress";
-  public static final String BACKUP_START_TIME = "startTs";
-  public static final String BACKUP_INPROGRESS_PHASE = "phase";
-  public static final String BACKUP_COMPLETE_TIME = "completeTs";
-  public static final String BACKUP_FAIL_TIME = "failedTs";
-  public static final String BACKUP_FAIL_PHASE = "failedphase";
-  public static final String BACKUP_FAIL_MSG = "failedmessage";
-  public static final String BACKUP_ROOT_PATH = "targetRootDir";
-  public static final String BACKUP_REQUEST_TABLE_LIST = "tablelist";
-  public static final String BACKUP_REQUEST_TYPE = "type";
-  public static final String BACKUP_BYTES_COPIED = "bytescopied";
-  public static final String BACKUP_ANCESTORS = "ancestors";
-  public static final String BACKUP_EXISTINGSNAPSHOT = "snapshot";
-
-  public static final String BACKUP_TYPE_FULL = "full";
-  public static final String BACKUP_TYPE_INCR = "incremental";
-
-  // delimiter in tablename list in restore command
-  public static final String TABLENAME_DELIMITER_IN_COMMAND = ",";
-
-  // delimiter in znode data
-  public static final String ZNODE_DATA_DELIMITER = ",";
-
-  public static final String CONF_STAGING_ROOT = "snapshot.export.staging.root";
-
-  public static final String BACKUPID_PREFIX = "backup_";
-
-  public static enum BACKUP_COMMAND {
-    CREATE, CANCEL, DELETE, DESCRIBE, HISTORY, STATUS, CONVERT, MERGE, STOP, SHOW, HELP,
-  }
-
-  private BackupRestoreConstants() {
-    // Can't be instantiated with this ctor.
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/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
deleted file mode 100644
index c26e1d4..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/BackupStatus.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.Serializable;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * Backup status and related information encapsulated for a table.
- * At this moment only TargetDir and SnapshotName is encapsulated here.
- * future Jira will be implemented for progress, bytesCopies, phase, etc.
- */
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BackupStatus implements Serializable {
-
-  private static final long serialVersionUID = -5968397963548535982L;
-
-  // table name for backup
-  private String 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(String 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 String getTable() {
-    return table;
-  }
-}


[3/6] hbase git commit: HBASE-14030 Revert due to pending review comments

Posted by te...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
deleted file mode 100644
index ae21b33..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
+++ /dev/null
@@ -1,496 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeSet;
-
-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.BackupManifest.BackupImage;
-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.log4j.Level;
-import org.apache.log4j.Logger;
-
-/**
- * The main class which interprets the given arguments and trigger restore operation.
- */
-
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public final class RestoreClient {
-
-  private static final Log LOG = LogFactory.getLog(RestoreClient.class);
-
-  private static Options opt;
-  private static Configuration conf;
-  private static Set<BackupImage> lastRestoreImagesSet;
-
-  // delimiter in tablename list in restore command
-  private static final String DELIMITER_IN_COMMAND = ",";
-
-  private static final String OPTION_OVERWRITE = "overwrite";
-  private static final String OPTION_CHECK = "check";
-  private static final String OPTION_AUTOMATIC = "automatic";
-
-  private static final String USAGE =
-      "Usage: hbase restore <backup_root_path> <backup_id> <tables> [tableMapping] \n"
-          + "       [-overwrite] [-check] [-automatic]\n"
-          + " backup_root_path  The parent location where the backup images are stored\n"
-          + " backup_id         The id identifying the backup image\n"
-          + " table(s)          Table(s) from the backup image to be restored.\n"
-          + "                   Tables are separated by comma.\n"
-          + " Options:\n"
-          + "   tableMapping    A comma separated list of target tables.\n"
-          + "                   If specified, each table in <tables> must have a mapping.\n"
-          + "   -overwrite      With this option, restore overwrites to the existing table "
-          + "if there's any in\n"
-          + "                   restore target. The existing table must be online before restore.\n"
-          + "   -check          With this option, restore sequence and dependencies are checked\n"
-          + "                   and verified without executing the restore\n"
-          + "   -automatic      With this option, all the dependencies are automatically restored\n"
-          + "                   together with this backup image following the correct order.\n"
-          + "                   The restore dependencies can be checked by using \"-check\" "
-          + "option,\n"
-          + "                   or using \"hbase backup describe\" command. Without this option, "
-          + "only\n" + "                   this backup image is restored\n";
-
-  private RestoreClient(){
-    throw new AssertionError("Instantiating utility class...");
-  }
-
-  protected static void init() throws IOException {
-    // define supported options
-    opt = new Options();
-    opt.addOption(OPTION_OVERWRITE, false,
-        "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");
-
-    conf = getConf();
-
-    // disable irrelevant loggers to avoid it mess up command output
-    disableUselessLoggers();
-  }
-
-  public static void main(String[] args) throws IOException {
-    init();
-    parseAndRun(args);
-  }
-
-  private static void parseAndRun(String[] args) {
-    CommandLine cmd = null;
-    try {
-      cmd = new PosixParser().parse(opt, args);
-    } catch (ParseException e) {
-      LOG.error("Could not parse command", e);
-      System.exit(-1);
-    }
-
-    // enable debug logging
-    Logger backupClientLogger = Logger.getLogger("org.apache.hadoop.hbase.backup");
-    if (cmd.hasOption("debug")) {
-      backupClientLogger.setLevel(Level.DEBUG);
-    }
-
-    // whether to overwrite to existing table if any, false by default
-    boolean isOverwrite = cmd.hasOption(OPTION_OVERWRITE);
-    if (isOverwrite) {
-      LOG.debug("Found -overwrite option in restore command, "
-          + "will overwrite to existing table if any in the restore target");
-    }
-
-    // whether to only check the dependencies, false by default
-    boolean check = cmd.hasOption(OPTION_CHECK);
-    if (check) {
-      LOG.debug("Found -check option in restore command, "
-          + "will check and verify the dependencies");
-    }
-
-    // whether to restore all dependencies, false by default
-    boolean autoRestore = cmd.hasOption(OPTION_AUTOMATIC);
-    if (autoRestore) {
-      LOG.debug("Found -automatic option in restore command, "
-          + "will automatically retore all the dependencies");
-    }
-
-    // parse main restore command options
-    String[] remainArgs = cmd.getArgs();
-    if (remainArgs.length < 3) {
-      System.out.println("ERROR: missing arguments");
-      System.out.println(USAGE);
-      System.exit(-1);
-    }
-
-    String backupRootDir = remainArgs[0];
-    String backupId = remainArgs[1];
-    String tables = remainArgs[2];
-
-    String tableMapping = (remainArgs.length > 3) ? remainArgs[3] : null;
-
-    String[] sTableArray = (tables != null) ? tables.split(DELIMITER_IN_COMMAND) : null;
-    String[] tTableArray = (tableMapping != null) ? tableMapping.split(DELIMITER_IN_COMMAND) : null;
-
-    if (tableMapping != null && tTableArray != null && (sTableArray.length != tTableArray.length)) {
-      System.err.println("ERROR: table mapping mismatch: " + tables + " : " + tableMapping);
-      System.out.println(USAGE);
-      System.exit(-1);
-    }
-
-    try {
-      HBackupFileSystem hBackupFS = new HBackupFileSystem(conf, new Path(backupRootDir), backupId);
-      restore_stage1(hBackupFS, backupRootDir, backupId, check, autoRestore, sTableArray,
-        tTableArray, isOverwrite);
-    } catch (IOException e) {
-      System.err.println("ERROR: " + e.getMessage());
-      System.exit(-1);
-    }
-  }
-
-  /**
-   * Restore operation. Stage 1: validate backupManifest, and check target tables
-   * @param hBackupFS to access the backup image
-   * @param backupRootDir The root dir for backup image
-   * @param backupId The backup id for image to be restored
-   * @param check True if only do dependency check
-   * @param autoRestore True if automatically restore following the dependency
-   * @param sTableArray The array of tables to be restored
-   * @param tTableArray The array of mapping tables to restore to
-   * @param isOverwrite True then do restore overwrite if target table exists, otherwise fail the
-   *          request if target table exists
-   * @return True if only do dependency check
-   * @throws IOException if any failure during restore
-   */
-  public static boolean restore_stage1(HBackupFileSystem hBackupFS, String backupRootDir,
-      String backupId, boolean check, boolean autoRestore, String[] sTableArray,
-      String[] tTableArray, boolean isOverwrite) throws IOException {
-
-    HashMap<String, BackupManifest> backupManifestMap = new HashMap<String, BackupManifest>();
-    // check and load backup image manifest for the tables
-    hBackupFS.checkImageManifestExist(backupManifestMap, sTableArray);
-
-    try {
-      // Check and validate the backup image and its dependencies
-      if (check || autoRestore) {
-        if (validate(backupManifestMap)) {
-          LOG.info("Checking backup images: ok");
-        } else {
-          String errMsg = "Some dependencies are missing for restore";
-          LOG.error(errMsg);
-          throw new IOException(errMsg);
-        }
-      }
-
-      // return true if only for check
-      if (check) {
-        return true;
-      }
-
-      if (tTableArray == null) {
-        tTableArray = sTableArray;
-      }
-
-      // check the target tables
-      checkTargetTables(tTableArray, isOverwrite);
-
-      // start restore process
-      Set<BackupImage> restoreImageSet =
-          restore_stage2(hBackupFS, backupManifestMap, sTableArray, tTableArray, autoRestore);
-
-      LOG.info("Restore for " + Arrays.asList(sTableArray) + " are successful!");
-      lastRestoreImagesSet = restoreImageSet;
-
-    } catch (IOException e) {
-      LOG.error("ERROR: restore failed with error: " + e.getMessage());
-      throw e;
-    }
-
-    // not only for check, return false
-    return false;
-  }
-
-  /**
-   * Get last restore image set. The value is globally set for the latest finished restore.
-   * @return the last restore image set
-   */
-  public static Set<BackupImage> getLastRestoreImagesSet() {
-    return lastRestoreImagesSet;
-  }
-
-  private static boolean validate(HashMap<String, BackupManifest> backupManifestMap)
-      throws IOException {
-    boolean isValid = true;
-
-    for (Entry<String, BackupManifest> manifestEntry : backupManifestMap.entrySet()) {
-
-      String table = manifestEntry.getKey();
-      TreeSet<BackupImage> imageSet = new TreeSet<BackupImage>();
-
-      ArrayList<BackupImage> depList = manifestEntry.getValue().getDependentListByTable(table);
-      if (depList != null && !depList.isEmpty()) {
-        imageSet.addAll(depList);
-      }
-
-      // todo merge
-      LOG.debug("merge will be implemented in future jira");
-      // BackupUtil.clearMergedImages(table, imageSet, conf);
-
-      LOG.info("Dependent image(s) from old to new:");
-      for (BackupImage image : imageSet) {
-        String imageDir =
-            HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(), table);
-        if (!HBackupFileSystem.checkPathExist(imageDir, getConf())) {
-          LOG.error("ERROR: backup image does not exist: " + imageDir);
-          isValid = false;
-          break;
-        }
-        // TODO More validation?
-        LOG.info("Backup image: " + image.getBackupId() + " for '" + table + "' is available");
-      }
-    }
-    return isValid;
-  }
-
-  /**
-   * Validate target Tables
-   * @param tTableArray: target tables
-   * @param isOverwrite overwrite existing table
-   * @throws IOException exception
-   */
-  private static void checkTargetTables(String[] tTableArray, boolean isOverwrite)
-      throws IOException {
-    ArrayList<String> existTableList = new ArrayList<String>();
-    ArrayList<String> disabledTableList = new ArrayList<String>();
-
-    // check if the tables already exist
-    HBaseAdmin admin = null;
-    Connection conn = null;
-    try {
-      conn = ConnectionFactory.createConnection(conf);
-      admin = (HBaseAdmin) conn.getAdmin();
-      for (String tableName : tTableArray) {
-        if (admin.tableExists(TableName.valueOf(tableName))) {
-          existTableList.add(tableName);
-          if (admin.isTableDisabled(TableName.valueOf(tableName))) {
-            disabledTableList.add(tableName);
-          }
-        } else {
-          LOG.info("HBase table " + tableName
-              + " does not exist. It will be create during backup process");
-        }
-      }
-    } finally {
-      if (admin != null) {
-        admin.close();
-      }
-      if (conn != null) {
-        conn.close();
-      }
-    }
-
-    if (existTableList.size() > 0) {
-      if (!isOverwrite) {
-        LOG.error("Existing table found in the restore target, please add \"-overwrite\" "
-            + "option in the command if you mean to restore to these existing tables");
-        LOG.info("Existing table list in restore target: " + existTableList);
-        throw new IOException("Existing table found in target while no \"-overwrite\" "
-            + "option found");
-      } else {
-        if (disabledTableList.size() > 0) {
-          LOG.error("Found offline table in the restore target, "
-              + "please enable them before restore with \"-overwrite\" option");
-          LOG.info("Offline table list in restore target: " + disabledTableList);
-          throw new IOException(
-              "Found offline table in the target when restore with \"-overwrite\" option");
-        }
-      }
-    }
-
-  }
-
-  /**
-   * Restore operation. Stage 2: resolved Backup Image dependency
-   * @param hBackupFS to access the backup image
-   * @param backupManifestMap : tableName,  Manifest
-   * @param sTableArray The array of tables to be restored
-   * @param tTableArray The array of mapping tables to restore to
-   * @param autoRestore : yes, restore all the backup images on the dependency list
-   * @return set of BackupImages restored
-   * @throws IOException exception
-   */
-  private static Set<BackupImage> restore_stage2(HBackupFileSystem hBackupFS,
-    HashMap<String, BackupManifest> backupManifestMap, String[] sTableArray,
-    String[] tTableArray, boolean autoRestore) throws IOException {
-    TreeSet<BackupImage> restoreImageSet = new TreeSet<BackupImage>();
-
-    for (int i = 0; i < sTableArray.length; i++) {
-      restoreImageSet.clear();
-      String table = sTableArray[i];
-      BackupManifest manifest = backupManifestMap.get(table);
-      if (autoRestore) {
-        // Get the image list of this backup for restore in time order from old
-        // to new.
-        TreeSet<BackupImage> restoreList =
-            new TreeSet<BackupImage>(manifest.getDependentListByTable(table));
-        LOG.debug("need to clear merged Image. to be implemented in future jira");
-
-        for (BackupImage image : restoreList) {
-          restoreImage(image, table, tTableArray[i]);
-        }
-        restoreImageSet.addAll(restoreList);
-      } else {
-        BackupImage image = manifest.getBackupImage();
-        List<BackupImage> depList = manifest.getDependentListByTable(table);
-        // The dependency list always contains self.
-        if (depList != null && depList.size() > 1) {
-          LOG.warn("Backup image " + image.getBackupId() + " depends on other images.\n"
-              + "this operation will only restore the delta contained within backupImage "
-              + image.getBackupId());
-        }
-        restoreImage(image, table, tTableArray[i]);
-        restoreImageSet.add(image);
-      }
-
-      if (autoRestore) {
-        if (restoreImageSet != null && !restoreImageSet.isEmpty()) {
-          LOG.info("Restore includes the following image(s):");
-          for (BackupImage image : restoreImageSet) {
-            LOG.info("  Backup: "
-                + image.getBackupId()
-                + " "
-                + HBackupFileSystem.getTableBackupDir(image.getRootDir(), image.getBackupId(),
-                  table));
-          }
-        }
-      }
-
-    }
-    return restoreImageSet;
-  }
-
-  /**
-   * Restore operation handle each backupImage
-   * @param image: backupImage
-   * @param sTable: table to be restored
-   * @param tTable: table to be restored to
-   * @throws IOException exception
-   */
-  private static void restoreImage(BackupImage image, String sTable, String tTable)
-      throws IOException {
-
-    Configuration conf = getConf();
-
-    String rootDir = image.getRootDir();
-    LOG.debug("Image root dir " + rootDir);
-    String backupId = image.getBackupId();
-
-    HBackupFileSystem hFS = new HBackupFileSystem(conf, new Path(rootDir), backupId);
-    RestoreUtil restoreTool = new RestoreUtil(conf, hFS);
-    BackupManifest manifest = hFS.getManifest(sTable);
-
-    Path tableBackupPath = hFS.getTableBackupPath(sTable);
-
-    // todo: convert feature will be provided in a future jira
-    boolean converted = false;
-
-    if (manifest.getType().equals(BackupRestoreConstants.BACKUP_TYPE_FULL) || converted) {
-      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from "
-          + (converted ? "converted" : "full") + " backup image " + tableBackupPath.toString());
-      restoreTool.fullRestoreTable(tableBackupPath, sTable, tTable, converted);
-    } else { // incremental Backup
-      String logBackupDir =
-          HBackupFileSystem.getLogBackupDir(image.getRootDir(), image.getBackupId());
-      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from incremental backup image "
-          + logBackupDir);
-      restoreTool.incrementalRestoreTable(logBackupDir, new String[] { sTable },
-        new String[] { tTable });
-    }
-
-    LOG.info(sTable + " has been successfully restored to " + tTable);
-  }
-
-  /**
-   * Set the configuration from a given one.
-   * @param newConf A new given configuration
-   */
-  public synchronized static void setConf(Configuration newConf) {
-    conf = newConf;
-  }
-
-  /**
-   * Get and merge Hadoop and HBase configuration.
-   * @throws IOException exception
-   */
-  protected static Configuration getConf() {
-    if (conf == null) {
-      synchronized (RestoreClient.class) {
-        conf = new Configuration();
-        HBaseConfiguration.merge(conf, HBaseConfiguration.create());
-      }
-    }
-    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());
-
-    // disable other related log to avoid mess up command output
-    Logger otherLogger = Logger.getLogger("org.apache.hadoop.hbase.io.hfile");
-    otherLogger.setLevel(Level.OFF);
-    otherLogger = Logger.getLogger("org.apache.hadoop.hbase.util");
-    otherLogger.setLevel(Level.OFF);
-    otherLogger = Logger.getLogger("org.apache.hadoop.hbase.mapreduce");
-    otherLogger.setLevel(Level.OFF);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java
deleted file mode 100644
index bdb7988..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreUtil.java
+++ /dev/null
@@ -1,503 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.NavigableSet;
-
-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.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-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.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
-import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALSplitter;
-
-/**
- * A collection for methods used by multiple classes to restore HBase tables.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class RestoreUtil {
-
-  public static final Log LOG = LogFactory.getLog(RestoreUtil.class);
-
-  protected Configuration conf = null;
-
-  protected HBackupFileSystem hBackupFS = null;
-
-  // store table name and snapshot dir mapping
-  private final HashMap<String, Path> snapshotMap = new HashMap<String, Path>();
-
-  public RestoreUtil(Configuration conf, HBackupFileSystem hBackupFS) throws IOException {
-    this.conf = conf;
-    this.hBackupFS = hBackupFS;
-  }
-
-  /**
-   * During incremental backup operation. Call WalPlayer to replay WAL in backup image Currently
-   * tableNames and newTablesNames only contain single table, will be expanded to multiple tables in
-   * the future
-   * @param logDir : incremental backup folders, which contains WAL
-   * @param tableNames : source tableNames(table names were backuped)
-   * @param newTableNames : target tableNames(table names to be restored to)
-   * @throws IOException exception
-   */
-  public void incrementalRestoreTable(String logDir, String[] tableNames, String[] newTableNames)
-      throws IOException {
-
-    if (tableNames.length != newTableNames.length) {
-      throw new IOException("Number of source tables adn taget Tables does not match!");
-    }
-
-    // for incremental backup image, expect the table already created either by user or previous
-    // full backup. Here, check that all new tables exists
-    HBaseAdmin admin = null;
-    Connection conn = null;
-    try {
-      conn = ConnectionFactory.createConnection(conf);
-      admin = (HBaseAdmin) conn.getAdmin();
-      for (String tableName : newTableNames) {
-        if (!admin.tableExists(TableName.valueOf(tableName))) {
-          admin.close();
-          throw new IOException("HBase table " + tableName
-            + " does not exist. Create the table first, e.g. by restoring a full backup.");
-        }
-      }
-      IncrementalRestoreService restoreService =
-          BackupRestoreServiceFactory.getIncrementalRestoreService(conf);
-
-      restoreService.run(logDir, tableNames, newTableNames);
-    } finally {
-      if (admin != null) {
-        admin.close();
-      }
-      if(conn != null){
-        conn.close();
-      }
-    }
-  }
-
-  public void fullRestoreTable(Path tableBackupPath, String tableName, String newTableName,
-      boolean converted) throws IOException {
-
-    restoreTableAndCreate(tableName, newTableName, tableBackupPath, converted);
-  }
-
-  private void restoreTableAndCreate(String tableName, String newTableName, Path tableBackupPath,
-       boolean converted) throws IOException {
-    if (newTableName == null || newTableName.equals("")) {
-      newTableName = tableName;
-    }
-
-    FileSystem fileSys = tableBackupPath.getFileSystem(this.conf);
-
-    // get table descriptor first
-    HTableDescriptor tableDescriptor = null;
-
-    Path tableSnapshotPath = hBackupFS.getTableSnapshotPath(tableName);
-
-    if (fileSys.exists(tableSnapshotPath)) {
-      // snapshot path exist means the backup path is in HDFS
-      // check whether snapshot dir already recorded for target table
-      if (snapshotMap.get(tableName) != null) {
-        SnapshotDescription desc =
-            SnapshotDescriptionUtils.readSnapshotInfo(fileSys, tableSnapshotPath);
-        SnapshotManifest manifest = SnapshotManifest.open(conf, fileSys, tableSnapshotPath, desc);
-        tableDescriptor = manifest.getTableDescriptor();
-        LOG.debug("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString()
-          + " while tableName = " + tableName);
-        // HBase 96.0 and 98.0
-        // tableDescriptor =
-        // FSTableDescriptors.getTableDescriptorFromFs(fileSys, snapshotMap.get(tableName));
-      } else {
-        tableDescriptor = hBackupFS.getTableDesc(tableName);
-        LOG.debug("tableSnapshotPath=" + tableSnapshotPath.toString());
-        snapshotMap.put(tableName, hBackupFS.getTableInfoPath(tableName));
-      }
-      if (tableDescriptor == null) {
-        LOG.debug("Found no table descriptor in the snapshot dir, previous schema would be lost");
-      }
-    } else if (converted) {
-      // first check if this is a converted backup image
-      LOG.error("convert will be supported in a future jira");
-    }
-
-    Path tableArchivePath = hBackupFS.getTableArchivePath(tableName);
-    if (tableArchivePath == null) {
-      if (tableDescriptor != null) {
-        // find table descriptor but no archive dir means the table is empty, create table and exit
-        LOG.debug("find table descriptor but no archive dir for table " + tableName
-          + ", will only create table");
-        tableDescriptor.setName(Bytes.toBytes(newTableName));
-        checkAndCreateTable(tableBackupPath, tableName, newTableName, null, tableDescriptor);
-        return;
-      } else {
-        throw new IllegalStateException("Cannot restore hbase table because directory '"
-            + " tableArchivePath is null.");
-      }
-    }
-
-    if (tableDescriptor == null) {
-      tableDescriptor = new HTableDescriptor(newTableName);
-    } else {
-      tableDescriptor.setName(Bytes.toBytes(newTableName));
-    }
-
-    if (!converted) {
-      // record all region dirs:
-      // load all files in dir
-      try {
-        ArrayList<Path> regionPathList = hBackupFS.getRegionList(tableName);
-
-        // should only try to create the table with all region informations, so we could pre-split
-        // the regions in fine grain
-        checkAndCreateTable(tableBackupPath, tableName, newTableName, regionPathList,
-          tableDescriptor);
-        if (tableArchivePath != null) {
-          // start real restore through bulkload
-          // if the backup target is on local cluster, special action needed
-          Path tempTableArchivePath = hBackupFS.checkLocalAndBackup(tableArchivePath);
-          if (tempTableArchivePath.equals(tableArchivePath)) {
-            LOG.debug("TableArchivePath for bulkload using existPath: " + tableArchivePath);
-          } else {
-            regionPathList = hBackupFS.getRegionList(tempTableArchivePath); // point to the tempDir
-            LOG.debug("TableArchivePath for bulkload using tempPath: " + tempTableArchivePath);
-          }
-
-          LoadIncrementalHFiles loader = createLoader(tempTableArchivePath, false);
-          for (Path regionPath : regionPathList) {
-            String regionName = regionPath.toString();
-            LOG.debug("Restoring HFiles from directory " + regionName);
-            String[] args = { regionName, newTableName };
-            loader.run(args);
-          }
-        }
-        // restore the recovered.edits if exists
-        replayRecoveredEditsIfAny(tableBackupPath, tableName, tableDescriptor);
-      } catch (Exception e) {
-        throw new IllegalStateException("Cannot restore hbase table", e);
-      }
-    } else {
-      LOG.debug("convert will be supported in a future jira");
-    }
-  }
-
-  /**
-   * Replay recovered edits from backup.
-   */
-  private void replayRecoveredEditsIfAny(Path tableBackupPath, String tableName,
-      HTableDescriptor newTableHtd) throws IOException {
-
-    LOG.debug("Trying to replay the recovered.edits if exist to the target table "
-        + newTableHtd.getNameAsString() + " from the backup of table " + tableName + ".");
-
-    FileSystem fs = tableBackupPath.getFileSystem(this.conf);
-    ArrayList<Path> regionDirs = hBackupFS.getRegionList(tableName);
-
-    if (regionDirs == null || regionDirs.size() == 0) {
-      LOG.warn("No recovered.edits to be replayed for empty backup of table " + tableName + ".");
-      return;
-    }
-
-    Connection conn = null;
-    try {
-
-      conn = ConnectionFactory.createConnection(conf);
-
-      for (Path regionDir : regionDirs) {
-        // OLD: NavigableSet<Path> files = HLogUtil.getSplitEditFilesSorted(fs, regionDir);
-        NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regionDir);
-
-        if (files == null || files.isEmpty()) {
-          LOG.warn("No recovered.edits found for the region " + regionDir.getName() + ".");
-          return;
-        }
-
-        for (Path edits : files) {
-          if (edits == null || !fs.exists(edits)) {
-            LOG.warn("Null or non-existent edits file: " + edits);
-            continue;
-          }
-
-          HTable table = null;
-          try {
-            table = (HTable) conn.getTable(newTableHtd.getTableName());
-            replayRecoveredEdits(table, fs, edits);
-            table.flushCommits();
-            table.close();
-          } catch (IOException e) {
-            boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
-            if (skipErrors) {
-              Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
-              LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
-                + "=true so continuing. Renamed " + edits + " as " + p, e);
-            } else {
-              throw e;
-            }
-          } finally {
-            if (table != null) {
-              table.close();
-            }
-          }
-        } // for each edit file under a region
-      } // for each region
-
-    } finally {
-      if (conn != null) {
-        conn.close();
-      }
-    }
-  }
-
-  /**
-   * Restore process for an edit entry.
-   * @param htable The target table of restore
-   * @param key HLog key
-   * @param val KVs
-   * @throws IOException exception
-   */
-  private void restoreEdit(HTable htable, WALKey key, WALEdit val) throws IOException {
-    Put put = null;
-    Delete del = null;
-    Cell lastKV = null;
-    for (Cell kv : val.getCells()) {
-      // filtering HLog meta entries, see HLog.completeCacheFlushLogEdit
-      if (WALEdit.isMetaEditFamily(CellUtil.cloneFamily(kv))) {
-        continue;
-      }
-
-      // A WALEdit may contain multiple operations (HBASE-3584) and/or
-      // multiple rows (HBASE-5229).
-      // Aggregate as much as possible into a single Put/Delete
-      // operation before apply the action to the table.
-      if (lastKV == null || lastKV.getTypeByte() != kv.getTypeByte()
-          || !CellUtil.matchingRow(lastKV, kv)) {
-        // row or type changed, write out aggregate KVs.
-        if (put != null) {
-          applyAction(htable, put);
-        }
-        if (del != null) {
-          applyAction(htable, del);
-        }
-
-        if (CellUtil.isDelete(kv)) {
-          del = new Delete(CellUtil.cloneRow(kv));
-        } else {
-          put = new Put(CellUtil.cloneRow(kv));
-        }
-      }
-      if (CellUtil.isDelete(kv)) {
-        del.addDeleteMarker(kv);
-      } else {
-        put.add(kv);
-      }
-      lastKV = kv;
-    }
-    // write residual KVs
-    if (put != null) {
-      applyAction(htable, put);
-    }
-    if (del != null) {
-      applyAction(htable, del);
-    }
-  }
-
-  /**
-   * Apply an action (Put/Delete) to table.
-   * @param table table
-   * @param action action
-   * @throws IOException exception
-   */
-  private void applyAction(HTable table, Mutation action) throws IOException {
-    // The actions are not immutable, so we defensively copy them
-    if (action instanceof Put) {
-      Put put = new Put((Put) action);
-      // put.setWriteToWAL(false);
-      // why do not we do WAL?
-      put.setDurability(Durability.SKIP_WAL);
-      table.put(put);
-    } else if (action instanceof Delete) {
-      Delete delete = new Delete((Delete) action);
-      table.delete(delete);
-    } else {
-      throw new IllegalArgumentException("action must be either Delete or Put");
-    }
-  }
-
-  /**
-   * Replay the given edits.
-   * @param htable The target table of restore
-   * @param fs File system
-   * @param edits Recovered.edits to be replayed
-   * @throws IOException exception
-   */
-  private void replayRecoveredEdits(HTable htable, FileSystem fs, Path edits) throws IOException {
-    LOG.debug("Replaying edits from " + edits + "; path=" + edits);
-
-    WAL.Reader reader = null;
-    try {
-      reader = WALFactory.createReader(fs, edits, this.conf);
-      long editsCount = 0;
-      WAL.Entry entry;
-
-      try {
-        while ((entry = reader.next()) != null) {
-          restoreEdit(htable, entry.getKey(), entry.getEdit());
-          editsCount++;
-        }
-        LOG.debug(editsCount + " edits from " + edits + " have been replayed.");
-
-      } catch (EOFException eof) {
-        Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
-        String msg =
-            "Encountered EOF. Most likely due to Master failure during "
-                + "log spliting, so we have this data in another edit.  "
-                + "Continuing, but renaming " + edits + " as " + p;
-        LOG.warn(msg, eof);
-      } catch (IOException ioe) {
-        // If the IOE resulted from bad file format,
-        // then this problem is idempotent and retrying won't help
-        if (ioe.getCause() instanceof ParseException) {
-          Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
-          String msg =
-              "File corruption encountered!  " + "Continuing, but renaming " + edits + " as " + p;
-          LOG.warn(msg, ioe);
-        } else {
-          // other IO errors may be transient (bad network connection,
-          // checksum exception on one datanode, etc). throw & retry
-          throw ioe;
-        }
-      }
-    } finally {
-      if (reader != null) {
-        reader.close();
-      }
-    }
-  }
-
-  /**
-   * Create a {@link LoadIncrementalHFiles} instance to be used to restore the HFiles of a full
-   * backup.
-   * @return the {@link LoadIncrementalHFiles} instance
-   * @throws IOException exception
-   */
-  private LoadIncrementalHFiles createLoader(Path tableArchivePath, boolean multipleTables)
-      throws IOException {
-    // set configuration for restore:
-    // LoadIncrementalHFile needs more time
-    // <name>hbase.rpc.timeout</name> <value>600000</value>
-    // calculates
-    Integer milliSecInMin = 60000;
-    Integer previousMillis = this.conf.getInt("hbase.rpc.timeout", 0);
-    Integer numberOfFilesInDir =
-        multipleTables ? hBackupFS.getMaxNumberOfFilesInSubDir(tableArchivePath) : hBackupFS
-            .getNumberOfFilesInDir(tableArchivePath);
-    Integer calculatedMillis = numberOfFilesInDir * milliSecInMin; // 1 minute per file
-    Integer resultMillis = Math.max(calculatedMillis, previousMillis);
-    if (resultMillis > previousMillis) {
-      LOG.info("Setting configuration for restore with LoadIncrementalHFile: "
-          + "hbase.rpc.timeout to " + calculatedMillis / milliSecInMin
-          + " minutes, to handle the number of files in backup " + tableArchivePath);
-      this.conf.setInt("hbase.rpc.timeout", resultMillis);
-    }
-
-    LoadIncrementalHFiles loader = null;
-    try {
-      loader = new LoadIncrementalHFiles(this.conf);
-    } catch (Exception e1) {
-      throw new IOException(e1);
-    }
-    return loader;
-  }
-
-  /**
-   * Prepare the table for bulkload, most codes copied from
-   * {@link LoadIncrementalHFiles#createTable(String, String)}
-   * @param tableBackupPath path
-   * @param tableName table name
-   * @param targetTableName target table name
-   * @param regionDirList region directory list
-   * @param htd table descriptor
-   * @throws IOException exception
-   */
-  private void checkAndCreateTable(Path tableBackupPath, String tableName, String targetTableName,
-      ArrayList<Path> regionDirList, HTableDescriptor htd) throws IOException {
-    HBaseAdmin hbadmin = null;
-    Connection conn = null;
-    try {
-      conn = ConnectionFactory.createConnection(conf);
-      hbadmin = (HBaseAdmin) conn.getAdmin();
-      if (hbadmin.tableExists(TableName.valueOf(targetTableName))) {
-        LOG.info("Using exising target table '" + targetTableName + "'");
-      } else {
-        LOG.info("Creating target table '" + targetTableName + "'");
-
-        // if no region dir given, create the table and return
-        if (regionDirList == null || regionDirList.size() == 0) {
-
-          hbadmin.createTable(htd);
-          return;
-        }
-
-        byte[][] keys = hBackupFS.generateBoundaryKeys(regionDirList);
-
-        // create table using table decriptor and region boundaries
-        hbadmin.createTable(htd, keys);
-      }
-    } catch (Exception e) {
-      throw new IOException(e);
-    } finally {
-      if (hbadmin != null) {
-        hbadmin.close();
-      }
-      if(conn != null){
-        conn.close();
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java
deleted file mode 100644
index a3b5db5..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceBackupCopyService.java
+++ /dev/null
@@ -1,292 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.mapreduce;
-
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.backup.BackupCopyService;
-import org.apache.hadoop.hbase.backup.BackupHandler;
-import org.apache.hadoop.hbase.backup.BackupUtil;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.snapshot.ExportSnapshot;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.tools.DistCp;
-import org.apache.hadoop.tools.DistCpConstants;
-import org.apache.hadoop.tools.DistCpOptions;
-/**
- * Copier for backup operation. Basically, there are 2 types of copy. One is copying from snapshot,
- * which bases on extending ExportSnapshot's function with copy progress reporting to ZooKeeper
- * implementation. The other is copying for incremental log files, which bases on extending 
- * DistCp's function with copy progress reporting to ZooKeeper implementation.
- *
- * For now this is only a wrapper. The other features such as progress and increment backup will be
- * implemented in future jira
- */
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class MapReduceBackupCopyService implements BackupCopyService {
-  private static final Log LOG = LogFactory.getLog(MapReduceBackupCopyService.class);
-
-  private Configuration conf;
-  // private static final long BYTES_PER_MAP = 2 * 256 * 1024 * 1024;
-
-  // Accumulated progress within the whole backup process for the copy operation
-  private float progressDone = 0.1f;
-  private long bytesCopied = 0;
-  private static float INIT_PROGRESS = 0.1f;
-
-  // The percentage of the current copy task within the whole task if multiple time copies are
-  // needed. The default value is 100%, which means only 1 copy task for the whole.
-  private float subTaskPercntgInWholeTask = 1f;
-
-  public MapReduceBackupCopyService() {
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  /**
-   * Get the current copy task percentage within the whole task if multiple copies are needed.
-   * @return the current copy task percentage
-   */
-  public float getSubTaskPercntgInWholeTask() {
-    return subTaskPercntgInWholeTask;
-  }
-
-  /**
-   * Set the current copy task percentage within the whole task if multiple copies are needed. Must
-   * be called before calling
-   * {@link #copy(BackupHandler, Configuration, Type, String[])}
-   * @param subTaskPercntgInWholeTask The percentage of the copy subtask
-   */
-  public void setSubTaskPercntgInWholeTask(float subTaskPercntgInWholeTask) {
-    this.subTaskPercntgInWholeTask = subTaskPercntgInWholeTask;
-  }
-
-  class SnapshotCopy extends ExportSnapshot {
-    private BackupHandler backupHandler;
-    private String table;
-
-    public SnapshotCopy(BackupHandler backupHandler, String table) {
-      super();
-      this.backupHandler = backupHandler;
-      this.table = table;
-    }
-
-    public BackupHandler getBackupHandler() {
-      return this.backupHandler;
-    }
-
-    public String getTable() {
-      return this.table;
-    }
-  }  
-  
-  // Extends DistCp for progress updating to hbase:backup
-  // during backup. Using DistCpV2 (MAPREDUCE-2765).
-  // Simply extend it and override execute() method to get the 
-  // Job reference for progress updating.
-  // Only the argument "src1, [src2, [...]] dst" is supported, 
-  // no more DistCp options.
-  class BackupDistCp extends DistCp {
-
-    private BackupHandler backupHandler;
-
-    public BackupDistCp(Configuration conf, DistCpOptions options, BackupHandler backupHandler)
-        throws Exception {
-      super(conf, options);
-      this.backupHandler = backupHandler;
-    }
-
-    @Override
-    public Job execute() throws Exception {
-
-      // reflection preparation for private methods and fields
-      Class<?> classDistCp = org.apache.hadoop.tools.DistCp.class;
-      Method methodCreateMetaFolderPath = classDistCp.getDeclaredMethod("createMetaFolderPath");
-      Method methodCreateJob = classDistCp.getDeclaredMethod("createJob");
-      Method methodCreateInputFileListing =
-          classDistCp.getDeclaredMethod("createInputFileListing", Job.class);
-      Method methodCleanup = classDistCp.getDeclaredMethod("cleanup");
-
-      Field fieldInputOptions = classDistCp.getDeclaredField("inputOptions");
-      Field fieldMetaFolder = classDistCp.getDeclaredField("metaFolder");
-      Field fieldJobFS = classDistCp.getDeclaredField("jobFS");
-      Field fieldSubmitted = classDistCp.getDeclaredField("submitted");
-
-      methodCreateMetaFolderPath.setAccessible(true);
-      methodCreateJob.setAccessible(true);
-      methodCreateInputFileListing.setAccessible(true);
-      methodCleanup.setAccessible(true);
-
-      fieldInputOptions.setAccessible(true);
-      fieldMetaFolder.setAccessible(true);
-      fieldJobFS.setAccessible(true);
-      fieldSubmitted.setAccessible(true);
-
-      // execute() logic starts here
-      assert fieldInputOptions.get(this) != null;
-      assert getConf() != null;
-
-      Job job = null;
-      try {
-        synchronized (this) {
-          // Don't cleanup while we are setting up.
-          fieldMetaFolder.set(this, methodCreateMetaFolderPath.invoke(this));
-          fieldJobFS.set(this, ((Path) fieldMetaFolder.get(this)).getFileSystem(getConf()));
-
-          job = (Job) methodCreateJob.invoke(this);
-        }
-        methodCreateInputFileListing.invoke(this, job);
-
-        // Get the total length of the source files
-        List<Path> srcs = ((DistCpOptions) fieldInputOptions.get(this)).getSourcePaths();
-        long totalSrcLgth = 0;
-        for (Path aSrc : srcs) {
-          totalSrcLgth += BackupUtil.getFilesLength(aSrc.getFileSystem(getConf()), aSrc);
-        }
-
-        // submit the copy job
-        job.submit();
-        fieldSubmitted.set(this, true);
-
-        // after submit the MR job, set its handler in backup handler for cancel process
-        // this.backupHandler.copyJob = job;
-
-        // Update the copy progress to ZK every 0.5s if progress value changed
-        int progressReportFreq =
-            this.getConf().getInt("hbase.backup.progressreport.frequency", 500);
-        float lastProgress = progressDone;
-        while (!job.isComplete()) {
-          float newProgress =
-              progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS);
-
-          if (newProgress > lastProgress) {
-
-            BigDecimal progressData =
-                new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP);
-            String newProgressStr = progressData + "%";
-            LOG.info("Progress: " + newProgressStr);
-            this.backupHandler.updateProgress(newProgressStr, bytesCopied);
-            LOG.debug("Backup progress data updated to hbase:backup: \"Progress: " + newProgressStr
-              + ".\"");
-            lastProgress = newProgress;
-          }
-          Thread.sleep(progressReportFreq);
-        }
-
-        // update the progress data after copy job complete
-        float newProgress =
-            progressDone + job.mapProgress() * subTaskPercntgInWholeTask * (1 - INIT_PROGRESS);
-        BigDecimal progressData =
-            new BigDecimal(newProgress * 100).setScale(1, BigDecimal.ROUND_HALF_UP);
-
-        String newProgressStr = progressData + "%";
-        LOG.info("Progress: " + newProgressStr);
-
-        // accumulate the overall backup progress
-        progressDone = newProgress;
-        bytesCopied += totalSrcLgth;
-
-        this.backupHandler.updateProgress(newProgressStr, bytesCopied);
-        LOG.debug("Backup progress data updated to hbase:backup: \"Progress: " + newProgressStr
-          + " - " + bytesCopied + " bytes copied.\"");
-
-      } finally {
-        if (!fieldSubmitted.getBoolean(this)) {
-          methodCleanup.invoke(this);
-        }
-      }
-
-      String jobID = job.getJobID().toString();
-      job.getConfiguration().set(DistCpConstants.CONF_LABEL_DISTCP_JOB_ID, jobID);
-
-      LOG.debug("DistCp job-id: " + jobID);
-      return job;
-    }
-
-  }
-
-  /**
-   * Do backup copy based on different types.
-   * @param handler The backup handler reference
-   * @param conf The hadoop configuration
-   * @param copyType The backup copy type
-   * @param options Options for customized ExportSnapshot or DistCp
-   * @throws Exception exception
-   */
-  public int copy(BackupHandler handler, Configuration conf, BackupCopyService.Type copyType,
-      String[] options) throws IOException {
-
-    int res = 0;
-
-    try {
-      if (copyType == Type.FULL) {
-        SnapshotCopy snapshotCp =
-            new SnapshotCopy(handler, handler.getBackupContext().getTableBySnapshot(options[1]));
-        LOG.debug("Doing SNAPSHOT_COPY");
-        // Make a new instance of conf to be used by the snapshot copy class.
-        snapshotCp.setConf(new Configuration(conf));
-        res = snapshotCp.run(options);
-      } else if (copyType == Type.INCREMENTAL) {
-        LOG.debug("Doing COPY_TYPE_DISTCP");
-        setSubTaskPercntgInWholeTask(1f);
-
-        BackupDistCp distcp = new BackupDistCp(new Configuration(conf), null, handler);
-        // Handle a special case where the source file is a single file.
-        // In this case, distcp will not create the target dir. It just take the
-        // target as a file name and copy source file to the target (as a file name).
-        // We need to create the target dir before run distcp.
-        LOG.debug("DistCp options: " + Arrays.toString(options));
-        if (options.length == 2) {
-          Path dest = new Path(options[1]);
-          FileSystem destfs = dest.getFileSystem(conf);
-          if (!destfs.exists(dest)) {
-            destfs.mkdirs(dest);
-          }
-        }
-
-        res = distcp.run(options);
-      }
-      return res;
-
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java
deleted file mode 100644
index deefbf7..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceRestoreService.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.mapreduce;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.backup.HBackupFileSystem;
-import org.apache.hadoop.hbase.backup.IncrementalRestoreService;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.mapreduce.WALPlayer;
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class MapReduceRestoreService implements IncrementalRestoreService {
-  public static final Log LOG = LogFactory.getLog(MapReduceRestoreService.class);
-
-  private WALPlayer player;
-
-  public MapReduceRestoreService() {
-    this.player = new WALPlayer();
-  }
-
-  @Override
-  public void run(String logDir, String[] tableNames, String[] newTableNames) throws IOException {
-    String tableStr = HBackupFileSystem.join(tableNames);
-    String newTableStr = HBackupFileSystem.join(newTableNames);
-
-    // WALPlayer reads all files in arbitrary directory structure and creates a Map task for each
-    // log file
-
-    String[] playerArgs = { logDir, tableStr, newTableStr };
-    LOG.info("Restore incremental backup from directory " + logDir + " from hbase tables "
-        + HBackupFileSystem.join(tableNames) + " to tables "
-        + HBackupFileSystem.join(newTableNames));
-    try {
-      player.run(playerArgs);
-    } catch (Exception e) {
-      throw new IOException("cannot restore from backup directory " + logDir
-        + " (check Hadoop and HBase logs) " + e);
-    }
-  }
-
-  @Override
-  public Configuration getConf() {
-    return player.getConf();
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.player.setConf(conf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
deleted file mode 100644
index 4712548..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/BackupLogCleaner.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.backup.master;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.backup.BackupSystemTable;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
-
-
-
-
-/**
- * Implementation of a log cleaner that checks if a log is still scheduled for
- * incremental backup before deleting it when its TTL is over.
- */
-@InterfaceStability.Evolving
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class BackupLogCleaner extends BaseLogCleanerDelegate {
-  private static final Log LOG = LogFactory.getLog(BackupLogCleaner.class);
-
-  private boolean stopped = false;
-
-  public BackupLogCleaner() {
-  }
-
-  @Override
-  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
-    // all members of this class are null if backup is disabled,
-    // so we cannot filter the files
-    if (this.getConf() == null) {
-      return files;
-    }
-
-    try {
-      final BackupSystemTable table = BackupSystemTable.getTable(getConf());
-      // If we do not have recorded backup sessions
-      if (table.hasBackupSessions() == false) {
-        return files;
-      }
-      return Iterables.filter(files, new Predicate<FileStatus>() {
-        @Override
-        public boolean apply(FileStatus file) {
-          try {
-            String wal = file.getPath().toString();
-            boolean logInSystemTable = table.checkWALFile(wal);
-            if (LOG.isDebugEnabled()) {
-              if (logInSystemTable) {
-                LOG.debug("Found log file in hbase:backup, deleting: " + wal);
-              } else {
-                LOG.debug("Didn't find this log in hbase:backup, keeping: " + wal);
-              }
-            }
-            return logInSystemTable;
-          } catch (IOException e) {
-            LOG.error(e);
-            return false;// keep file for a while, HBase failed
-          }
-        }
-      });
-    } catch (IOException e) {
-      LOG.error("Failed to get hbase:backup table, therefore will keep all files", e);
-      // nothing to delete
-      return new ArrayList<FileStatus>();
-    }
-
-  }
-
-  @Override
-  public void setConf(Configuration config) {
-    // If backup is disabled, keep all members null
-    if (!config.getBoolean(HConstants.BACKUP_ENABLE_KEY, HConstants.BACKUP_ENABLE_DEFAULT)) {
-      LOG.warn("Backup is disabled - allowing all wals to be deleted");
-      return;
-    }
-    super.setConf(config);
-  }
-
-  @Override
-  public void stop(String why) {
-    if (this.stopped) {
-      return;
-    }
-    this.stopped = true;
-    LOG.info("Stopping BackupLogCleaner");
-  }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
deleted file mode 100644
index f96682f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/master/LogRollMasterProcedureManager.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.master;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ThreadPoolExecutor;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.errorhandling.ForeignException;
-import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.MetricsMaster;
-import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
-import org.apache.hadoop.hbase.procedure.Procedure;
-import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
-import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
-import org.apache.zookeeper.KeeperException;
-
-public class LogRollMasterProcedureManager extends MasterProcedureManager {
-
-  public static final String ROLLLOG_PROCEDURE_SIGNATURE = "rolllog-proc";
-  public static final String ROLLLOG_PROCEDURE_NAME = "rolllog";
-  private static final Log LOG = LogFactory.getLog(LogRollMasterProcedureManager.class);
-
-  private MasterServices master;
-  private ProcedureCoordinator coordinator;
-  private boolean done;
-
-  @Override
-  public void stop(String why) {
-    LOG.info("stop: " + why);
-  }
-
-  @Override
-  public boolean isStopped() {
-    return false;
-  }
-
-  @Override
-  public void initialize(MasterServices master, MetricsMaster metricsMaster)
-      throws KeeperException, IOException, UnsupportedOperationException {
-    this.master = master;
-    this.done = false;
-
-    // setup the default procedure coordinator
-    String name = master.getServerName().toString();
-    ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, 1);
-    BaseCoordinatedStateManager coordManager =
-        (BaseCoordinatedStateManager) CoordinatedStateManagerFactory
-        .getCoordinatedStateManager(master.getConfiguration());
-    coordManager.initialize(master);
-
-    ProcedureCoordinatorRpcs comms =
-        coordManager.getProcedureCoordinatorRpcs(getProcedureSignature(), name);
-
-    this.coordinator = new ProcedureCoordinator(comms, tpool);
-  }
-
-  @Override
-  public String getProcedureSignature() {
-    return ROLLLOG_PROCEDURE_SIGNATURE;
-  }
-
-  @Override
-  public void execProcedure(ProcedureDescription desc) throws IOException {
-    this.done = false;
-    // start the process on the RS
-    ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(desc.getInstance());
-    List<ServerName> serverNames = master.getServerManager().getOnlineServersList();
-    List<String> servers = new ArrayList<String>();
-    for (ServerName sn : serverNames) {
-      servers.add(sn.toString());
-    }
-    Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), new byte[0], servers);
-    if (proc == null) {
-      String msg = "Failed to submit distributed procedure for '" + desc.getInstance() + "'";
-      LOG.error(msg);
-      throw new IOException(msg);
-    }
-
-    try {
-      // wait for the procedure to complete. A timer thread is kicked off that should cancel this
-      // if it takes too long.
-      proc.waitForCompleted();
-      LOG.info("Done waiting - exec procedure for " + desc.getInstance());
-      LOG.info("Distributed roll log procedure is successful!");
-      this.done = true;
-    } catch (InterruptedException e) {
-      ForeignException ee =
-          new ForeignException("Interrupted while waiting for roll log procdure to finish", e);
-      monitor.receive(ee);
-      Thread.currentThread().interrupt();
-    } catch (ForeignException e) {
-      ForeignException ee =
-          new ForeignException("Exception while waiting for roll log procdure to finish", e);
-      monitor.receive(ee);
-    }
-    monitor.rethrowException();
-  }
-
-  @Override
-  public boolean isProcedureDone(ProcedureDescription desc) throws IOException {
-    return done;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
deleted file mode 100644
index 618748e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedure.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.regionserver;
-
-import java.util.HashMap;
-import java.util.concurrent.Callable;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.backup.BackupSystemTable;
-import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
-import org.apache.hadoop.hbase.errorhandling.ForeignException;
-import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
-import org.apache.hadoop.hbase.procedure.ProcedureMember;
-import org.apache.hadoop.hbase.procedure.Subprocedure;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
-
-
-/**
- * This backup subprocedure implementation forces a log roll on the RS.
- */
-public class LogRollBackupSubprocedure extends Subprocedure {
-  private static final Log LOG = LogFactory.getLog(LogRollBackupSubprocedure.class);
-
-  private final RegionServerServices rss;
-  private final LogRollBackupSubprocedurePool taskManager;
-  private FSHLog hlog;
-
-  public LogRollBackupSubprocedure(RegionServerServices rss, ProcedureMember member,
-      ForeignExceptionDispatcher errorListener, long wakeFrequency, long timeout,
-      LogRollBackupSubprocedurePool taskManager) {
-
-    super(member, LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, errorListener,
-      wakeFrequency, timeout);
-    LOG.info("Constructing a LogRollBackupSubprocedure.");
-    this.rss = rss;
-    this.taskManager = taskManager;
-  }
-
-  /**
-   * Callable task. TODO. We don't need a thread pool to execute roll log. This can be simplified
-   * with no use of subprocedurepool.
-   */
-  class RSRollLogTask implements Callable<Void> {
-    RSRollLogTask() {
-    }
-
-    @Override
-    public Void call() throws Exception {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("++ DRPC started: " + rss.getServerName());
-      }
-      hlog = (FSHLog) rss.getWAL(null);
-      long filenum = hlog.getFilenum();
-
-      LOG.info("Trying to roll log in backup subprocedure, current log number: " + filenum);
-      hlog.rollWriter(true);
-      LOG.info("After roll log in backup subprocedure, current log number: " + hlog.getFilenum());
-      // write the log number to hbase:backup.
-      BackupSystemTable table = BackupSystemTable.getTable(rss.getConfiguration());
-      // sanity check, good for testing
-      HashMap<String, String> serverTimestampMap = table.readRegionServerLastLogRollResult();
-      String host = rss.getServerName().getHostname();
-      String sts = serverTimestampMap.get(host);
-      if (sts != null && Long.parseLong(sts) > filenum) {
-        LOG.warn("Won't update server's last roll log result: current=" + sts + " new=" + filenum);
-        return null;
-      }
-      table.writeRegionServerLastLogRollResult(host, Long.toString(filenum));
-      // TODO: potential leak of HBase connection
-      // BackupSystemTable.close();
-      return null;
-    }
-
-  }
-
-  private void rolllog() throws ForeignException {
-
-    monitor.rethrowException();
-
-    taskManager.submitTask(new RSRollLogTask());
-    monitor.rethrowException();
-
-    // wait for everything to complete.
-    taskManager.waitForOutstandingTasks();
-    monitor.rethrowException();
-
-  }
-
-  @Override
-  public void acquireBarrier() throws ForeignException {
-    // do nothing, executing in inside barrier step.
-  }
-
-  /**
-   * do a log roll.
-   * @return some bytes
-   */
-  @Override
-  public byte[] insideBarrier() throws ForeignException {
-    rolllog();
-    // FIXME
-    return null;
-  }
-
-  /**
-   * Cancel threads if they haven't finished.
-   */
-  @Override
-  public void cleanup(Exception e) {
-    taskManager.abort("Aborting log roll subprocedure tasks for backup due to error", e);
-  }
-
-  /**
-   * Hooray!
-   */
-  public void releaseBarrier() {
-    // NO OP
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
deleted file mode 100644
index 1ca638c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollBackupSubprocedurePool.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.regionserver;
-
-import java.io.Closeable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.DaemonThreadFactory;
-import org.apache.hadoop.hbase.errorhandling.ForeignException;
-
-/**
- * Handle running each of the individual tasks for completing a backup procedure
- * on a regionserver.
- */
-public class LogRollBackupSubprocedurePool implements Closeable, Abortable {
-  private static final Log LOG = LogFactory.getLog(LogRollBackupSubprocedurePool.class);
-
-  /** Maximum number of concurrent snapshot region tasks that can run concurrently */
-  private static final String CONCURENT_BACKUP_TASKS_KEY = "hbase.backup.region.concurrentTasks";
-  private static final int DEFAULT_CONCURRENT_BACKUP_TASKS = 3;
-
-  private final ExecutorCompletionService<Void> taskPool;
-  private final ThreadPoolExecutor executor;
-  private volatile boolean aborted;
-  private final List<Future<Void>> futures = new ArrayList<Future<Void>>();
-  private final String name;
-
-  public LogRollBackupSubprocedurePool(String name, Configuration conf) {
-    // configure the executor service
-    long keepAlive =
-        conf.getLong(LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_KEY,
-          LogRollRegionServerProcedureManager.BACKUP_TIMEOUT_MILLIS_DEFAULT);
-    int threads = conf.getInt(CONCURENT_BACKUP_TASKS_KEY, DEFAULT_CONCURRENT_BACKUP_TASKS);
-    this.name = name;
-    executor =
-        new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.SECONDS,
-          new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("rs(" + name
-            + ")-backup-pool"));
-    taskPool = new ExecutorCompletionService<Void>(executor);
-  }
-
-  /**
-   * Submit a task to the pool.
-   */
-  public void submitTask(final Callable<Void> task) {
-    Future<Void> f = this.taskPool.submit(task);
-    futures.add(f);
-  }
-
-  /**
-   * Wait for all of the currently outstanding tasks submitted via {@link #submitTask(Callable)}
-   * @return <tt>true</tt> on success, <tt>false</tt> otherwise
-   * @throws ForeignException exception
-   */
-  public boolean waitForOutstandingTasks() throws ForeignException {
-    LOG.debug("Waiting for backup procedure to finish.");
-
-    try {
-      for (Future<Void> f : futures) {
-        f.get();
-      }
-      return true;
-    } catch (InterruptedException e) {
-      if (aborted) {
-        throw new ForeignException("Interrupted and found to be aborted while waiting for tasks!",
-            e);
-      }
-      Thread.currentThread().interrupt();
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof ForeignException) {
-        throw (ForeignException) e.getCause();
-      }
-      throw new ForeignException(name, e.getCause());
-    } finally {
-      // close off remaining tasks
-      for (Future<Void> f : futures) {
-        if (!f.isDone()) {
-          f.cancel(true);
-        }
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Attempt to cleanly shutdown any running tasks - allows currently running tasks to cleanly
-   * finish
-   */
-  @Override
-  public void close() {
-    executor.shutdown();
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    if (this.aborted) {
-      return;
-    }
-
-    this.aborted = true;
-    LOG.warn("Aborting because: " + why, e);
-    this.executor.shutdownNow();
-  }
-
-  @Override
-  public boolean isAborted() {
-    return this.aborted;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
deleted file mode 100644
index aca190c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/regionserver/LogRollRegionServerProcedureManager.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.backup.regionserver;
-
-
-import java.io.IOException;
-import java.util.concurrent.ThreadPoolExecutor;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
-import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
-import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
-import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
-import org.apache.hadoop.hbase.procedure.ProcedureMember;
-import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
-import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
-import org.apache.hadoop.hbase.procedure.Subprocedure;
-import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-
-/**
- * This manager class handles the work dealing with backup for a {@link HRegionServer}.
- * <p>
- * This provides the mechanism necessary to kick off a backup specific {@link Subprocedure} that is
- * responsible by this region server. If any failures occur with the subprocedure, the manager's
- * procedure member notifies the procedure coordinator to abort all others.
- * <p>
- * On startup, requires {@link #start()} to be called.
- * <p>
- * On shutdown, requires org.apache.hadoop.hbase.procedure.ProcedureMember.close() to be
- * called
- */
-public class LogRollRegionServerProcedureManager extends RegionServerProcedureManager {
-
-  private static final Log LOG = LogFactory.getLog(LogRollRegionServerProcedureManager.class);
-
-  /** Conf key for number of request threads to start backup on regionservers */
-  public static final String BACKUP_REQUEST_THREADS_KEY = "hbase.backup.region.pool.threads";
-  /** # of threads for backup work on the rs. */
-  public static final int BACKUP_REQUEST_THREADS_DEFAULT = 10;
-
-  public static final String BACKUP_TIMEOUT_MILLIS_KEY = "hbase.backup.timeout";
-  public static final long BACKUP_TIMEOUT_MILLIS_DEFAULT = 60000;
-
-  /** Conf key for millis between checks to see if backup work completed or if there are errors */
-  public static final String BACKUP_REQUEST_WAKE_MILLIS_KEY = "hbase.backup.region.wakefrequency";
-  /** Default amount of time to check for errors while regions finish backup work */
-  private static final long BACKUP_REQUEST_WAKE_MILLIS_DEFAULT = 500;
-
-  private RegionServerServices rss;
-  private ProcedureMemberRpcs memberRpcs;
-  private ProcedureMember member;
-
-  /**
-   * Create a default backup procedure manager
-   */
-  public LogRollRegionServerProcedureManager() {
-  }
-
-  /**
-   * Start accepting backup procedure requests.
-   */
-  @Override
-  public void start() {
-    this.memberRpcs.start(rss.getServerName().toString(), member);
-    LOG.info("Started region server backup manager.");
-  }
-
-  /**
-   * Close <tt>this</tt> and all running backup procedure tasks
-   * @param force forcefully stop all running tasks
-   * @throws IOException exception
-   */
-  @Override
-  public void stop(boolean force) throws IOException {
-    String mode = force ? "abruptly" : "gracefully";
-    LOG.info("Stopping RegionServerBackupManager " + mode + ".");
-
-    try {
-      this.member.close();
-    } finally {
-      this.memberRpcs.close();
-    }
-  }
-
-  /**
-   * If in a running state, creates the specified subprocedure for handling a backup procedure.
-   * @return Subprocedure to submit to the ProcedureMemeber.
-   */
-  public Subprocedure buildSubprocedure() {
-
-    // don't run a backup if the parent is stop(ping)
-    if (rss.isStopping() || rss.isStopped()) {
-      throw new IllegalStateException("Can't start backup procedure on RS: " + rss.getServerName()
-        + ", because stopping/stopped!");
-    }
-
-    LOG.info("Attempting to run a roll log procedure for backup.");
-    ForeignExceptionDispatcher errorDispatcher = new ForeignExceptionDispatcher();
-    Configuration conf = rss.getConfiguration();
-    long timeoutMillis = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT);
-    long wakeMillis =
-        conf.getLong(BACKUP_REQUEST_WAKE_MILLIS_KEY, BACKUP_REQUEST_WAKE_MILLIS_DEFAULT);
-
-    LogRollBackupSubprocedurePool taskManager =
-        new LogRollBackupSubprocedurePool(rss.getServerName().toString(), conf);
-    return new LogRollBackupSubprocedure(rss, member, errorDispatcher, wakeMillis, timeoutMillis,
-      taskManager);
-
-  }
-
-  /**
-   * Build the actual backup procedure runner that will do all the 'hard' work
-   */
-  public class BackupSubprocedureBuilder implements SubprocedureFactory {
-
-    @Override
-    public Subprocedure buildSubprocedure(String name, byte[] data) {
-      return LogRollRegionServerProcedureManager.this.buildSubprocedure();
-    }
-  }
-
-  @Override
-  public void initialize(RegionServerServices rss) throws IOException {
-    this.rss = rss;
-    BaseCoordinatedStateManager coordManager =
-        (BaseCoordinatedStateManager) CoordinatedStateManagerFactory.getCoordinatedStateManager(rss
-          .getConfiguration());
-    coordManager.initialize(rss);
-    this.memberRpcs =
-        coordManager
-        .getProcedureMemberRpcs(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE);
-
-    // read in the backup handler configuration properties
-    Configuration conf = rss.getConfiguration();
-    long keepAlive = conf.getLong(BACKUP_TIMEOUT_MILLIS_KEY, BACKUP_TIMEOUT_MILLIS_DEFAULT);
-    int opThreads = conf.getInt(BACKUP_REQUEST_THREADS_KEY, BACKUP_REQUEST_THREADS_DEFAULT);
-    // create the actual cohort member
-    ThreadPoolExecutor pool =
-        ProcedureMember.defaultPool(rss.getServerName().toString(), opThreads, keepAlive);
-    this.member = new ProcedureMember(memberRpcs, pool, new BackupSubprocedureBuilder());
-  }
-
-  @Override
-  public String getProcedureSignature() {
-    return "backup-proc";
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
index 3342743..ae36f08 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/BaseCoordinatedStateManager.java
@@ -17,11 +17,7 @@
  */
 package org.apache.hadoop.hbase.coordination;
 
-import java.io.IOException;
-
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
-import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.Server;
 
@@ -55,21 +51,8 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan
    * Method to retrieve coordination for split log worker
    */
   public abstract  SplitLogWorkerCoordination getSplitLogWorkerCoordination();
-  
   /**
    * Method to retrieve coordination for split log manager
    */
   public abstract SplitLogManagerCoordination getSplitLogManagerCoordination();
-  /**
-   * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs}
-   */
-  public abstract ProcedureCoordinatorRpcs
-    getProcedureCoordinatorRpcs(String procType, String coordNode) throws IOException;
-  
-  /**
-   * Method to retrieve {@link org.apache.hadoop.hbase.procedure.ProcedureMemberRpc}
-   */
-  public abstract ProcedureMemberRpcs
-    getProcedureMemberRpcs(String procType) throws IOException;
-    
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/449fb812/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
index 7cf4aab..3e89be7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkCoordinatedStateManager.java
@@ -17,15 +17,9 @@
  */
 package org.apache.hadoop.hbase.coordination;
 
-import java.io.IOException;
-
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
-import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
-import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs;
-import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 
 /**
@@ -55,21 +49,9 @@ public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
   @Override
   public SplitLogWorkerCoordination getSplitLogWorkerCoordination() {
     return splitLogWorkerCoordination;
-  }
-
+    }
   @Override
   public SplitLogManagerCoordination getSplitLogManagerCoordination() {
     return splitLogManagerCoordination;
   }
-
-  @Override
-  public ProcedureCoordinatorRpcs getProcedureCoordinatorRpcs(String procType, String coordNode)
-      throws IOException {
-    return new ZKProcedureCoordinatorRpcs(watcher, procType, coordNode);
-  }
-
-  @Override
-  public ProcedureMemberRpcs getProcedureMemberRpcs(String procType) throws IOException {
-    return new ZKProcedureMemberRpcs(watcher, procType);
-  }
 }