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/08/25 19:41:47 UTC

hbase git commit: HBASE-16458 Shorten backup / restore test execution time

Repository: hbase
Updated Branches:
  refs/heads/HBASE-7912 f09911fe8 -> 7bb0d3163


HBASE-16458 Shorten backup / restore test execution time


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

Branch: refs/heads/HBASE-7912
Commit: 7bb0d31637ed39f2dffb7586b03f4718913a6cd5
Parents: f09911f
Author: tedyu <yu...@gmail.com>
Authored: Thu Aug 25 12:41:15 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Aug 25 12:41:15 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/backup/TestBackupAdmin.java    | 186 -------------------
 .../hadoop/hbase/backup/TestBackupBase.java     |   5 +-
 .../hbase/backup/TestBackupShowHistory.java     |  76 ++++----
 ...tBackupShowHistoryFromBackupDestination.java | 126 -------------
 .../hadoop/hbase/backup/TestFullBackup.java     |  61 ------
 .../hbase/backup/TestIncrementalBackup.java     |  62 +++----
 .../backup/TestIncrementalBackupNoDataLoss.java | 124 -------------
 7 files changed, 62 insertions(+), 578 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7bb0d316/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
deleted file mode 100644
index 0ec8b5d..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupAdmin.java
+++ /dev/null
@@ -1,186 +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.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.BackupAdmin;
-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.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.google.common.collect.Lists;
-
-@Category(LargeTests.class)
-public class TestBackupAdmin extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestBackupAdmin.class);
-  //implement all test cases in 1 test since incremental backup/restore has dependencies
-  @Test
-  public void TestIncBackupRestoreWithAdminAPI() throws Exception {
-    // #1 - create full backup for all tables
-    LOG.info("create full backup image for all tables");
-
-    List<TableName> tables = Lists.newArrayList(table1, table2, table3, table4);
-    HBaseAdmin admin = null;
-    BackupAdmin backupAdmin = null;
-    Connection conn = ConnectionFactory.createConnection(conf1);
-    admin = (HBaseAdmin) conn.getAdmin();
-    backupAdmin =  admin.getBackupAdmin();
-    BackupRequest request = new BackupRequest();
-    request.setBackupType(BackupType.FULL).setTableList(tables).setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdFull = backupAdmin.backupTables(request);
-
-    assertTrue(checkSucceeded(backupIdFull));
-
-    // #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
-    tables = Lists.newArrayList(table1, table2, table3);
-    request = new BackupRequest();
-    request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
-    .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdIncMultiple = backupAdmin.backupTables(request);
-    assertTrue(checkSucceeded(backupIdIncMultiple));
-
-    // #4 - restore full backup for all tables, without overwrite
-    TableName[] tablesRestoreFull =
-        new TableName[] { table1, table2, table3, table4 };
-
-    TableName[] tablesMapFull =
-        new TableName[] { table1_restore, table2_restore, table3_restore, table4_restore };
-
-    RestoreRequest restoreRequest = new RestoreRequest();
-    restoreRequest.setBackupRootDir(BACKUP_ROOT_DIR).setBackupId(backupIdFull).
-      setCheck(false).setOverwrite(false).
-      setFromTables(tablesRestoreFull).setToTables(tablesMapFull);
-    
-    backupAdmin.restore(restoreRequest);
-    
-    // #5.1 - check tables for full restore
-    
-    assertTrue(admin.tableExists(table1_restore));
-    assertTrue(admin.tableExists(table2_restore));
-    assertTrue(admin.tableExists(table3_restore));
-    assertTrue(admin.tableExists(table4_restore));
-
-
-    // #5.2 - checking row count of tables for full restore
-    HTable hTable = (HTable) conn.getTable(table1_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(table2_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(table3_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(table4_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-
-    // #6 - restore incremental backup for multiple tables, with overwrite
-    TableName[] tablesRestoreIncMultiple =
-        new TableName[] { table1, table2, table3 };
-    TableName[] tablesMapIncMultiple =
-        new TableName[] { table1_restore, table2_restore, table3_restore };
-    
-    restoreRequest = new RestoreRequest();
-    restoreRequest.setBackupRootDir(BACKUP_ROOT_DIR).setBackupId(backupIdIncMultiple).
-      setCheck(false).setOverwrite(true).
-      setFromTables(tablesRestoreIncMultiple).setToTables(tablesMapIncMultiple);
-    
-    backupAdmin.restore(restoreRequest);
-    
-    hTable = (HTable) conn.getTable(table1_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(table2_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(table3_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-
-    // #7 - incremental backup for single, empty table
-
-    tables = toList(table4.getNameAsString());
-    request = new BackupRequest();
-    request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
-    .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdIncEmpty = admin.getBackupAdmin().backupTables(request);
-
-
-    // #8 - restore incremental backup for single empty table, with overwrite
-    TableName[] tablesRestoreIncEmpty = new TableName[] { table4 };
-    TableName[] tablesMapIncEmpty = new TableName[] { table4_restore };
-    
-    restoreRequest = new RestoreRequest();
-    restoreRequest.setBackupRootDir(BACKUP_ROOT_DIR).setBackupId(backupIdIncEmpty).
-      setCheck(false).setOverwrite(true).
-      setFromTables(tablesRestoreIncEmpty).setToTables(tablesMapIncEmpty);
-    
-    backupAdmin.restore(restoreRequest);   
-
-    hTable = (HTable) conn.getTable(table4_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-    admin.close();
-    conn.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bb0d316/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 7be9f58..3982b1d 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BackupAdmin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+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.Put;
@@ -84,6 +85,7 @@ public class TestBackupBase {
 
   protected static String BACKUP_ROOT_DIR = "/backupUT";
   protected static String BACKUP_REMOTE_ROOT_DIR = "/backupUT";
+  protected static String provider = "defaultProvider";
 
   /**
    * @throws java.lang.Exception
@@ -94,7 +96,7 @@ public class TestBackupBase {
     conf1 = TEST_UTIL.getConfiguration();
     conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
     // Set MultiWAL (with 2 default WAL files per RS)
-    conf1.set(WALFactory.WAL_PROVIDER, "multiwal");
+    conf1.set(WALFactory.WAL_PROVIDER, provider);
     TEST_UTIL.startMiniZKCluster();
     MiniZooKeeperCluster miniZK = TEST_UTIL.getZkCluster();
 
@@ -179,6 +181,7 @@ public class TestBackupBase {
     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.setDurability(Durability.SKIP_WAL);
       p.addColumn(famName, qualName, Bytes.toBytes("val" + i));
       table.put(p);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bb0d316/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
index 145a060..4a94cb9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistory.java
@@ -26,7 +26,9 @@ import java.util.List;
 
 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.backup.util.BackupClientUtil;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
@@ -39,6 +41,18 @@ public class TestBackupShowHistory extends TestBackupBase {
 
   private static final Log LOG = LogFactory.getLog(TestBackupShowHistory.class);
 
+  private boolean findBackup(List<BackupInfo> history, String backupId) {
+    assertTrue(history.size() > 0);
+    boolean success = false;
+    for (BackupInfo info: history){
+      if (info.getBackupId().equals(backupId)){
+        success = true;
+        break;
+      }
+    }
+    return success;
+  }
+
   /**
    * Verify that full backup is created on a single table with data correctly. Verify that history
    * works as expected
@@ -55,68 +69,50 @@ public class TestBackupShowHistory extends TestBackupBase {
     LOG.info("backup complete");
 
     List<BackupInfo> history = getBackupAdmin().getHistory(10);
-    assertTrue(history.size() > 0);
-    boolean success = false;
-    for(BackupInfo info: history){
-      if(info.getBackupId().equals(backupId)){
-        success = true; break;
-      }
-    }
-    assertTrue(success);
-    LOG.info("show_history");
-
-  }
-
-  @Test
-  public void testBackupHistoryCommand() throws Exception {
-
-    LOG.info("test backup history on a single table with data: command-line");
+    assertTrue(findBackup(history, backupId));
+    history = BackupClientUtil.getHistory(conf1, 10, null, new Path(BACKUP_ROOT_DIR));
+    assertTrue(findBackup(history, backupId));
     
-    List<TableName> tableList = Lists.newArrayList(table1);
-    String backupId = fullTableBackup(tableList);
-    assertTrue(checkSucceeded(backupId));
-    LOG.info("backup complete");
-
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     System.setOut(new PrintStream(baos));
 
-    String[] args = new String[]{"history",  "-n", "10" }; 
+    String[] args = new String[]{"history",  "-n", "10", "-path", BACKUP_ROOT_DIR }; 
     // Run backup
     int ret = ToolRunner.run(conf1, new BackupDriver(), args);
     assertTrue(ret == 0);
     LOG.info("show_history");
     String output = baos.toString();
-    LOG.info(baos.toString());
+    LOG.info(output);
+    baos.close();
     assertTrue(output.indexOf(backupId) > 0);
-  }  
-  
-  
-  @Test
-  public void testBackupHistoryOneTable() throws Exception {
-
-    LOG.info("test backup history on a single table with data");
-    
-    List<TableName> tableList = Lists.newArrayList(table1);
-    String backupId1 = fullTableBackup(tableList);
-    assertTrue(checkSucceeded(backupId1));
-    LOG.info("backup complete: "+table1);
 
     tableList = Lists.newArrayList(table2);
     String backupId2 = fullTableBackup(tableList);
     assertTrue(checkSucceeded(backupId2));
     LOG.info("backup complete: "+ table2);
     
-    List<BackupInfo> history = getBackupAdmin().getHistory(10, table1);
+    history = getBackupAdmin().getHistory(10, table1);
     assertTrue(history.size() > 0);
     boolean success = true;
-    for(BackupInfo info: history){
-      if(!info.getTableNames().contains(table1)){
-        success = false; break;
+    for (BackupInfo info: history){
+      if (!info.getTableNames().contains(table1)){
+        success = false;
+        break;
+      }
+    }
+    assertTrue(success);
+    history = BackupClientUtil.getHistory(conf1, 10, table1, new Path(BACKUP_ROOT_DIR));
+    assertTrue(history.size() > 0);
+    success = true;
+    for (BackupInfo info: history){
+      if (!info.getTableNames().contains(table1)){
+        success = false;
+        break;
       }
     }
     assertTrue(success);
-    LOG.info("show_history");
 
+    LOG.info("show_history");
   }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bb0d316/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistoryFromBackupDestination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistoryFromBackupDestination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistoryFromBackupDestination.java
deleted file mode 100644
index 512e737..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestBackupShowHistoryFromBackupDestination.java
+++ /dev/null
@@ -1,126 +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.ByteArrayOutputStream;
-import java.io.PrintStream;
-import java.util.List;
-
-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.backup.util.BackupClientUtil;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.google.common.collect.Lists;
-
-@Category(LargeTests.class)
-public class TestBackupShowHistoryFromBackupDestination extends TestBackupBase {
-
-  private static final Log LOG = 
-      LogFactory.getLog(TestBackupShowHistoryFromBackupDestination.class);
-
-  /**
-   * Verify that full backup is created on a single table with data correctly. 
-   * Verify that history works as expected
-   * @throws Exception
-   */
-  @Test
-  public void testBackupHistory() throws Exception {
-
-    LOG.info("test backup history on a single table with data");
-    
-    List<TableName> tableList = Lists.newArrayList(table1);
-    String backupId = fullTableBackup(tableList);
-    assertTrue(checkSucceeded(backupId));
-    LOG.info("backup complete");
-
-    List<BackupInfo> history = BackupClientUtil.getHistory(conf1, 10, null, 
-      new Path(BACKUP_ROOT_DIR));
-    assertTrue(history.size() > 0);
-    boolean success = false;
-    for(BackupInfo info: history){
-      if(info.getBackupId().equals(backupId)){
-        success = true; break;
-      }
-    }
-    assertTrue(success);
-    LOG.info("show_history");
-
-  }
-
-  @Test
-  public void testBackupHistoryCommand() throws Exception {
-
-    LOG.info("test backup history on a single table with data: command-line");
-    
-    List<TableName> tableList = Lists.newArrayList(table1);
-    String backupId = fullTableBackup(tableList);
-    assertTrue(checkSucceeded(backupId));
-    LOG.info("backup complete");
-
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(baos));
-
-    String[] args = new String[]{"history",  "-n", "10", "-path", BACKUP_ROOT_DIR }; 
-    // Run backup
-    int ret = ToolRunner.run(conf1, new BackupDriver(), args);
-    assertTrue(ret == 0);
-    LOG.info("show_history");
-    String output = baos.toString();
-    LOG.info(baos.toString());
-    assertTrue(output.indexOf(backupId) > 0);
-  }  
-  
-  
-  @Test
-  public void testBackupHistoryOneTable() throws Exception {
-
-    LOG.info("test backup history on a single table with data");
-    
-    List<TableName> tableList = Lists.newArrayList(table1);
-    String backupId1 = fullTableBackup(tableList);
-    assertTrue(checkSucceeded(backupId1));
-    LOG.info("backup complete: "+table1);
-
-    tableList = Lists.newArrayList(table2);
-    String backupId2 = fullTableBackup(tableList);
-    assertTrue(checkSucceeded(backupId2));
-    LOG.info("backup complete: "+ table2);
-    
-    List<BackupInfo> history = BackupClientUtil.getHistory(conf1, 10, table1, new Path(BACKUP_ROOT_DIR));
-    assertTrue(history.size() > 0);
-    boolean success = true;
-    for(BackupInfo info: history){
-      if(!info.getTableNames().contains(table1)){
-        success = false; break;
-      }
-    }
-    assertTrue(success);
-    LOG.info("show_history");
-
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bb0d316/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
index 3d37d07..6fd38bb 100644
--- 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
@@ -38,56 +38,6 @@ 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");
-    List<TableName> tables = Lists.newArrayList(table1);
-    String backupId = fullTableBackup(tables);
-    assertTrue(checkSucceeded(backupId));
-    LOG.info("backup complete for " + backupId);
-  }
-
-  /**
-   * Verify that full backup is created on a single table with data correctly.
-   * @throws Exception
-   */
-  @Test
-  public void testFullBackupSingleCommand() throws Exception {
-    LOG.info("test full backup on a single table with data: command-line");
-    try(BackupSystemTable table = new BackupSystemTable(TEST_UTIL.getConnection())) {    
-      int before = table.getBackupHistory().size();
-      String[] args = new String[]{"create", "full", BACKUP_ROOT_DIR,  table1.getNameAsString() }; 
-      // Run backup
-      int ret = ToolRunner.run(conf1, new BackupDriver(), args);
-      assertTrue(ret == 0);
-      ArrayList<BackupInfo> backups = table.getBackupHistory(); 
-      int after = table.getBackupHistory().size();
-      assertTrue(after == before +1);
-      for(BackupInfo data : backups){
-        String backupId = data.getBackupId();
-        assertTrue(checkSucceeded(backupId));
-      }
-    }
-    LOG.info("backup complete");
-  }
-
-
-  /**
-   * 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");
-    List<TableName> tables = Lists.newArrayList(table1, table1);
-    String backupId = fullTableBackup(tables);
-    assertTrue(checkSucceeded(backupId));
-  }
-
   @Test
   public void testFullBackupMultipleCommand() throws Exception {
     LOG.info("test full backup on a multiple tables with data: command-line");
@@ -108,17 +58,6 @@ public class TestFullBackup extends TestBackupBase {
     }
     LOG.info("backup complete");
   }
-  /**
-   * 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 = fullTableBackup(null);
-    assertTrue(checkSucceeded(backupId));
-
-  }
   
   @Test
   public void testFullBackupAllCommand() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bb0d316/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 bfd9d01..9a2ad89 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.backup;
 
 import static org.junit.Assert.assertTrue;
 
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -36,20 +38,34 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Lists;
 
 @Category(LargeTests.class)
+@RunWith(Parameterized.class)
 public class TestIncrementalBackup extends TestBackupBase {
   private static final Log LOG = LogFactory.getLog(TestIncrementalBackup.class);
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    provider = "multiwal";
+    List<Object[]> params = new ArrayList<Object[]>();
+    params.add(new Object[] {Boolean.TRUE});
+    return params;
+  }
+  public TestIncrementalBackup(Boolean b) {
+  }
+
   //implement all test cases in 1 test since incremental backup/restore has dependencies
   @Test
   public void TestIncBackupRestore() throws Exception {
     // #1 - create full backup for all tables
     LOG.info("create full backup image for all tables");
 
-    List<TableName> tables = Lists.newArrayList(table1, table2, table3, table4);
+    List<TableName> tables = Lists.newArrayList(table1, table2);
     HBaseAdmin admin = null;
     Connection conn = ConnectionFactory.createConnection(conf1);
     admin = (HBaseAdmin) conn.getAdmin();
@@ -87,7 +103,7 @@ public class TestIncrementalBackup extends TestBackupBase {
     LOG.debug("written " + 5 + " rows to " + table2);
 
     // #3 - incremental backup for multiple tables
-    tables = Lists.newArrayList(table1, table2, table3);
+    tables = Lists.newArrayList(table1, table2);
     request = new BackupRequest();
     request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
     .setTargetRootDir(BACKUP_ROOT_DIR);
@@ -96,10 +112,10 @@ public class TestIncrementalBackup extends TestBackupBase {
 
     // #4 - restore full backup for all tables, without overwrite
     TableName[] tablesRestoreFull =
-        new TableName[] { table1, table2, table3, table4 };
+        new TableName[] { table1, table2 };
 
     TableName[] tablesMapFull =
-        new TableName[] { table1_restore, table2_restore, table3_restore, table4_restore };
+        new TableName[] { table1_restore, table2_restore };
 
     BackupAdmin client = getBackupAdmin();
     client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdFull, false,
@@ -110,8 +126,6 @@ public class TestIncrementalBackup extends TestBackupBase {
     HBaseAdmin hAdmin = TEST_UTIL.getHBaseAdmin();
     assertTrue(hAdmin.tableExists(table1_restore));
     assertTrue(hAdmin.tableExists(table2_restore));
-    assertTrue(hAdmin.tableExists(table3_restore));
-    assertTrue(hAdmin.tableExists(table4_restore));
 
     hAdmin.close();
 
@@ -124,19 +138,11 @@ public class TestIncrementalBackup extends TestBackupBase {
     Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH));
     hTable.close();
 
-    hTable = (HTable) conn.getTable(table3_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-
-    hTable = (HTable) conn.getTable(table4_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-
     // #6 - restore incremental backup for multiple tables, with overwrite
     TableName[] tablesRestoreIncMultiple =
-        new TableName[] { table1, table2, table3 };
+        new TableName[] { table1, table2 };
     TableName[] tablesMapIncMultiple =
-        new TableName[] { table1_restore, table2_restore, table3_restore };
+        new TableName[] { table1_restore, table2_restore };
     client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncMultiple, false,
       tablesRestoreIncMultiple, tablesMapIncMultiple, true));
 
@@ -148,30 +154,6 @@ public class TestIncrementalBackup extends TestBackupBase {
     Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
     hTable.close();
 
-    hTable = (HTable) conn.getTable(table3_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
-
-    // #7 - incremental backup for single, empty table
-
-    tables = toList(table4.getNameAsString());
-    request = new BackupRequest();
-    request.setBackupType(BackupType.INCREMENTAL).setTableList(tables)
-    .setTargetRootDir(BACKUP_ROOT_DIR);
-    String backupIdIncEmpty = admin.getBackupAdmin().backupTables(request);
-
-
-    // #8 - restore incremental backup for single empty table, with overwrite
-    TableName[] tablesRestoreIncEmpty = new TableName[] { table4 };
-    TableName[] tablesMapIncEmpty = new TableName[] { table4_restore };
-
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdIncEmpty, false,
-      tablesRestoreIncEmpty,
-      tablesMapIncEmpty, true));
-
-    hTable = (HTable) conn.getTable(table4_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(0));
-    hTable.close();
     admin.close();
     conn.close();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bb0d316/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
deleted file mode 100644
index c3ad7d4..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupNoDataLoss.java
+++ /dev/null
@@ -1,124 +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.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.BackupAdmin;
-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.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.google.common.collect.Lists;
-
-@Category(LargeTests.class)
-public class TestIncrementalBackupNoDataLoss extends TestBackupBase {
-  private static final Log LOG = LogFactory.getLog(TestIncrementalBackupNoDataLoss.class);
-
-  // implement all test cases in 1 test since incremental backup/restore has dependencies
-  @Test
-  public void TestIncBackupRestore() throws Exception {
-
-    // #1 - create full backup for all tables
-    LOG.info("create full backup image for all tables");
-    List<TableName> tables = Lists.newArrayList(table1, table2);   
-    String backupIdFull = fullTableBackup(tables);
-    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 table1
-
-    tables = Lists.newArrayList(table1);
-    String backupIdInc1 = incrementalTableBackup(tables);
-    assertTrue(checkSucceeded(backupIdInc1));
-
-    // #4 - incremental backup for table2
-
-    tables = Lists.newArrayList(table2);
-    String backupIdInc2 = incrementalTableBackup(tables);
-    assertTrue(checkSucceeded(backupIdInc2));
-    // #5 - restore incremental backup for table1
-    TableName[] tablesRestoreInc1 = new TableName[] { table1 };
-    TableName[] tablesMapInc1 = new TableName[] { table1_restore };
-
-    if (TEST_UTIL.getAdmin().tableExists(table1_restore)) {
-      TEST_UTIL.deleteTable(table1_restore);
-    }
-    if (TEST_UTIL.getAdmin().tableExists(table2_restore)) {
-      TEST_UTIL.deleteTable(table2_restore);
-    }
-
-    BackupAdmin client = getBackupAdmin();
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdInc1, false, tablesRestoreInc1,
-      tablesMapInc1, false));
-
-    HTable hTable = (HTable) conn.getTable(table1_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH * 2));
-    hTable.close();
-
-    // #5 - restore incremental backup for table2
-    
-    TableName[] tablesRestoreInc2 = new TableName[] { table2 };
-    TableName[] tablesMapInc2 = new TableName[] { table2_restore };
-
-    client.restore(createRestoreRequest(BACKUP_ROOT_DIR, backupIdInc2, false, tablesRestoreInc2,
-      tablesMapInc2, false));
-
-    hTable = (HTable) conn.getTable(table2_restore);
-    Assert.assertThat(TEST_UTIL.countRows(hTable), CoreMatchers.equalTo(NB_ROWS_IN_BATCH + 5));
-    hTable.close();
-
-    conn.close();
-  }
-
-}