You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 03:59:52 UTC

svn commit: r1181336 [3/3] - in /hbase/branches/0.89: ./ bin/ conf/ src/main/java/org/apache/hadoop/hbase/io/ src/main/java/org/apache/hadoop/hbase/master/ src/main/java/org/apache/hadoop/hbase/regionserver/ src/main/java/org/apache/hadoop/hbase/region...

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java?rev=1181336&r1=1181335&r2=1181336&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java Tue Oct 11 01:59:50 2011
@@ -25,12 +25,11 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hbase.HBaseTestCase;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueTestUtil;
 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
 import org.apache.hadoop.hbase.util.Bytes;
 
 
@@ -53,6 +52,7 @@ public class TestQueryMatcher extends HB
 
   long ttl = Long.MAX_VALUE;
   KeyComparator rowComparator;
+  private Scan scan;
 
   public void setUp() throws Exception {
     super.setUp();
@@ -74,6 +74,7 @@ public class TestQueryMatcher extends HB
     get.addColumn(fam2, col2);
     get.addColumn(fam2, col4);
     get.addColumn(fam2, col5);
+    this.scan = new Scan(get);
 
     rowComparator = KeyValue.KEY_COMPARATOR;
 
@@ -85,29 +86,31 @@ public class TestQueryMatcher extends HB
     //of just byte []
 
     //Expected result
-    List<MatchCode> expected = new ArrayList<MatchCode>();
-    expected.add(MatchCode.SKIP);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.SKIP);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.DONE);
+    List<MatchCode> expected = new ArrayList<ScanQueryMatcher.MatchCode>();
+    expected.add(ScanQueryMatcher.MatchCode.SEEK_NEXT_COL);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.SEEK_NEXT_COL);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.DONE);
 
-    QueryMatcher qm = new QueryMatcher(get, fam2,
+    // 2,4,5
+    ScanQueryMatcher qm = new ScanQueryMatcher(scan, fam2,
         get.getFamilyMap().get(fam2), ttl, rowComparator, 1);
 
     List<KeyValue> memstore = new ArrayList<KeyValue>();
-    memstore.add(new KeyValue(row1, fam2, col1, data));
-    memstore.add(new KeyValue(row1, fam2, col2, data));
-    memstore.add(new KeyValue(row1, fam2, col3, data));
-    memstore.add(new KeyValue(row1, fam2, col4, data));
-    memstore.add(new KeyValue(row1, fam2, col5, data));
+    memstore.add(new KeyValue(row1, fam2, col1, 1, data));
+    memstore.add(new KeyValue(row1, fam2, col2, 1, data));
+    memstore.add(new KeyValue(row1, fam2, col3, 1, data));
+    memstore.add(new KeyValue(row1, fam2, col4, 1, data));
+    memstore.add(new KeyValue(row1, fam2, col5, 1, data));
 
     memstore.add(new KeyValue(row2, fam1, col1, data));
 
-    List<MatchCode> actual = new ArrayList<MatchCode>();
+    List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
+    qm.setRow(memstore.get(0).getRow());
 
-    for(KeyValue kv : memstore){
+    for (KeyValue kv : memstore){
       actual.add(qm.match(kv));
     }
 
@@ -128,27 +131,29 @@ public class TestQueryMatcher extends HB
     //of just byte []
 
     //Expected result
-    List<MatchCode> expected = new ArrayList<MatchCode>();
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.NEXT);
+    List<MatchCode> expected = new ArrayList<ScanQueryMatcher.MatchCode>();
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.DONE);
 
-    QueryMatcher qm = new QueryMatcher(get, fam2, null, ttl, rowComparator, 1);
+    ScanQueryMatcher qm = new ScanQueryMatcher(scan, fam2, null, ttl, rowComparator, 1);
 
     List<KeyValue> memstore = new ArrayList<KeyValue>();
-    memstore.add(new KeyValue(row1, fam2, col1, data));
-    memstore.add(new KeyValue(row1, fam2, col2, data));
-    memstore.add(new KeyValue(row1, fam2, col3, data));
-    memstore.add(new KeyValue(row1, fam2, col4, data));
-    memstore.add(new KeyValue(row1, fam2, col5, data));
-    memstore.add(new KeyValue(row2, fam1, col1, data));
+    memstore.add(new KeyValue(row1, fam2, col1, 1, data));
+    memstore.add(new KeyValue(row1, fam2, col2, 1, data));
+    memstore.add(new KeyValue(row1, fam2, col3, 1, data));
+    memstore.add(new KeyValue(row1, fam2, col4, 1, data));
+    memstore.add(new KeyValue(row1, fam2, col5, 1, data));
+    memstore.add(new KeyValue(row2, fam1, col1, 1, data));
+
+    List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
 
-    List<MatchCode> actual = new ArrayList<MatchCode>();
+    qm.setRow(memstore.get(0).getRow());
 
