You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2016/02/23 04:56:46 UTC

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

Repository: hbase
Updated Branches:
  refs/heads/master 2966eee60 -> 2d66cd86d


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/2d66cd86
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2d66cd86
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2d66cd86

Branch: refs/heads/master
Commit: 2d66cd86d08120887efe05b65fa53bd667a5be76
Parents: 2966eee
Author: anoopsjohn <an...@gmail.com>
Authored: Tue Feb 23 09:21:37 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Tue Feb 23 09:21:37 2016 +0530

----------------------------------------------------------------------
 .../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/2d66cd86/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 3412c5c..1096f5e 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
@@ -62,9 +62,7 @@ public class InclusiveStopFilter extends FilterBase {
   public boolean filterRowKey(Cell firstRowCell) {
     // if stopRowKey is <= buffer, then true, filter row.
     int cmp = CellComparator.COMPARATOR.compareRows(firstRowCell, stopRowKey, 0, stopRowKey.length);
-    if (cmp > 0) {
-      done = true;
-    }
+    done = reversed ? cmp < 0 : cmp > 0;
     return done;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d66cd86/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 1e89685..1dd6616 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
@@ -717,6 +717,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