You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/13 19:58:45 UTC

svn commit: r1445852 - /hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java

Author: jmhsieh
Date: Wed Feb 13 18:58:44 2013
New Revision: 1445852

URL: http://svn.apache.org/r1445852
Log:
HBASE-7536 Add test that confirms that multiple concurrent snapshot requests are rejected


Modified:
    hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java?rev=1445852&r1=1445851&r2=1445852&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestFlushSnapshotFromClient.java Wed Feb 13 18:58:44 2013
@@ -25,11 +25,11 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.CountDownLatch;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.TableNotF
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -297,7 +296,94 @@ public class TestFlushSnapshotFromClient
     // make sure we don't have any snapshots
     SnapshotTestingUtils.assertNoSnapshots(admin);
     LOG.debug("------- Flush-Snapshot Create List Destroy-------------");
+  }
+
+  /**
+   * Demonstrate that we reject snapshot requests if there is a snapshot currently running.
+   */
+  @Test(timeout=60000)
+  public void testConcurrentSnapshottingAttempts() throws IOException, InterruptedException {
+    int ssNum = 10;
+    HBaseAdmin admin = UTIL.getHBaseAdmin();
+    // make sure we don't fail on listing snapshots
+    SnapshotTestingUtils.assertNoSnapshots(admin);
+    // load the table so we have some data
+    UTIL.loadTable(new HTable(UTIL.getConfiguration(), TABLE_NAME), TEST_FAM);
+    // and wait until everything stabilizes
+    HRegionServer rs = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
+    List<HRegion> onlineRegions = rs.getOnlineRegions(TABLE_NAME);
+    for (HRegion region : onlineRegions) {
+      region.waitForFlushesAndCompactions();
+    }
+
+    // build descriptions
+    SnapshotDescription[] descs = new SnapshotDescription[ssNum];
+    for (int i = 0; i < ssNum; i++) {
+      SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
+      builder.setTable(STRING_TABLE_NAME);
+      builder.setName("ss"+i);
+      builder.setType(SnapshotDescription.Type.FLUSH);
+      descs[i] = builder.build();
+    }
+
+    final CountDownLatch toBeSubmitted = new CountDownLatch(ssNum);
+    // We'll have one of these per thread
+    class SSRunnable implements Runnable {
+      SnapshotDescription ss;
+      SSRunnable(SnapshotDescription ss) {
+        this.ss = ss;
+      }
+
+      @Override
+      public void run() {
+        try {
+          HBaseAdmin admin = UTIL.getHBaseAdmin();
+          LOG.info("Submitting snapshot request: " + SnapshotDescriptionUtils.toString(ss));
+          admin.takeSnapshotAsync(ss);
+        } catch (Exception e) {
+          LOG.info("Exception during snapshot request: " + SnapshotDescriptionUtils.toString(ss)
+              + ".  This is ok, we expect some", e);
+        }
+        LOG.info("Submitted snapshot request: " + SnapshotDescriptionUtils.toString(ss));
+        toBeSubmitted.countDown();
+      }
+    };
+
+    // kick each off its own thread
+    for (int i=0 ; i < ssNum; i++) {
+      new Thread(new SSRunnable(descs[i])).start();
+    }
+
+    // wait until all have been submitted
+    toBeSubmitted.await();
+
+    // loop until all are done.
+    while (true) {
+      int doneCount = 0;
+      for (SnapshotDescription ss : descs) {
+        try {
+          if (admin.isSnapshotFinished(ss)) {
+            doneCount++;
+          }
+        } catch (Exception e) {
+          LOG.warn("Got an exception when checking for snapshot " + ss.getName(), e);
+          doneCount++;
+        }
+      }
+      if (doneCount == descs.length) {
+        break;
+      }
+      Thread.sleep(100);
+    }
+
+    // dump for debugging
+    logFSTree(new Path(UTIL.getConfiguration().get(HConstants.HBASE_DIR)));
 
+    List<SnapshotDescription> taken = admin.listSnapshots();
+    int takenSize = taken.size();
+    LOG.info("Taken " + takenSize + " snapshots:  " + taken);
+    assertTrue("We expect at least 1 request to be rejected because of we concurrently" +
+        " issued many requests", takenSize < ssNum && takenSize > 0);
   }
 
   private void logFSTree(Path root) throws IOException {