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 2016/06/24 22:40:45 UTC

hbase git commit: HBASE-16037 Make automatic mode default one (Vladimir)

Repository: hbase
Updated Branches:
  refs/heads/HBASE-7912 7b50cbd2b -> 14170122c


HBASE-16037 Make automatic mode default one (Vladimir)


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

Branch: refs/heads/HBASE-7912
Commit: 14170122c4dcb19b719e594505901cf9e8e00071
Parents: 7b50cbd
Author: tedyu <yu...@gmail.com>
Authored: Fri Jun 24 15:40:38 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jun 24 15:40:38 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/backup/RestoreClient.java      |   3 +-
 .../hadoop/hbase/backup/RestoreRequest.java     |  10 --
 .../hadoop/hbase/client/HBaseBackupAdmin.java   |   1 -
 .../hadoop/hbase/backup/RestoreDriver.java      |  15 +--
 .../hbase/backup/impl/RestoreClientImpl.java    | 115 +++++--------------
 .../hadoop/hbase/backup/TestBackupAdmin.java    |   6 +-
 .../hadoop/hbase/backup/TestBackupBase.java     |   4 +-
 .../hbase/backup/TestBackupDeleteRestore.java   |   2 +-
 .../hadoop/hbase/backup/TestFullRestore.java    |  12 +-
 .../hbase/backup/TestIncrementalBackup.java     |   6 +-
 .../TestIncrementalBackupDeleteTable.java       |   4 +-
 .../backup/TestIncrementalBackupNoDataLoss.java |   4 +-
 .../hadoop/hbase/backup/TestRemoteRestore.java  |   2 +-
 .../hbase/backup/TestRestoreBoundaryTests.java  |   4 +-
 14 files changed, 51 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
