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 21:58:32 UTC

svn commit: r1445918 [25/29] - in /hbase/branches/hbase-7290: ./ bin/ conf/ dev-support/ hbase-client/ hbase-common/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/ hbase-common/src/ma...

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java Wed Feb 13 20:58:23 2013
@@ -21,13 +21,21 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseTestCase;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
+import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
 import org.junit.experimental.categories.Category;
 
 @Category(SmallTests.class)
@@ -43,6 +51,28 @@ public class TestKeepDeletes extends HBa
   private final byte[] c0 = COLUMNS[0];
   private final byte[] c1 = COLUMNS[1];
 
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    /* HBASE-6832: [WINDOWS] Tests should use explicit timestamp for Puts, and not rely on
+     * implicit RS timing.
+     * Use an explicit timer (IncrementingEnvironmentEdge) so that the put, delete
+     * compact timestamps are tracked. Otherwise, forced major compaction will not purge
+     * Delete's having the same timestamp. see ScanQueryMatcher.match():
+     * if (retainDeletesInOutput
+     *     || (!isUserScan && (EnvironmentEdgeManager.currentTimeMillis() - timestamp)
+     *     <= timeToPurgeDeletes) ... )
+     *
+     */
+    EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    super.tearDown();
+    EnvironmentEdgeManager.reset();
+  }
+
   /**
    * Make sure that deleted rows are retained.
    * Family delete markers are deleted.
@@ -55,7 +85,7 @@ public class TestKeepDeletes extends HBa
         HConstants.FOREVER, true);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis();
+    long ts = EnvironmentEdgeManager.currentTimeMillis();
     Put p = new Put(T1, ts);
     p.add(c0, c0, T1);
     region.put(p);
@@ -70,8 +100,8 @@ public class TestKeepDeletes extends HBa
     region.put(p);
 
     // now place a delete marker at ts+2
-    Delete d = new Delete(T1, ts+2, null);
-    region.delete(d, null, true);
+    Delete d = new Delete(T1, ts+2);
+    region.delete(d, true);
 
     // a raw scan can see the delete markers
     // (one for each column family)
@@ -81,14 +111,14 @@ public class TestKeepDeletes extends HBa
     Get g = new Get(T1);
     g.setMaxVersions();
     g.setTimeRange(0L, ts+2);
-    Result r = region.get(g, null);
+    Result r = region.get(g);
     checkResult(r, c0, c0, T2,T1);
 
     // flush
     region.flushcache();
 
     // yep, T2 still there, T1 gone
-    r = region.get(g, null);
+    r = region.get(g);
     checkResult(r, c0, c0, T2);
 
     // major compact
@@ -100,12 +130,12 @@ public class TestKeepDeletes extends HBa
     assertEquals(1, countDeleteMarkers(region));
 
     // still there (even after multiple compactions)
-    r = region.get(g, null);
+    r = region.get(g);
     checkResult(r, c0, c0, T2);
 
     // a timerange that includes the delete marker won't see past rows
     g.setTimeRange(0L, ts+4);
-    r = region.get(g, null);
+    r = region.get(g);
     assertTrue(r.isEmpty());
 
     // two more puts, this will expire the older puts.
@@ -121,7 +151,7 @@ public class TestKeepDeletes extends HBa
     p = new Put(T1, ts);
     p.add(c0, c0, T1);
     region.put(p);
-    r = region.get(g, null);
+    r = region.get(g);
     assertTrue(r.isEmpty());
 
     region.flushcache();
@@ -130,7 +160,7 @@ public class TestKeepDeletes extends HBa
 
     // verify that the delete marker itself was collected
     region.put(p);
-    r = region.get(g, null);
+    r = region.get(g);
     checkResult(r, c0, c0, T1);
     assertEquals(0, countDeleteMarkers(region));
 
@@ -138,7 +168,7 @@ public class TestKeepDeletes extends HBa
   }
 
   /**
-   * Even when the store does not keep deletes a "raw" scan will 
+   * Even when the store does not keep deletes a "raw" scan will
    * return everything it can find (unless discarding cells is guaranteed
    * to have no effect).
    * Assuming this the desired behavior. Could also disallow "raw" scanning
@@ -151,14 +181,14 @@ public class TestKeepDeletes extends HBa
         HConstants.FOREVER, false);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis();
+    long ts = EnvironmentEdgeManager.currentTimeMillis();
     Put p = new Put(T1, ts);
     p.add(c0, c0, T1);
     region.put(p);
 
-    Delete d = new Delete(T1, ts, null);
+    Delete d = new Delete(T1, ts);
     d.deleteColumn(c0, c0, ts);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
     // scan still returns delete markers and deletes rows
     Scan s = new Scan();
@@ -195,19 +225,19 @@ public class TestKeepDeletes extends HBa
         HConstants.FOREVER, false);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis();  
+    long ts = EnvironmentEdgeManager.currentTimeMillis();
     Put p = new Put(T1, ts);
     p.add(c0, c0, T1);
     region.put(p);
-    Delete d = new Delete(T1, ts+2, null);
+    Delete d = new Delete(T1, ts+2);
     d.deleteColumn(c0, c0, ts);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
     // "past" get does not see rows behind delete marker
     Get g = new Get(T1);
     g.setMaxVersions();
     g.setTimeRange(0L, ts+1);
-    Result r = region.get(g, null);
+    Result r = region.get(g);
     assertTrue(r.isEmpty());
 
     // "past" scan does not see rows behind delete marker
@@ -242,7 +272,7 @@ public class TestKeepDeletes extends HBa
     s.setRaw(true);
     s.setMaxVersions();
     s.addColumn(c0, c0);
-    
+
     try {
       InternalScanner scan = region.getScanner(s);
       fail("raw scanner with columns should have failed");
@@ -261,7 +291,7 @@ public class TestKeepDeletes extends HBa
         HConstants.FOREVER, true);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis();
+    long ts = EnvironmentEdgeManager.currentTimeMillis();
     Put p = new Put(T1, ts);
     p.add(c0, c0, T1);
     region.put(p);
@@ -272,16 +302,16 @@ public class TestKeepDeletes extends HBa
     p.add(c0, c0, T3);
     region.put(p);
 
-    Delete d = new Delete(T1, ts+1, null);
-    region.delete(d, null, true);
+    Delete d = new Delete(T1, ts+1);
+    region.delete(d, true);
 
-    d = new Delete(T1, ts+2, null);
+    d = new Delete(T1, ts+2);
     d.deleteColumn(c0, c0, ts+2);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
-    d = new Delete(T1, ts+3, null);
+    d = new Delete(T1, ts+3);
     d.deleteColumns(c0, c0, ts+3);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
     Scan s = new Scan();
     s.setRaw(true);
@@ -307,23 +337,23 @@ public class TestKeepDeletes extends HBa
         HConstants.FOREVER, true);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis();
+    long ts = EnvironmentEdgeManager.currentTimeMillis();
 
-    Delete d = new Delete(T1, ts, null);
+    Delete d = new Delete(T1, ts);
     d.deleteColumns(c0, c0, ts);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
-    d = new Delete(T1, ts, null);
+    d = new Delete(T1, ts);
     d.deleteFamily(c0);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
-    d = new Delete(T1, ts, null);
+    d = new Delete(T1, ts);
     d.deleteColumn(c0, c0, ts+1);
-    region.delete(d, null, true);
-    
-    d = new Delete(T1, ts, null);
+    region.delete(d, true);
+
+    d = new Delete(T1, ts);
     d.deleteColumn(c0, c0, ts+2);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
     // 1 family marker, 1 column marker, 2 version markers
     assertEquals(4, countDeleteMarkers(region));
@@ -349,7 +379,7 @@ public class TestKeepDeletes extends HBa
         HConstants.FOREVER, true);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis();
+    long ts = EnvironmentEdgeManager.currentTimeMillis();
 
     Put p = new Put(T1, ts);
     p.add(c0, c0, T1);
@@ -361,21 +391,21 @@ public class TestKeepDeletes extends HBa
     region.put(p);
 
     // all the following deletes affect the put
-    Delete d = new Delete(T1, ts, null);
+    Delete d = new Delete(T1, ts);
     d.deleteColumns(c0, c0, ts);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
-    d = new Delete(T1, ts, null);
+    d = new Delete(T1, ts);
     d.deleteFamily(c0, ts);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
-    d = new Delete(T1, ts, null);
+    d = new Delete(T1, ts);
     d.deleteColumn(c0, c0, ts+1);
-    region.delete(d, null, true);
-    
-    d = new Delete(T1, ts, null);
+    region.delete(d, true);
+
+    d = new Delete(T1, ts);
     d.deleteColumn(c0, c0, ts+2);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
     // 1 family marker, 1 column marker, 2 version markers
     assertEquals(4, countDeleteMarkers(region));
@@ -411,7 +441,7 @@ public class TestKeepDeletes extends HBa
         HConstants.FOREVER, true);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis();
+    long ts = EnvironmentEdgeManager.currentTimeMillis();
     Put p = new Put(T1, ts);
     p.add(c0, c0, T1);
     p.add(c0, c1, T1);
@@ -440,22 +470,22 @@ public class TestKeepDeletes extends HBa
     p.add(c1, c1, T2);
     region.put(p);
 
-    Delete d = new Delete(T1, ts+2, null);
+    Delete d = new Delete(T1, ts+2);
     d.deleteColumns(c0, c0, ts+2);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
-    d = new Delete(T1, ts+2, null);
+    d = new Delete(T1, ts+2);
     d.deleteFamily(c1, ts+2);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
-    d = new Delete(T2, ts+2, null);
+    d = new Delete(T2, ts+2);
     d.deleteFamily(c0, ts+2);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
     // add an older delete, to make sure it is filtered
-    d = new Delete(T1, ts-10, null);
+    d = new Delete(T1, ts-10);
     d.deleteFamily(c1, ts-10);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
     // ts + 2 does NOT include the delete at ts+2
     checkGet(region, T1, c0, c0, ts+2, T2, T1);
@@ -492,7 +522,7 @@ public class TestKeepDeletes extends HBa
         HConstants.FOREVER, true);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis();
+    long ts = EnvironmentEdgeManager.currentTimeMillis();
     Put p = new Put(T1, ts);
     p.add(c0, c0, T1);
     region.put(p);
@@ -502,19 +532,19 @@ public class TestKeepDeletes extends HBa
     p = new Put(T1, ts-10);
     p.add(c0, c1, T1);
     region.put(p);
-    
-    Delete d = new Delete(T1, ts, null);
+
+    Delete d = new Delete(T1, ts);
     // test corner case (Put and Delete have same TS)
     d.deleteColumns(c0, c0, ts);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
-    d = new Delete(T1, ts+1, null);
+    d = new Delete(T1, ts+1);
     d.deleteColumn(c0, c0, ts+1);
-    region.delete(d, null, true);
-    
-    d = new Delete(T1, ts+3, null);
+    region.delete(d, true);
+
+    d = new Delete(T1, ts+3);
     d.deleteColumn(c0, c0, ts+3);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
     region.flushcache();
     region.compactStores(true);
@@ -527,7 +557,7 @@ public class TestKeepDeletes extends HBa
     p = new Put(T1, ts+2);
     p.add(c0, c0, T2);
     region.put(p);
-    
+
     // delete, put, delete, delete, put
     assertEquals(3, countDeleteMarkers(region));
 
@@ -584,7 +614,7 @@ public class TestKeepDeletes extends HBa
         HConstants.FOREVER, true);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis();
+    long ts = EnvironmentEdgeManager.currentTimeMillis();
 
     Put p = new Put(T1, ts);
     p.add(c0, c0, T1);
@@ -601,11 +631,11 @@ public class TestKeepDeletes extends HBa
     region.put(p);
 
     // family markers are each family
-    Delete d = new Delete(T1, ts+1, null);
-    region.delete(d, null, true);
+    Delete d = new Delete(T1, ts+1);
+    region.delete(d, true);
 
-    d = new Delete(T2, ts+2, null);
-    region.delete(d, null, true);
+    d = new Delete(T2, ts+2);
+    region.delete(d, true);
 
     Scan s = new Scan(T1);
     s.setTimeRange(0, ts+1);
@@ -634,7 +664,7 @@ public class TestKeepDeletes extends HBa
     HTableDescriptor htd = createTableDescriptor(getName(), 3, 1000, 1, true);
     HRegion region = createNewHRegion(htd, null, null);
 
-    long ts = System.currentTimeMillis() - 2000; // 2s in the past
+    long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000; // 2s in the past
 
     Put p = new Put(T1, ts);
     p.add(c0, c0, T3);
@@ -652,17 +682,17 @@ public class TestKeepDeletes extends HBa
     // all puts now are just retained because of min versions = 3
 
     // place a family delete marker
-    Delete d = new Delete(T1, ts-1, null);
-    region.delete(d, null, true);
+    Delete d = new Delete(T1, ts-1);
+    region.delete(d, true);
     // and a column delete marker
-    d = new Delete(T1, ts-2, null);
+    d = new Delete(T1, ts-2);
     d.deleteColumns(c0, c0, ts-1);
-    region.delete(d, null, true);
+    region.delete(d, true);
 
     Get g = new Get(T1);
     g.setMaxVersions();
     g.setTimeRange(0L, ts-2);
-    Result r = region.get(g, null);
+    Result r = region.get(g);
     checkResult(r, c0, c0, T1,T0);
 
     // 3 families, one column delete marker
@@ -672,7 +702,7 @@ public class TestKeepDeletes extends HBa
     // no delete marker removes by the flush
     assertEquals(4, countDeleteMarkers(region));
 
-    r = region.get(g, null);
+    r = region.get(g);
     checkResult(r, c0, c0, T1);
     p = new Put(T1, ts+1);
     p.add(c0, c0, T4);
@@ -681,7 +711,7 @@ public class TestKeepDeletes extends HBa
 
     assertEquals(4, countDeleteMarkers(region));
 
-    r = region.get(g, null);
+    r = region.get(g);
     checkResult(r, c0, c0, T1);
 
     // this will push out the last put before
@@ -709,7 +739,7 @@ public class TestKeepDeletes extends HBa
     g.addColumn(fam, col);
     g.setMaxVersions();
     g.setTimeRange(0L, time);
-    Result r = region.get(g, null);
+    Result r = region.get(g);
     checkResult(r, fam, col, vals);
 
   }

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java Wed Feb 13 20:58:23 2013
@@ -133,13 +133,13 @@ public class TestMinVersions extends HBa
 
       Get g = new Get(T1);
       g.setMaxVersions();
-      Result r = region.get(g, null); // this'll use ScanWildcardColumnTracker
+      Result r = region.get(g); // this'll use ScanWildcardColumnTracker
       checkResult(r, c0, T3,T2,T1);
 
       g = new Get(T1);
       g.setMaxVersions();
       g.addColumn(c0, c0);
-      r = region.get(g, null);  // this'll use ExplicitColumnTracker
+      r = region.get(g);  // this'll use ExplicitColumnTracker
       checkResult(r, c0, T3,T2,T1);
     } finally {
       HRegion.closeHRegion(region);
@@ -169,18 +169,18 @@ public class TestMinVersions extends HBa
       p.add(c0, c0, T3);
       region.put(p);
 
-      Delete d = new Delete(T1, ts-1, null);
-      region.delete(d, null, true);
+      Delete d = new Delete(T1, ts-1);
+      region.delete(d, true);
 
       Get g = new Get(T1);
       g.setMaxVersions();
-      Result r = region.get(g, null);  // this'll use ScanWildcardColumnTracker
+      Result r = region.get(g);  // this'll use ScanWildcardColumnTracker
       checkResult(r, c0, T3);
 
       g = new Get(T1);
       g.setMaxVersions();
       g.addColumn(c0, c0);
-      r = region.get(g, null);  // this'll use ExplicitColumnTracker
+      r = region.get(g);  // this'll use ExplicitColumnTracker
       checkResult(r, c0, T3);
 
       // now flush/compact
@@ -190,13 +190,13 @@ public class TestMinVersions extends HBa
       // try again
       g = new Get(T1);
       g.setMaxVersions();
-      r = region.get(g, null);  // this'll use ScanWildcardColumnTracker
+      r = region.get(g);  // this'll use ScanWildcardColumnTracker
       checkResult(r, c0, T3);
 
       g = new Get(T1);
       g.setMaxVersions();
       g.addColumn(c0, c0);
-      r = region.get(g, null);  // this'll use ExplicitColumnTracker
+      r = region.get(g);  // this'll use ExplicitColumnTracker
       checkResult(r, c0, T3);
     } finally {
       HRegion.closeHRegion(region);
@@ -241,18 +241,18 @@ public class TestMinVersions extends HBa
       // now the latest change is in the memstore,
       // but it is not the latest version
 
-      Result r = region.get(new Get(T1), null);
+      Result r = region.get(new Get(T1));
       checkResult(r, c0, T4);
 
       Get g = new Get(T1);
       g.setMaxVersions();
-      r = region.get(g, null); // this'll use ScanWildcardColumnTracker
+      r = region.get(g); // this'll use ScanWildcardColumnTracker
       checkResult(r, c0, T4,T3);
 
       g = new Get(T1);
       g.setMaxVersions();
       g.addColumn(c0, c0);
-      r = region.get(g, null);  // this'll use ExplicitColumnTracker
+      r = region.get(g);  // this'll use ExplicitColumnTracker
       checkResult(r, c0, T4,T3);
 
       p = new Put(T1, ts+1);
@@ -263,13 +263,13 @@ public class TestMinVersions extends HBa
 
       g = new Get(T1);
       g.setMaxVersions();
-      r = region.get(g, null);  // this'll use ScanWildcardColumnTracker
+      r = region.get(g);  // this'll use ScanWildcardColumnTracker
       checkResult(r, c0, T5,T4);
 
       g = new Get(T1);
       g.setMaxVersions();
       g.addColumn(c0, c0);
-      r = region.get(g, null);  // this'll use ExplicitColumnTracker
+      r = region.get(g);  // this'll use ExplicitColumnTracker
       checkResult(r, c0, T5,T4);
     } finally {
       HRegion.closeHRegion(region);
@@ -308,30 +308,30 @@ public class TestMinVersions extends HBa
       p.add(c0, c0, T4);
       region.put(p);
 
-      Result r = region.get(new Get(T1), null);
+      Result r = region.get(new Get(T1));
       checkResult(r, c0, T4);
 
       Get g = new Get(T1);
       g.setTimeRange(0L, ts+1);
-      r = region.get(g, null);
+      r = region.get(g);
       checkResult(r, c0, T4);
 
   // oldest version still exists
       g.setTimeRange(0L, ts-2);
-      r = region.get(g, null);
+      r = region.get(g);
       checkResult(r, c0, T1);
 
       // gets see only available versions
       // even before compactions
       g = new Get(T1);
       g.setMaxVersions();
-      r = region.get(g, null); // this'll use ScanWildcardColumnTracker
+      r = region.get(g); // this'll use ScanWildcardColumnTracker
       checkResult(r, c0, T4,T3);
 
       g = new Get(T1);
       g.setMaxVersions();
       g.addColumn(c0, c0);
-      r = region.get(g, null);  // this'll use ExplicitColumnTracker
+      r = region.get(g);  // this'll use ExplicitColumnTracker
       checkResult(r, c0, T4,T3);
 
       // now flush
@@ -340,7 +340,7 @@ public class TestMinVersions extends HBa
       // with HBASE-4241 a flush will eliminate the expired rows
       g = new Get(T1);
       g.setTimeRange(0L, ts-2);
-      r = region.get(g, null);
+      r = region.get(g);
       assertTrue(r.isEmpty());
 
       // major compaction
@@ -349,17 +349,17 @@ public class TestMinVersions extends HBa
       // after compaction the 4th version is still available
       g = new Get(T1);
       g.setTimeRange(0L, ts+1);
-      r = region.get(g, null);
+      r = region.get(g);
       checkResult(r, c0, T4);
 
       // so is the 3rd
       g.setTimeRange(0L, ts);
-      r = region.get(g, null);
+      r = region.get(g);
       checkResult(r, c0, T3);
 
       // but the 2nd and earlier versions are gone
       g.setTimeRange(0L, ts-1);
-      r = region.get(g, null);
+      r = region.get(g);
       assertTrue(r.isEmpty());
     } finally {
       HRegion.closeHRegion(region);
@@ -407,14 +407,14 @@ public class TestMinVersions extends HBa
       g.addColumn(c1,c1);
       g.setFilter(new TimestampsFilter(tss));
       g.setMaxVersions();
-      Result r = region.get(g, null);
+      Result r = region.get(g);
       checkResult(r, c1, T2,T1);
 
       g = new Get(T1);
       g.addColumn(c0,c0);
       g.setFilter(new TimestampsFilter(tss));
       g.setMaxVersions();
-      r = region.get(g, null);
+      r = region.get(g);
       checkResult(r, c0, T2,T1);
 
       // now flush/compact
@@ -425,14 +425,14 @@ public class TestMinVersions extends HBa
       g.addColumn(c1,c1);
       g.setFilter(new TimestampsFilter(tss));
       g.setMaxVersions();
-      r = region.get(g, null);
+      r = region.get(g);
       checkResult(r, c1, T2);
 
       g = new Get(T1);
       g.addColumn(c0,c0);
       g.setFilter(new TimestampsFilter(tss));
       g.setMaxVersions();
-      r = region.get(g, null);
+      r = region.get(g);
       checkResult(r, c0, T2);
     } finally {
       HRegion.closeHRegion(region);

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java Wed Feb 13 20:58:23 2013
@@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.client.Sc
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Before;
+import org.apache.hbase.cell.CellComparator;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -107,7 +107,7 @@ public class TestMultiColumnScanner {
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private final Compression.Algorithm comprAlgo;
-  private final StoreFile.BloomType bloomType;
+  private final BloomType bloomType;
   private final DataBlockEncoding dataBlockEncoding;
 
   // Some static sanity-checking.
@@ -133,7 +133,7 @@ public class TestMultiColumnScanner {
   }
 
   public TestMultiColumnScanner(Compression.Algorithm comprAlgo,
-      StoreFile.BloomType bloomType, boolean useDataBlockEncoding) {
+      BloomType bloomType, boolean useDataBlockEncoding) {
     this.comprAlgo = comprAlgo;
     this.bloomType = bloomType;
     this.dataBlockEncoding = useDataBlockEncoding ? DataBlockEncoding.PREFIX :
@@ -214,7 +214,7 @@ public class TestMultiColumnScanner {
               deletedSomething = true;
             }
           if (deletedSomething)
-            region.delete(d, null, true);
+            region.delete(d, true);
         }
       }
       region.flushcache();
@@ -262,8 +262,8 @@ public class TestMultiColumnScanner {
             }
             assertTrue("Scanner returned additional key/value: " + kv + ", "
                 + queryInfo + deleteInfo + ";", kvPos < kvs.size());
-            assertEquals("Scanner returned wrong key/value; " + queryInfo
-                + deleteInfo + ";", kvs.get(kvPos), kv);
+            assertTrue("Scanner returned wrong key/value; " + queryInfo
+                + deleteInfo + ";", CellComparator.equalsIgnoreMvccVersion(kvs.get(kvPos), (kv)));
             ++kvPos;
             ++numResults;
           }

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java Wed Feb 13 20:58:23 2013
@@ -167,7 +167,7 @@ public class TestParallelPut extends HBa
     // run a get and see if the value matches
     Get get = new Get(row);
     get.addColumn(familiy, qualifier);
-    Result result = region.get(get, null);
+    Result result = region.get(get);
     assertEquals(1, result.size());
 
     KeyValue kv = result.raw()[0];

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java Wed Feb 13 20:58:23 2013
@@ -85,7 +85,7 @@ public class TestPriorityRpc {
     HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class);
     Mockito.when(mockRS.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
     Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
-    Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(true);
+    Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(true);
     qosFunction.setRegionServer(mockRS);
     assertTrue (qosFunction.apply(rpcRequest) == HConstants.HIGH_QOS);
   }
@@ -132,14 +132,14 @@ public class TestPriorityRpc {
     Mockito.when(mockRegionScanner.getRegionInfo()).thenReturn(mockRegionInfo);
     Mockito.when(mockRS.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
     Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
-    Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(true);
+    Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(true);
 
     qosFunction.setRegionServer(mockRS);
 
     assertTrue (qosFunction.apply(rpcRequest) == HConstants.HIGH_QOS);
 
     //the same as above but with non-meta region
-    Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(false);
+    Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(false);
     assertTrue (qosFunction.apply(rpcRequest) == HConstants.NORMAL_QOS);
   }
 

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java Wed Feb 13 20:58:23 2013
@@ -149,6 +149,8 @@ public class TestRegionServerMetrics {
     metricsHelper.assertCounterGt("totalRequestCount", requests + 39, serverSource);
     metricsHelper.assertCounterGt("readRequestCount", readRequests + 9, serverSource);
     metricsHelper.assertCounterGt("writeRequestCount", writeRequests + 29, serverSource);
+
+    table.close();
   }
 
   @Test
@@ -176,6 +178,8 @@ public class TestRegionServerMetrics {
     metricsHelper.assertGauge("putsWithoutWALCount", 1, serverSource);
     long minLength = row.length + cf.length + qualifier.length + val.length;
     metricsHelper.assertGaugeGt("putsWithoutWALSize", minLength, serverSource);
+
+    t.close();
   }
 
   @Test
@@ -203,6 +207,8 @@ public class TestRegionServerMetrics {
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
     metricsHelper.assertGauge("storeCount", stores +1, serverSource);
     metricsHelper.assertGauge("storeFileCount", storeFiles + 1, serverSource);
+
+    t.close();
   }
 
   @Test
@@ -237,6 +243,8 @@ public class TestRegionServerMetrics {
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
     metricsHelper.assertCounter("checkMutateFailedCount", 1, serverSource);
     metricsHelper.assertCounter("checkMutatePassedCount", 1, serverSource);
+
+    t.close();
   }
 
   @Test
@@ -267,6 +275,8 @@ public class TestRegionServerMetrics {
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
     metricsHelper.assertCounter("incrementNumOps", 13, serverSource);
+
+    t.close();
   }
 
   @Test
@@ -297,5 +307,7 @@ public class TestRegionServerMetrics {
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
     metricsHelper.assertCounter("appendNumOps", 73, serverSource);
+
+    t.close();
   }
 }

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java Wed Feb 13 20:58:23 2013
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java Wed Feb 13 20:58:23 2013
@@ -78,14 +78,14 @@ public class TestResettingCounters {
       }
 
       // increment odd qualifiers 5 times and flush
-      for (int i=0;i<5;i++) region.increment(odd, null, false);
+      for (int i=0;i<5;i++) region.increment(odd, false);
       region.flushcache();
 
       // increment even qualifiers 5 times
-      for (int i=0;i<5;i++) region.increment(even, null, false);
+      for (int i=0;i<5;i++) region.increment(even, false);
 
       // increment all qualifiers, should have value=6 for all
-      Result result = region.increment(all, null, false);
+      Result result = region.increment(all, false);
       assertEquals(numQualifiers, result.size());
       KeyValue [] kvs = result.raw();
       for (int i=0;i<kvs.length;i++) {

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java Wed Feb 13 20:58:23 2013
@@ -41,8 +41,6 @@ import org.apache.hadoop.hbase.SmallTest
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -74,7 +72,7 @@ public class TestScanWithBloomError {
   private static final byte[] ROW_BYTES = Bytes.toBytes(ROW);
   private static NavigableSet<Integer> allColIds = new TreeSet<Integer>();
   private HRegion region;
-  private StoreFile.BloomType bloomType;
+  private BloomType bloomType;
   private FileSystem fs;
   private Configuration conf;
 
@@ -84,13 +82,13 @@ public class TestScanWithBloomError {
   @Parameters
   public static final Collection<Object[]> parameters() {
     List<Object[]> configurations = new ArrayList<Object[]>();
-    for (StoreFile.BloomType bloomType : StoreFile.BloomType.values()) {
+    for (BloomType bloomType : BloomType.values()) {
       configurations.add(new Object[] { bloomType });
     }
     return configurations;
   }
 
-  public TestScanWithBloomError(StoreFile.BloomType bloomType) {
+  public TestScanWithBloomError(BloomType bloomType) {
     this.bloomType = bloomType;
   }
 

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanner.java Wed Feb 13 20:58:23 2013
@@ -234,7 +234,7 @@ public class TestScanner extends HBaseTe
 
       // Write information to the meta table
 
-      Put put = new Put(ROW_KEY, System.currentTimeMillis(), null);
+      Put put = new Put(ROW_KEY, System.currentTimeMillis());
 
       put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
           REGION_INFO.toByteArray());
@@ -261,7 +261,7 @@ public class TestScanner extends HBaseTe
 
       String address = HConstants.LOCALHOST_IP + ":" + HBaseTestingUtility.randomFreePort();
 
-      put = new Put(ROW_KEY, System.currentTimeMillis(), null);
+      put = new Put(ROW_KEY, System.currentTimeMillis());
       put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
           Bytes.toBytes(address));
 
@@ -299,7 +299,7 @@ public class TestScanner extends HBaseTe
 
       address = "bar.foo.com:4321";
 
-      put = new Put(ROW_KEY, System.currentTimeMillis(), null);
+      put = new Put(ROW_KEY, System.currentTimeMillis());
 
       put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
           Bytes.toBytes(address));
@@ -428,7 +428,7 @@ public class TestScanner extends HBaseTe
   private void getRegionInfo() throws IOException {
     Get get = new Get(ROW_KEY);
     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-    Result result = region.get(get, null);
+    Result result = region.get(get);
     byte [] bytes = result.value();
     validateRegionInfo(bytes);
   }
@@ -498,7 +498,7 @@ public class TestScanner extends HBaseTe
       Delete dc = new Delete(firstRowBytes);
       /* delete column1 of firstRow */
       dc.deleteColumns(fam1, col1);