-    for(KeyValue kv : memstore){
+    for(KeyValue kv : memstore) {
       actual.add(qm.match(kv));
     }
 
@@ -164,7 +169,7 @@ public class TestQueryMatcher extends HB
 
 
   /**
-   * Verify that {@link QueryMatcher} only skips expired KeyValue
+   * Verify that {@link ScanQueryMatcher} only skips expired KeyValue
    * instances and does not exit early from the row (skipping
    * later non-expired KeyValues).  This version mimics a Get with
    * explicitly specified column qualifiers.
@@ -176,15 +181,15 @@ public class TestQueryMatcher extends HB
 
     long testTTL = 1000;
     MatchCode [] expected = new MatchCode[] {
-        MatchCode.SKIP,
-        MatchCode.INCLUDE,
-        MatchCode.SKIP,
-        MatchCode.INCLUDE,
-        MatchCode.SKIP,
-        MatchCode.NEXT
+        ScanQueryMatcher.MatchCode.SEEK_NEXT_COL,
+        ScanQueryMatcher.MatchCode.INCLUDE,
+        ScanQueryMatcher.MatchCode.SEEK_NEXT_COL,
+        ScanQueryMatcher.MatchCode.INCLUDE,
+        ScanQueryMatcher.MatchCode.SEEK_NEXT_COL,
+        ScanQueryMatcher.MatchCode.DONE
     };
 
-    QueryMatcher qm = new QueryMatcher(get, fam2,
+    ScanQueryMatcher qm = new ScanQueryMatcher(scan, fam2,
         get.getFamilyMap().get(fam2), testTTL, rowComparator, 1);
 
     long now = System.currentTimeMillis();
@@ -197,6 +202,8 @@ public class TestQueryMatcher extends HB
         new KeyValue(row2, fam1, col1, now-10, data)
     };
 
+    qm.setRow(kvs[0].getRow());
+
     List<MatchCode> actual = new ArrayList<MatchCode>(kvs.length);
     for (KeyValue kv : kvs) {
       actual.add( qm.match(kv) );
@@ -214,7 +221,7 @@ public class TestQueryMatcher extends HB
 
 
   /**
-   * Verify that {@link QueryMatcher} only skips expired KeyValue
+   * Verify that {@link ScanQueryMatcher} only skips expired KeyValue
    * instances and does not exit early from the row (skipping
    * later non-expired KeyValues).  This version mimics a Get with
    * wildcard-inferred column qualifiers.
@@ -226,15 +233,15 @@ public class TestQueryMatcher extends HB
 
     long testTTL = 1000;
     MatchCode [] expected = new MatchCode[] {
-        MatchCode.INCLUDE,
-        MatchCode.INCLUDE,
-        MatchCode.SKIP,
-        MatchCode.INCLUDE,
-        MatchCode.SKIP,
-        MatchCode.NEXT
+        ScanQueryMatcher.MatchCode.INCLUDE,
+        ScanQueryMatcher.MatchCode.INCLUDE,
+        ScanQueryMatcher.MatchCode.SEEK_NEXT_COL,
+        ScanQueryMatcher.MatchCode.INCLUDE,
+        ScanQueryMatcher.MatchCode.SEEK_NEXT_COL,
+        ScanQueryMatcher.MatchCode.DONE
     };
 
-    QueryMatcher qm = new QueryMatcher(get, fam2,
+    ScanQueryMatcher qm = new ScanQueryMatcher(scan, fam2,
         null, testTTL, rowComparator, 1);
 
     long now = System.currentTimeMillis();
@@ -246,8 +253,9 @@ public class TestQueryMatcher extends HB
         new KeyValue(row1, fam2, col5, now-10000, data),
         new KeyValue(row2, fam1, col1, now-10, data)
     };
+    qm.setRow(kvs[0].getRow());
 
-    List<MatchCode> actual = new ArrayList<MatchCode>(kvs.length);
+    List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>(kvs.length);
     for (KeyValue kv : kvs) {
       actual.add( qm.match(kv) );
     }

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWildcardColumnTracker.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWildcardColumnTracker.java?rev=1181336&r1=1181335&r2=1181336&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWildcardColumnTracker.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWildcardColumnTracker.java Tue Oct 11 01:59:50 2011
@@ -24,7 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hbase.HBaseTestCase;
-import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
+import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
 import org.apache.hadoop.hbase.util.Bytes;
 
 public class TestScanWildcardColumnTracker extends HBaseTestCase {
@@ -32,7 +32,6 @@ public class TestScanWildcardColumnTrack
   final static int VERSIONS = 2;
 
   public void testCheckColumn_Ok() {
-    //Create a WildcardColumnTracker
     ScanWildcardColumnTracker tracker =
       new ScanWildcardColumnTracker(VERSIONS);
 
@@ -45,15 +44,15 @@ public class TestScanWildcardColumnTrack
 
     //Setting up expected result
     List<MatchCode> expected = new ArrayList<MatchCode>();
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
 
-    List<MatchCode> actual = new ArrayList<MatchCode>();
+    List<ScanQueryMatcher.MatchCode> actual = new ArrayList<MatchCode>();
 
     for(byte [] qualifier : qualifiers) {
-      MatchCode mc = tracker.checkColumn(qualifier, 0, qualifier.length);
+      ScanQueryMatcher.MatchCode mc = tracker.checkColumn(qualifier, 0, qualifier.length);
       actual.add(mc);
     }
 
@@ -64,7 +63,6 @@ public class TestScanWildcardColumnTrack
   }
 
   public void testCheckColumn_EnforceVersions() {
-    //Create a WildcardColumnTracker
     ScanWildcardColumnTracker tracker =
       new ScanWildcardColumnTracker(VERSIONS);
 
@@ -76,13 +74,13 @@ public class TestScanWildcardColumnTrack
     qualifiers.add(Bytes.toBytes("qualifer2"));
 
     //Setting up expected result
-    List<MatchCode> expected = new ArrayList<MatchCode>();
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.INCLUDE);
-    expected.add(MatchCode.SKIP);
-    expected.add(MatchCode.INCLUDE);
+    List<ScanQueryMatcher.MatchCode> expected = new ArrayList<MatchCode>();
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
+    expected.add(ScanQueryMatcher.MatchCode.SKIP);
+    expected.add(ScanQueryMatcher.MatchCode.INCLUDE);
 
-    List<MatchCode> actual = new ArrayList<MatchCode>();
+    List<MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>();
 
     for(byte [] qualifier : qualifiers) {
       MatchCode mc = tracker.checkColumn(qualifier, 0, qualifier.length);
@@ -96,7 +94,6 @@ public class TestScanWildcardColumnTrack
   }
 
   public void DisabledTestCheckColumn_WrongOrder() {
-    //Create a WildcardColumnTracker
     ScanWildcardColumnTracker tracker =
       new ScanWildcardColumnTracker(VERSIONS);
 

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java?rev=1181336&r1=1181335&r2=1181336&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java Tue Oct 11 01:59:50 2011
@@ -101,7 +101,7 @@ public class TestStore extends TestCase 
     Iterator<byte[]> iter = qualifiers.iterator();
     while(iter.hasNext()){
       byte [] next = iter.next();
-      expected.add(new KeyValue(row, family, next, null));
+      expected.add(new KeyValue(row, family, next, 1, (byte[])null));
       get.addColumn(family, next);
     }
   }
@@ -142,8 +142,8 @@ public class TestStore extends TestCase 
   public void testEmptyStoreFile() throws IOException {
     init(this.getName());
     // Write a store file.
-    this.store.add(new KeyValue(row, family, qf1, null));
-    this.store.add(new KeyValue(row, family, qf2, null));
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
     flush(1);
     // Now put in place an empty store file.  Its a little tricky.  Have to
     // do manually with hacked in sequence id.
@@ -152,7 +152,7 @@ public class TestStore extends TestCase 
     long seqid = f.getMaxSequenceId();
     Configuration c = HBaseConfiguration.create();
     FileSystem fs = FileSystem.get(c);
-    StoreFile.Writer w = StoreFile.createWriter(fs, storedir, 
+    StoreFile.Writer w = StoreFile.createWriter(fs, storedir,
         StoreFile.DEFAULT_BLOCKSIZE_SMALL);
     w.appendMetadata(seqid + 1, false);
     w.close();
@@ -163,7 +163,10 @@ public class TestStore extends TestCase 
       this.store.getFamily(), fs, c);
     System.out.println(this.store.getHRegionInfo().getEncodedName());
     assertEquals(2, this.store.getStorefilesCount());
-    this.store.get(get, qualifiers, result);
+
+    result = HBaseTestingUtility.getFromStoreFile(store,
+        get.getRow(),
+        qualifiers);
     assertEquals(1, result.size());
   }
 
@@ -175,15 +178,16 @@ public class TestStore extends TestCase 
     init(this.getName());
 
     //Put data in memstore
-    this.store.add(new KeyValue(row, family, qf1, null));
-    this.store.add(new KeyValue(row, family, qf2, null));
-    this.store.add(new KeyValue(row, family, qf3, null));
-    this.store.add(new KeyValue(row, family, qf4, null));
-    this.store.add(new KeyValue(row, family, qf5, null));
-    this.store.add(new KeyValue(row, family, qf6, null));
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null));
 
     //Get
-    this.store.get(get, qualifiers, result);
+    result = HBaseTestingUtility.getFromStoreFile(store,
+        get.getRow(), qualifiers);
 
     //Compare
     assertCheck();
@@ -197,25 +201,28 @@ public class TestStore extends TestCase 
     init(this.getName());
 
     //Put data in memstore
-    this.store.add(new KeyValue(row, family, qf1, null));
-    this.store.add(new KeyValue(row, family, qf2, null));
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
     //flush
     flush(1);
 
     //Add more data
-    this.store.add(new KeyValue(row, family, qf3, null));
-    this.store.add(new KeyValue(row, family, qf4, null));
+    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null));
     //flush
     flush(2);
 
     //Add more data
-    this.store.add(new KeyValue(row, family, qf5, null));
-    this.store.add(new KeyValue(row, family, qf6, null));
+    this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null));
     //flush
     flush(3);
 
     //Get
-    this.store.get(get, qualifiers, result);
+    result = HBaseTestingUtility.getFromStoreFile(store,
+        get.getRow(),
+        qualifiers);
+    //this.store.get(get, qualifiers, result);
 
     //Need to sort the result since multiple files
     Collections.sort(result, KeyValue.COMPARATOR);
@@ -232,23 +239,24 @@ public class TestStore extends TestCase 
     init(this.getName());
 
     //Put data in memstore
-    this.store.add(new KeyValue(row, family, qf1, null));
-    this.store.add(new KeyValue(row, family, qf2, null));
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
     //flush
     flush(1);
 
     //Add more data
-    this.store.add(new KeyValue(row, family, qf3, null));
-    this.store.add(new KeyValue(row, family, qf4, null));
+    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null));
     //flush
     flush(2);
 
     //Add more data
-    this.store.add(new KeyValue(row, family, qf5, null));
-    this.store.add(new KeyValue(row, family, qf6, null));
+    this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null));
 
     //Get
-    this.store.get(get, qualifiers, result);
+    result = HBaseTestingUtility.getFromStoreFile(store,
+        get.getRow(), qualifiers);
 
     //Need to sort the result since multiple files
     Collections.sort(result, KeyValue.COMPARATOR);
@@ -278,7 +286,7 @@ public class TestStore extends TestCase 
    * test the internal details of how ICV works, especially during a flush scenario.
    */
   public void testIncrementColumnValue_ICVDuringFlush()
-    throws IOException {
+      throws IOException, InterruptedException {
     init(this.getName());
 
     long oldValue = 1L;
@@ -295,6 +303,9 @@ public class TestStore extends TestCase 
         System.currentTimeMillis(),
         Bytes.toBytes(oldValue)));
 
+    // sleep 2 ms to space out the increments.
+    Thread.sleep(2);
+
     // update during the snapshot.
     long ret = this.store.updateColumnValue(row, family, qf1, newValue);
 
@@ -316,7 +327,7 @@ public class TestStore extends TestCase 
     NavigableSet<byte[]> cols = new TreeSet<byte[]>();
     cols.add(qf1);
 
-    this.store.get(get, cols, results);
+    results = HBaseTestingUtility.getFromStoreFile(store, get);
     assertEquals(2, results.size());
 
     long ts1 = results.get(0).getTimestamp();
@@ -347,9 +358,9 @@ public class TestStore extends TestCase 
     init(getName(), conf);
 
     LOG.info("Adding some data");
-    this.store.add(new KeyValue(row, family, qf1, null));
-    this.store.add(new KeyValue(row, family, qf2, null));
-    this.store.add(new KeyValue(row, family, qf3, null));
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
+    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
 
     LOG.info("Before flush, we should have no files");
     FileStatus[] files = fs.listStatus(store.getHomedir());
@@ -469,41 +480,32 @@ public class TestStore extends TestCase 
       this.store.add(kv);
     }
 