index acbcb44..07f573e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreClient.java
@@ -35,7 +35,6 @@ public interface RestoreClient {
    * @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
@@ -44,6 +43,6 @@ public interface RestoreClient {
    */
   public  void restore(
       String backupRootDir,
-      String backupId, boolean check, boolean autoRestore, TableName[] sTableArray,
+      String backupId, boolean check, TableName[] sTableArray,
       TableName[] tTableArray, boolean isOverwrite) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
index 770e588..c78f0bc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/backup/RestoreRequest.java
@@ -32,7 +32,6 @@ public class RestoreRequest {
   private String backupRootDir;  
   private String backupId;  
   private boolean check = false;
-  private boolean autorestore = true;
   private TableName[] fromTables;
   private TableName[] toTables;
   private boolean overwrite = false;
@@ -67,15 +66,6 @@ public class RestoreRequest {
     return this;
   }
 
-  public boolean isAutorestore() {
-    return autorestore;
-  }
-
-  public RestoreRequest setAutorestore(boolean autorestore) {
-    this.autorestore = autorestore;
-    return this;
-  }
-
   public TableName[] getFromTables() {
     return fromTables;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseBackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseBackupAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseBackupAdmin.java
index a6532df..81413c6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseBackupAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseBackupAdmin.java
@@ -203,7 +203,6 @@ public class HBaseBackupAdmin implements BackupAdmin {
     client.restore(request.getBackupRootDir(), 
                    request.getBackupId(), 
                    request.isCheck(), 
-                   request.isAutorestore(), 
                    request.getFromTables(), 
                    request.getToTables(), 
                    request.isOverwrite());

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
index 0dba079..c66ac6e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/RestoreDriver.java
@@ -65,12 +65,6 @@ public class RestoreDriver extends AbstractHBaseTool {
           + "                   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"
           + "   -set set_name   Backup set to restore, mutually exclusive with table list <tables>.";
 
     
@@ -114,12 +108,7 @@ public class RestoreDriver extends AbstractHBaseTool {
           + "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");
-    }
+    LOG.debug("Will automatically restore all the dependencies");
 
     // parse main restore command options
     String[] remainArgs = cmd.getArgs();
@@ -166,7 +155,7 @@ public class RestoreDriver extends AbstractHBaseTool {
     
     RestoreClient client = BackupRestoreClientFactory.getRestoreClient(getConf());
     try{
-      client.restore(backupRootDir, backupId, check, autoRestore, sTableArray,
+      client.restore(backupRootDir, backupId, check, sTableArray,
         tTableArray, isOverwrite);
     } catch (Exception e){
       e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
index 1f7305f..a04fc08 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/impl/RestoreClientImpl.java
@@ -70,7 +70,6 @@ public final class RestoreClientImpl implements RestoreClient {
    * @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
@@ -79,7 +78,7 @@ public final class RestoreClientImpl implements RestoreClient {
    */
   @Override
   public void restore(String backupRootDir,
-      String backupId, boolean check, boolean autoRestore, TableName[] sTableArray,
+      String backupId, boolean check, TableName[] sTableArray,
       TableName[] tTableArray, boolean isOverwrite) throws IOException {
 
     HashMap<TableName, BackupManifest> backupManifestMap = new HashMap<>();
@@ -87,10 +86,9 @@ public final class RestoreClientImpl implements RestoreClient {
     Path rootPath = new Path(backupRootDir);
     HBackupFileSystem.checkImageManifestExist(backupManifestMap, sTableArray, conf, rootPath,
       backupId);
-
     try {
       // Check and validate the backup image and its dependencies
-      if (check || autoRestore) {
+      if (check) {
         if (validate(backupManifestMap)) {
           LOG.info("Checking backup images: ok");
         } else {
@@ -103,16 +101,11 @@ public final class RestoreClientImpl implements RestoreClient {
       if (tTableArray == null) {
         tTableArray = sTableArray;
       }
-
       // check the target tables
       checkTargetTables(tTableArray, isOverwrite);
-
-      // start restore process
-      
-      restoreStage(backupManifestMap, sTableArray, tTableArray, autoRestore, isOverwrite);
-
+      // start restore process      
+      restoreStage(backupManifestMap, sTableArray, tTableArray, isOverwrite);
       LOG.info("Restore for " + Arrays.asList(sTableArray) + " are successful!");
-
     } catch (IOException e) {
       LOG.error("ERROR: restore failed with error: " + e.getMessage());
       throw e;
@@ -120,7 +113,6 @@ public final class RestoreClientImpl implements RestoreClient {
 
   }
 
-
   private  boolean validate(HashMap<TableName, BackupManifest> backupManifestMap)
       throws IOException {
     boolean isValid = true;
@@ -195,7 +187,6 @@ public final class RestoreClientImpl implements RestoreClient {
         }
       }
     }
-
   }
 
   /**
@@ -203,53 +194,36 @@ public final class RestoreClientImpl implements RestoreClient {
    * @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 void restoreStage(
-    HashMap<TableName, BackupManifest> backupManifestMap, TableName[] sTableArray,
-    TableName[] tTableArray, boolean autoRestore, boolean isOverwrite) throws IOException {
+  private void restoreStage(HashMap<TableName, BackupManifest> backupManifestMap,
+      TableName[] sTableArray, TableName[] tTableArray, boolean isOverwrite) throws IOException {
     TreeSet<BackupImage> restoreImageSet = new TreeSet<BackupImage>();
-    boolean truncateIfExists = autoRestore && isOverwrite;
+    boolean truncateIfExists = isOverwrite;
     try {
       for (int i = 0; i < sTableArray.length; i++) {
         TableName 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.
-          List<BackupImage> list = new ArrayList<BackupImage>();
-          list.add(manifest.getBackupImage());
-          List<BackupImage> depList = manifest.getDependentListByTable(table);
-          list.addAll(depList);
-          TreeSet<BackupImage> restoreList = new TreeSet<BackupImage>(list);
-          LOG.debug("need to clear merged Image. to be implemented in future jira");
-          restoreImages(restoreList.iterator(), table, tTableArray[i], truncateIfExists);
-          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));
-            }
+        // Get the image list of this backup for restore in time order from old
+        // to new.
+        List<BackupImage> list = new ArrayList<BackupImage>();
+        list.add(manifest.getBackupImage());
+        List<BackupImage> depList = manifest.getDependentListByTable(table);
+        list.addAll(depList);
+        TreeSet<BackupImage> restoreList = new TreeSet<BackupImage>(list);
+        LOG.debug("need to clear merged Image. to be implemented in future jira");
+        restoreImages(restoreList.iterator(), table, tTableArray[i], truncateIfExists);
+        restoreImageSet.addAll(restoreList);
+
+        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));
           }
         }
       }
@@ -262,43 +236,6 @@ public final class RestoreClientImpl implements RestoreClient {
   }
 
   /**
-   * 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  void restoreImage(BackupImage image, TableName sTable, TableName tTable)
-      throws IOException {
-    String rootDir = image.getRootDir();
-    String backupId = image.getBackupId();
-
-    Path rootPath = new Path(rootDir);
-    RestoreServerUtil restoreTool = new RestoreServerUtil(conf, rootPath, backupId);
-    BackupManifest manifest = HBackupFileSystem.getManifest(sTable, conf, rootPath, backupId);
-
-    Path tableBackupPath = HBackupFileSystem.getTableBackupPath(sTable, rootPath,  backupId);
-
-    // TODO: convert feature will be provided in a future JIRA
-    boolean converted = false;
-
-    if (manifest.getType() == BackupType.FULL || converted) {
-      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from "
-          + (converted ? "converted" : "full") + " backup image " + tableBackupPath.toString());
-      restoreTool.fullRestoreTable(tableBackupPath, sTable, tTable, converted, false);
-    } else { // incremental Backup
-      String logBackupDir =
-          HBackupFileSystem.getLogBackupDir(image.getRootDir(), image.getBackupId());
-      LOG.info("Restoring '" + sTable + "' to '" + tTable + "' from incremental backup image "
-          + logBackupDir);
-      restoreTool.incrementalRestoreTable(new Path[]{ new Path(logBackupDir)}, new TableName[] { sTable },
-        new TableName[] { tTable });
-    }
-
-    LOG.info(sTable + " has been successfully restored to " + tTable);
-  }
-  
-  /**
    * Restore operation handle each backupImage in iterator
    * @param it: backupImage iterator - ascending
    * @param sTable: table to be restored

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupAdmin.java
index 3630d87..0ec8b5d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupAdmin.java
@@ -101,7 +101,7 @@ public class TestBackupAdmin extends TestBackupBase {
 
     RestoreRequest restoreRequest = new RestoreRequest();
     restoreRequest.setBackupRootDir(BACKUP_ROOT_DIR).setBackupId(backupIdFull).
-      setCheck(false).setAutorestore(false).setOverwrite(false).
+      setCheck(false).setOverwrite(false).
       setFromTables(tablesRestoreFull).setToTables(tablesMapFull);
     
     backupAdmin.restore(restoreRequest);
@@ -139,7 +139,7 @@ public class TestBackupAdmin extends TestBackupBase {
     
     restoreRequest = new RestoreRequest();
     restoreRequest.setBackupRootDir(BACKUP_ROOT_DIR).setBackupId(backupIdIncMultiple).
-      setCheck(false).setAutorestore(false).setOverwrite(true).
+      setCheck(false).setOverwrite(true).
       setFromTables(tablesRestoreIncMultiple).setToTables(tablesMapIncMultiple);
     
     backupAdmin.restore(restoreRequest);
@@ -171,7 +171,7 @@ public class TestBackupAdmin extends TestBackupBase {
     
     restoreRequest = new RestoreRequest();
     restoreRequest.setBackupRootDir(BACKUP_ROOT_DIR).setBackupId(backupIdIncEmpty).
-      setCheck(false).setAutorestore(false).setOverwrite(true).
+      setCheck(false).setOverwrite(true).
       setFromTables(tablesRestoreIncEmpty).setToTables(tablesMapIncEmpty);
     
     backupAdmin.restore(restoreRequest);   

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/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
index 31cf65a..da1c6c1 100644
--- 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
@@ -259,10 +259,10 @@ public class TestBackupBase {
    */
   public  RestoreRequest createRestoreRequest(
       String backupRootDir,
-      String backupId, boolean check, boolean autoRestore, TableName[] fromTables,
+      String backupId, boolean check, TableName[] fromTables,
       TableName[] toTables, boolean isOverwrite) {
     RestoreRequest request = new RestoreRequest();
-    request.setBackupRootDir(backupRootDir).setBackupId(backupId).setCheck(check).setAutorestore(autoRestore).
+    request.setBackupRootDir(backupRootDir).setBackupId(backupId).setCheck(check).
     setFromTables(fromTables).setToTables(toTables).setOverwrite(isOverwrite);
     return request;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
index 702e8e7..2e04f1a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupDeleteRestore.java
@@ -60,7 +60,7 @@ public class TestBackupDeleteRestore extends TestBackupBase {
     TableName[] tableset = new TableName[] { table1 };
     TableName[] tablemap = null;//new TableName[] { table1_restore };
     BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, true, tableset, tablemap, true));
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, tablemap, true));
     
     
     int numRowsAfterRestore = TEST_UTIL.countRows(table1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/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
index 1e23842..0eddbfe 100644
--- 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
@@ -52,7 +52,7 @@ public class TestFullRestore extends TestBackupBase {
     TableName[] tableset = new TableName[] { table1 };
     TableName[] tablemap = new TableName[] { table1_restore };
     BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap, false));
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,  tableset, tablemap, false));
     HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
     assertTrue(hba.tableExists(table1_restore));
     TEST_UTIL.deleteTable(table1_restore);
@@ -96,7 +96,7 @@ public class TestFullRestore extends TestBackupBase {
     TableName[] restore_tableset = new TableName[] { table2, table3 };
     TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
     BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, false,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
       restore_tableset, tablemap, false));
     HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
     assertTrue(hba.tableExists(table2_restore));
@@ -154,7 +154,7 @@ public class TestFullRestore extends TestBackupBase {
 
     TableName[] tableset = new TableName[] { table1 };
     BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, false, tableset, null,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, null,
       true));
   }
 
@@ -198,7 +198,7 @@ public class TestFullRestore extends TestBackupBase {
 
     TableName[] restore_tableset = new TableName[] { table2, table3 };
     BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId,
       false, restore_tableset, null, true));
   }
 
@@ -245,7 +245,7 @@ public class TestFullRestore extends TestBackupBase {
     TableName[] tableset = new TableName[] { TableName.valueOf("faketable") };
     TableName[] tablemap = new TableName[] { table1_restore };
     BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, tablemap,
       false));
   }
 
@@ -292,7 +292,7 @@ public class TestFullRestore extends TestBackupBase {
     TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
     BackupAdmin client = getBackupAdmin();
     client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false,
-      false, restore_tableset, tablemap, false));
+      restore_tableset, tablemap, false));
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/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
index 988b156..0f35026 100644
--- 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
@@ -99,7 +99,7 @@ public class TestIncrementalBackup extends TestBackupBase {
         new TableName[] { table1_restore, table2_restore, table3_restore, table4_restore };
 
     BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false, false,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false,
       tablesRestoreFull,
       tablesMapFull, false));
 
