HBASE-2211 Add a new Filter that checks a single column value but does not emit it.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@909658 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-02-12 23:29:13 +00:00
parent 03c67da3c0
commit ba598c6bed
4 changed files with 169 additions and 2 deletions

View File

@ -0,0 +1,88 @@
/**
* Copyright 2009 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 org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
/**
* A {@link Filter} that checks a single column value, but does not emit the
* tested column. This will enable a performance boost over
* {@link SingleColumnValueFilter}, if the tested column value is not actually
* needed as input (besides for the filtering itself).
*/
public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
/**
* Writable constructor, do not use.
*/
public SingleColumnValueExcludeFilter() {
super();
}
/**
* Constructor for binary compare of the value of a single column. If the
* column is found and the condition passes, all columns of the row will be
* emitted; except for the tested column value. If the column is not found or
* the condition fails, the row will not be emitted.
*
* @param family name of column family
* @param qualifier name of column qualifier
* @param compareOp operator
* @param value value to compare column values against
*/
public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier,
CompareOp compareOp, byte[] value) {
super(family, qualifier, compareOp, value);
}
/**
* Constructor for binary compare of the value of a single column. If the
* column is found and the condition passes, all columns of the row will be
* emitted; except for the tested column value. If the condition fails, the
* row will not be emitted.
* <p>
* Use the filterIfColumnMissing flag to set whether the rest of the columns
* in a row will be emitted if the specified column to check is not found in
* the row.
*
* @param family name of column family
* @param qualifier name of column qualifier
* @param compareOp operator
* @param comparator Comparator to use.
*/
public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier,
CompareOp compareOp, WritableByteArrayComparable comparator) {
super(family, qualifier, compareOp, comparator);
}
public ReturnCode filterKeyValue(KeyValue keyValue) {
ReturnCode superRetCode = super.filterKeyValue(keyValue);
if (superRetCode == ReturnCode.INCLUDE) {
// If the current column is actually the tested column,
// we will skip it instead.
if (keyValue.matchingColumn(this.columnFamily, this.columnQualifier)) {
return ReturnCode.SKIP;
}
}
return superRetCode;
}
}

View File

@ -63,8 +63,8 @@ import org.apache.hadoop.hbase.util.Bytes;
public class SingleColumnValueFilter implements Filter {
static final Log LOG = LogFactory.getLog(SingleColumnValueFilter.class);
private byte [] columnFamily;
private byte [] columnQualifier;
protected byte [] columnFamily;
protected byte [] columnQualifier;
private CompareOp compareOp;
private WritableByteArrayComparable comparator;
private boolean foundColumn = false;

View File

@ -149,6 +149,7 @@ public class HbaseObjectWritable implements Writable, Configurable {
addToMap(InclusiveStopFilter.class, code++);
addToMap(ColumnCountGetFilter.class, code++);
addToMap(SingleColumnValueFilter.class, code++);
addToMap(SingleColumnValueExcludeFilter.class, code++);
addToMap(BinaryComparator.class, code++);
addToMap(CompareFilter.class, code++);
addToMap(RowFilter.class, code++);

View File

@ -0,0 +1,78 @@
/**
* 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 junit.framework.TestCase;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Tests for {@link SingleColumnValueExcludeFilter}. Because this filter
* extends {@link SingleColumnValueFilter}, only the added functionality is
* tested. That is, method filterKeyValue(KeyValue).
*
* @author ferdy
*
*/
public class TestSingleColumnValueExcludeFilter extends TestCase {
private static final byte[] ROW = Bytes.toBytes("test");
private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test");
private static final byte[] COLUMN_QUALIFIER = Bytes.toBytes("foo");
private static final byte[] COLUMN_QUALIFIER_2 = Bytes.toBytes("foo_2");
private static final byte[] VAL_1 = Bytes.toBytes("a");
private static final byte[] VAL_2 = Bytes.toBytes("ab");
/**
* Test the overridden functionality of filterKeyValue(KeyValue)
* @throws Exception
*/
public void testFilterKeyValue() throws Exception {
Filter filter = new SingleColumnValueExcludeFilter(COLUMN_FAMILY, COLUMN_QUALIFIER,
CompareOp.EQUAL, VAL_1);
// A 'match' situation
KeyValue kv;
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
// INCLUDE expected because test column has not yet passed
assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
// Test column will pass (will match), will SKIP because test columns are excluded
assertTrue("testedMatch", filter.filterKeyValue(kv) == Filter.ReturnCode.SKIP);
// Test column has already passed and matched, all subsequent columns are INCLUDE
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
assertFalse("allRemainingWhenMatch", filter.filterAllRemaining());
// A 'mismatch' situation
filter.reset();
// INCLUDE expected because test column has not yet passed
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
// Test column will pass (wont match), expect NEXT_ROW
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
assertTrue("testedMismatch", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
// After a mismatch (at least with LatestVersionOnly), subsequent columns are EXCLUDE
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("otherColumn", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
}
}