Index: src/test/org/apache/hadoop/hbase/filter/TestFilter.java =================================================================== --- src/test/org/apache/hadoop/hbase/filter/TestFilter.java (revision 933367) +++ src/test/org/apache/hadoop/hbase/filter/TestFilter.java (working copy) @@ -994,4 +994,97 @@ 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 + s.setFilter(new ColumnPaginationFilter(1,1)); + verifyScan(s, expectedRows, expectedKeys); + this.verifyScanFull(s, expectedKVs); + + // Page 3; 1 Result per page + s.setFilter(new ColumnPaginationFilter(3,1)); + verifyScan(s, expectedRows, expectedKeys); + this.verifyScanFull(s, expectedKVs2); + + // Page 2; 2 Results per page + s.setFilter(new ColumnPaginationFilter(2,2)); + expectedKeys = 2; + verifyScan(s, expectedRows, expectedKeys); + this.verifyScanFull(s, expectedKVs3); + + // Page 8; 20 Results per page (no results) + s.setFilter(new ColumnPaginationFilter(8,20)); + expectedKeys = 0; + expectedRows = 0; + verifyScan(s, expectedRows, 0); + this.verifyScanFull(s, expectedKVs4); + } } Index: src/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java =================================================================== --- src/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (revision 0) +++ src/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (revision 0) @@ -0,0 +1,75 @@ +package org.apache.hadoop.hbase.filter; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.hbase.KeyValue; + +/** + * A filter, based on the ColumnCountGetFilter, takes two arguments: page (x) and pageSize (y). + * This filter can be used for row-based indexing, where references to other tables are stored across many columns, + * in order to efficient lookups and paginated results for end users. + */ +public class ColumnPaginationFilter implements Filter +{ + private int page = 0; + private int pageSize = 0; + private int count = 0; + private int offset = 0; + + /** + * Used during serialization. Do not use. + */ + public ColumnPaginationFilter() + { + super(); + } + + public ColumnPaginationFilter(final int page, final int pageSize) + { + this.page = page; + this.pageSize = pageSize; + offset = (page - 1) * pageSize; + } + + public boolean filterAllRemaining() + { + return false; + } + + public ReturnCode filterKeyValue(KeyValue v) + { + ReturnCode code = (count < offset || count >= offset + pageSize) ? ReturnCode.SKIP : ReturnCode.INCLUDE; + count++; + return code; + } + + public boolean filterRow() + { + this.count = 0; + return false; + } + + public boolean filterRowKey(byte[] buffer, int offset, int length) + { + return false; + } + + public void reset() + { + this.count = 0; + } + + public void readFields(DataInput in) throws IOException + { + this.page = in.readInt(); + this.pageSize = in.readInt(); + } + + public void write(DataOutput out) throws IOException + { + out.writeInt(this.page); + out.writeInt(this.pageSize); + } +} \ No newline at end of file