@@ -134,7 +134,7 @@ public class TestIncrementalBackup extends TestBackupBase {
         new TableName[] { table1, table2, table3 };
     TableName[] tablesMapIncMultiple =
         new TableName[] { table1_restore, table2_restore, table3_restore };
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple, false, false,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple, false,
       tablesRestoreIncMultiple, tablesMapIncMultiple, true));
 
     hTable = (HTable) conn.getTable(table1_restore);
@@ -162,7 +162,7 @@ public class TestIncrementalBackup extends TestBackupBase {
     TableName[] tablesRestoreIncEmpty = new TableName[] { table4 };
     TableName[] tablesMapIncEmpty = new TableName[] { table4_restore };
 
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncEmpty, false, false,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncEmpty, false,
       tablesRestoreIncEmpty,
       tablesMapIncEmpty, true));
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
index ad42d48..ad11548 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupDeleteTable.java
@@ -101,7 +101,7 @@ public class TestIncrementalBackupDeleteTable extends TestBackupBase {
         new TableName[] { table1_restore, table2_restore };
 
     BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false, false,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false,
       tablesRestoreFull,
       tablesMapFull, false));
 
@@ -126,7 +126,7 @@ public class TestIncrementalBackupDeleteTable extends TestBackupBase {
         new TableName[] { table1 };
     TableName[] tablesMapIncMultiple =
         new TableName[] { table1_restore };
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple, false, false,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple, false,
       tablesRestoreIncMultiple, tablesMapIncMultiple, true));
 
     hTable = (HTable) conn.getTable(table1_restore);

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java
index c67f63f..c3ad7d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java
@@ -99,7 +99,7 @@ public class TestIncrementalBackupNoDataLoss extends TestBackupBase {
     }
 
     BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdInc1, false, true, tablesRestoreInc1,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdInc1, false, tablesRestoreInc1,
       tablesMapInc1, false));
 
     HTable hTable = (HTable) conn.getTable(table1_restore);