-    NavigableSet<byte[]> columns = new ConcurrentSkipListSet<byte[]>(
-        Bytes.BYTES_COMPARATOR);
-    columns.add(qf1);
     List<KeyValue> result;
     Get get = new Get(Bytes.toBytes(1));
     get.addColumn(family,qf1);
 
     get.setTimeRange(0,15);
-    result = new ArrayList<KeyValue>();
-    this.store.get(get, columns, result);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
     assertTrue(result.size()>0);
 
     get.setTimeRange(40,90);
-    result = new ArrayList<KeyValue>();
-    this.store.get(get, columns, result);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
     assertTrue(result.size()>0);
 
     get.setTimeRange(10,45);
-    result = new ArrayList<KeyValue>();
-    this.store.get(get, columns, result);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
     assertTrue(result.size()>0);
 
     get.setTimeRange(80,145);
-    result = new ArrayList<KeyValue>();
-    this.store.get(get, columns, result);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
     assertTrue(result.size()>0);
 
     get.setTimeRange(1,2);
-    result = new ArrayList<KeyValue>();
-    this.store.get(get, columns, result);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
     assertTrue(result.size()>0);
 
     get.setTimeRange(90,200);
-    result = new ArrayList<KeyValue>();
-    this.store.get(get, columns, result);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
     assertTrue(result.size()==0);
   }
 }
