HBASE-1582 Translate ColumnValueFilter and RowFilterSet to the new Filter interface

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@789557 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-06-30 04:44:33 +00:00
parent ac9b0d924f
commit a8821d10f7
24 changed files with 776 additions and 82 deletions

View File

@ -233,6 +233,8 @@ Release 0.20.0 - Unreleased
HBASE-1580 Store scanner does not consult filter.filterRow at end of scan
(Clint Morgan via Stack)
HBASE-1437 broken links in hbase.org
HBASE-1582 Translate ColumnValueFilter and RowFilterSet to the new Filter
interface
IMPROVEMENTS
HBASE-1089 Add count of regions on filesystem to master UI; add percentage

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.io.Writable;
/**
* Single row result of a {@link Get} or {@link Scan} query.
* Backed by array of KeyValues.
*/
public class Result implements Writable {
private KeyValue [] kvs = null;
@ -84,7 +85,7 @@ public class Result implements Writable {
}
/**
* Directly return the unsorted array of KeyValues in this Result.
* Return the unsorted array of KeyValues backing this Result instance.
* @return unsorted array of KeyValues
*/
public KeyValue[] raw() {
@ -92,7 +93,7 @@ public class Result implements Writable {
}
/**
* Return a sorted list of the KeyValue's in this result.
* Create a sorted list of the KeyValue's in this result.
*
* @return The sorted list of KeyValue's.
*/
@ -108,7 +109,7 @@ public class Result implements Writable {
* @return sorted array of KeyValues
*/
public KeyValue[] sorted() {
if(isEmpty()) {
if (isEmpty()) {
return null;
}
Arrays.sort(kvs, (Comparator<KeyValue>)KeyValue.COMPARATOR);
@ -230,7 +231,7 @@ public class Result implements Writable {
*/
public byte [] getValue(byte [] family, byte [] qualifier) {
Map.Entry<Long,byte[]> entry = getKeyValue(family, qualifier);
return entry == null ?null :entry.getValue();
return entry == null? null: entry.getValue();
}
public Cell getCellValue(byte[] family, byte[] qualifier) {
@ -281,11 +282,8 @@ public class Result implements Writable {
private NavigableMap<Long, byte[]> getVersionMap(
NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap, byte [] qualifier) {
if(qualifier != null) {
return qualifierMap.get(qualifier);
} else {
return qualifierMap.get(new byte[0]);
}
return qualifier != null?
qualifierMap.get(qualifier): qualifierMap.get(new byte[0]);
}
/**
@ -341,7 +339,7 @@ public class Result implements Writable {
* @return value of the first column
*/
public byte [] value() {
if(isEmpty()) {
if (isEmpty()) {
return null;
}
return kvs[0].getValue();
@ -352,15 +350,14 @@ public class Result implements Writable {
* @return true if empty
*/
public boolean isEmpty() {
return (this.kvs == null || this.kvs.length == 0);
return this.kvs == null || this.kvs.length == 0;
}
/**
*
* @return the size of the underlying KeyValue []
*/
public int size() {
return (this.kvs == null ? 0 : this.kvs.length);
return this.kvs == null? 0: this.kvs.length;
}
/**

View File

@ -22,9 +22,14 @@ Provides HBase Client
<h2>Table of Contents</h2>
<ul>
<li><a href="#overview">Overview</a></li>
<li><a href="#client_example">Example API Usage</a></li>
</ul>
<h2><a name="overview">Overview</a></h2>
<p>
</p>
<h2><a name="client_example">Example API Usage</a></h2>
<p>Once you have a running HBase, you probably want a way to hook your application up to it.

View File

@ -0,0 +1,217 @@
/**
* 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 java.util.HashSet;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
/**
* Implementation of {@link Filter} that represents a Set of Filters
* which will be evaluated with a specified boolean operator MUST_PASS_ALL
* (!AND) or MUST_PASS_ONE (!OR). Since you can use Filter Sets as children
* of Filter Sets, you can create a hierarchy of filters to be evaluated.
* <p>TODO: Fix creation of Configuration on serialization and deserialization.
*/
public class FilterSet implements Filter {
/** set operator */
public static enum Operator {
/** !AND */
MUST_PASS_ALL,
/** !OR */
MUST_PASS_ONE
}
private Operator operator = Operator.MUST_PASS_ALL;
private Set<Filter> filters = new HashSet<Filter>();
/**
* Default constructor, filters nothing. Required though for RPC
* deserialization.
*/
public FilterSet() {
super();
}
/**
* Constructor that takes a set of {@link Filter}s. The default operator
* MUST_PASS_ALL is assumed.
*
* @param rowFilters
*/
public FilterSet(final Set<Filter> rowFilters) {
this.filters = rowFilters;
}
/**
* Constructor that takes a set of {@link Filter}s and an operator.
*
* @param operator Operator to process filter set with.
* @param rowFilters Set of row filters.
*/
public FilterSet(final Operator operator, final Set<Filter> rowFilters) {
this.filters = rowFilters;
this.operator = operator;
}
/**
* Get the operator.
*
* @return operator
*/
public Operator getOperator() {
return operator;
}
/**
* Get the filters.
*
* @return filters
*/
public Set<Filter> getFilters() {
return filters;
}
/**
* Add a filter.
*
* @param filter
*/
public void addFilter(Filter filter) {
this.filters.add(filter);
}
public void reset() {
for (Filter filter : filters) {
filter.reset();
}
}
public boolean filterRowKey(byte[] rowKey, int offset, int length) {
for (Filter filter : filters) {
if (operator == Operator.MUST_PASS_ALL) {
if (filter.filterAllRemaining()
|| filter.filterRowKey(rowKey, offset, length)) {
return true;
}
} else if (operator == Operator.MUST_PASS_ONE) {
if (!filter.filterAllRemaining()
&& !filter.filterRowKey(rowKey, offset, length)) {
return false;
}
}
}
return operator == Operator.MUST_PASS_ONE;
}
public boolean filterAllRemaining() {
for (Filter filter : filters) {
if (filter.filterAllRemaining()) {
if (operator == Operator.MUST_PASS_ALL) {
return true;
}
} else {
if (operator == Operator.MUST_PASS_ONE) {
return false;
}
}
}
return operator == Operator.MUST_PASS_ONE;
}
public ReturnCode filterKeyValue(KeyValue v) {
for (Filter filter : filters) {
if (operator == Operator.MUST_PASS_ALL) {
if (filter.filterAllRemaining()) {
return ReturnCode.NEXT_ROW;
}
switch (filter.filterKeyValue(v)) {
case INCLUDE:
continue;
case NEXT_ROW:
case SKIP:
return ReturnCode.SKIP;
}
} else if (operator == Operator.MUST_PASS_ONE) {
if (filter.filterAllRemaining()) {
continue;
}
switch (filter.filterKeyValue(v)) {
case INCLUDE:
return ReturnCode.INCLUDE;
case NEXT_ROW:
case SKIP:
continue;
}
}
}
return operator == Operator.MUST_PASS_ONE?
ReturnCode.SKIP: ReturnCode.INCLUDE;
}
public boolean filterRow() {
for (Filter filter : filters) {
if (operator == Operator.MUST_PASS_ALL) {
if (filter.filterAllRemaining()
|| filter.filterRow()) {
return true;
}
} else if (operator == Operator.MUST_PASS_ONE) {
if (!filter.filterAllRemaining()
&& !filter.filterRow()) {
return false;
}
}
}
return operator == Operator.MUST_PASS_ONE;
}
public void readFields(final DataInput in) throws IOException {
Configuration conf = new HBaseConfiguration();
byte opByte = in.readByte();
operator = Operator.values()[opByte];
int size = in.readInt();
if (size > 0) {
filters = new HashSet<Filter>();
for (int i = 0; i < size; i++) {
Filter filter = (Filter)HbaseObjectWritable.readObject(in, conf);
filters.add(filter);
}
}
}
public void write(final DataOutput out) throws IOException {
Configuration conf = new HBaseConfiguration();
out.writeByte(operator.ordinal());
out.writeInt(filters.size());
for (Filter filter : filters) {
HbaseObjectWritable.writeObject(out, filter, filter.getClass(), conf);
}
}
}

View File

@ -33,23 +33,21 @@ import java.io.DataInput;
*
* Use this filter to include the stop row, eg: [A,Z].
*/
public class RowInclusiveStopFilter implements Filter {
public class InclusiveStopFilter implements Filter {
private byte [] stopRowKey;
public RowInclusiveStopFilter() {
public InclusiveStopFilter() {
super();
}
public RowInclusiveStopFilter(final byte [] stopRowKey) {
public InclusiveStopFilter(final byte [] stopRowKey) {
this.stopRowKey = stopRowKey;
}
@Override
public void reset() {
// noop, no state
}
@Override
public boolean filterRowKey(byte[] buffer, int offset, int length) {
if (buffer == null) {
if (this.stopRowKey == null) {
@ -61,28 +59,23 @@ public class RowInclusiveStopFilter implements Filter {
return Bytes.compareTo(stopRowKey, 0, stopRowKey.length, buffer, offset, length) < 0;
}
@Override
public boolean filterAllRemaining() {
return false;
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
// include everything.
return ReturnCode.INCLUDE;
}
@Override
public boolean filterRow() {
return false;
}
@Override
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.stopRowKey);
}
@Override
public void readFields(DataInput in) throws IOException {
this.stopRowKey = Bytes.readByteArray(in);
}

View File

@ -79,13 +79,11 @@ public class PageFilter implements Filter {
out.writeLong(pageSize);
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
this.rowsAccepted++;
return filterAllRemaining()? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
}
@Override
public boolean filterRow() {
return filterAllRemaining();
}

View File

@ -0,0 +1,78 @@
/*
* 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.util.Bytes;
import java.io.DataOutput;
import java.io.IOException;
import java.io.DataInput;
/**
* Pass results that have same row prefix.
*/
public class PrefixFilter implements Filter {
protected byte [] prefix = null;
public PrefixFilter(final byte [] prefix) {
this.prefix = prefix;
}
public PrefixFilter() {
super();
}
public void reset() {
// Noop
}
public boolean filterRowKey(byte[] buffer, int offset, int length) {
if (buffer == null || this.prefix == null)
return true;
if (length < prefix.length)
return true;
// if they are equal, return false => pass row
// else return true, filter row
return Bytes.compareTo(buffer, offset, this.prefix.length, this.prefix, 0,
this.prefix.length) != 0;
}
public boolean filterAllRemaining() {
return false;
}
public ReturnCode filterKeyValue(KeyValue v) {
return ReturnCode.INCLUDE;
}
public boolean filterRow() {
return false;
}
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.prefix);
}
public void readFields(DataInput in) throws IOException {
this.prefix = Bytes.readByteArray(in);
}
}

View File

@ -50,11 +50,11 @@ import org.apache.hadoop.hbase.util.Bytes;
* </pre>
*/
public class RegexStringComparator implements WritableByteArrayComparable {
private Pattern pattern;
/** Nullary constructor for Writable */
public RegexStringComparator() {
super();
}
/**
@ -78,5 +78,4 @@ public class RegexStringComparator implements WritableByteArrayComparable {
public void write(DataOutput out) throws IOException {
out.writeUTF(pattern.toString());
}
}

View File

@ -41,12 +41,10 @@ public class RowPrefixFilter implements Filter {
super();
}
@Override
public void reset() {
// Noop
}
@Override
public boolean filterRowKey(byte[] buffer, int offset, int length) {
if (buffer == null || this.prefix == null)
return true;
@ -58,27 +56,22 @@ public class RowPrefixFilter implements Filter {
this.prefix.length) != 0;
}
@Override
public boolean filterAllRemaining() {
return false;
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
return ReturnCode.INCLUDE;
}
@Override
public boolean filterRow() {
return false;
}
@Override
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.prefix);
}
@Override
public void readFields(DataInput in) throws IOException {
this.prefix = Bytes.readByteArray(in);
}

View File

@ -0,0 +1,226 @@
/**
* Copyright 2008 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 java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.ObjectWritable;
/**
* This filter is used to filter based on the value of a given column. It takes
* an operator (equal, greater, not equal, etc) and either a byte [] value or a
* byte [] comparator. If we have a byte [] value then we just do a
* lexicographic compare. If this is not sufficient (eg you want to deserialize
* a long and then compare it to a fixed long value), then you can pass in your
* own comparator instead.
* */
public class ValueFilter implements Filter {
static final Log LOG = LogFactory.getLog(ValueFilter.class);
/** Comparison operators. */
public enum CompareOp {
/** less than */
LESS,
/** less than or equal to */
LESS_OR_EQUAL,
/** equals */
EQUAL,
/** not equal */
NOT_EQUAL,
/** greater than or equal to */
GREATER_OR_EQUAL,
/** greater than */
GREATER;
}
private byte[] columnName;
private CompareOp compareOp;
private byte[] value;
private WritableByteArrayComparable comparator;
private boolean filterIfColumnMissing;
ValueFilter() {
// for Writable
}
/**
* Constructor.
*
* @param columnName name of column
* @param compareOp operator
* @param value value to compare column values against
*/
public ValueFilter(final byte[] columnName, final CompareOp compareOp,
final byte[] value) {
this(columnName, compareOp, value, true);
}
/**
* Constructor.
*
* @param columnName name of column
* @param compareOp operator
* @param value value to compare column values against
* @param filterIfColumnMissing if true then we will filter rows that don't
* have the column.
*/
public ValueFilter(final byte[] columnName, final CompareOp compareOp,
final byte[] value, boolean filterIfColumnMissing) {
this.columnName = columnName;
this.compareOp = compareOp;
this.value = value;
this.filterIfColumnMissing = filterIfColumnMissing;
}
/**
* Constructor.
*
* @param columnName name of column
* @param compareOp operator
* @param comparator Comparator to use.
*/
public ValueFilter(final byte[] columnName, final CompareOp compareOp,
final WritableByteArrayComparable comparator) {
this(columnName, compareOp, comparator, true);
}
/**
* Constructor.
*
* @param columnName name of column
* @param compareOp operator
* @param comparator Comparator to use.
* @param filterIfColumnMissing if true then we will filter rows that don't
* have the column.
*/
public ValueFilter(final byte[] columnName, final CompareOp compareOp,
final WritableByteArrayComparable comparator,
boolean filterIfColumnMissing) {
this.columnName = columnName;
this.compareOp = compareOp;
this.comparator = comparator;
this.filterIfColumnMissing = filterIfColumnMissing;
}
public boolean filterRowKey(byte[] rowKey, int offset, int length) {
return false;
}
private boolean filterThisRow = false;
private boolean foundColValue = false;
public ReturnCode filterKeyValue(KeyValue keyValue) {
if (Bytes.compareTo(keyValue.getColumn(), this.columnName) != 0) {
return ReturnCode.INCLUDE;
}
LOG.info("Found column [" + Bytes.toString(columnName) + "] in row ["
+ Bytes.toString(keyValue.getRow()) + "]");
foundColValue = true;
boolean filtered = filterColumnValue(keyValue.getBuffer(), keyValue
.getValueOffset(), keyValue.getValueLength());
if (filtered) {
LOG.info("filtered it");
filterThisRow = true;
return ReturnCode.NEXT_ROW;
}
return ReturnCode.INCLUDE;
}
private boolean filterColumnValue(final byte[] data, final int offset,
final int length) {
int compareResult;
if (comparator != null) {
compareResult = comparator.compareTo(Arrays.copyOfRange(data, offset,
offset + length));
} else {
compareResult = Bytes.compareTo(value, 0, value.length, data, offset,
length);
}
switch (compareOp) {
case LESS:
return compareResult <= 0;
case LESS_OR_EQUAL:
return compareResult < 0;
case EQUAL:
return compareResult != 0;
case NOT_EQUAL:
return compareResult == 0;
case GREATER_OR_EQUAL:
return compareResult > 0;
case GREATER:
return compareResult >= 0;
default:
throw new RuntimeException("Unknown Compare op " + compareOp.name());
}
}
public boolean filterAllRemaining() {
return false;
}
public boolean filterRow() {
boolean result = filterThisRow || (filterIfColumnMissing && !foundColValue);
filterThisRow = false;
foundColValue = false;
LOG.info("Deciding " + (result ? "" : " not ") + "to filter");
return result;
}
public void reset() {
// Nothing.
}
public void readFields(final DataInput in) throws IOException {
int valueLen = in.readInt();
if (valueLen > 0) {
value = new byte[valueLen];
in.readFully(value);
}
columnName = Bytes.readByteArray(in);
compareOp = CompareOp.valueOf(in.readUTF());
comparator = (WritableByteArrayComparable) ObjectWritable.readObject(in,
new HBaseConfiguration());
filterIfColumnMissing = in.readBoolean();
}
public void write(final DataOutput out) throws IOException {
if (value == null) {
out.writeInt(0);
} else {
out.writeInt(value.length);
out.write(value);
}
Bytes.writeByteArray(out, columnName);
out.writeUTF(compareOp.name());
ObjectWritable.writeObject(out, comparator,
WritableByteArrayComparable.class, new HBaseConfiguration());
out.writeBoolean(filterIfColumnMissing);
}
}

View File

@ -29,19 +29,18 @@ import java.io.DataInput;
/**
* A wrapper filter that filters everything after the first filtered row.
*/
public class RowWhileMatchFilter implements Filter {
public class WhileMatchFilter implements Filter {
private boolean filterAllRemaining = false;
private Filter filter;
public RowWhileMatchFilter() {
public WhileMatchFilter() {
super();
}
public RowWhileMatchFilter(Filter filter) {
public WhileMatchFilter(Filter filter) {
this.filter = filter;
}
@Override
public void reset() {
// no state.
}
@ -50,36 +49,30 @@ public class RowWhileMatchFilter implements Filter {
filterAllRemaining = filterAllRemaining || value;
}
@Override
public boolean filterRowKey(byte[] buffer, int offset, int length) {
changeFAR(filter.filterRowKey(buffer, offset, length));
return filterAllRemaining();
}
@Override
public boolean filterAllRemaining() {
return this.filterAllRemaining || this.filter.filterAllRemaining();
}
@Override
public ReturnCode filterKeyValue(KeyValue v) {
ReturnCode c = filter.filterKeyValue(v);
changeFAR(c != ReturnCode.INCLUDE);
return c;
}
@Override
public boolean filterRow() {
return false;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeUTF(this.filter.getClass().getName());
this.filter.write(out);
}
@Override
public void readFields(DataInput in) throws IOException {
String className = in.readUTF();
try {

View File

@ -23,7 +23,7 @@
<p>Since HBase 0.20.0, {@link org.apache.hadoop.hbase.filter.Filter} is the new Interface used filtering.
It replaces the deprecated {@link org.apache.hadoop.hbase.filter.RowFilterInterface}.
Filters run the extent of a table unless you wrap your filter in a
{@link org.apache.hadoop.hbase.filter.RowWhileMatchFilter}.
{@link WhileMatchFilter}.
The latter returns as soon as the filter stops matching.
</p>
*/

View File

@ -44,21 +44,14 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.filter.RowFilterSet;
import org.apache.hadoop.hbase.filter.RowWhileMatchFilter;
import org.apache.hadoop.hbase.filter.RowPrefixFilter;
import org.apache.hadoop.hbase.filter.PageFilter;
import org.apache.hadoop.hbase.filter.RowInclusiveStopFilter;
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.hbase.filter.*;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.io.MapWritable;
import org.apache.hadoop.io.ObjectWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
@ -145,10 +138,10 @@ public class HbaseObjectWritable implements Writable, Configurable {
addToMap(Result[].class, code++);
addToMap(Scan.class, code++);
addToMap(RowWhileMatchFilter.class, code++);
addToMap(RowPrefixFilter.class, code++);
addToMap(WhileMatchFilter.class, code++);
addToMap(PrefixFilter.class, code++);
addToMap(PageFilter.class, code++);
addToMap(RowInclusiveStopFilter.class, code++);
addToMap(InclusiveStopFilter.class, code++);
addToMap(ColumnCountGetFilter.class, code++);
}

View File

@ -27,13 +27,9 @@ import java.rmi.UnexpectedException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.SortedMap;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.logging.Log;

View File

@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.filter.PageFilter;
import org.apache.hadoop.hbase.filter.RowWhileMatchFilter;
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Hash;
@ -436,7 +436,7 @@ public class PerformanceEvaluation implements HConstants {
void testRow(final int i) throws IOException {
Scan scan = new Scan(getRandomRow(this.rand, this.totalRows));
scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
scan.setFilter(new RowWhileMatchFilter(new PageFilter(120)));
scan.setFilter(new WhileMatchFilter(new PageFilter(120)));
ResultScanner s = this.table.getScanner(scan);
//int count = 0;
for (Result rr = null; (rr = s.next()) != null;) {

View File

@ -0,0 +1,111 @@
/**
* 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 java.util.HashSet;
import java.util.Set;
import org.apache.hadoop.hbase.util.Bytes;
import junit.framework.TestCase;
/**
* Tests filter sets
*
*/
public class TestFilterSet extends TestCase {
static final int MAX_PAGES = 5;
static final char FIRST_CHAR = 'a';
static final char LAST_CHAR = 'e';
static byte[] GOOD_BYTES = Bytes.toBytes("abc");
static byte[] BAD_BYTES = Bytes.toBytes("def");
/**
* Test "must pass one"
* @throws Exception
*/
public void testMPONE() throws Exception {
Set<Filter> filters = new HashSet<Filter>();
filters.add(new PageFilter(MAX_PAGES));
filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
Filter filterMPONE =
new FilterSet(FilterSet.Operator.MUST_PASS_ONE, filters);
/* Filter must do all below steps:
* <ul>
* <li>{@link #reset()}</li>
* <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
* <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
* if false, we will also call</li>
* <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
* <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
* filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
* </li>
* </ul>
*/
filterMPONE.reset();
assertFalse(filterMPONE.filterAllRemaining());
byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
}
/**
* Test "must pass all"
* @throws Exception
*/
public void testMPALL() throws Exception {
Set<Filter> filters = new HashSet<Filter>();
filters.add(new PageFilter(MAX_PAGES));
filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
Filter filterMPALL =
new FilterSet(FilterSet.Operator.MUST_PASS_ALL, filters);
}
/**
* Test serialization
* @throws Exception
*/
public void testSerialization() throws Exception {
Set<Filter> filters = new HashSet<Filter>();
filters.add(new PageFilter(MAX_PAGES));
filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
Filter filterMPALL =
new FilterSet(FilterSet.Operator.MUST_PASS_ALL, filters);
// Decompose filterMPALL to bytes.
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(stream);
filterMPALL.write(out);
out.close();
byte[] buffer = stream.toByteArray();
// Recompose filterMPALL.
DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
FilterSet newFilter = new FilterSet();
newFilter.readFields(in);
// TODO: Run TESTS!!!
}
}

View File

@ -0,0 +1,100 @@
/*
* 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 junit.framework.TestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.UnsupportedEncodingException;
public class TestPrefixFilter extends TestCase {
Filter mainFilter;
static final char FIRST_CHAR = 'a';
static final char LAST_CHAR = 'e';
static final String HOST_PREFIX = "org.apache.site-";
static byte [] GOOD_BYTES = null;
static {
try {
GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
} catch (UnsupportedEncodingException e) {
fail();
}
}
protected void setUp() throws Exception {
super.setUp();
this.mainFilter = new PrefixFilter(Bytes.toBytes(HOST_PREFIX));
}
public void testPrefixOnRow() throws Exception {
prefixRowTests(mainFilter);
}
public void testPrefixOnRowInsideWhileMatchRow() throws Exception {
prefixRowTests(new WhileMatchFilter(this.mainFilter), true);
}
public void testSerialization() throws Exception {
// Decompose mainFilter to bytes.
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(stream);
mainFilter.write(out);
out.close();
byte[] buffer = stream.toByteArray();
// Recompose filter.
DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
Filter newFilter = new PrefixFilter();
newFilter.readFields(in);
// Ensure the serialization preserved the filter by running all test.
prefixRowTests(newFilter);
}
private void prefixRowTests(Filter filter) throws Exception {
prefixRowTests(filter, false);
}
private void prefixRowTests(Filter filter, boolean lastFilterAllRemaining)
throws Exception {
for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
byte [] t = createRow(c);
assertFalse("Failed with character " + c,
filter.filterRowKey(t, 0, t.length));
assertFalse(filter.filterAllRemaining());
}
String yahooSite = "com.yahoo.www";
byte [] yahooSiteBytes = Bytes.toBytes(yahooSite);
assertTrue("Failed with character " +
yahooSite, filter.filterRowKey(yahooSiteBytes, 0, yahooSiteBytes.length));
assertEquals(filter.filterAllRemaining(), lastFilterAllRemaining);
}
private byte [] createRow(final char c) {
return Bytes.toBytes(HOST_PREFIX + Character.toString(c));
}
}

View File

@ -55,7 +55,7 @@ public class TestRowPrefixFilter extends TestCase {
}
public void testPrefixOnRowInsideWhileMatchRow() throws Exception {
prefixRowTests(new RowWhileMatchFilter(this.mainFilter), true);
prefixRowTests(new WhileMatchFilter(this.mainFilter), true);
}
public void testSerialization() throws Exception {

View File

@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.filter.StopRowFilter;
import org.apache.hadoop.hbase.filter.RowPrefixFilter;
import org.apache.hadoop.hbase.filter.PrefixFilter;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.WritableComparator;
@ -81,9 +81,9 @@ public class TestHbaseObjectWritable extends TestCase {
RowFilterInterface.class);
assertTrue(obj instanceof StopRowFilter);
// Check that filters can be serialized
obj = doType(conf, new RowPrefixFilter(HConstants.EMPTY_BYTE_ARRAY),
RowPrefixFilter.class);
assertTrue(obj instanceof RowPrefixFilter);
obj = doType(conf, new PrefixFilter(HConstants.EMPTY_BYTE_ARRAY),
PrefixFilter.class);
assertTrue(obj instanceof PrefixFilter);
}
private Object doType(final HBaseConfiguration conf, final Object value,

View File

@ -3,7 +3,6 @@ package org.apache.hadoop.hbase.io;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;

View File

@ -24,14 +24,8 @@ import junit.framework.TestCase;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.RowInclusiveStopFilter;
import org.apache.hadoop.hbase.filter.RowWhileMatchFilter;
import org.apache.hadoop.hbase.filter.RowPrefixFilter;
import org.apache.hadoop.hbase.filter.RowFilterInterface;
import org.apache.hadoop.hbase.filter.WhileMatchRowFilter;
import org.apache.hadoop.hbase.filter.PrefixRowFilter;
import org.apache.hadoop.hbase.filter.InclusiveStopRowFilter;
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
import org.apache.hadoop.hbase.filter.*;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.IOException;
@ -402,8 +396,8 @@ public class TestStoreScanner extends TestCase {
// Test new and old row prefix filters.
public void testNewRowPrefixFilter() throws IOException {
Filter f = new RowWhileMatchFilter(
new RowPrefixFilter(Bytes.toBytes("R:")));
Filter f = new WhileMatchFilter(
new PrefixFilter(Bytes.toBytes("R:")));
Scan s = new Scan(Bytes.toBytes("R:7"));
s.setFilter(f);
@ -442,7 +436,7 @@ public class TestStoreScanner extends TestCase {
// Test new and old row-inclusive stop filter.
public void testNewRowInclusiveStopFilter() throws IOException {
Filter f = new RowWhileMatchFilter(new RowInclusiveStopFilter(Bytes.toBytes("R:3")));
Filter f = new WhileMatchFilter(new InclusiveStopFilter(Bytes.toBytes("R:3")));
Scan scan = new Scan();
scan.setFilter(f);