hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ramkris...@apache.org
Subject svn commit: r1455990 - in /hbase/branches/0.95/hbase-server/src: main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
Date Wed, 13 Mar 2013 15:11:03 GMT
Author: ramkrishna
Date: Wed Mar 13 15:11:02 2013
New Revision: 1455990

URL: http://svn.apache.org/r1455990
Log:
HBASE-8012 - Reseek should position to the beginning of file for the first time it is invoked
with a KV smaller than the first KV in file (Raymond Liu)


Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java?rev=1455990&r1=1455989&r2=1455990&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
(original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
Wed Mar 13 15:11:02 2013
@@ -53,11 +53,6 @@ public class StoreFileScanner implements
 
   private boolean enforceMVCC = false;
 
-  //The variable, realSeekDone, may cheat on store file scanner for the
-  // multi-column bloom-filter optimization.
-  // So this flag shows whether this storeFileScanner could do a reseek.
-  private boolean isReseekable = false;
-
   private static final AtomicLong seekCount = new AtomicLong();
 
   private ScanQueryMatcher matcher;
@@ -148,7 +143,6 @@ public class StoreFileScanner implements
           return false;
         }
 
-        this.isReseekable = true;
         cur = hfs.getKeyValue();
 
         return skipKVsNewerThanReadpoint();
@@ -242,6 +236,12 @@ public class StoreFileScanner implements
     //This function is similar to seekAtOrAfter function
     int result = s.reseekTo(k.getBuffer(), k.getKeyOffset(), k.getKeyLength());
     if (result <= 0) {
+      // If up to now scanner is not seeked yet, this means passed KV is smaller
+      // than first KV in file, and it is the first time we seek on this file.
+      // So we also need to work from the start of file.
+      if (!s.isSeeked()) {
+        return  s.seekTo();
+      }
       return true;
     } else {
       // passed KV is larger than current KV in file, if there is a next
@@ -346,7 +346,7 @@ public class StoreFileScanner implements
     if (realSeekDone)
       return;
 
-    if (delayedReseek && this.isReseekable) {
+    if (delayedReseek) {
       reseek(delayedSeekKV);
     } else {
       seek(delayedSeekKV);

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java?rev=1455990&r1=1455989&r2=1455990&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
(original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
Wed Mar 13 15:11:02 2013
@@ -609,6 +609,33 @@ public class TestStoreFile extends HBase
         + ", expected no more than " + maxFalsePos, falsePos <= maxFalsePos);
   }
 
+  /**
+   * Test for HBASE-8012
+   */
+  public void testReseek() throws Exception {
+    // write the file
+    Path f = new Path(ROOT_DIR, getName());
+
+    // Make a store file and write data to it.
+    StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf,
+         this.fs, 8 * 1024)
+            .withFilePath(f)
+            .build();
+
+    writeStoreFile(writer);
+    writer.close();
+
+    StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf, DataBlockEncoding.NONE);
+
+    // Now do reseek with empty KV to position to the beginning of the file
+
+    KeyValue k = KeyValue.createFirstOnRow(HConstants.EMPTY_BYTE_ARRAY);
+    StoreFileScanner s = reader.getStoreFileScanner(false, false);
+    s.reseek(k);
+
+    assertNotNull("Intial reseek should position at the beginning of the file", s.peek());
+  }
+
   public void testBloomTypes() throws Exception {
     float err = (float) 0.01;
     FileSystem fs = FileSystem.getLocal(conf);



Mime
View raw message