\ No newline at end of file

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java?rev=1181336&r1=1181335&r2=1181336&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java Tue Oct 11 01:59:50 2011
@@ -19,6 +19,9 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.HashMap;
@@ -31,9 +34,10 @@ import org.apache.commons.logging.impl.L
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestCase;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -49,9 +53,13 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.log4j.Level;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 /** JUnit test case for HLog */
-public class TestHLog extends HBaseTestCase {
+public class TestHLog  {
   private static final Log LOG = LogFactory.getLog(TestHLog.class);
   {
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
@@ -61,45 +69,57 @@ public class TestHLog extends HBaseTestC
     ((Log4JLogger)HLog.LOG).getLogger().setLevel(Level.ALL);
   }
 
-  private Path dir;
-  private Path oldLogDir;
-  private MiniDFSCluster cluster;
+  private static Configuration conf;
+  private static FileSystem fs;
+  private static Path dir;
+  private static MiniDFSCluster cluster;
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Path hbaseDir;
+  private static Path oldLogDir;
 
-  @Override
+  @Before
   public void setUp() throws Exception {
-    // Make block sizes small.
-    this.conf.setInt("dfs.blocksize", 1024 * 1024);
-    this.conf.setInt("hbase.regionserver.flushlogentries", 1);
-    // needed for testAppendClose()
-    conf.setBoolean("dfs.support.append", true);
-    // quicker heartbeat interval for faster DN death notification
-    conf.setInt("heartbeat.recheck.interval", 5000);
-    conf.setInt("dfs.heartbeat.interval", 1);
-    conf.setInt("dfs.socket.timeout", 5000);
-    // faster failover with cluster.shutdown();fs.close() idiom
-    conf.setInt("ipc.client.connect.max.retries", 1);
-    conf.setInt("dfs.client.block.recovery.retries", 1);
 
-    cluster = new MiniDFSCluster(conf, 3, true, (String[])null);
-    // Set the hbase.rootdir to be the home directory in mini dfs.
-    this.conf.set(HConstants.HBASE_DIR,
-      this.cluster.getFileSystem().getHomeDirectory().toString());
-    super.setUp();
-    this.dir = new Path("/hbase", getName());
-    if (fs.exists(dir)) {
-      fs.delete(dir, true);
+    FileStatus[] entries = fs.listStatus(new Path("/"));
+    for (FileStatus dir : entries) {
+      fs.delete(dir.getPath(), true);
     }
-    this.oldLogDir = new Path(this.dir, HConstants.HREGION_OLDLOGDIR_NAME);
 
   }
 
-  @Override
+  @After
   public void tearDown() throws Exception {
-    if (this.fs.exists(this.dir)) {
-      this.fs.delete(this.dir, true);
-    }
-    shutdownDfs(cluster);
-    super.tearDown();
+  }
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // Make block sizes small.
+    TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
+    TEST_UTIL.getConfiguration().setInt(
+        "hbase.regionserver.flushlogentries", 1);
+    // needed for testAppendClose()
+    TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
+    // quicker heartbeat interval for faster DN death notification
+    TEST_UTIL.getConfiguration().setInt("heartbeat.recheck.interval", 5000);
+    TEST_UTIL.getConfiguration().setInt("dfs.heartbeat.interval", 1);
+    TEST_UTIL.getConfiguration().setInt("dfs.socket.timeout", 5000);
+    // faster failover with cluster.shutdown();fs.close() idiom
+    TEST_UTIL.getConfiguration()
+        .setInt("ipc.client.connect.max.retries", 1);
+    TEST_UTIL.getConfiguration().setInt(
+        "dfs.client.block.recovery.retries", 1);
+    TEST_UTIL.startMiniCluster(3);
+
+    conf = TEST_UTIL.getConfiguration();
+    cluster = TEST_UTIL.getDFSCluster();
+    fs = cluster.getFileSystem();
+
+    hbaseDir = new Path(TEST_UTIL.getConfiguration().get("hbase.rootdir"));
+    oldLogDir = new Path(hbaseDir, ".oldlogs");
+    dir = new Path(hbaseDir, getName());
+  }
+  private static String getName() {
+    // TODO Auto-generated method stub
+    return "TestHLog";
   }
 
   /**
@@ -107,12 +127,13 @@ public class TestHLog extends HBaseTestC
    * would fail.
    * @throws IOException
    */
+  @Test
   public void testSplit() throws IOException {
 
     final byte [] tableName = Bytes.toBytes(getName());
     final byte [] rowName = tableName;
-    Path logdir = new Path(this.dir, HConstants.HREGION_LOGDIR_NAME);
-    HLog log = new HLog(this.fs, logdir, this.oldLogDir, this.conf, null);
+    Path logdir = new Path(dir, HConstants.HREGION_LOGDIR_NAME);
+    HLog log = new HLog(fs, logdir, oldLogDir, conf, null);
     final int howmany = 3;
     HRegionInfo[] infos = new HRegionInfo[3];
     for(int i = 0; i < howmany; i++) {
@@ -139,9 +160,9 @@ public class TestHLog extends HBaseTestC
         log.rollWriter();
       }
       log.close();
-      Path splitsdir = new Path(this.dir, "splits");
+      Path splitsdir = new Path(dir, "splits");
       List<Path> splits =
-        HLog.splitLog(splitsdir, logdir, this.oldLogDir, this.fs, conf);
+        HLog.splitLog(splitsdir, logdir, oldLogDir, fs, conf);
       verifySplits(splits, howmany);
       log = null;
     } finally {
@@ -155,10 +176,11 @@ public class TestHLog extends HBaseTestC
    * Test new HDFS-265 sync.
    * @throws Exception
    */
+  @Test
   public void Broken_testSync() throws Exception {
     byte [] bytes = Bytes.toBytes(getName());
     // First verify that using streams all works.
-    Path p = new Path(this.dir, getName() + ".fsdos");
+    Path p = new Path(dir, getName() + ".fsdos");
     FSDataOutputStream out = fs.create(p);
     out.write(bytes);
     out.sync();
@@ -169,8 +191,8 @@ public class TestHLog extends HBaseTestC
     assertEquals(bytes.length, read);
     out.close();
     in.close();
-    Path subdir = new Path(this.dir, "hlogdir");
-    HLog wal = new HLog(this.fs, subdir, this.oldLogDir, this.conf, null);
+    Path subdir = new Path(dir, "hlogdir");
+    HLog wal = new HLog(fs, subdir, oldLogDir, conf, null);
     final int total = 20;
 
     HRegionInfo info = new HRegionInfo(new HTableDescriptor(bytes),
@@ -238,6 +260,7 @@ public class TestHLog extends HBaseTestC
    * Test the findMemstoresWithEditsOlderThan method.
    * @throws IOException
    */
+  @Test
   public void testFindMemstoresWithEditsOlderThan() throws IOException {
     Map<byte [], Long> regionsToSeqids = new HashMap<byte [], Long>();
     for (int i = 0; i < 10; i++) {
@@ -264,7 +287,7 @@ public class TestHLog extends HBaseTestC
     assertEquals(howmany, splits.size());
     for (int i = 0; i < splits.size(); i++) {
       LOG.info("Verifying=" + splits.get(i));
-      HLog.Reader reader = HLog.getReader(this.fs, splits.get(i), conf);
+      HLog.Reader reader = HLog.getReader(fs, splits.get(i), conf);
       try {
         int count = 0;
         String previousRegion = null;
@@ -272,7 +295,6 @@ public class TestHLog extends HBaseTestC
         HLog.Entry entry = new HLog.Entry();
         while((entry = reader.next(entry)) != null) {
           HLogKey key = entry.getKey();
-          WALEdit kv = entry.getEdit();
           String region = Bytes.toString(key.getRegionName());
           // Assert that all edits are for same region.
           if (previousRegion != null) {
@@ -296,14 +318,14 @@ public class TestHLog extends HBaseTestC
   // 2. HDFS-988 (SafeMode should freeze file operations
   //              [FSNamesystem.nextGenerationStampForBlock])
   // 3. HDFS-142 (on restart, maintain pendingCreates)
+  @Test
   public void testAppendClose() throws Exception {
-    this.conf.setBoolean("dfs.support.append", true);
     byte [] tableName = Bytes.toBytes(getName());
     HRegionInfo regioninfo = new HRegionInfo(new HTableDescriptor(tableName),
         HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
-    Path subdir = new Path(this.dir, "hlogdir");
-    Path archdir = new Path(this.dir, "hlogdir_archive");
-    HLog wal = new HLog(this.fs, subdir, archdir, this.conf, null);
+    Path subdir = new Path(dir, "hlogdir");
+    Path archdir = new Path(dir, "hlogdir_archive");
+    HLog wal = new HLog(fs, subdir, archdir, conf, null);
     final int total = 20;
 
     for (int i = 0; i < total; i++) {
@@ -313,12 +335,14 @@ public class TestHLog extends HBaseTestC
     }
     // Now call sync to send the data to HDFS datanodes
     wal.sync(true);
+     int namenodePort = cluster.getNameNodePort();
     final Path walPath = wal.computeFilename();
 
+
     // Stop the cluster.  (ensure restart since we're sharing MiniDFSCluster)
     try {
-      this.cluster.getNameNode().setSafeMode(SafeModeAction.SAFEMODE_ENTER);
-      this.cluster.shutdown();
+      cluster.getNameNode().setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      cluster.shutdown();
       try {
         // wal.writer.close() will throw an exception,
         // but still call this since it closes the LogSyncer thread first
@@ -326,19 +350,23 @@ public class TestHLog extends HBaseTestC
       } catch (IOException e) {
         LOG.info(e);
       }
-      this.fs.close(); // closing FS last so DFSOutputStream can't call close
+      fs.close(); // closing FS last so DFSOutputStream can't call close
       LOG.info("STOPPED first instance of the cluster");
     } finally {
       // Restart the cluster
-      this.cluster = new MiniDFSCluster(conf, 2, false, null);
-      this.cluster.waitActive();
-      this.fs = cluster.getFileSystem();
+      while (cluster.isClusterUp()){
+        LOG.error("Waiting for cluster to go down");
+        Thread.sleep(1000);
+      }
+      cluster = new MiniDFSCluster(namenodePort, conf, 5, false, true, true, null, null, null, null);
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
       LOG.info("START second instance.");
     }
 
     // set the lease period to be 1 second so that the
     // namenode triggers lease recovery upon append request
-    Method setLeasePeriod = this.cluster.getClass()
+    Method setLeasePeriod = cluster.getClass()
       .getDeclaredMethod("setLeasePeriod", new Class[]{Long.TYPE, Long.TYPE});
     setLeasePeriod.setAccessible(true);
     setLeasePeriod.invoke(cluster,
@@ -350,8 +378,8 @@ public class TestHLog extends HBaseTestC
     }
 
     // Now try recovering the log, like the HMaster would do
-    final FileSystem recoveredFs = this.fs;
-    final Configuration rlConf = this.conf;
+    final FileSystem recoveredFs = fs;
+    final Configuration rlConf = conf;
 
     class RecoverLogThread extends Thread {
       public Exception exception = null;
@@ -378,9 +406,9 @@ public class TestHLog extends HBaseTestC
 
     // Make sure you can read all the content
     SequenceFile.Reader reader
-      = new SequenceFile.Reader(this.fs, walPath, this.conf);
+      = new SequenceFile.Reader(fs, walPath, conf);
     int count = 0;
-    HLogKey key = HLog.newKey(this.conf);
+    HLogKey key = HLog.newKey(conf);
     WALEdit val = new WALEdit();
     while (reader.next(key, val)) {
       count++;
@@ -395,12 +423,13 @@ public class TestHLog extends HBaseTestC
    * Tests that we can write out an edit, close, and then read it back in again.
    * @throws IOException
    */
+  @Test
   public void testEditAdd() throws IOException {
     final int COL_COUNT = 10;
     final byte [] tableName = Bytes.toBytes("tablename");
     final byte [] row = Bytes.toBytes("row");
     HLog.Reader reader = null;
-    HLog log = new HLog(fs, dir, this.oldLogDir, this.conf, null);
+    HLog log = new HLog(fs, dir, oldLogDir, conf, null);
     try {
       // Write columns named 1, 2, 3, etc. and then values of single byte
       // 1, 2, 3...
@@ -463,13 +492,13 @@ public class TestHLog extends HBaseTestC
   /**
    * @throws IOException
    */
+  @Test
   public void testAppend() throws IOException {
     final int COL_COUNT = 10;
     final byte [] tableName = Bytes.toBytes("tablename");
     final byte [] row = Bytes.toBytes("row");
-    this.conf.setBoolean("dfs.support.append", true);
     Reader reader = null;
-    HLog log = new HLog(this.fs, dir, this.oldLogDir, this.conf, null);
+    HLog log = new HLog(fs, dir, oldLogDir, conf, null);
     try {
       // Write columns named 1, 2, 3, etc. and then values of single byte
       // 1, 2, 3...
@@ -530,12 +559,12 @@ public class TestHLog extends HBaseTestC
    * Test that we can visit entries before they are appended
    * @throws Exception
    */
+  @Test
   public void testVisitors() throws Exception {
     final int COL_COUNT = 10;
     final byte [] tableName = Bytes.toBytes("tablename");
     final byte [] row = Bytes.toBytes("row");
-    this.conf.setBoolean("dfs.support.append", true);
-    HLog log = new HLog(this.fs, dir, this.oldLogDir, this.conf, null);
+    HLog log = new HLog(fs, dir, oldLogDir, conf, null);
     DumbLogEntriesVisitor visitor = new DumbLogEntriesVisitor();
     log.addLogEntryVisitor(visitor);
     long timestamp = System.currentTimeMillis();

Added: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java?rev=1181336&view=auto
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java (added)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogMethods.java Tue Oct 11 01:59:50 2011
@@ -0,0 +1,81 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.NavigableSet;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.junit.Test;
+
+/**
+ * Simple testing of a few HLog methods.
+ */
+public class TestHLogMethods {
+  private final HBaseTestingUtility util = new HBaseTestingUtility();
+
+  /**
+   * Assert that getSplitEditFilesSorted returns files in expected order and
+   * that it skips moved-aside files.
+   * @throws IOException
+   */
+  @Test public void testGetSplitEditFilesSorted() throws IOException {
+    FileSystem fs = FileSystem.get(util.getConfiguration());
+    Path regiondir = HBaseTestingUtility.getTestDir("regiondir");
+    fs.delete(regiondir, true);
+    fs.mkdirs(regiondir);
+    Path recoverededits = HLog.getRegionDirRecoveredEditsDir(regiondir);
+    String first = HLog.formatRecoveredEditsFileName(-1);
+    createFile(fs, recoverededits, first);
+    createFile(fs, recoverededits, HLog.formatRecoveredEditsFileName(0));
+    createFile(fs, recoverededits, HLog.formatRecoveredEditsFileName(1));
+    createFile(fs, recoverededits, HLog.formatRecoveredEditsFileName(11));
+    createFile(fs, recoverededits, HLog.formatRecoveredEditsFileName(2));
+    createFile(fs, recoverededits, HLog.formatRecoveredEditsFileName(50));
+    String last = HLog.formatRecoveredEditsFileName(Long.MAX_VALUE);
+    createFile(fs, recoverededits, last);
+    createFile(fs, recoverededits,
+      Long.toString(Long.MAX_VALUE) + "." + System.currentTimeMillis());
+    NavigableSet<Path> files = HLog.getSplitEditFilesSorted(fs, regiondir);
+    assertEquals(7, files.size());
+    assertEquals(files.pollFirst().getName(), first);
+    assertEquals(files.pollLast().getName(), last);
+    assertEquals(files.pollFirst().getName(),
+      HLog.formatRecoveredEditsFileName(0));
+    assertEquals(files.pollFirst().getName(),
+      HLog.formatRecoveredEditsFileName(1));
+    assertEquals(files.pollFirst().getName(),
+      HLog.formatRecoveredEditsFileName(2));
+    assertEquals(files.pollFirst().getName(),
+      HLog.formatRecoveredEditsFileName(11));
+  }
+
+  private void createFile(final FileSystem fs, final Path testdir,
+      final String name)
+  throws IOException {
+    FSDataOutputStream fdos = fs.create(new Path(testdir, name), true);
+    fdos.close();
+  }
+}
\ No newline at end of file

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java?rev=1181336&r1=1181335&r2=1181336&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java Tue Oct 11 01:59:50 2011
@@ -476,9 +476,6 @@ public class TestHLogSplit {
     assertEquals(0, compareHLogSplitDirs(firstSplitPath, splitPath));
   }
 
-
-
-
   /**
    * This thread will keep writing to the file after the split process has started
    * It simulates a region server that was considered dead but woke up and wrote
@@ -610,11 +607,14 @@ public class TestHLogSplit {
     }
   }
 
-  private Path getLogForRegion(Path rootdir, byte[] table, String region) {
-    return new Path(HRegion.getRegionDir(HTableDescriptor
-            .getTableDir(rootdir, table),
-            HRegionInfo.encodeRegionName(region.getBytes())),
-            HLog.RECOVERED_EDITS);
+  private Path getLogForRegion(Path rootdir, byte[] table, String region)
+  throws IOException {
+    Path tdir = HTableDescriptor.getTableDir(rootdir, table);
+    Path editsdir = HLog.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
+      HRegionInfo.encodeRegionName(region.getBytes())));
+    FileStatus [] files = this.fs.listStatus(editsdir);
+    assertEquals(1, files.length);
+    return files[0].getPath();
   }
 
   private void corruptHLog(Path path, Corruptions corruption, boolean close,
@@ -722,8 +722,15 @@ public class TestHLogSplit {
     FileStatus[] f2 = fs.listStatus(p2);
 
     for (int i=0; i<f1.length; i++) {
-      if (!logsAreEqual(new Path(f1[i].getPath(), HLog.RECOVERED_EDITS),
-              new Path(f2[i].getPath(), HLog.RECOVERED_EDITS))) {
+      // Regions now have a directory named RECOVERED_EDITS_DIR and in here
+      // are split edit files.  In below presume only 1.
+      Path rd1 = HLog.getRegionDirRecoveredEditsDir(f1[i].getPath());
+      FileStatus [] rd1fs = fs.listStatus(rd1);
+      assertEquals(1, rd1fs.length);
+      Path rd2 = HLog.getRegionDirRecoveredEditsDir(f2[i].getPath());
+      FileStatus [] rd2fs = fs.listStatus(rd2);
+      assertEquals(1, rd2fs.length);
+      if (!logsAreEqual(rd1fs[0].getPath(), rd2fs[0].getPath())) {
         return -1;
       }
     }
@@ -745,6 +752,4 @@ public class TestHLogSplit {
     }
     return true;
   }
-
-
 }

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1181336&r1=1181335&r2=1181336&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Tue Oct 11 01:59:50 2011
@@ -42,8 +42,8 @@ import org.apache.hadoop.hbase.client.Ge
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.regionserver.FlushRequester;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -115,6 +115,57 @@ public class TestWALReplay {
   }
 
   /**
+   * Tests for hbase-2727.
+   * @throws Exception
+   * @see https://issues.apache.org/jira/browse/HBASE-2727
+   */
+  @Test
+  public void test2727() throws Exception {
+    // Test being able to have > 1 set of edits in the recovered.edits directory.
+    // Ensure edits are replayed properly.
+    final String tableNameStr = "test2727";
+    HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
+    Path basedir = new Path(hbaseRootDir, tableNameStr);
+    deleteDir(basedir);
+
+    final byte [] tableName = Bytes.toBytes(tableNameStr);
+    final byte [] rowName = tableName;
+
+    HLog wal1 = createWAL(this.conf);
+    // Add 1k to each family.
+    final int countPerFamily = 1000;
+    for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
+      addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal1);
+    }
+    wal1.close();
+    runWALSplit(this.conf);
+
+    HLog wal2 = createWAL(this.conf);
+    // Up the sequenceid so that these edits are after the ones added above.
+    wal2.setSequenceNumber(wal1.getSequenceNumber());
+    // Add 1k to each family.
+    for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
+      addWALEdits(tableName, hri, rowName, hcd.getName(), countPerFamily, ee, wal2);
+    }
+    wal2.close();
+    runWALSplit(this.conf);
+
+    HLog wal3 = createWAL(this.conf);
+    wal3.setSequenceNumber(wal2.getSequenceNumber());
+    try {
+      final HRegion region = new HRegion(basedir, wal3, this.fs, this.conf, hri,
+          null);
+      long seqid = region.initialize();
+      assertTrue(seqid > wal3.getSequenceNumber());
+
+      // TODO: Scan all.
+      region.close();
+    } finally {
+      wal3.closeAndDelete();
+    }
+  }
+
+  /**
    * Test case of HRegion that is only made out of bulk loaded files.  Assert
    * that we don't 'crash'.
    * @throws IOException
@@ -210,8 +261,8 @@ public class TestWALReplay {
     HLog wal2 = createWAL(this.conf);
     HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, null) {
       @Override
-      protected void restoreEdit(KeyValue kv) throws IOException {
-        super.restoreEdit(kv);
+      protected boolean restoreEdit(Store s, KeyValue kv) {
+        super.restoreEdit(s, kv);
         throw new RuntimeException("Called when it should not have been!");
       }
     };
@@ -221,7 +272,7 @@ public class TestWALReplay {
     assertTrue(seqid + result.size() < seqid2);
 
     // Next test.  Add more edits, then 'crash' this region by stealing its wal
-    // out from under it and assert that replay of the log addes the edits back
+    // out from under it and assert that replay of the log adds the edits back
     // correctly when region is opened again.
     for (HColumnDescriptor hcd: hri.getTableDesc().getFamilies()) {
       addRegionEdits(rowName, hcd.getName(), countPerFamily, this.ee, region2, "y");
@@ -242,9 +293,10 @@ public class TestWALReplay {
     final AtomicInteger countOfRestoredEdits = new AtomicInteger(0);
     HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, null) {
       @Override
-      protected void restoreEdit(KeyValue kv) throws IOException {
-        super.restoreEdit(kv);
+      protected boolean restoreEdit(Store s, KeyValue kv) {
+        boolean b = super.restoreEdit(s, kv);
         countOfRestoredEdits.incrementAndGet();
+        return b;
       }
     };
     long seqid3 = region3.initialize();
@@ -317,14 +369,20 @@ public class TestWALReplay {
     newConf.setInt("hbase.hregion.memstore.flush.size", 1024 * 100);
     // Make a new wal for new region.
     HLog newWal = createWAL(newConf);
+    final AtomicInteger flushcount = new AtomicInteger(0);
     try {
-      TestFlusher flusher = new TestFlusher();
       final HRegion region = new HRegion(basedir, newWal, newFS, newConf, hri,
-          flusher);
-      flusher.r = region;
+          null) {
+        protected boolean internalFlushcache(HLog wal, long myseqid)
+        throws IOException {
+          boolean b = super.internalFlushcache(wal, myseqid);
+          flushcount.incrementAndGet();
+          return b;
+        };
+      };
       long seqid = region.initialize();
-      // Assert we flushed.
-      assertTrue(flusher.count > 0);
+      // We flushed during init.
+      assertTrue(flushcount.get() > 0);
       assertTrue(seqid > wal.getSequenceNumber());
 
       Get get = new Get(rowName);
@@ -338,23 +396,6 @@ public class TestWALReplay {
     }
   }
 
-  // Flusher used in this test.  Keep count of how often we are called and
-  // actually run the flush inside here.
-  class TestFlusher implements FlushRequester {
-    private int count = 0;
-    private HRegion r;
-
-    @Override
-    public void request(HRegion region) {
-      count++;
-      try {
-        r.flushcache();
-      } catch (IOException e) {
-        throw new RuntimeException("Exception flushing", e);
-      }
-    }
-  }
-
   private void addWALEdits (final byte [] tableName, final HRegionInfo hri,
       final byte [] rowName, final byte [] family,
       final int count, EnvironmentEdge ee, final HLog wal)

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java?rev=1181336&r1=1181335&r2=1181336&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestByteBloomFilter.java Tue Oct 11 01:59:50 2011
@@ -20,13 +20,10 @@
 
 package org.apache.hadoop.hbase.util;
 
-import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.nio.ByteBuffer;
 import java.util.BitSet;
-import java.util.Random;
 
 import junit.framework.TestCase;
 
@@ -141,61 +138,4 @@ public class TestByteBloomFilter extends
 
     // test: foldFactor > log(max/actual)
   }
-
-  public void testDynamicBloom() throws Exception {
-    int keyInterval = 1000;
-    float err = (float)0.01;
-    BitSet valid = new BitSet(keyInterval*4);
-
-    DynamicByteBloomFilter bf1 = new DynamicByteBloomFilter(keyInterval, err,
-        Hash.MURMUR_HASH);
-    bf1.allocBloom();
-    
-    long seed = System.currentTimeMillis();
-    Random r = new Random(seed);
-    System.out.println("seed = " + seed);
-
-    for (int i = 0; i < keyInterval*4; ++i) { // add
-      if (r.nextBoolean()) {
-        bf1.add(Bytes.toBytes(i));
-        valid.set(i);
-
-        // we assume only 2 blooms in this test, so exit before a 3rd is made
-        if (bf1.getKeyCount() == 2000) {
-          break;
-        }
-      }
-    }
-    assertTrue(2 <= bf1.bloomCount());
-    System.out.println("keys added = " + bf1.getKeyCount());
-
-    // test serialization/deserialization
-    ByteArrayOutputStream metaOut = new ByteArrayOutputStream();
-    ByteArrayOutputStream dataOut = new ByteArrayOutputStream();
-    bf1.getMetaWriter().write(new DataOutputStream(metaOut));
-    bf1.getDataWriter().write(new DataOutputStream(dataOut));
-    ByteBuffer bb = ByteBuffer.wrap(dataOut.toByteArray()); 
-    DynamicByteBloomFilter newBf1 = new DynamicByteBloomFilter(
-        ByteBuffer.wrap(metaOut.toByteArray()));
-
-    int falsePositives = 0;
-    for (int i = 0; i < keyInterval*4; ++i) { // check
-      if (newBf1.contains(Bytes.toBytes(i), bb)) {
-        if (!valid.get(i)) ++falsePositives;
-      } else {
-        if (valid.get(i)) {
-          assert false;
-        }
-      }
-    }
-    
-    // Dynamic Blooms are a little sneaky.  The error rate currently isn't
-    // 'err', it's err * bloomCount.  bloomCount == 2000/1000 == 2 in this case
-    // So, the actual error rate should be roughly:
-    //    (keyInterval*2) * err * bloomCount
-    // allow some tolerance
-    System.out.println("False positives: " + falsePositives);
-    assertTrue(falsePositives <= (keyInterval*5)*err); 
-  }
-
 }