You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2010/04/20 23:23:27 UTC
svn commit: r936067 - in /hadoop/hbase/branches/0.20: CHANGES.txt
src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
src/test/org/apache/hadoop/hbase/filter/TestFilter.java
Author: stack
Date: Tue Apr 20 21:23:27 2010
New Revision: 936067
URL: http://svn.apache.org/viewvc?rev=936067&view=rev
Log:
HBASE-2438 Addition of a Column Pagination Filter
Modified:
hadoop/hbase/branches/0.20/CHANGES.txt
hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java
Modified: hadoop/hbase/branches/0.20/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/CHANGES.txt?rev=936067&r1=936066&r2=936067&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.20/CHANGES.txt Tue Apr 20 21:23:27 2010
@@ -161,6 +161,7 @@ Release 0.20.4 - Unreleased
HBASE-2349 Backport HBaseTestingUtility to branch (will include adding
support for junit4 to branch)
HBASE-2403 [stargate] client HTable interface to REST connector
+ HBASE-2438 Addition of a Column Pagination Filter (Paul Kist via Stack)
Release 0.20.3 - January 25th, 2010
INCOMPATIBLE CHANGES
Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java?rev=936067&r1=936066&r2=936067&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java Tue Apr 20 21:23:27 2010
@@ -155,6 +155,7 @@ public class HbaseObjectWritable impleme
addToMap(SkipFilter.class, code++);
addToMap(WritableByteArrayComparable.class, code++);
addToMap(FirstKeyOnlyFilter.class, code++);
+ addToMap(ColumnPaginationFilter.class, code++);
addToMap(Delete [].class, code++);
Modified: hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java?rev=936067&r1=936066&r2=936067&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java (original)
+++ hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java Tue Apr 20 21:23:27 2010
@@ -1103,4 +1103,97 @@ public class TestFilter extends HBaseTes
assertEquals("Expected " + kvs.length + " total keys but scanned " + idx,
kvs.length, idx);
}
+
+
+ public void testColumnPaginationFilter() throws Exception {
+
+ // Set of KVs (page: 1; pageSize: 1) - the first set of 1 column per row
+ KeyValue [] expectedKVs = {
+ // testRowOne-0
+ new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+ // testRowOne-2
+ new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+ // testRowOne-3
+ new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
+ // testRowTwo-0
+ new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+ // testRowTwo-2
+ new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+ // testRowTwo-3
+ new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1])
+ };
+
+
+ // Set of KVs (page: 3; pageSize: 1) - the third set of 1 column per row
+ KeyValue [] expectedKVs2 = {
+ // testRowOne-0
+ new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+ // testRowOne-2
+ new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+ // testRowOne-3
+ new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+ // testRowTwo-0
+ new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+ // testRowTwo-2
+ new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+ // testRowTwo-3
+ new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+ };
+
+ // Set of KVs (page: 2; pageSize 2) - the 2nd set of 2 columns per row
+ KeyValue [] expectedKVs3 = {
+ // testRowOne-0
+ new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+ new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
+ // testRowOne-2
+ new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+ new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
+ // testRowOne-3
+ new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
+ new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
+ // testRowTwo-0
+ new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+ new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
+ // testRowTwo-2
+ new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+ new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
+ // testRowTwo-3
+ new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+ new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
+ };
+
+
+ // Set of KVs (page: 2; pageSize 2) - the 2nd set of 2 columns per row
+ KeyValue [] expectedKVs4 = {
+
+ };
+
+ long expectedRows = this.numRows;
+ long expectedKeys = 1;
+ Scan s = new Scan();
+
+
+ // Page 1; 1 Column per page (Limit 1, Offset 0)
+ s.setFilter(new ColumnPaginationFilter(1,0));
+ verifyScan(s, expectedRows, expectedKeys);
+ this.verifyScanFull(s, expectedKVs);
+
+ // Page 3; 1 Result per page (Limit 1, Offset 2)
+ s.setFilter(new ColumnPaginationFilter(1,2));
+ verifyScan(s, expectedRows, expectedKeys);
+ this.verifyScanFull(s, expectedKVs2);
+
+ // Page 2; 2 Results per page (Limit 2, Offset 2)
+ s.setFilter(new ColumnPaginationFilter(2,2));
+ expectedKeys = 2;
+ verifyScan(s, expectedRows, expectedKeys);
+ this.verifyScanFull(s, expectedKVs3);
+
+ // Page 8; 20 Results per page (no results) (Limit 20, Offset 140)
+ s.setFilter(new ColumnPaginationFilter(20,140));
+ expectedKeys = 0;
+ expectedRows = 0;
+ verifyScan(s, expectedRows, 0);
+ this.verifyScanFull(s, expectedKVs4);
+ }
}