HBASE-2438 Addition of a Column Pagination Filter

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@936070 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-04-20 21:28:49 +00:00
parent 0f17cbf487
commit 2b2f1d1670
4 changed files with 285 additions and 0 deletions

View File

@ -546,6 +546,7 @@ Release 0.21.0 - Unreleased
HBASE-2327 [EC2] Allocate elastic IP addresses for ZK and master nodes HBASE-2327 [EC2] Allocate elastic IP addresses for ZK and master nodes
HBASE-2319 [stargate] multiuser mode: request shaping HBASE-2319 [stargate] multiuser mode: request shaping
HBASE-2403 [stargate] client HTable interface to REST connector HBASE-2403 [stargate] client HTable interface to REST connector
HBASE-2438 Addition of a Column Pagination Filter (Paul Kist via Stack)
OPTIMIZATIONS OPTIMIZATIONS
HBASE-410 [testing] Speed up the test suite HBASE-410 [testing] Speed up the test suite

View File

@ -0,0 +1,97 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
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: limit and offset.
* 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 limit = 0;
private int offset = 0;
private int count = 0;
/**
* Used during serialization. Do not use.
*/
public ColumnPaginationFilter()
{
super();
}
public ColumnPaginationFilter(final int limit, final int offset)
{
this.limit = limit;
this.offset = offset;
}
public boolean filterAllRemaining()
{
return false;
}
public ReturnCode filterKeyValue(KeyValue v)
{
if(count >= offset + limit)
{
return ReturnCode.NEXT_ROW;
}
ReturnCode code = count < offset ? 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.limit = in.readInt();
this.offset = in.readInt();
}
public void write(DataOutput out) throws IOException
{
out.writeInt(this.limit);
out.writeInt(this.offset);
}
}

View File

@ -0,0 +1,94 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.filter;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
import junit.framework.TestCase;
/**
* Test for the ColumnPaginationFilter, used mainly to test the successful serialization of the filter.
* More test functionality can be found within {@link org.apache.hadoop.hbase.filter.TestFilter#testColumnPaginationFilter()}
*/
public class TestColumnPaginationFilter extends TestCase
{
private static final byte[] ROW = Bytes.toBytes("row_1_test");
private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test");
private static final byte[] VAL_1 = Bytes.toBytes("a");
private static final byte [] COLUMN_QUALIFIER = Bytes.toBytes("foo");
private Filter columnPaginationFilter;
@Override
protected void setUp() throws Exception {
super.setUp();
columnPaginationFilter = getColumnPaginationFilter();
}
private Filter getColumnPaginationFilter() {
return new ColumnPaginationFilter(1,0);
}
private Filter serializationTest(Filter filter) throws Exception {
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(stream);
filter.write(out);
out.close();
byte[] buffer = stream.toByteArray();
DataInputStream in =
new DataInputStream(new ByteArrayInputStream(buffer));
Filter newFilter = new ColumnPaginationFilter();
newFilter.readFields(in);
return newFilter;
}
/**
* The more specific functionality tests are contained within the TestFilters class. This class is mainly for testing
* serialization
*
* @param filter
* @throws Exception
*/
private void basicFilterTests(ColumnPaginationFilter filter) throws Exception
{
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
}
/**
* Tests serialization
* @throws Exception
*/
public void testSerialization() throws Exception {
Filter newFilter = serializationTest(columnPaginationFilter);
basicFilterTests((ColumnPaginationFilter)newFilter);
}
}

View File

@ -1103,4 +1103,97 @@ public class TestFilter extends HBaseTestCase {
assertEquals("Expected " + kvs.length + " total keys but scanned " + idx, assertEquals("Expected " + kvs.length + " total keys but scanned " + idx,
kvs.length, 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);
}
} }