-      r.delete(dc, null, true);
+      r.delete(dc, true);
       r.flushcache();
 
       addContent(hri, Bytes.toString(fam1), Bytes.toString(col1),

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java Wed Feb 13 20:58:23 2013
@@ -44,8 +44,6 @@ import org.apache.hadoop.hbase.client.De
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
 import org.junit.Before;
@@ -115,7 +113,7 @@ public class TestSeekOptimizations {
   private List<KeyValue> expectedKVs = new ArrayList<KeyValue>();
 
   private Compression.Algorithm comprAlgo;
-  private StoreFile.BloomType bloomType;
+  private BloomType bloomType;
 
   private long totalSeekDiligent, totalSeekLazy;
   
@@ -128,7 +126,7 @@ public class TestSeekOptimizations {
   }
 
   public TestSeekOptimizations(Compression.Algorithm comprAlgo,
-      StoreFile.BloomType bloomType) {
+      BloomType bloomType) {
     this.comprAlgo = comprAlgo;
     this.bloomType = bloomType;
   }
@@ -421,7 +419,7 @@ public class TestSeekOptimizations {
 
         region.put(put);
         if (!del.isEmpty()) {
-          region.delete(del, null, true);
+          region.delete(del, true);
         }
 
         // Add remaining timestamps (those we have not deleted) to expected

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java Wed Feb 13 20:58:23 2013
@@ -18,96 +18,116 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.client.Get;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
-import org.apache.hadoop.hbase.ipc.ProtocolSignature;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.CountRequest;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.CountResponse;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.HelloRequest;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.HelloResponse;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.IncrementCountRequest;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.IncrementCountResponse;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.NoopRequest;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.NoopResponse;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.PingRequest;
+import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.PingResponse;
+import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.ipc.VersionedProtocol;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
-import com.google.common.collect.Lists;
 import org.junit.experimental.categories.Category;
 
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
 @Category(MediumTests.class)
 public class TestServerCustomProtocol {
-  /* Test protocol */
-  public static interface PingProtocol extends CoprocessorProtocol {
-    public String ping();
-    public int getPingCount();
-    public int incrementCount(int diff);
-    public String hello(String name);
-    public void noop();
-  }
+  private static final Log LOG = LogFactory.getLog(TestServerCustomProtocol.class);
+  static final String WHOAREYOU = "Who are you?";
+  static final String NOBODY = "nobody";
+  static final String HELLO = "Hello, ";
 
   /* Test protocol implementation */
-  public static class PingHandler implements Coprocessor, PingProtocol, VersionedProtocol {
-    static long VERSION = 1;
+  public static class PingHandler extends PingProtos.PingService
+  implements Coprocessor, CoprocessorService {
     private int counter = 0;
-    @Override
-    public String ping() {
-      counter++;
-      return "pong";
-    }
 
     @Override
-    public int getPingCount() {
-      return counter;
+    public void start(CoprocessorEnvironment env) throws IOException {
+      if (env instanceof RegionCoprocessorEnvironment) return;
+      throw new CoprocessorException("Must be loaded on a table region!");
     }
 
     @Override
-    public int incrementCount(int diff) {
-      counter += diff;
-      return counter;
+    public void stop(CoprocessorEnvironment env) throws IOException {
+      // Nothing to do.
     }
 
     @Override
-    public String hello(String name) {
-      if (name == null) {
-        return "Who are you?";
-      } else if ("nobody".equals(name)) {
-        return null;
-      }
-      return "Hello, "+name;
+    public void ping(RpcController controller, PingRequest request,
+        RpcCallback<PingResponse> done) {
+      this.counter++;
+      done.run(PingResponse.newBuilder().setPong("pong").build());
     }
 
     @Override
-    public void noop() {
-      // do nothing, just test void return type
+    public void count(RpcController controller, CountRequest request,
+        RpcCallback<CountResponse> done) {
+      done.run(CountResponse.newBuilder().setCount(this.counter).build());
     }
 
     @Override
-    public ProtocolSignature getProtocolSignature(
-        String protocol, long version, int clientMethodsHashCode)
-    throws IOException {
-      return new ProtocolSignature(VERSION, null);
+    public void increment(RpcController controller,
+        IncrementCountRequest request, RpcCallback<IncrementCountResponse> done) {
+      this.counter += request.getDiff();
+      done.run(IncrementCountResponse.newBuilder().setCount(this.counter).build());
     }
 
     @Override
-    public long getProtocolVersion(String s, long l) throws IOException {
-      return VERSION;
+    public void hello(RpcController controller, HelloRequest request,
+        RpcCallback<HelloResponse> done) {
+      if (!request.hasName()) done.run(HelloResponse.newBuilder().setResponse(WHOAREYOU).build());
+      else if (request.getName().equals(NOBODY)) done.run(HelloResponse.newBuilder().build());
+      else done.run(HelloResponse.newBuilder().setResponse(HELLO + request.getName()).build());
     }
 
     @Override
-    public void start(CoprocessorEnvironment env) throws IOException {
+    public void noop(RpcController controller, NoopRequest request,
+        RpcCallback<NoopResponse> done) {
+      done.run(NoopResponse.newBuilder().build());
     }
 
     @Override
-    public void stop(CoprocessorEnvironment env) throws IOException {
+    public Service getService() {
+      return this;
     }
   }
 
@@ -122,19 +142,19 @@ public class TestServerCustomProtocol {
   private static final byte[] ROW_BC = Bytes.toBytes("bcc");
 
   private static HBaseTestingUtility util = new HBaseTestingUtility();
-  private static MiniHBaseCluster cluster = null;
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
-        PingHandler.class.getName());
-    util.startMiniCluster(1);
-    cluster = util.getMiniHBaseCluster();
+      PingHandler.class.getName());
+    util.startMiniCluster();
+  }
 
+  @Before
+  public void before()  throws Exception {
     HTable table = util.createTable(TEST_TABLE, TEST_FAMILY);
     util.createMultiRegions(util.getConfiguration(), table, TEST_FAMILY,
-        new byte[][]{ HConstants.EMPTY_BYTE_ARRAY,
-            ROW_B, ROW_C});
+      new byte[][]{ HConstants.EMPTY_BYTE_ARRAY, ROW_B, ROW_C});
 
     Put puta = new Put( ROW_A );
     puta.add(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
@@ -149,162 +169,260 @@ public class TestServerCustomProtocol {
     table.put(putc);
   }
 
+  @After
+  public void after() throws Exception {
+    util.deleteTable(TEST_TABLE);
+  }
+
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     util.shutdownMiniCluster();
   }
 
   @Test
-  public void testSingleProxy() throws Exception {
+  public void testSingleProxy() throws Throwable {
     HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
-
-    PingProtocol pinger = table.coprocessorProxy(PingProtocol.class, ROW_A);
-    String result = pinger.ping();
-    assertEquals("Invalid custom protocol response", "pong", result);
-    result = pinger.hello("George");
-    assertEquals("Invalid custom protocol response", "Hello, George", result);
-    result = pinger.hello(null);
-    assertEquals("Should handle NULL parameter", "Who are you?", result);
-    result = pinger.hello("nobody");
-    assertNull(result);
-    int cnt = pinger.getPingCount();
-    assertTrue("Count should be incremented", cnt > 0);
-    int newcnt = pinger.incrementCount(5);
-    assertEquals("Counter should have incremented by 5", cnt+5, newcnt);
+    Map<byte [], String> results = ping(table, null, null);
+    // There are three regions so should get back three results.
+    assertEquals(3, results.size());
+    for (Map.Entry<byte [], String> e: results.entrySet()) {
+      assertEquals("Invalid custom protocol response", "pong", e.getValue());
+    }
+    hello(table, "George", HELLO + "George");
+    LOG.info("Did george");
+    hello(table, null, "Who are you?");
+    LOG.info("Who are you");
+    hello(table, NOBODY, null);
+    LOG.info(NOBODY);
+    Map<byte [], Integer> intResults = table.coprocessorService(PingProtos.PingService.class,
+      null, null,
+      new Batch.Call<PingProtos.PingService, Integer>() {
+        @Override
+        public Integer call(PingProtos.PingService instance) throws IOException {
+          BlockingRpcCallback<PingProtos.CountResponse> rpcCallback =
+            new BlockingRpcCallback<PingProtos.CountResponse>();
+          instance.count(null, PingProtos.CountRequest.newBuilder().build(), rpcCallback);
+          return rpcCallback.get().getCount();
+        }
+      });
+    int count = -1;
+    for (Map.Entry<byte [], Integer> e: intResults.entrySet()) {
+      assertTrue(e.getValue() > 0);
+      count = e.getValue();
+    }
+    final int diff = 5;
+    intResults = table.coprocessorService(PingProtos.PingService.class,
+      null, null,
+      new Batch.Call<PingProtos.PingService, Integer>() {
+        @Override
+        public Integer call(PingProtos.PingService instance) throws IOException {
+          BlockingRpcCallback<PingProtos.IncrementCountResponse> rpcCallback =
+            new BlockingRpcCallback<PingProtos.IncrementCountResponse>();
+          instance.increment(null, PingProtos.IncrementCountRequest.newBuilder().setDiff(diff).build(),
+            rpcCallback);
+          return rpcCallback.get().getCount();
+        }
+      });
+    // There are three regions so should get back three results.
+    assertEquals(3, results.size());
+    for (Map.Entry<byte [], Integer> e: intResults.entrySet()) {
+      assertEquals(e.getValue().intValue(), count + diff);
+    }
+    table.close();
   }
 
-  @Test
-  public void testSingleMethod() throws Throwable {
-    HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
+  private Map<byte [], String> hello(final HTable table, final String send, final String response)
+  throws ServiceException, Throwable {
+    Map<byte [], String> results = hello(table, send);
+    for (Map.Entry<byte [], String> e: results.entrySet()) {
+      assertEquals("Invalid custom protocol response", response, e.getValue());
+    }
+    return results;
+  }
 
-    List<? extends Row> rows = Lists.newArrayList(
-        new Get(ROW_A), new Get(ROW_B), new Get(ROW_C));
+  private Map<byte [], String> hello(final HTable table, final String send)
+  throws ServiceException, Throwable {
+    return hello(table, send, null, null);
+  }
+
+  private Map<byte [], String> hello(final HTable table, final String send, final byte [] start,
+      final byte [] end)
+  throws ServiceException, Throwable {
+    return table.coprocessorService(PingProtos.PingService.class,
+        start, end,
+        new Batch.Call<PingProtos.PingService, String>() {
+          @Override
+          public String call(PingProtos.PingService instance) throws IOException {
+            BlockingRpcCallback<PingProtos.HelloResponse> rpcCallback =
+              new BlockingRpcCallback<PingProtos.HelloResponse>();
+            PingProtos.HelloRequest.Builder builder = PingProtos.HelloRequest.newBuilder();
+            if (send != null) builder.setName(send);
+            instance.hello(null, builder.build(), rpcCallback);
+            PingProtos.HelloResponse r = rpcCallback.get();
+            return r != null && r.hasResponse()? r.getResponse(): null;
+          }
+        });
+  }
 
-    Batch.Call<PingProtocol,String> call =  Batch.forMethod(PingProtocol.class,
-        "ping");
-    Map<byte[],String> results =
-        table.coprocessorExec(PingProtocol.class, ROW_A, ROW_C, call);
+  private Map<byte [], String> compoundOfHelloAndPing(final HTable table, final byte [] start,
+      final byte [] end)
+  throws ServiceException, Throwable {
+    return table.coprocessorService(PingProtos.PingService.class,
+        start, end,
+        new Batch.Call<PingProtos.PingService, String>() {
+          @Override
+          public String call(PingProtos.PingService instance) throws IOException {
+            BlockingRpcCallback<PingProtos.HelloResponse> rpcCallback =
+              new BlockingRpcCallback<PingProtos.HelloResponse>();
+            PingProtos.HelloRequest.Builder builder = PingProtos.HelloRequest.newBuilder();
+            // Call ping on same instance.  Use result calling hello on same instance.
+            builder.setName(doPing(instance));
+            instance.hello(null, builder.build(), rpcCallback);
+            PingProtos.HelloResponse r = rpcCallback.get();
+            return r != null && r.hasResponse()? r.getResponse(): null;
+          }
+        });
+  }
 
+  private Map<byte [], String> noop(final HTable table, final byte [] start,
+      final byte [] end)
+  throws ServiceException, Throwable {
+    return table.coprocessorService(PingProtos.PingService.class, start, end,
+        new Batch.Call<PingProtos.PingService, String>() {
+          @Override
+          public String call(PingProtos.PingService instance) throws IOException {
+            BlockingRpcCallback<PingProtos.NoopResponse> rpcCallback =
+              new BlockingRpcCallback<PingProtos.NoopResponse>();
+            PingProtos.NoopRequest.Builder builder = PingProtos.NoopRequest.newBuilder();
+            instance.noop(null, builder.build(), rpcCallback);
+            rpcCallback.get();
+            // Looks like null is expected when void.  That is what the test below is looking for
+            return null;
+          }
+        });
+  }
 
+  @Test
+  public void testSingleMethod() throws Throwable {
+    HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
+    Map<byte [], String> results = table.coprocessorService(PingProtos.PingService.class,
+      null, ROW_A,
+      new Batch.Call<PingProtos.PingService, String>() {
+        @Override
+        public String call(PingProtos.PingService instance) throws IOException {
+          BlockingRpcCallback<PingProtos.PingResponse> rpcCallback =
+            new BlockingRpcCallback<PingProtos.PingResponse>();
+          instance.ping(null, PingProtos.PingRequest.newBuilder().build(), rpcCallback);
+          return rpcCallback.get().getPong();
+        }
+      });
+    // Should have gotten results for 1 of the three regions only since we specified
+    // rows from 1 region
+    assertEquals(1, results.size());
     verifyRegionResults(table, results, ROW_A);
-    verifyRegionResults(table, results, ROW_B);
-    verifyRegionResults(table, results, ROW_C);
-
-    Batch.Call<PingProtocol,String> helloCall =
-      Batch.forMethod(PingProtocol.class, "hello", "NAME");
-    results =
-        table.coprocessorExec(PingProtocol.class, ROW_A, ROW_C, helloCall);
-
 
+    final String name = "NAME";
+    results = hello(table, name, null, ROW_A);
+    // Should have gotten results for 1 of the three regions only since we specified
+    // rows from 1 region
+    assertEquals(1, results.size());
     verifyRegionResults(table, results, "Hello, NAME", ROW_A);
-    verifyRegionResults(table, results, "Hello, NAME", ROW_B);
-    verifyRegionResults(table, results, "Hello, NAME", ROW_C);
+    table.close();
   }
 
   @Test
   public void testRowRange() throws Throwable {
     HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
-
-    // test empty range
-    Map<byte[],String> results = table.coprocessorExec(PingProtocol.class,
-        null, null, new Batch.Call<PingProtocol,String>() {
-          public String call(PingProtocol instance) {
-            return instance.ping();
-          }
-        });
-    // should contain all three rows/regions
+    for (Entry<HRegionInfo, ServerName> e: table.getRegionLocations().entrySet()) {
+      LOG.info("Region " + e.getKey().getRegionNameAsString() + ", servername=" + e.getValue());
+    }
+    // Here are what regions looked like on a run:
+    //
+    // test,,1355943549657.c65d4822d8bdecc033a96451f3a0f55d.
+    // test,bbb,1355943549661.110393b070dd1ed93441e0bc9b3ffb7e.
+    // test,ccc,1355943549665.c3d6d125141359cbbd2a43eaff3cdf74.
+
+    Map<byte [], String> results = ping(table, null, ROW_A);
+    // Should contain first region only.
+    assertEquals(1, results.size());
     verifyRegionResults(table, results, ROW_A);
-    verifyRegionResults(table, results, ROW_B);
-    verifyRegionResults(table, results, ROW_C);
 
-    // test start row + empty end
-    results = table.coprocessorExec(PingProtocol.class, ROW_BC, null,
-        new Batch.Call<PingProtocol,String>() {
-          public String call(PingProtocol instance) {
-            return instance.ping();
-          }
-        });
+    // Test start row + empty end
+    results = ping(table, ROW_BC, null);
+    assertEquals(2, results.size());
     // should contain last 2 regions
-    HRegionLocation loc = table.getRegionLocation(ROW_A);
+    HRegionLocation loc = table.getRegionLocation(ROW_A, true);
     assertNull("Should be missing region for row aaa (prior to start row)",
-        results.get(loc.getRegionInfo().getRegionName()));
+      results.get(loc.getRegionInfo().getRegionName()));
     verifyRegionResults(table, results, ROW_B);
     verifyRegionResults(table, results, ROW_C);
 
     // test empty start + end
-    results = table.coprocessorExec(PingProtocol.class, null, ROW_BC,
-        new Batch.Call<PingProtocol,String>() {
-          public String call(PingProtocol instance) {
-            return instance.ping();
-          }
-        });
+    results = ping(table, null, ROW_BC);
     // should contain the first 2 regions
+    assertEquals(2, results.size());
     verifyRegionResults(table, results, ROW_A);
     verifyRegionResults(table, results, ROW_B);
-    loc = table.getRegionLocation(ROW_C);
+    loc = table.getRegionLocation(ROW_C, true);
     assertNull("Should be missing region for row ccc (past stop row)",
         results.get(loc.getRegionInfo().getRegionName()));
 
     // test explicit start + end
-    results = table.coprocessorExec(PingProtocol.class, ROW_AB, ROW_BC,
-        new Batch.Call<PingProtocol,String>() {
-          public String call(PingProtocol instance) {
-            return instance.ping();
-          }
-        });
+    results = ping(table, ROW_AB, ROW_BC);
     // should contain first 2 regions
+    assertEquals(2, results.size());
     verifyRegionResults(table, results, ROW_A);
     verifyRegionResults(table, results, ROW_B);
-    loc = table.getRegionLocation(ROW_C);
+    loc = table.getRegionLocation(ROW_C, true);
     assertNull("Should be missing region for row ccc (past stop row)",
         results.get(loc.getRegionInfo().getRegionName()));
 
     // test single region
-    results = table.coprocessorExec(PingProtocol.class, ROW_B, ROW_BC,
-        new Batch.Call<PingProtocol,String>() {
-          public String call(PingProtocol instance) {
-            return instance.ping();
-          }
-        });
+    results = ping(table, ROW_B, ROW_BC);
     // should only contain region bbb
+    assertEquals(1, results.size());
     verifyRegionResults(table, results, ROW_B);
-    loc = table.getRegionLocation(ROW_A);
+    loc = table.getRegionLocation(ROW_A, true);
     assertNull("Should be missing region for row aaa (prior to start)",
         results.get(loc.getRegionInfo().getRegionName()));
-    loc = table.getRegionLocation(ROW_C);
+    loc = table.getRegionLocation(ROW_C, true);
     assertNull("Should be missing region for row ccc (past stop row)",
         results.get(loc.getRegionInfo().getRegionName()));
+    table.close();
+  }
+
+  private Map<byte [], String> ping(final HTable table, final byte [] start, final byte [] end)
+  throws ServiceException, Throwable {
+    return table.coprocessorService(PingProtos.PingService.class, start, end,
+      new Batch.Call<PingProtos.PingService, String>() {
+        @Override
+        public String call(PingProtos.PingService instance) throws IOException {
+          return doPing(instance);
+        }
+      });
+  }
+
+  private static String doPing(PingProtos.PingService instance) throws IOException {
+    BlockingRpcCallback<PingProtos.PingResponse> rpcCallback =
+        new BlockingRpcCallback<PingProtos.PingResponse>();
+      instance.ping(null, PingProtos.PingRequest.newBuilder().build(), rpcCallback);
+      return rpcCallback.get().getPong();
   }
 
   @Test
-  public void testCompountCall() throws Throwable {
+  public void testCompoundCall() throws Throwable {
     HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
-
-    Map<byte[],String> results = table.coprocessorExec(PingProtocol.class,
-        ROW_A, ROW_C,
-        new Batch.Call<PingProtocol,String>() {
-          public String call(PingProtocol instance) {
-            return instance.hello(instance.ping());
-          }
-        });
-
+    Map<byte [], String> results = compoundOfHelloAndPing(table, ROW_A, ROW_C);
     verifyRegionResults(table, results, "Hello, pong", ROW_A);
     verifyRegionResults(table, results, "Hello, pong", ROW_B);
     verifyRegionResults(table, results, "Hello, pong", ROW_C);
+    table.close();
   }
 
   @Test
   public void testNullCall() throws Throwable {
     HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
-
-    Map<byte[],String> results = table.coprocessorExec(PingProtocol.class,
-        ROW_A, ROW_C,
-        new Batch.Call<PingProtocol,String>() {
-          public String call(PingProtocol instance) {
-            return instance.hello(null);
-          }
-        });
-
+    Map<byte[],String> results = hello(table, null, ROW_A, ROW_C);
     verifyRegionResults(table, results, "Who are you?", ROW_A);
     verifyRegionResults(table, results, "Who are you?", ROW_B);
     verifyRegionResults(table, results, "Who are you?", ROW_C);
@@ -313,33 +431,16 @@ public class TestServerCustomProtocol {
   @Test
   public void testNullReturn() throws Throwable {
     HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
-
-    Map<byte[],String> results = table.coprocessorExec(PingProtocol.class,
-        ROW_A, ROW_C,
-        new Batch.Call<PingProtocol,String>(){
-          public String call(PingProtocol instance) {
-            return instance.hello("nobody");
-          }
-        });
-
+    Map<byte[],String> results = hello(table, "nobody", ROW_A, ROW_C);
     verifyRegionResults(table, results, null, ROW_A);
     verifyRegionResults(table, results, null, ROW_B);
     verifyRegionResults(table, results, null, ROW_C);
   }
 
   @Test
-  public void testVoidReturnType() throws Throwable {
+  public void testEmptyReturnType() throws Throwable {
     HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
-
-    Map<byte[],Object> results = table.coprocessorExec(PingProtocol.class,
-        ROW_A, ROW_C,
-        new Batch.Call<PingProtocol,Object>(){
-          public Object call(PingProtocol instance) {
-            instance.noop();
-            return null;
-          }
-        });
-
+    Map<byte[],String> results = noop(table, ROW_A, ROW_C);
     assertEquals("Should have results from three regions", 3, results.size());
     // all results should be null
     for (Object v : results.values()) {
@@ -353,16 +454,19 @@ public class TestServerCustomProtocol {
   }
 
   private void verifyRegionResults(HTable table,
-      Map<byte[],String> results, String expected, byte[] row)
+      Map<byte[], String> results, String expected, byte[] row)
   throws Exception {
-    HRegionLocation loc = table.getRegionLocation(row);
+    for (Map.Entry<byte [], String> e: results.entrySet()) {
+      LOG.info("row=" + Bytes.toString(row) + ", expected=" + expected +
+       ", result key=" + Bytes.toString(e.getKey()) +
+       ", value=" + e.getValue());
+    }
+    HRegionLocation loc = table.getRegionLocation(row, true);
     byte[] region = loc.getRegionInfo().getRegionName();
     assertTrue("Results should contain region " +
-        Bytes.toStringBinary(region)+" for row '"+Bytes.toStringBinary(row)+"'",
-        results.containsKey(region));
+      Bytes.toStringBinary(region) + " for row '" + Bytes.toStringBinary(row)+ "'",
+      results.containsKey(region));
     assertEquals("Invalid result for row '"+Bytes.toStringBinary(row)+"'",
-        expected, results.get(region));
+      expected, results.get(region));
   }
-
 }
-

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java Wed Feb 13 20:58:23 2013
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.MediumTes
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.SplitLogCounters;
 import org.apache.hadoop.hbase.SplitLogTask;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -56,29 +57,34 @@ public class TestSplitLogWorker {
   private ZooKeeperWatcher zkw;
   private SplitLogWorker slw;
 
-  private void waitForCounter(AtomicLong ctr, long oldval, long newval,
-      long timems) {
+  private void waitForCounter(AtomicLong ctr, long oldval, long newval, long timems)
+      throws Exception {
     assertTrue("ctr=" + ctr.get() + ", oldval=" + oldval + ", newval=" + newval,
       waitForCounterBoolean(ctr, oldval, newval, timems));
   }
 
-  private boolean waitForCounterBoolean(AtomicLong ctr, long oldval, long newval,
-      long timems) {
-    long curt = System.currentTimeMillis();
-    long endt = curt + timems;
-    while (curt < endt) {
-      if (ctr.get() == oldval) {
-        try {
-          Thread.sleep(10);
-        } catch (InterruptedException e) {
-        }
-        curt = System.currentTimeMillis();
-      } else {
-        assertEquals(newval, ctr.get());
-        return true;
+  private boolean waitForCounterBoolean(final AtomicLong ctr, final long oldval, long newval,
+      long timems) throws Exception {
+
+    return waitForCounterBoolean(ctr, oldval, newval, timems, true);
+  }
+
+  private boolean waitForCounterBoolean(final AtomicLong ctr, final long oldval, long newval,
+      long timems, boolean failIfTimeout) throws Exception {
+
+    long timeWaited = TEST_UTIL.waitFor(timems, 10, failIfTimeout,
+      new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return (ctr.get() != oldval);
       }
+    });
+
+    if( timeWaited > 0) {
+      // when not timed out
+      assertEquals(newval, ctr.get());
     }
-    return false;
+    return true;
   }
 
   @Before
@@ -134,7 +140,7 @@ public class TestSplitLogWorker {
     SplitLogWorker slw = new SplitLogWorker(zkw, TEST_UTIL.getConfiguration(), RS, neverEndingTask);
     slw.start();
     try {
-      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1500);
       byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS));
       SplitLogTask slt = SplitLogTask.parseFrom(bytes);
       assertTrue(slt.isOwned(RS));
@@ -170,10 +176,10 @@ public class TestSplitLogWorker {
     slw1.start();
     slw2.start();
     try {
-      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1500);
       // Assert that either the tot_wkr_failed_to_grab_task_owned count was set of if
       // not it, that we fell through to the next counter in line and it was set.
-      assertTrue(waitForCounterBoolean(SplitLogCounters.tot_wkr_failed_to_grab_task_owned, 0, 1, 1000) ||
+      assertTrue(waitForCounterBoolean(SplitLogCounters.tot_wkr_failed_to_grab_task_owned, 0, 1, 1500, false) ||
           SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.get() == 1);
       byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TRFT));
       SplitLogTask slt = SplitLogTask.parseFrom(bytes);
@@ -201,14 +207,14 @@ public class TestSplitLogWorker {
         new SplitLogTask.Unassigned(MANAGER).toByteArray(), Ids.OPEN_ACL_UNSAFE,
         CreateMode.PERSISTENT);
 
-      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1500);
       assertEquals(1, slw.taskReadySeq);
       byte [] bytes = ZKUtil.getData(zkw, PATH);
       SplitLogTask slt = SplitLogTask.parseFrom(bytes);
       assertTrue(slt.isOwned(SRV));
       slt = new SplitLogTask.Unassigned(MANAGER);
       ZKUtil.setData(zkw, PATH, slt.toByteArray());
-      waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1500);
     } finally {
       stopSplitLogWorker(slw);
     }
@@ -229,7 +235,7 @@ public class TestSplitLogWorker {
       zkw.getRecoverableZooKeeper().create(PATH1, unassignedManager.toByteArray(),
         Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 
-      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1500);
       // now the worker is busy doing the above task
 
       // create another task
@@ -241,9 +247,9 @@ public class TestSplitLogWorker {
       final ServerName anotherWorker = new ServerName("another-worker,1,1");
       SplitLogTask slt = new SplitLogTask.Owned(anotherWorker);
       ZKUtil.setData(zkw, PATH1, slt.toByteArray());
-      waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1500);
 
-      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, 1000);
+      waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, 1500);
       assertEquals(2, slw.taskReadySeq);
       byte [] bytes = ZKUtil.getData(zkw, PATH2);
       slt = SplitLogTask.parseFrom(bytes);
@@ -268,25 +274,25 @@ public class TestSplitLogWorker {
     zkw.getRecoverableZooKeeper().create(task,slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
       CreateMode.PERSISTENT);
 
-    waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1000);
+    waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, 1500);
     // now the worker is busy doing the above task
 
     // preempt the task, have it owned by another worker
     ZKUtil.setData(zkw, task, slt.toByteArray());
-    waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1000);
+    waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, 1500);
 
     // create a RESCAN node
     String rescan = ZKSplitLog.getEncodedNodeName(zkw, "RESCAN");
     rescan = zkw.getRecoverableZooKeeper().create(rescan, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
       CreateMode.PERSISTENT_SEQUENTIAL);
 
-    waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, 1000);
+    waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, 1500);
     // RESCAN node might not have been processed if the worker became busy
     // with the above task. preempt the task again so that now the RESCAN
     // node is processed
     ZKUtil.setData(zkw, task, slt.toByteArray());
-    waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 1, 2, 1000);
-    waitForCounter(SplitLogCounters.tot_wkr_task_acquired_rescan, 0, 1, 1000);
+    waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 1, 2, 1500);
+    waitForCounter(SplitLogCounters.tot_wkr_task_acquired_rescan, 0, 1, 1500);
 
     List<String> nodes = ZKUtil.listChildrenNoWatch(zkw, zkw.splitLogZNode);
     LOG.debug(nodes);

Modified: hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java?rev=1445918&r1=1445917&r2=1445918&view=diff
==============================================================================
--- hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java (original)
+++ hbase/branches/hbase-7290/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java Wed Feb 13 20:58:23 2013
@@ -202,21 +202,6 @@ public class TestSplitTransaction {
     assertFalse(st.prepare());
   }
 
-  @Test public void testWholesomeSplitWithHFileV1() throws IOException {
-    int defaultVersion = TEST_UTIL.getConfiguration().getInt(
-        HFile.FORMAT_VERSION_KEY, 2);
-    TEST_UTIL.getConfiguration().setInt(HFile.FORMAT_VERSION_KEY, 1);
-    try {
-      for (Store store : this.parent.stores.values()) {
-        store.getFamily().setBloomFilterType(StoreFile.BloomType.ROW);
-      }
-      testWholesomeSplit();
-    } finally {
-      TEST_UTIL.getConfiguration().setInt(HFile.FORMAT_VERSION_KEY,
-          defaultVersion);
-    }
-  }
-
   @Test public void testWholesomeSplit() throws IOException {
     final int rowcount = TEST_UTIL.loadRegion(this.parent, CF, true);
     assertTrue(rowcount > 0);