From 23e8c8e7ba9f701fa6f2e9c2f28641c6a8cf6b29 Mon Sep 17 00:00:00 2001 From: Reid Chan Date: Thu, 8 Feb 2018 14:00:56 +0800 Subject: [PATCH] HBASE-19950 Introduce a ColumnValueFilter --- .../hadoop/hbase/filter/ColumnValueFilter.java | 212 +++++++++++++++++++++ .../apache/hadoop/hbase/filter/CompareFilter.java | 4 +- .../apache/hadoop/hbase/filter/ParseFilter.java | 4 +- .../src/main/protobuf/Filter.proto | 7 + hbase-protocol/src/main/protobuf/Filter.proto | 9 +- .../org/apache/hadoop/hbase/filter/TestFilter.java | 125 ++++++++++++ .../hbase/filter/TestFilterSerialization.java | 9 + .../hadoop/hbase/filter/TestParseFilter.java | 11 ++ 8 files changed, 377 insertions(+), 4 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java new file mode 100644 index 0000000000..e7690dd8cb --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java @@ -0,0 +1,212 @@ +/** + * + * 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.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.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.util.Bytes; +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.yetus.audience.InterfaceAudience; + +import java.io.IOException; +import java.util.ArrayList; + +/** + * Different from {@link SingleColumnValueFilter} which returns an entire row + * when specified condition is matched, {@link ColumnValueFilter} return the matched cell only. + *

+ * 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 byte[] family; + private byte[] qualifier; + private CompareOperator op; + private ByteArrayComparable comparator; + + 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 = Preconditions.checkNotNull(qualifier, "qualifier should not be null"); + 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 boolean hasFilterRow() { + return true; + } + + @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 ReturnCode.NEXT_COL; + } + // 2. Check value match: + // True means fiter out, just skip this cell, else include it. + return compareValue(getCompareOperator(), getComparator(), c) ? + ReturnCode.SKIP : ReturnCode.INCLUDE; + } + + private boolean compareValue(final CompareOperator op, final ByteArrayComparable comparator, + final Cell cell) { + int compareResult = PrivateCellUtil.compareValue(cell, comparator); + return CompareFilter.compare(op, compareResult); + } + + public static Filter createFilterFromArguments(ArrayList 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(); + } + + 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())); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java index 6e6ca6079c..c796445948 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java @@ -39,9 +39,11 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; *

* To filter by row key, use {@link RowFilter}. *

+ * To filter by column family, use {@link FamilyFilter}. + *

* To filter by column qualifier, use {@link QualifierFilter}. *

- * To filter by value, use {@link SingleColumnValueFilter}. + * To filter by value, use {@link ValueFilter}. *

* These filters can be wrapped with {@link SkipFilter} and {@link WhileMatchFilter} * to add more control. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java index e984212985..716322cff9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java @@ -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 - *

* @param compareOpAsByteArray the comparatorOperator symbol as a byte array * @return the Compare Operator */ diff --git a/hbase-protocol-shaded/src/main/protobuf/Filter.proto b/hbase-protocol-shaded/src/main/protobuf/Filter.proto index 743498532b..b0d6da68c3 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Filter.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Filter.proto @@ -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; +} diff --git a/hbase-protocol/src/main/protobuf/Filter.proto b/hbase-protocol/src/main/protobuf/Filter.proto index 1fa6697823..8a0a9bf502 100644 --- a/hbase-protocol/src/main/protobuf/Filter.proto +++ b/hbase-protocol/src/main/protobuf/Filter.proto @@ -168,4 +168,11 @@ message RowRange { message MultiRowRangeFilter { repeated RowRange row_range_list = 1; -} \ No newline at end of file +} + +message ColumnValueFilter { + required bytes family = 1; + required bytes qualifier = 2; + required CompareType compare_op = 3; + required Comparator comparator = 4; +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index 3c5be6303a..0b84258e67 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -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,127 @@ public class TestFilter { } + @Test + public void testColumnValueFilter() throws Exception { + // Prepare test rows: + int versions = 2; + for (int i = 0; i < versions; i++) { + for (byte[] r1 : ROWS_ONE) { + Put p = new Put(r1).setDurability(Durability.SKIP_WAL); + for (byte[] q5 : QUALIFIERS_FIVE) { + p.addColumn(FAMILIES[0], q5, VALUES[0 + i]); + p.addColumn(FAMILIES[1], q5, VALUES[0 + i]); + } + this.region.put(p); + this.region.flush(true); + } + } + for (int i = 0; i < versions; i++) { + for (byte[] r2 : ROWS_TWO) { + Put p = new Put(r2).setDurability(Durability.SKIP_WAL); + for (byte[] q5 : QUALIFIERS_FIVE) { + p.addColumn(FAMILIES[0], q5, VALUES[1 - i]); + p.addColumn(FAMILIES[1], q5, VALUES[1 - i]); + } + this.region.put(p); + this.region.flush(true); + } + } + + // 1. Test = f[0]:q5[0]:v[1] + ColumnValueFilter cvf = + new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.EQUAL, VALUES[1]); + Scan scan1 = new Scan().setFilter(cvf); + 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]) }; + verifyScan(scan1, 8, 1); + verifyScanFull(scan1, expectedEquals); + // 2. Test > f[0]:q5[0]:v[0] + cvf = + new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.GREATER, VALUES[0]); + Scan scan2 = new Scan().setFilter(cvf); + 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]) }; + verifyScan(scan2, 8, 1); + verifyScanFull(scan2, expectedGreater); + // 3. Test >= f[0]:q5[0]:v[0] + // also test readAllVersions(), since FAMILIES[0] allow multiple versions. + cvf = new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.GREATER_OR_EQUAL, + VALUES[0]); + Scan scan3 = new Scan().setFilter(cvf).readAllVersions(); + KeyValue[] expectedGreaterOrEqual = + { // Test ROWS_ONE + 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]), + // Test ROWS_TWO + 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]) }; + verifyScan(scan3, 8, 2); + verifyScanFull(scan3, expectedGreaterOrEqual); + // 4. Test < f[1]:q5[1]:v[1], FAMILIES[1] doesn't support multiple versions + cvf = new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.LESS, VALUES[1]); + Scan scan4 = new Scan().setFilter(cvf); + 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]) }; + verifyScan(scan4, 4, 1); + verifyScanFull(scan4, expectedLess); + // 5. Test <= f[1]:q5[0]:v[1] + cvf = new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.LESS_OR_EQUAL, + VALUES[1]); + Scan scan5 = new Scan().setFilter(cvf); + 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]) }; + verifyScan(scan5, 8, 1); + verifyScanFull(scan5, expectedLessOrEqual); + // 6. Test != f[1]:q5[0]:v[1] + cvf = new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, + VALUES[1]); + Scan scan6 = new Scan().setFilter(cvf); + 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]) }; + verifyScan(scan6, 4, 1); + verifyScanFull(scan6, expectedNotEqual); + } + private void verifyScan(Scan s, long expectedRows, long expectedKeys) throws IOException { InternalScanner scanner = this.region.getScanner(s); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java index 2a0d1f4b96..b0d876b78d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java @@ -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)))); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java index d30958ccb5..cee0fb3502 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java @@ -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)); + } } -- 2.15.0