commit a210eb243de1f6e2f64810cfab25f2b261aa7e33 Author: Todd Lipcon Date: Sun May 23 19:46:38 2010 -0700 HBASE-2589. Fix TestHRegion to wait until put thread has started before beginning reading diff --git src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 44e7e6d..99feb2f 100644 --- src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -2104,9 +2104,11 @@ public class TestHRegion extends HBaseTestCase { initHRegion(tableName, method, families); PutThread putThread = new PutThread(numRows, families, qualifiers); putThread.start(); + putThread.waitForFirstPut(); + FlushThread flushThread = new FlushThread(); flushThread.start(); - + Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1")); // scan.setFilter(new RowFilter(CompareFilter.CompareOp.EQUAL, // new BinaryComparator(Bytes.toBytes("row0")))); @@ -2153,6 +2155,8 @@ public class TestHRegion extends HBaseTestCase { protected class PutThread extends Thread { private volatile boolean done; + private volatile int numPutsFinished = 0; + private Throwable error = null; private int numRows; private byte[][] families; @@ -2165,6 +2169,17 @@ public class TestHRegion extends HBaseTestCase { this.qualifiers = qualifiers; } + /** + * Block until this thread has put at least one row. + */ + public void waitForFirstPut() throws InterruptedException { + // wait until put thread actually puts some data + while (numPutsFinished == 0) { + checkNoError(); + Thread.sleep(50); + } + } + public void done() { done = true; synchronized (this) { @@ -2181,7 +2196,6 @@ public class TestHRegion extends HBaseTestCase { @Override public void run() { done = false; - int val = 0; while (!done) { try { for (int r = 0; r < numRows; r++) { @@ -2189,18 +2203,19 @@ public class TestHRegion extends HBaseTestCase { Put put = new Put(row); for (byte[] family : families) { for (byte[] qualifier : qualifiers) { - put.add(family, qualifier, (long) val, - Bytes.toBytes(val)); + put.add(family, qualifier, (long) numPutsFinished, + Bytes.toBytes(numPutsFinished)); } } // System.out.println("Putting of kvsetsize=" + put.size()); region.put(put); - if (val > 0 && val % 47 == 0) { - System.out.println("put iteration = " + val); - Delete delete = new Delete(row, (long)val-30, null); + numPutsFinished++; + if (numPutsFinished > 0 && numPutsFinished % 47 == 0) { + System.out.println("put iteration = " + numPutsFinished); + Delete delete = new Delete(row, (long)numPutsFinished-30, null); region.delete(delete, null, true); } - val++; + numPutsFinished++; } } catch (IOException e) { LOG.error("error while putting records", e); @@ -2244,6 +2259,8 @@ public class TestHRegion extends HBaseTestCase { initHRegion(tableName, method, families); PutThread putThread = new PutThread(numRows, families, qualifiers); putThread.start(); + putThread.waitForFirstPut(); + FlushThread flushThread = new FlushThread(); flushThread.start();