You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2013/07/31 12:16:25 UTC

svn commit: r1508801 - in /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase: client/ master/cleaner/ snapshot/

Author: mbertozzi
Date: Wed Jul 31 10:16:25 2013
New Revision: 1508801

URL: http://svn.apache.org/r1508801
Log:
HBASE-9090 cleanup snapshot tests setup/teardown code

Modified:
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java?rev=1508801&r1=1508800&r2=1508801&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCloneSnapshotFromClient.java Wed Jul 31 10:16:25 2013
@@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.LargeTest
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.MD5Hash;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -96,7 +96,7 @@ public class TestCloneSnapshotFromClient
     snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
 
     // create Table and disable it
-    createTable(tableName, FAMILY);
+    SnapshotTestingUtils.createTable(TEST_UTIL, tableName, FAMILY);
     admin.disableTable(tableName);
 
     // take an empty snapshot
@@ -106,7 +106,7 @@ public class TestCloneSnapshotFromClient
     try {
       // enable table and insert data
       admin.enableTable(tableName);
-      loadData(table, 500, FAMILY);
+      SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY);
       snapshot0Rows = TEST_UTIL.countRows(table);
       admin.disableTable(tableName);
 
@@ -115,7 +115,7 @@ public class TestCloneSnapshotFromClient
 
       // enable table and insert more data
       admin.enableTable(tableName);
-      loadData(table, 500, FAMILY);
+      SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY);
       snapshot1Rows = TEST_UTIL.countRows(table);
       admin.disableTable(tableName);
 
@@ -134,13 +134,8 @@ public class TestCloneSnapshotFromClient
     if (admin.tableExists(tableName)) {
       TEST_UTIL.deleteTable(tableName);
     }
-    admin.deleteSnapshot(snapshotName0);
-    admin.deleteSnapshot(snapshotName1);
-
-    // Ensure the archiver to be empty
-    MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    mfs.getFileSystem().delete(
-      new Path(mfs.getRootDir(), HConstants.HFILE_ARCHIVE_DIRECTORY), true);
+    SnapshotTestingUtils.deleteAllSnapshots(admin);
+    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
   }
 
   @Test(expected=SnapshotDoesNotExistException.class)