@@ -111,7 +111,7 @@ public class TestIncrementalBackupNoDataLoss extends TestBackupBase {
     TableName[] tablesRestoreInc2 = new TableName[] { table2 };
     TableName[] tablesMapInc2 = new TableName[] { table2_restore };
 
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdInc2, false, true, tablesRestoreInc2,
+    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdInc2, false, tablesRestoreInc2,
       tablesMapInc2, false));
 
     hTable = (HTable) conn.getTable(table2_restore);

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/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
index f40ed9d..1be2aa2 100644
--- 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
@@ -39,7 +39,7 @@ public class TestRemoteRestore extends TestBackupBase {
     LOG.info("backup complete");
     TableName[] tableset = new TableName[] { table1 };
     TableName[] tablemap = new TableName[] { table1_restore };
-    getBackupAdmin().restore(createRestoreRequest(BACKUP_REMOTE_ROOT_DIR, backupId, false, false, tableset,
+    getBackupAdmin().restore(createRestoreRequest(BACKUP_REMOTE_ROOT_DIR, backupId, false, tableset,
       tablemap, false));
     HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
     assertTrue(hba.tableExists(table1_restore));

http://git-wip-us.apache.org/repos/asf/hbase/blob/14170122/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
index c86b4b4..d2308d1 100644
--- 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
@@ -46,7 +46,7 @@ public class TestRestoreBoundaryTests extends TestBackupBase {
     LOG.info("backup complete");
     TableName[] tableset = new TableName[] { table1 };
     TableName[] tablemap = new TableName[] { table1_restore };
-    getBackupAdmin().restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, false, tableset, tablemap,
+    getBackupAdmin().restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, tableset, tablemap,
       false));
     HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();
     assertTrue(hba.tableExists(table1_restore));
@@ -65,7 +65,7 @@ public class TestRestoreBoundaryTests extends TestBackupBase {
     String backupId = fullTableBackup(tables);
     TableName[] restore_tableset = new TableName[] { table2, table3};
     TableName[] tablemap = new TableName[] { table2_restore, table3_restore };
-    getBackupAdmin().restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, false, restore_tableset,
+    getBackupAdmin().restore(createRestoreRequest(BACKUP_ROOT_DIR, backupId, false, restore_tableset,
       tablemap,
       false));
     HBaseAdmin hba = TEST_UTIL.getHBaseAdmin();