HBASE-19950 Introduce a ColumnValueFilter
Signed-off-by: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
a458d7c400
commit
4ef6319af0
|
@ -0,0 +1,241 @@
|
|||
/**
|
||||
*
|
||||
* 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.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
|
||||
|
||||
/**
|
||||
* Different from {@link SingleColumnValueFilter} which returns an <b>entire</b> row
|
||||
* when specified condition is matched, {@link ColumnValueFilter} return the matched cell only.
|
||||
* <p>
|
||||
* This filter is used to filter cells based on column and value.
|
||||
* It takes a {@link org.apache.hadoop.hbase.CompareOperator} operator (<, <=, =, !=, >, >=), and
|
||||
* and a {@link ByteArrayComparable} comparator.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
public class ColumnValueFilter extends FilterBase {
|
||||
private final byte[] family;
|
||||
private final byte[] qualifier;
|
||||
private final CompareOperator op;
|
||||
private final ByteArrayComparable comparator;
|
||||
|
||||
// This flag is used to speed up seeking cells when matched column is found, such that following
|
||||
// columns in the same row can be skipped faster by NEXT_ROW instead of NEXT_COL.
|
||||
private boolean columnFound = false;
|
||||
|
||||
public ColumnValueFilter(final byte[] family, final byte[] qualifier,
|
||||
final CompareOperator op, final byte[] value) {
|
||||
this(family, qualifier, op, new BinaryComparator(value));
|
||||
}
|
||||
|
||||
public ColumnValueFilter(final byte[] family, final byte[] qualifier,
|
||||
final CompareOperator op,
|
||||
final ByteArrayComparable comparator) {
|
||||
this.family = Preconditions.checkNotNull(family, "family should not be null.");
|
||||
this.qualifier = qualifier == null ? new byte[0] : qualifier;
|
||||
this.op = Preconditions.checkNotNull(op, "CompareOperator should not be null");
|
||||
this.comparator = Preconditions.checkNotNull(comparator, "Comparator should not be null");
|
||||
}
|
||||
|
||||
/**
|
||||
* @return operator
|
||||
*/
|
||||
public CompareOperator getCompareOperator() {
|
||||
return op;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the comparator
|
||||
*/
|
||||
public ByteArrayComparable getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the column family
|
||||
*/
|
||||
public byte[] getFamily() {
|
||||
return family;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the qualifier
|
||||
*/
|
||||
public byte[] getQualifier() {
|
||||
return qualifier;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
columnFound = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean filterRowKey(Cell cell) throws IOException {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReturnCode filterCell(Cell c) throws IOException {
|
||||
// 1. Check column match
|
||||
if (!CellUtil.matchingColumn(c, this.family, this.qualifier)) {
|
||||
return columnFound ? ReturnCode.NEXT_ROW : ReturnCode.NEXT_COL;
|
||||
}
|
||||
// Column found
|
||||
columnFound = true;
|
||||
// 2. Check value match:
|
||||
// True means filter out, just skip this cell, else include it.
|
||||
return compareValue(getCompareOperator(), getComparator(), c) ?
|
||||
ReturnCode.SKIP : ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is used to determine a cell should be included or filtered out.
|
||||
* @param op one of operators {@link CompareOperator}
|
||||
* @param comparator comparator used to compare cells.
|
||||
* @param cell cell to be compared.
|
||||
* @return true means cell should be filtered out, included otherwise.
|
||||
*/
|
||||
private boolean compareValue(final CompareOperator op, final ByteArrayComparable comparator,
|
||||
final Cell cell) {
|
||||
if (op == CompareOperator.NO_OP) {
|
||||
return true;
|
||||
}
|
||||
int compareResult = PrivateCellUtil.compareValue(cell, comparator);
|
||||
return CompareFilter.compare(op, compareResult);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creating this filter by reflection, it is used by {@link ParseFilter},
|
||||
* @param filterArguments arguments for creating a ColumnValueFilter
|
||||
* @return a ColumnValueFilter
|
||||
*/
|
||||
public static Filter createFilterFromArguments(ArrayList<byte[]> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 4,
|
||||
"Expect 4 arguments: %s", filterArguments.size());
|
||||
byte[] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
|
||||
byte[] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
|
||||
CompareOperator operator = ParseFilter.createCompareOperator(filterArguments.get(2));
|
||||
ByteArrayComparable comparator =
|
||||
ParseFilter.createComparator(ParseFilter.removeQuotesFromByteArray(filterArguments.get(3)));
|
||||
|
||||
if (comparator instanceof RegexStringComparator ||
|
||||
comparator instanceof SubstringComparator) {
|
||||
if (operator != CompareOperator.EQUAL &&
|
||||
operator != CompareOperator.NOT_EQUAL) {
|
||||
throw new IllegalArgumentException("A regexstring comparator and substring comparator " +
|
||||
"can only be used with EQUAL and NOT_EQUAL");
|
||||
}
|
||||
}
|
||||
|
||||
return new ColumnValueFilter(family, qualifier, operator, comparator);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return A pb instance to represent this instance.
|
||||
*/
|
||||
FilterProtos.ColumnValueFilter convert() {
|
||||
FilterProtos.ColumnValueFilter.Builder builder =
|
||||
FilterProtos.ColumnValueFilter.newBuilder();
|
||||
|
||||
builder.setFamily(UnsafeByteOperations.unsafeWrap(this.family));
|
||||
builder.setQualifier(UnsafeByteOperations.unsafeWrap(this.qualifier));
|
||||
builder.setCompareOp(HBaseProtos.CompareType.valueOf(this.op.name()));
|
||||
builder.setComparator(ProtobufUtil.toComparator(this.comparator));
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse protobuf bytes to a ColumnValueFilter
|
||||
* @param pbBytes pbBytes
|
||||
* @return a ColumnValueFilter
|
||||
* @throws DeserializationException deserialization exception
|
||||
*/
|
||||
public static ColumnValueFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
|
||||
FilterProtos.ColumnValueFilter proto;
|
||||
try {
|
||||
proto = FilterProtos.ColumnValueFilter.parseFrom(pbBytes);
|
||||
} catch (InvalidProtocolBufferException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
|
||||
final CompareOperator compareOp = CompareOperator.valueOf(proto.getCompareOp().name());
|
||||
final ByteArrayComparable comparator;
|
||||
try {
|
||||
comparator = ProtobufUtil.toComparator(proto.getComparator());
|
||||
} catch (IOException ioe) {
|
||||
throw new DeserializationException(ioe);
|
||||
}
|
||||
|
||||
return new ColumnValueFilter(proto.getFamily().toByteArray(),
|
||||
proto.getQualifier().toByteArray(), compareOp, comparator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] toByteArray() throws IOException {
|
||||
return convert().toByteArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean areSerializedFieldsEqual(Filter o) {
|
||||
if (o == this) {
|
||||
return true;
|
||||
} else if (!(o instanceof ColumnValueFilter)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
ColumnValueFilter other = (ColumnValueFilter) o;
|
||||
return Bytes.equals(this.getFamily(), other.getFamily()) &&
|
||||
Bytes.equals(this.getQualifier(), other.getQualifier()) &&
|
||||
this.getCompareOperator().equals(other.getCompareOperator()) &&
|
||||
this.getComparator().areSerializedFieldsEqual(other.getComparator());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFamilyEssential(byte[] name) throws IOException {
|
||||
return Bytes.equals(name, this.family);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format("%s (%s, %s, %s, %s)",
|
||||
getClass().getSimpleName(), Bytes.toStringBinary(this.family),
|
||||
Bytes.toStringBinary(this.qualifier), this.op.name(),
|
||||
Bytes.toStringBinary(this.comparator.getValue()));
|
||||
}
|
||||
}
|
|
@ -25,23 +25,27 @@ import java.util.ArrayList;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
/**
|
||||
* This is a generic filter to be used to filter by comparison. It takes an
|
||||
* operator (equal, greater, not equal, etc) and a byte [] comparator.
|
||||
* <p>
|
||||
* To filter by row key, use {@link RowFilter}.
|
||||
* <p>
|
||||
* To filter by column family, use {@link FamilyFilter}.
|
||||
* <p>
|
||||
* To filter by column qualifier, use {@link QualifierFilter}.
|
||||
* <p>
|
||||
* To filter by value, use {@link SingleColumnValueFilter}.
|
||||
* To filter by value, use {@link ValueFilter}.
|
||||
* <p>
|
||||
* These filters can be wrapped with {@link SkipFilter} and {@link WhileMatchFilter}
|
||||
* to add more control.
|
||||
|
|
|
@ -93,6 +93,8 @@ public class ParseFilter {
|
|||
"SingleColumnValueExcludeFilter");
|
||||
filterHashMap.put("DependentColumnFilter", ParseConstants.FILTER_PACKAGE + "." +
|
||||
"DependentColumnFilter");
|
||||
filterHashMap.put("ColumnValueFilter", ParseConstants.FILTER_PACKAGE + "." +
|
||||
"ColumnValueFilter");
|
||||
|
||||
// Creates the operatorPrecedenceHashMap
|
||||
operatorPrecedenceHashMap = new HashMap<>();
|
||||
|
@ -769,8 +771,6 @@ public class ParseFilter {
|
|||
|
||||
/**
|
||||
* Takes a compareOperator symbol as a byte array and returns the corresponding CompareOperator
|
||||
* @deprecated Since 2.0
|
||||
* <p>
|
||||
* @param compareOpAsByteArray the comparatorOperator symbol as a byte array
|
||||
* @return the Compare Operator
|
||||
*/
|
||||
|
|
|
@ -170,3 +170,10 @@ message RowRange {
|
|||
message MultiRowRangeFilter {
|
||||
repeated RowRange row_range_list = 1;
|
||||
}
|
||||
|
||||
message ColumnValueFilter {
|
||||
required bytes family = 1;
|
||||
required bytes qualifier = 2;
|
||||
required CompareType compare_op = 3;
|
||||
required Comparator comparator = 4;
|
||||
}
|
||||
|
|
|
@ -168,4 +168,11 @@ message RowRange {
|
|||
|
||||
message MultiRowRangeFilter {
|
||||
repeated RowRange row_range_list = 1;
|
||||
}
|
||||
}
|
||||
|
||||
message ColumnValueFilter {
|
||||
required bytes family = 1;
|
||||
required bytes qualifier = 2;
|
||||
required CompareType compare_op = 3;
|
||||
required Comparator comparator = 4;
|
||||
}
|
||||
|
|
|
@ -132,6 +132,10 @@ public class TestFilter {
|
|||
Bytes.toBytes("testQualifierFour-2"), Bytes.toBytes("testQualifierFour-3")
|
||||
};
|
||||
|
||||
private static final byte [][] QUALIFIERS_FIVE = {
|
||||
Bytes.toBytes("testQualifierFive-0"), Bytes.toBytes("testQualifierFive-1")
|
||||
};
|
||||
|
||||
private static final byte [][] VALUES = {
|
||||
Bytes.toBytes("testValueOne"), Bytes.toBytes("testValueTwo")
|
||||
};
|
||||
|
@ -1663,6 +1667,157 @@ public class TestFilter {
|
|||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testColumnValueFilter() throws Exception {
|
||||
// Prepare test rows:
|
||||
for (int i = 0; i < 2; i++) {
|
||||
for (int j = 0; j < ROWS_ONE.length; j++) {
|
||||
Put p1 = new Put(ROWS_ONE[j]).setDurability(Durability.SKIP_WAL);
|
||||
Put p2 = new Put(ROWS_TWO[j]).setDurability(Durability.SKIP_WAL);
|
||||
for (byte[] q5 : QUALIFIERS_FIVE) {
|
||||
p1.addColumn(FAMILIES[0], q5, VALUES[0 + i]).addColumn(FAMILIES[1], q5, VALUES[0 + i]);
|
||||
p2.addColumn(FAMILIES[0], q5, VALUES[1 - i]).addColumn(FAMILIES[1], q5, VALUES[1 - i]);
|
||||
}
|
||||
this.region.put(p1);
|
||||
this.region.put(p2);
|
||||
}
|
||||
this.region.flush(true);
|
||||
}
|
||||
// 1. Test = f[0]:q5[0]:v[1]
|
||||
Scan scan = new Scan().setFilter(
|
||||
new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.EQUAL, VALUES[1]));
|
||||
KeyValue[] expectedEquals =
|
||||
{ new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) };
|
||||
verifyScanFull(scan, expectedEquals);
|
||||
// 2. Test > f[0]:q5[0]:v[0]
|
||||
scan.setFilter(
|
||||
new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.GREATER, VALUES[0]));
|
||||
KeyValue[] expectedGreater =
|
||||
{ new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) };
|
||||
verifyScanFull(scan, expectedGreater);
|
||||
// 3. Test >= f[0]:q5[0]:v[0]
|
||||
// also test readAllVersions(), since FAMILIES[0] allow multiple versions.
|
||||
scan.readAllVersions().setFilter(new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0],
|
||||
CompareOperator.GREATER_OR_EQUAL, VALUES[0]));
|
||||
KeyValue[] expectedGreaterOrEqual =
|
||||
{ new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) };
|
||||
verifyScanFull(scan, expectedGreaterOrEqual);
|
||||
// 4. Test < f[1]:q5[1]:v[1], FAMILIES[1] doesn't support multiple versions
|
||||
scan.readVersions(1).setFilter(new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1],
|
||||
CompareOperator.LESS, VALUES[1]));
|
||||
KeyValue[] expectedLess =
|
||||
{ new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
|
||||
verifyScanFull(scan, expectedLess);
|
||||
// 5. Test <= f[1]:q5[0]:v[1]
|
||||
scan.setFilter(new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1],
|
||||
CompareOperator.LESS_OR_EQUAL, VALUES[1]));
|
||||
KeyValue[] expectedLessOrEqual =
|
||||
{ new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
|
||||
verifyScanFull(scan, expectedLessOrEqual);
|
||||
// 6. Test != f[1]:q5[1]:v[1]
|
||||
scan.setFilter(
|
||||
new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1]));
|
||||
KeyValue[] expectedNotEqual =
|
||||
{ new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
|
||||
verifyScanFull(scan, expectedNotEqual);
|
||||
// 7. Test FilterList(MUST_PASS_ONE) combining ColumnValueFilter and QualifierFilter
|
||||
// (ColumnValueFilter, != f[1]:q5[1]:v[1]) || (QualifierFilter, = q5[0])
|
||||
List<Filter> orFilters = new ArrayList<>(2);
|
||||
orFilters.add(
|
||||
new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1]));
|
||||
orFilters.add(
|
||||
new QualifierFilter(CompareOperator.EQUAL, new BinaryComparator(QUALIFIERS_FIVE[0])));
|
||||
scan.setFilter(new FilterList(Operator.MUST_PASS_ONE, orFilters));
|
||||
KeyValue[] expectedMustPassOne =
|
||||
{ new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[1], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
|
||||
new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) }; // this pass scvf
|
||||
verifyScanFull(scan, expectedMustPassOne);
|
||||
// 8. Test FilterList(MUST_PASS_ALL) combining ColumnValueFilter and RowFilter
|
||||
// (ColumnValueFilter, != f[1]:q5[1]:v[1]) && (RowFilter, = prefix:"testRow")
|
||||
List<Filter> andFilters = new ArrayList<>(2);
|
||||
andFilters.add(
|
||||
new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1]));
|
||||
andFilters.add(new RowFilter(CompareOperator.EQUAL,
|
||||
new BinaryPrefixComparator(Bytes.toBytes("testRow"))));
|
||||
scan.setFilter(new FilterList(Operator.MUST_PASS_ALL, andFilters));
|
||||
KeyValue[] expectedMustPassAll =
|
||||
{ new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
|
||||
verifyScanFull(scan, expectedMustPassAll);
|
||||
// 9. Test specified columns with FilterList(MUST_PASS_ONE) which sused in case 7.
|
||||
// Result is different from case 7, because column is strongly constrained by specified columns
|
||||
Scan anotherScan = new Scan().addColumn(FAMILIES[1], QUALIFIERS_FIVE[1])
|
||||
.setFilter(new FilterList(Operator.MUST_PASS_ONE, orFilters));
|
||||
KeyValue[] expectedValues =
|
||||
{ new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
|
||||
new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
|
||||
verifyScanFull(anotherScan, expectedValues);
|
||||
}
|
||||
|
||||
private void verifyScan(Scan s, long expectedRows, long expectedKeys)
|
||||
throws IOException {
|
||||
InternalScanner scanner = this.region.getScanner(s);
|
||||
|
|
|
@ -341,4 +341,13 @@ public class TestFilterSerialization {
|
|||
assertTrue(multiRowRangeFilter.areSerializedFieldsEqual(
|
||||
ProtobufUtil.toFilter(ProtobufUtil.toFilter(multiRowRangeFilter))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testColumnValueFilter() throws Exception {
|
||||
ColumnValueFilter columnValueFilter =
|
||||
new ColumnValueFilter(Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
|
||||
CompareOperator.EQUAL, Bytes.toBytes("value"));
|
||||
assertTrue(columnValueFilter.areSerializedFieldsEqual(
|
||||
ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnValueFilter))));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -685,4 +685,15 @@ public class TestParseFilter {
|
|||
assertEquals(clazz, filter.getClass());
|
||||
return clazz.cast(filter);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testColumnValueFilter() throws IOException {
|
||||
String filterString = "ColumnValueFilter ('family', 'qualifier', <, 'binaryprefix:value')";
|
||||
ColumnValueFilter cvf = doTestFilter(filterString, ColumnValueFilter.class);
|
||||
assertEquals("family", new String(cvf.getFamily(), StandardCharsets.UTF_8));
|
||||
assertEquals("qualifier", new String(cvf.getQualifier(), StandardCharsets.UTF_8));
|
||||
assertEquals(CompareOperator.LESS, cvf.getCompareOperator());
|
||||
assertTrue(cvf.getComparator() instanceof BinaryPrefixComparator);
|
||||
assertEquals("value", new String(cvf.getComparator().getValue(), StandardCharsets.UTF_8));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue