You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2016/03/16 00:33:42 UTC

hbase git commit: HBASE-15247 InclusiveStopFilter does not respect reverse Filter property. (Amal Joshy)

Repository: hbase
Updated Branches:
  refs/heads/0.98 f0ddd3c06 -> 92eea6db5


HBASE-15247 InclusiveStopFilter does not respect reverse Filter property. (Amal Joshy)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/92eea6db
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/92eea6db
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/92eea6db

Branch: refs/heads/0.98
Commit: 92eea6db5cbe0abfd4acf5575929863bac2d8c02
Parents: f0ddd3c
Author: anoopsjohn <an...@gmail.com>
Authored: Tue Feb 23 09:59:35 2016 +0530
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Mar 15 16:31:45 2016 -0700

----------------------------------------------------------------------
 .../hbase/filter/InclusiveStopFilter.java       |  4 +--
 .../apache/hadoop/hbase/filter/TestFilter.java  | 37 ++++++++++++++++++++
 2 files changed, 38 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/92eea6db/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
index 4ca6cf8..f274bdc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
@@ -70,9 +70,7 @@ public class InclusiveStopFilter extends FilterBase {
     int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length,
       buffer, offset, length);
 
-    if(cmp < 0) {
-      done = true;
-    }
+    done = reversed ? cmp > 0 : cmp < 0;
     return done;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/92eea6db/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
index 37a4db9..0fccc6b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
@@ -713,6 +713,43 @@ public class TestFilter {
   }
 
   @Test
+  public void testInclusiveStopFilterWithReverseScan() throws IOException {
+
+    // Grab rows from group one
+
+    // If we just use start/stop row, we get total/2 - 1 rows
+    long expectedRows = (this.numRows / 2) - 1;
+    long expectedKeys = this.colsPerRow;
+    Scan s = new Scan(Bytes.toBytes("testRowOne-3"), Bytes.toBytes("testRowOne-0"));
+    s.setReversed(true);
+    verifyScan(s, expectedRows, expectedKeys);
+
+    // Now use start row with inclusive stop filter
+    expectedRows = this.numRows / 2;
+    s = new Scan(Bytes.toBytes("testRowOne-3"));
+    s.setReversed(true);
+    s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowOne-0")));
+    verifyScan(s, expectedRows, expectedKeys);
+
+    // Grab rows from group two
+
+    // If we just use start/stop row, we get total/2 - 1 rows
+    expectedRows = (this.numRows / 2) - 1;
+    expectedKeys = this.colsPerRow;
+    s = new Scan(Bytes.toBytes("testRowTwo-3"), Bytes.toBytes("testRowTwo-0"));
+    s.setReversed(true);
+    verifyScan(s, expectedRows, expectedKeys);
+
+    // Now use start row with inclusive stop filter
+    expectedRows = this.numRows / 2;
+    s = new Scan(Bytes.toBytes("testRowTwo-3"));
+    s.setReversed(true);
+    s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowTwo-0")));
+    verifyScan(s, expectedRows, expectedKeys);
+
+  }
+
+  @Test
   public void testQualifierFilter() throws IOException {
 
     // Match two keys (one from each family) in half the rows