@@ -162,10 +157,9 @@ public class TestCloneSnapshotFromClient
       int snapshotRows) throws IOException, InterruptedException {
     // create a new table from snapshot
     admin.cloneSnapshot(snapshotName, tableName);
-    verifyRowCount(tableName, snapshotRows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, tableName, snapshotRows);
 
-    admin.disableTable(tableName);
-    admin.deleteTable(tableName);
+    TEST_UTIL.deleteTable(tableName);
   }
 
   /**
@@ -176,7 +170,7 @@ public class TestCloneSnapshotFromClient
     // Clone a table from the first snapshot
     byte[] clonedTableName = Bytes.toBytes("clonedtb1-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName0, clonedTableName);
-    verifyRowCount(clonedTableName, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
 
     // Take a snapshot of this cloned table.
     admin.disableTable(clonedTableName);
@@ -185,85 +179,45 @@ public class TestCloneSnapshotFromClient
     // Clone the snapshot of the cloned table
     byte[] clonedTableName2 = Bytes.toBytes("clonedtb2-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName2, clonedTableName2);
-    verifyRowCount(clonedTableName2, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
     admin.disableTable(clonedTableName2);
 
     // Remove the original table
-    admin.disableTable(tableName);
-    admin.deleteTable(tableName);
+    TEST_UTIL.deleteTable(tableName);
     waitCleanerRun();
 
     // Verify the first cloned table
     admin.enableTable(clonedTableName);
-    verifyRowCount(clonedTableName, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
 
     // Verify the second cloned table
     admin.enableTable(clonedTableName2);
-    verifyRowCount(clonedTableName2, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
     admin.disableTable(clonedTableName2);
 
     // Delete the first cloned table
-    admin.disableTable(clonedTableName);
-    admin.deleteTable(clonedTableName);
+    TEST_UTIL.deleteTable(clonedTableName);
     waitCleanerRun();
 
     // Verify the second cloned table
     admin.enableTable(clonedTableName2);
-    verifyRowCount(clonedTableName2, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName2, snapshot0Rows);
 
     // Clone a new table from cloned
     byte[] clonedTableName3 = Bytes.toBytes("clonedtb3-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName2, clonedTableName3);
-    verifyRowCount(clonedTableName3, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName3, snapshot0Rows);
 
     // Delete the cloned tables
-    admin.disableTable(clonedTableName2);
-    admin.deleteTable(clonedTableName2);
-    admin.disableTable(clonedTableName3);
-    admin.deleteTable(clonedTableName3);
+    TEST_UTIL.deleteTable(clonedTableName2);
+    TEST_UTIL.deleteTable(clonedTableName3);
     admin.deleteSnapshot(snapshotName2);
   }
 
   // ==========================================================================
   //  Helpers
   // ==========================================================================
-  private void createTable(final byte[] tableName, final byte[]... families) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    for (byte[] family: families) {
-      HColumnDescriptor hcd = new HColumnDescriptor(family);
-      htd.addFamily(hcd);
-    }
-    byte[][] splitKeys = new byte[16][];
-    byte[] hex = Bytes.toBytes("0123456789abcdef");
-    for (int i = 0; i < 16; ++i) {
-      splitKeys[i] = new byte[] { hex[i] };
-    }
-    admin.createTable(htd, splitKeys);
-  }
-
-  public void loadData(final HTable table, int rows, byte[]... families) throws IOException {
-    byte[] qualifier = Bytes.toBytes("q");
-    table.setAutoFlush(false);
-    while (rows-- > 0) {
-      byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
-      byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
-      Put put = new Put(key);
-      put.setDurability(Durability.SKIP_WAL);
-      for (byte[] family: families) {
-        put.add(family, qualifier, value);
-      }
-      table.put(put);
-    }
-    table.flushCommits();
-  }
-
   private void waitCleanerRun() throws InterruptedException {
     TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
   }
-
-  private void verifyRowCount(final byte[] tableName, long expectedRows) throws IOException {
-    HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
-    assertEquals(expectedRows, TEST_UTIL.countRows(table));
-    table.close();
-  }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java?rev=1508801&r1=1508800&r2=1508801&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java Wed Jul 31 10:16:25 2013
@@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.LargeTest
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.MD5Hash;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -100,7 +100,7 @@ public class TestRestoreSnapshotFromClie
     snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
 
     // create Table and disable it
-    createTable(tableName, FAMILY);
+    SnapshotTestingUtils.createTable(TEST_UTIL, tableName, FAMILY);
     admin.disableTable(tableName);
 
     // take an empty snapshot
@@ -110,7 +110,7 @@ public class TestRestoreSnapshotFromClie
     try {
       // enable table and insert data
       admin.enableTable(tableName);
-      loadData(table, 500, FAMILY);
+      SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY);
       snapshot0Rows = TEST_UTIL.countRows(table);
       admin.disableTable(tableName);
 
@@ -119,7 +119,7 @@ public class TestRestoreSnapshotFromClie
 
       // enable table and insert more data
       admin.enableTable(tableName);
-      loadData(table, 500, FAMILY);
+      SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, FAMILY);
       snapshot1Rows = TEST_UTIL.countRows(table);
       admin.disableTable(tableName);
 
@@ -135,43 +135,36 @@ public class TestRestoreSnapshotFromClie
 
   @After
   public void tearDown() throws Exception {
-    if (admin.tableExists(tableName)) {
-      TEST_UTIL.deleteTable(tableName);
-    }
-    admin.deleteSnapshot(snapshotName0);
-    admin.deleteSnapshot(snapshotName1);
-
-    // Ensure the archiver to be empty
-    MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    mfs.getFileSystem().delete(
-      new Path(mfs.getRootDir(), HConstants.HFILE_ARCHIVE_DIRECTORY), true);
+    TEST_UTIL.deleteTable(tableName);
+    SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
   }
 
   @Test
   public void testRestoreSnapshot() throws IOException {
-    verifyRowCount(tableName, snapshot1Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, tableName, snapshot1Rows);
 
     // Restore from snapshot-0
     admin.disableTable(tableName);
     admin.restoreSnapshot(snapshotName0);
     admin.enableTable(tableName);
-    verifyRowCount(tableName, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, tableName, snapshot0Rows);
 
     // Restore from emptySnapshot
     admin.disableTable(tableName);
     admin.restoreSnapshot(emptySnapshot);
     admin.enableTable(tableName);
-    verifyRowCount(tableName, 0);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, tableName, 0);
 
     // Restore from snapshot-1
     admin.disableTable(tableName);
     admin.restoreSnapshot(snapshotName1);
     admin.enableTable(tableName);
-    verifyRowCount(tableName, snapshot1Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, tableName, snapshot1Rows);
   }
 
   @Test
-  public void testRestoreSchemaChange() throws IOException {
+  public void testRestoreSchemaChange() throws Exception {
     byte[] TEST_FAMILY2 = Bytes.toBytes("cf2");
 
     HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
@@ -183,7 +176,7 @@ public class TestRestoreSnapshotFromClie
     assertEquals(2, table.getTableDescriptor().getFamilies().size());
     HTableDescriptor htd = admin.getTableDescriptor(tableName);
     assertEquals(2, htd.getFamilies().size());
-    loadData(table, 500, TEST_FAMILY2);
+    SnapshotTestingUtils.loadData(TEST_UTIL, table, 500, TEST_FAMILY2);
     long snapshot2Rows = snapshot1Rows + 500;
     assertEquals(snapshot2Rows, TEST_UTIL.countRows(table));
     assertEquals(500, TEST_UTIL.countRows(table, TEST_FAMILY2));
@@ -230,51 +223,20 @@ public class TestRestoreSnapshotFromClie
   public void testRestoreSnapshotOfCloned() throws IOException, InterruptedException {
     byte[] clonedTableName = Bytes.toBytes("clonedtb-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName0, clonedTableName);
-    verifyRowCount(clonedTableName, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
     admin.disableTable(clonedTableName);
     admin.snapshot(snapshotName2, clonedTableName);
-    admin.deleteTable(clonedTableName);
+    TEST_UTIL.deleteTable(clonedTableName);
     waitCleanerRun();
 
     admin.cloneSnapshot(snapshotName2, clonedTableName);
-    verifyRowCount(clonedTableName, snapshot0Rows);
-    admin.disableTable(clonedTableName);
-    admin.deleteTable(clonedTableName);
+    SnapshotTestingUtils.verifyRowCount(TEST_UTIL, clonedTableName, snapshot0Rows);
+    TEST_UTIL.deleteTable(clonedTableName);
   }
 
   // ==========================================================================
   //  Helpers
   // ==========================================================================
-  private void createTable(final byte[] tableName, final byte[]... families) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    for (byte[] family: families) {
-      HColumnDescriptor hcd = new HColumnDescriptor(family);
-      htd.addFamily(hcd);
-    }
-    byte[][] splitKeys = new byte[16][];
-    byte[] hex = Bytes.toBytes("0123456789abcdef");
-    for (int i = 0; i < 16; ++i) {
-      splitKeys[i] = new byte[] { hex[i] };
-    }
-    admin.createTable(htd, splitKeys);
-  }
-
-  public void loadData(final HTable table, int rows, byte[]... families) throws IOException {
-    byte[] qualifier = Bytes.toBytes("q");
-    table.setAutoFlush(false);
-    while (rows-- > 0) {
-      byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
-      byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
-      Put put = new Put(key);
-      put.setDurability(Durability.SKIP_WAL);
-      for (byte[] family: families) {
-        put.add(family, qualifier, value);
-      }
-      table.put(put);
-    }
-    table.flushCommits();
-  }
-
   private void waitCleanerRun() throws InterruptedException {
     TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
   }
@@ -290,10 +252,4 @@ public class TestRestoreSnapshotFromClie
     }
     return families;
   }
-
-  private void verifyRowCount(final byte[] tableName, long expectedRows) throws IOException {
-    HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
-    assertEquals(expectedRows, TEST_UTIL.countRows(table));
-    table.close();
-  }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java?rev=1508801&r1=1508800&r2=1508801&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java Wed Jul 31 10:16:25 2013
@@ -97,13 +97,8 @@ public class TestSnapshotCloneIndependen
   @After
   public void tearDown() throws Exception {
     UTIL.deleteTable(TABLE_NAME);
-    // and cleanup the archive directory
-    try {
-      UTIL.getTestFileSystem().delete(
-        new Path(UTIL.getDefaultRootDirPath(), HConstants.HFILE_ARCHIVE_DIRECTORY), true);
-    } catch (IOException e) {
-      LOG.warn("Failure to delete archive directory", e);
-    }
+    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
   }
 
   @AfterClass
@@ -373,4 +368,4 @@ public class TestSnapshotCloneIndependen
     Assert.assertTrue("The new family was not found. ",
       !clonedTableDescriptor.hasFamily(TEST_FAM_2));
   }
-}
\ No newline at end of file
+}

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java?rev=1508801&r1=1508800&r2=1508801&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java Wed Jul 31 10:16:25 2013
@@ -98,13 +98,8 @@ public class TestSnapshotFromClient {
   @After
   public void tearDown() throws Exception {
     UTIL.deleteTable(TABLE_NAME);
-    // and cleanup the archive directory
-    try {
-      UTIL.getTestFileSystem().delete(
-        new Path(UTIL.getDefaultRootDirPath(), HConstants.HFILE_ARCHIVE_DIRECTORY), true);
-    } catch (IOException e) {
-      LOG.warn("Failure to delete archive directory", e);
-    }
+    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
   }
 
   @AfterClass
@@ -263,4 +258,4 @@ public class TestSnapshotFromClient {
       LOG.info("Correctly failed to snapshot a non-existant table:" + e.getMessage());
     }
   }
-}
\ No newline at end of file
+}

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java?rev=1508801&r1=1508800&r2=1508801&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java Wed Jul 31 10:16:25 2013
@@ -134,22 +134,8 @@ public class TestSnapshotFromMaster {
   @After
   public void tearDown() throws Exception {
     UTIL.deleteTable(TABLE_NAME);
-
-    // delete the archive directory, if its exists
-    if (fs.exists(archiveDir)) {
-      if (!fs.delete(archiveDir, true)) {
-        throw new IOException("Couldn't delete archive directory (" + archiveDir
-            + " for an unknown reason");
-      }
-    }
-
-    // delete the snapshot directory, if its exists
-    if (fs.exists(snapshots)) {
-      if (!fs.delete(snapshots, true)) {
-        throw new IOException("Couldn't delete snapshots directory (" + snapshots
-            + " for an unknown reason");
-      }
-    }
+    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
   }
 
   @AfterClass

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java?rev=1508801&r1=1508800&r2=1508801&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java Wed Jul 31 10:16:25 2013
@@ -34,21 +34,29 @@ 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.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableNotEnabledException;
+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;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneResponse;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.MD5Hash;
 import org.junit.Assert;
 
 import com.google.protobuf.ServiceException;
@@ -433,4 +441,102 @@ public class SnapshotTestingUtils {
     SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), tableName, nonEmptyFamilyNames, emptyFamilyNames,
       rootDir, admin, fs, false, new Path(rootDir, HConstants.HREGION_LOGDIR_NAME), null);
   }
+
+  // ==========================================================================
+  //  Table Helpers
+  // ==========================================================================
+  public static void waitForTableToBeOnline(final HBaseTestingUtility util, final byte[] tableName)
+      throws IOException, InterruptedException {
+    HRegionServer rs = util.getRSForFirstRegionInTable(tableName);
+    List<HRegion> onlineRegions = rs.getOnlineRegions(tableName);
+    for (HRegion region : onlineRegions) {
+      region.waitForFlushesAndCompactions();
+    }
+    util.getHBaseAdmin().isTableAvailable(tableName);
+  }
+
+  public static void createTable(final HBaseTestingUtility util, final byte[] tableName,
+      final byte[]... families) throws IOException, InterruptedException {
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    for (byte[] family: families) {
+      HColumnDescriptor hcd = new HColumnDescriptor(family);
+      htd.addFamily(hcd);
+    }
+    byte[][] splitKeys = new byte[14][];
+    byte[] hex = Bytes.toBytes("123456789abcde");
+    for (int i = 0; i < splitKeys.length; ++i) {
+      splitKeys[i] = new byte[] { hex[i] };
+    }
+    util.getHBaseAdmin().createTable(htd, splitKeys);
+    waitForTableToBeOnline(util, tableName);
+    assertEquals(15, util.getHBaseAdmin().getTableRegions(tableName).size());
+  }
+
+  public static void loadData(final HBaseTestingUtility util, final byte[] tableName, int rows,
+      byte[]... families) throws IOException, InterruptedException {
+    loadData(util, new HTable(util.getConfiguration(), tableName), rows, families);
+  }
+
+  public static void loadData(final HBaseTestingUtility util, final HTable table, int rows,
+      byte[]... families) throws IOException, InterruptedException {
+    table.setAutoFlush(false);
+
+    // Ensure one row per region
+    assertTrue(rows >= 16);
+    for (byte k0: Bytes.toBytes("0123456789abcdef")) {
+      byte[] k = new byte[] { k0 };
+      byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), k);
+      byte[] key = Bytes.add(k, Bytes.toBytes(MD5Hash.getMD5AsHex(value)));
+      putData(table, families, key, value);
+      rows--;
+    }
+
+    // Add other extra rows. more rows, more files
+    while (rows-- > 0) {
+      byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
+      byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
+      putData(table, families, key, value);
+    }
+    table.flushCommits();
+
+    waitForTableToBeOnline(util, table.getTableName());
+  }
+
+  private static void putData(final HTable table, final byte[][] families,
+      final byte[] key, final byte[] value) throws IOException {
+    byte[] q = Bytes.toBytes("q");
+    Put put = new Put(key);
+    put.setDurability(Durability.SKIP_WAL);
+    for (byte[] family: families) {
+      put.add(family, q, value);
+    }
+    table.put(put);
+  }
+
+  public static void deleteAllSnapshots(final HBaseAdmin admin)
+      throws IOException {
+    // Delete all the snapshots
+    for (SnapshotDescription snapshot: admin.listSnapshots()) {
+      admin.deleteSnapshot(snapshot.getName());
+    }
+    SnapshotTestingUtils.assertNoSnapshots(admin);
+  }
+
+  public static void deleteArchiveDirectory(final HBaseTestingUtility util)
+      throws IOException {
+    // Ensure the archiver to be empty
+    MasterFileSystem mfs = util.getMiniHBaseCluster().getMaster().getMasterFileSystem();
+    Path archiveDir = new Path(mfs.getRootDir(), HConstants.HFILE_ARCHIVE_DIRECTORY);
+    mfs.getFileSystem().delete(archiveDir, true);
+  }
+
+  public static void verifyRowCount(final HBaseTestingUtility util, final byte[] tableName,
+      long expectedRows) throws IOException {
+    HTable table = new HTable(util.getConfiguration(), tableName);
+    try {
+      assertEquals(expectedRows, util.countRows(table));
+    } finally {
+      table.close();
+    }
+  }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java?rev=1508801&r1=1508800&r2=1508801&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java Wed Jul 31 10:16:25 2013
@@ -119,10 +119,9 @@ public class TestExportSnapshot {
 
   @After
   public void tearDown() throws Exception {
-    admin.disableTable(tableName);
-    admin.deleteSnapshot(snapshotName);
-    admin.deleteSnapshot(emptySnapshotName);
-    admin.deleteTable(tableName);
+    TEST_UTIL.deleteTable(tableName);
+    SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL);
     admin.close();
   }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java?rev=1508801&r1=1508800&r2=1508801&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java Wed Jul 31 10:16:25 2013
@@ -42,8 +42,6 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcServer;
@@ -56,7 +54,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -121,23 +118,15 @@ public class TestFlushSnapshotFromClient
 
   @Before
   public void setup() throws Exception {
-    createTable(TABLE_NAME, TEST_FAM);
+    SnapshotTestingUtils.createTable(UTIL, TABLE_NAME, TEST_FAM);
   }
 
   @After
   public void tearDown() throws Exception {
     UTIL.deleteTable(TABLE_NAME);
-    // and cleanup the archive directory
-    try {
-      UTIL.getTestFileSystem().delete(
-        new Path(UTIL.getDefaultRootDirPath(), HConstants.HFILE_ARCHIVE_DIRECTORY), true);
-    } catch (IOException e) {
-      LOG.warn("Failure to delete archive directory", e);
-    }
-    for (SnapshotDescription snapshot: UTIL.getHBaseAdmin().listSnapshots()) {
-      UTIL.getHBaseAdmin().deleteSnapshot(snapshot.getName());
-    }
-    SnapshotTestingUtils.assertNoSnapshots(UTIL.getHBaseAdmin());
+
+    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
   }
 
   @AfterClass
@@ -161,7 +150,7 @@ public class TestFlushSnapshotFromClient
 
     // put some stuff in the table
     HTable table = new HTable(UTIL.getConfiguration(), TABLE_NAME);
-    loadData(table, DEFAULT_NUM_ROWS, TEST_FAM);
+    SnapshotTestingUtils.loadData(UTIL, table, DEFAULT_NUM_ROWS, TEST_FAM);
 
     // get the name of all the regionservers hosting the snapshotted table
     Set<String> snapshotServers = new HashSet<String>();
@@ -252,9 +241,7 @@ public class TestFlushSnapshotFromClient
     // make sure we don't fail on listing snapshots
     SnapshotTestingUtils.assertNoSnapshots(admin);
     // load the table so we have some data
-    loadData(new HTable(UTIL.getConfiguration(), TABLE_NAME), numRows, TEST_FAM);
-    // and wait until everything stabilizes
-    waitForTableToBeOnline(TABLE_NAME);
+    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, numRows, TEST_FAM);
 
     // Take a snapshot
     String snapshotBeforeMergeName = "snapshotBeforeMerge";
@@ -263,7 +250,7 @@ public class TestFlushSnapshotFromClient
     // Clone the table
     String cloneBeforeMergeName = "cloneBeforeMerge";
     admin.cloneSnapshot(snapshotBeforeMergeName, cloneBeforeMergeName);
-    waitForTableToBeOnline(Bytes.toBytes(cloneBeforeMergeName));
+    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, Bytes.toBytes(cloneBeforeMergeName));
 
     // Merge two regions
     List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
@@ -287,11 +274,11 @@ public class TestFlushSnapshotFromClient
     // Clone the table
     String cloneAfterMergeName = "cloneAfterMerge";
     admin.cloneSnapshot(snapshotBeforeMergeName, cloneAfterMergeName);
-    waitForTableToBeOnline(Bytes.toBytes(cloneAfterMergeName));
+    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, Bytes.toBytes(cloneAfterMergeName));
 
-    verifyRowCount(TABLE_NAME, numRows);
-    verifyRowCount(Bytes.toBytes(cloneBeforeMergeName), numRows);
-    verifyRowCount(Bytes.toBytes(cloneAfterMergeName), numRows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, TABLE_NAME, numRows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, Bytes.toBytes(cloneBeforeMergeName), numRows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, Bytes.toBytes(cloneAfterMergeName), numRows);
 
     // test that we can delete the snapshot
     UTIL.deleteTable(cloneAfterMergeName);
@@ -305,9 +292,7 @@ public class TestFlushSnapshotFromClient
     // make sure we don't fail on listing snapshots
     SnapshotTestingUtils.assertNoSnapshots(admin);
     // load the table so we have some data
-    loadData(new HTable(UTIL.getConfiguration(), TABLE_NAME), numRows, TEST_FAM);
-    // and wait until everything stabilizes
-    waitForTableToBeOnline(TABLE_NAME);
+    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, numRows, TEST_FAM);
 
     // Merge two regions
     List<HRegionInfo> regions = admin.getTableRegions(TABLE_NAME);
@@ -334,10 +319,10 @@ public class TestFlushSnapshotFromClient
     // Clone the table
     String cloneName = "cloneMerge";
     admin.cloneSnapshot(snapshotName, cloneName);
-    waitForTableToBeOnline(Bytes.toBytes(cloneName));
+    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, Bytes.toBytes(cloneName));
 
-    verifyRowCount(TABLE_NAME, numRows);
-    verifyRowCount(Bytes.toBytes(cloneName), numRows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, TABLE_NAME, numRows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, Bytes.toBytes(cloneName), numRows);
 
     // test that we can delete the snapshot
     UTIL.deleteTable(cloneName);
@@ -353,9 +338,7 @@ public class TestFlushSnapshotFromClient
     // make sure we don't fail on listing snapshots
     SnapshotTestingUtils.assertNoSnapshots(admin);
     // load the table so we have some data
-    loadData(new HTable(UTIL.getConfiguration(), TABLE_NAME), DEFAULT_NUM_ROWS, TEST_FAM);
-    // and wait until everything stabilizes
-    waitForTableToBeOnline(TABLE_NAME);
+    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
 
     String snapshotName = "flushSnapshotCreateListDestroy";
     // test creating the snapshot
@@ -414,13 +397,10 @@ public class TestFlushSnapshotFromClient
     // make sure we don't fail on listing snapshots
     SnapshotTestingUtils.assertNoSnapshots(admin);
     // create second testing table
-    createTable(TABLE2_NAME, TEST_FAM);
+    SnapshotTestingUtils.createTable(UTIL, TABLE2_NAME, TEST_FAM);
     // load the table so we have some data
-    loadData(new HTable(UTIL.getConfiguration(), TABLE_NAME), DEFAULT_NUM_ROWS, TEST_FAM);
-    loadData(new HTable(UTIL.getConfiguration(), TABLE2_NAME), DEFAULT_NUM_ROWS, TEST_FAM);
-    // and wait until everything stabilizes
-    waitForTableToBeOnline(TABLE_NAME);
-    waitForTableToBeOnline(TABLE2_NAME);
+    SnapshotTestingUtils.loadData(UTIL, TABLE_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
+    SnapshotTestingUtils.loadData(UTIL, TABLE2_NAME, DEFAULT_NUM_ROWS, TEST_FAM);
 
     final CountDownLatch toBeSubmitted = new CountDownLatch(ssNum);
     // We'll have one of these per thread
@@ -512,16 +492,6 @@ public class TestFlushSnapshotFromClient
     FSUtils.logFileSystemState(UTIL.getDFSCluster().getFileSystem(), root, LOG);
   }
 
-  private void waitForTableToBeOnline(final byte[] tableName)
-  throws IOException, InterruptedException {
-    HRegionServer rs = UTIL.getRSForFirstRegionInTable(tableName);
-    List<HRegion> onlineRegions = rs.getOnlineRegions(tableName);
-    for (HRegion region : onlineRegions) {
-      region.waitForFlushesAndCompactions();
-    }
-    UTIL.getHBaseAdmin().isTableAvailable(tableName);
-  }
-
   private void waitRegionsAfterMerge(final long numRegionsAfterMerge)
       throws IOException, InterruptedException {
     HBaseAdmin admin = UTIL.getHBaseAdmin();
@@ -534,64 +504,6 @@ public class TestFlushSnapshotFromClient
         break;
       Thread.sleep(100);
     }
-    waitForTableToBeOnline(TABLE_NAME);
-  }
-
-  private void createTable(final byte[] tableName, final byte[]... families)
-      throws IOException, InterruptedException {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    for (byte[] family: families) {
-      HColumnDescriptor hcd = new HColumnDescriptor(family);
-      htd.addFamily(hcd);
-    }
-    byte[][] splitKeys = new byte[14][];
-    byte[] hex = Bytes.toBytes("123456789abcde");
-    for (int i = 0; i < splitKeys.length; ++i) {
-      splitKeys[i] = new byte[] { hex[i] };
-    }
-    UTIL.getHBaseAdmin().createTable(htd, splitKeys);
-    waitForTableToBeOnline(tableName);
-    assertEquals(15, UTIL.getHBaseAdmin().getTableRegions(TABLE_NAME).size());
-  }
-
-  public void loadData(final HTable table, int rows, byte[]... families)
-      throws IOException, InterruptedException {
-    table.setAutoFlush(false);
-
-    // Ensure one row per region
-    assertTrue(rows >= 16);
-    for (byte k0: Bytes.toBytes("0123456789abcdef")) {
-      byte[] k = new byte[] { k0 };
-      byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), k);
-      byte[] key = Bytes.add(k, Bytes.toBytes(MD5Hash.getMD5AsHex(value)));
-      putData(table, families, key, value);
-      rows--;
-    }
-
-    // Add other extra rows. more rows, more files
-    while (rows-- > 0) {
-      byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
-      byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
-      putData(table, families, key, value);
-    }
-    table.flushCommits();
-
-    waitForTableToBeOnline(table.getTableName());
-  }
-
-  private void putData(final HTable table, final byte[][] families,
-      final byte[] key, final byte[] value) throws IOException {
-    Put put = new Put(key);
-    put.setDurability(Durability.SKIP_WAL);
-    for (byte[] family: families) {
-      put.add(family, TEST_QUAL, value);
-    }
-    table.put(put);
-  }
-
-  private void verifyRowCount(final byte[] tableName, long expectedRows) throws IOException {
-    HTable table = new HTable(UTIL.getConfiguration(), tableName);
-    assertEquals(expectedRows, UTIL.countRows(table));
-    table.close();
+    SnapshotTestingUtils.waitForTableToBeOnline(UTIL, TABLE_NAME);
   }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java?rev=1508801&r1=1508800&r2=1508801&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreFlushSnapshotFromClient.java Wed Jul 31 10:16:25 2013
@@ -18,8 +18,10 @@
 package org.apache.hadoop.hbase.snapshot;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -36,6 +38,8 @@ import org.apache.hadoop.hbase.client.Du
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -57,9 +61,10 @@ import org.junit.experimental.categories
 public class TestRestoreFlushSnapshotFromClient {
   final Log LOG = LogFactory.getLog(getClass());
 
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
   private final byte[] FAMILY = Bytes.toBytes("cf");
+  private static final byte[] TEST_QUAL = Bytes.toBytes("q");
 
   private byte[] snapshotName0;
   private byte[] snapshotName1;
@@ -71,24 +76,24 @@ public class TestRestoreFlushSnapshotFro
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
-    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
-    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
-    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
-    TEST_UTIL.getConfiguration().setBoolean(
+    UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
+    UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
+    UTIL.getConfiguration().setInt("hbase.client.pause", 250);
+    UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
+    UTIL.getConfiguration().setBoolean(
         "hbase.master.enabletable.roundrobin", true);
 
     // Enable snapshot
-    TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
-    TEST_UTIL.getConfiguration().setLong(RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_KEY,
+    UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
+    UTIL.getConfiguration().setLong(RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_KEY,
       RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT * 2);
 
-    TEST_UTIL.startMiniCluster(3);
+    UTIL.startMiniCluster(3);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+    UTIL.shutdownMiniCluster();
   }
 
   /**
@@ -98,7 +103,7 @@ public class TestRestoreFlushSnapshotFro
    */
   @Before
   public void setup() throws Exception {
-    this.admin = TEST_UTIL.getHBaseAdmin();
+    this.admin = UTIL.getHBaseAdmin();
 
     long tid = System.currentTimeMillis();
     tableName = Bytes.toBytes("testtb-" + tid);
@@ -107,11 +112,11 @@ public class TestRestoreFlushSnapshotFro
     snapshotName2 = Bytes.toBytes("snaptb2-" + tid);
 
     // create Table and disable it
-    createTable(tableName, FAMILY);
-    HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
+    SnapshotTestingUtils.createTable(UTIL, tableName, FAMILY);
+    HTable table = new HTable(UTIL.getConfiguration(), tableName);
     try {
-      loadData(table, 500, FAMILY);
-      snapshot0Rows = TEST_UTIL.countRows(table);
+      SnapshotTestingUtils.loadData(UTIL, table, 500, FAMILY);
+      snapshot0Rows = UTIL.countRows(table);
       LOG.info("=== before snapshot with 500 rows");
       logFSTree();
 
@@ -123,8 +128,8 @@ public class TestRestoreFlushSnapshotFro
       logFSTree();
 
       // insert more data
-      loadData(table, 500, FAMILY);
-      snapshot1Rows = TEST_UTIL.countRows(table);
+      SnapshotTestingUtils.loadData(UTIL, table, 500, FAMILY);
+      snapshot1Rows = UTIL.countRows(table);
       LOG.info("=== before snapshot with 1000 rows");
       logFSTree();
 
@@ -140,14 +145,8 @@ public class TestRestoreFlushSnapshotFro
 
   @After
   public void tearDown() throws Exception {
-    TEST_UTIL.deleteTable(tableName);
-    admin.deleteSnapshot(snapshotName0);
-    admin.deleteSnapshot(snapshotName1);
-
-    // Ensure the archiver to be empty
-    MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
-    mfs.getFileSystem().delete(
-      new Path(mfs.getRootDir(), HConstants.HFILE_ARCHIVE_DIRECTORY), true);
+    SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
+    SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
   }
 
   @Test
@@ -157,7 +156,7 @@ public class TestRestoreFlushSnapshotFro
 
   @Test
   public void testRestoreSnapshot() throws IOException {
-    verifyRowCount(tableName, snapshot1Rows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshot1Rows);
 
     // Restore from snapshot-0
     admin.disableTable(tableName);
@@ -166,13 +165,13 @@ public class TestRestoreFlushSnapshotFro
     admin.enableTable(tableName);
     LOG.info("=== after restore with 500 row snapshot");
     logFSTree();
-    verifyRowCount(tableName, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshot0Rows);
 
     // Restore from snapshot-1
     admin.disableTable(tableName);
     admin.restoreSnapshot(snapshotName1);
     admin.enableTable(tableName);
-    verifyRowCount(tableName, snapshot1Rows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshot1Rows);
   }
 
   @Test(expected=SnapshotDoesNotExistException.class)
@@ -193,65 +192,29 @@ public class TestRestoreFlushSnapshotFro
       int snapshotRows) throws IOException, InterruptedException {
     // create a new table from snapshot
     admin.cloneSnapshot(snapshotName, tableName);
-    verifyRowCount(tableName, snapshotRows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, tableName, snapshotRows);
 
-    TEST_UTIL.deleteTable(tableName);
+    UTIL.deleteTable(tableName);
   }
 
   @Test
   public void testRestoreSnapshotOfCloned() throws IOException, InterruptedException {
     byte[] clonedTableName = Bytes.toBytes("clonedtb-" + System.currentTimeMillis());
     admin.cloneSnapshot(snapshotName0, clonedTableName);
-    verifyRowCount(clonedTableName, snapshot0Rows);
+    SnapshotTestingUtils.verifyRowCount(UTIL, clonedTableName, snapshot0Rows);
     admin.snapshot(Bytes.toString(snapshotName2), Bytes.toString(clonedTableName), SnapshotDescription.Type.FLUSH);
-    TEST_UTIL.deleteTable(clonedTableName);
+    UTIL.deleteTable(clonedTableName);
 
     admin.cloneSnapshot(snapshotName2, clonedTableName);
-    verifyRowCount(clonedTableName, snapshot0Rows);
-    TEST_UTIL.deleteTable(clonedTableName);
+    SnapshotTestingUtils.verifyRowCount(UTIL, clonedTableName, snapshot0Rows);
+    UTIL.deleteTable(clonedTableName);
   }
 
   // ==========================================================================
   //  Helpers
   // ==========================================================================
-  private void createTable(final byte[] tableName, final byte[]... families) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    for (byte[] family: families) {
-      HColumnDescriptor hcd = new HColumnDescriptor(family);
-      htd.addFamily(hcd);
-    }
-    byte[][] splitKeys = new byte[16][];
-    byte[] hex = Bytes.toBytes("0123456789abcdef");
-    for (int i = 0; i < 16; ++i) {
-      splitKeys[i] = new byte[] { hex[i] };
-    }
-    admin.createTable(htd, splitKeys);
-  }
-
-  public void loadData(final HTable table, int rows, byte[]... families) throws IOException {
-    byte[] qualifier = Bytes.toBytes("q");
-    table.setAutoFlush(false);
-    while (rows-- > 0) {
-      byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
-      byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
-      Put put = new Put(key);
-      put.setDurability(Durability.SKIP_WAL);
-      for (byte[] family: families) {
-        put.add(family, qualifier, value);
-      }
-      table.put(put);
-    }
-    table.flushCommits();
-  }
-
   private void logFSTree() throws IOException {
-    MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
+    MasterFileSystem mfs = UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
     FSUtils.logFileSystemState(mfs.getFileSystem(), mfs.getRootDir(), LOG);
   }
-
-  private void verifyRowCount(final byte[] tableName, long expectedRows) throws IOException {
-    HTable table = new HTable(TEST_UTIL.getConfiguration(), tableName);
-    assertEquals(expectedRows, TEST_UTIL.countRows(table));
-    table.close();
-  }
 }