From 033be3998730dc2b0bdb8ec919df781ca49a44da Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Wed, 6 Mar 2013 13:18:04 +0800 Subject: [PATCH] Make reseek work correctly when invoked for the first time. --- .../hbase/regionserver/StoreFileScanner.java | 14 +++++----- .../hadoop/hbase/regionserver/TestStoreFile.java | 27 ++++++++++++++++++++ 2 files changed, 34 insertions(+), 7 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index 9213f39..ea225c9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -53,11 +53,6 @@ public class StoreFileScanner implements KeyValueScanner { 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 KeyValueScanner { return false; } - this.isReseekable = true; cur = hfs.getKeyValue(); return skipKVsNewerThanReadpoint(); @@ -242,6 +236,12 @@ public class StoreFileScanner implements KeyValueScanner { //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 KeyValueScanner { if (realSeekDone) return; - if (delayedReseek && this.isReseekable) { + if (delayedReseek) { reseek(delayedSeekKV); } else { seek(delayedSeekKV); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index 81a313a..72a8b93 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -586,6 +586,33 @@ public class TestStoreFile extends HBaseTestCase { + ", 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); -- 1.7.9.5