From 46aa0810de7c6fd325e89827af7a2e88f435ea33 Mon Sep 17 00:00:00 2001 From: Reid Chan Date: Thu, 1 Feb 2018 15:25:09 +0800 Subject: [PATCH] HBASE-19824 SingleColumnValueFilter returns wrong result when used in shell command --- .../hbase/filter/SingleColumnValueFilter.java | 74 ++++++++++++-- .../src/main/protobuf/Filter.proto | 1 + hbase-protocol/src/main/protobuf/Filter.proto | 3 +- .../org/apache/hadoop/hbase/filter/TestFilter.java | 112 ++++++++++++++++++++- .../hadoop/hbase/filter/TestParseFilter.java | 14 ++- 5 files changed, 192 insertions(+), 12 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java index e5c83b1d72..9414a4354e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java @@ -78,6 +78,7 @@ public class SingleColumnValueFilter extends FilterBase { protected boolean matchedColumn = false; protected boolean filterIfMissing = false; protected boolean latestVersionOnly = true; + protected boolean fetchEntireRow = true; /** * Constructor for binary compare of the value of a single column. If the @@ -206,6 +207,14 @@ public class SingleColumnValueFilter extends FilterBase { this.latestVersionOnly = latestVersionOnly; } + protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier, + final CompareOperator op, org.apache.hadoop.hbase.filter.ByteArrayComparable comparator, + final boolean filterIfMissing, final boolean latestVersionOnly, + final boolean fetchEntireRow) { + this(family, qualifier, op, comparator, filterIfMissing, latestVersionOnly); + this.fetchEntireRow = fetchEntireRow; + } + /** * @return operator * @deprecated since 2.0.0. Will be removed in 3.0.0. Use {@link #getCompareOperator()} instead. @@ -254,7 +263,10 @@ public class SingleColumnValueFilter extends FilterBase { @Override public ReturnCode filterCell(final Cell c) { - // System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue())); + if (!fetchEntireRow) { + return fetchMatchColumnValueOnly(c); + } + if (this.matchedColumn) { // We already found and matched the single column, all keys now pass return ReturnCode.INCLUDE; @@ -278,6 +290,23 @@ public class SingleColumnValueFilter extends FilterBase { return CompareFilter.compare(this.op, compareResult); } + private ReturnCode fetchMatchColumnValueOnly(final Cell c) { + if (!CellUtil.matchingColumn(c, this.columnFamily, this.columnQualifier)) { + // Column mismatch, go to next column. + return ReturnCode.NEXT_COL; + } + this.foundColumn = true; + if (filterColumnValue(c)) { + // Column match, value mismatch, go to next row. + return ReturnCode.NEXT_ROW; + } + this.matchedColumn = true; + // Column and value match, check if it is latest version. + // If true, just include and this row is done. (since latest always comes first) + // If false, just include. + return this.latestVersionOnly ? ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW : ReturnCode.INCLUDE; + } + @Override public boolean filterRow() { // If column was found, return false if it was matched, true if it was not @@ -339,9 +368,29 @@ public class SingleColumnValueFilter extends FilterBase { this.latestVersionOnly = latestVersionOnly; } + /** + * Set whether entire row should be fetched if column and value is found. + * If true, the entire row will be fetched. + * If false, only fetch the match column and value of the row. + * @param fetchEntireRow flag + */ + public void setFetchEntireRow(boolean fetchEntireRow) { + this.fetchEntireRow = fetchEntireRow; + } + + /** + * Get whether entire row should be fetched if column and value is found. Default is true + * @return true if entire row should be fetched, + * otherwise only fetch match column value of the row. + */ + public boolean getFetchEntireRow() { + return fetchEntireRow; + } + public static Filter createFilterFromArguments(ArrayList filterArguments) { - Preconditions.checkArgument(filterArguments.size() == 4 || filterArguments.size() == 6, - "Expected 4 or 6 but got: %s", filterArguments.size()); + Preconditions.checkArgument( + filterArguments.size() == 4 || filterArguments.size() == 6 || filterArguments.size() == 7, + "Expected 4 or 6 or 7 but got: %s", filterArguments.size()); byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1)); CompareOperator op = ParseFilter.createCompareOperator(filterArguments.get(2)); @@ -360,12 +409,16 @@ public class SingleColumnValueFilter extends FilterBase { SingleColumnValueFilter filter = new SingleColumnValueFilter(family, qualifier, op, comparator); - if (filterArguments.size() == 6) { + if (filterArguments.size() >= 6) { boolean filterIfMissing = ParseFilter.convertByteArrayToBoolean(filterArguments.get(4)); boolean latestVersionOnly = ParseFilter.convertByteArrayToBoolean(filterArguments.get(5)); filter.setFilterIfMissing(filterIfMissing); filter.setLatestVersionOnly(latestVersionOnly); } + if (filterArguments.size() == 7) { + boolean fetchEntireRow = ParseFilter.convertByteArrayToBoolean(filterArguments.get(6)); + filter.setFetchEntireRow(fetchEntireRow); + } return filter; } @@ -383,6 +436,7 @@ public class SingleColumnValueFilter extends FilterBase { builder.setComparator(ProtobufUtil.toComparator(this.comparator)); builder.setFilterIfMissing(this.filterIfMissing); builder.setLatestVersionOnly(this.latestVersionOnly); + builder.setFetchEntireRow(this.fetchEntireRow); return builder.build(); } @@ -419,10 +473,11 @@ public class SingleColumnValueFilter extends FilterBase { throw new DeserializationException(ioe); } - return new SingleColumnValueFilter(proto.hasColumnFamily() ? proto.getColumnFamily() - .toByteArray() : null, proto.hasColumnQualifier() ? proto.getColumnQualifier() - .toByteArray() : null, compareOp, comparator, proto.getFilterIfMissing(), proto - .getLatestVersionOnly()); + return new SingleColumnValueFilter( + proto.hasColumnFamily() ? proto.getColumnFamily().toByteArray() : null, + proto.hasColumnQualifier() ? proto.getColumnQualifier().toByteArray() : null, + compareOp, comparator, proto.getFilterIfMissing(), + proto.getLatestVersionOnly(), proto.getFetchEntireRow()); } /** @@ -440,7 +495,8 @@ public class SingleColumnValueFilter extends FilterBase { && this.op.equals(other.op) && this.getComparator().areSerializedFieldsEqual(other.getComparator()) && this.getFilterIfMissing() == other.getFilterIfMissing() - && this.getLatestVersionOnly() == other.getLatestVersionOnly(); + && this.getLatestVersionOnly() == other.getLatestVersionOnly() + && this.getFetchEntireRow() == other.getFetchEntireRow(); } /** diff --git a/hbase-protocol-shaded/src/main/protobuf/Filter.proto b/hbase-protocol-shaded/src/main/protobuf/Filter.proto index 743498532b..b8c5e0f6f8 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Filter.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Filter.proto @@ -139,6 +139,7 @@ message SingleColumnValueFilter { required Comparator comparator = 4; optional bool filter_if_missing = 5; optional bool latest_version_only = 6; + optional bool fetch_entire_row = 7; } message SkipFilter { diff --git a/hbase-protocol/src/main/protobuf/Filter.proto b/hbase-protocol/src/main/protobuf/Filter.proto index 1fa6697823..9705a31d18 100644 --- a/hbase-protocol/src/main/protobuf/Filter.proto +++ b/hbase-protocol/src/main/protobuf/Filter.proto @@ -138,6 +138,7 @@ message SingleColumnValueFilter { required Comparator comparator = 4; optional bool filter_if_missing = 5; optional bool latest_version_only = 6; + optional bool fetch_entire_row = 7; } message SkipFilter { @@ -168,4 +169,4 @@ message RowRange { message MultiRowRangeFilter { repeated RowRange row_range_list = 1; -} \ No newline at end of file +} 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..244a4177ac 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,11 @@ 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") }; @@ -174,7 +179,6 @@ public class TestFilter { } this.region.put(p); } - // Flush this.region.flush(true); @@ -1660,7 +1664,113 @@ public class TestFilter { s.setFilter(scvf); kvs = new KeyValue [] { srcKVs[3] }; verifyScanFull(s, kvs); + } + @Test + public void testSingleColumnValueFilterFetchMatchColumnValueOnly() throws Exception { + // Create new another rows, column family and value to test (2 versions). + 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); + } + } + 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); + } + } + // 1. Test = + SingleColumnValueFilter scvf = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], + CompareOperator.EQUAL, VALUES[1]); + scvf.setFetchEntireRow(false); + Scan scan1 = new Scan().setFilter(scvf); + 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]) + }; + verifyScanFull(scan1, expectedEquals); + // 2. Test > + scvf = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], + CompareOperator.GREATER, VALUES[0]); + scvf.setFetchEntireRow(false); + Scan scan2 = new Scan().setFilter(scvf); + 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]) + }; + scan2.setFilter(scvf); + verifyScanFull(scan2, expectedGreater); + // 3. Test >= + scvf = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], + CompareOperator.GREATER_OR_EQUAL, VALUES[0]); + scvf.setFetchEntireRow(false); + Scan scan3 = new Scan().setFilter(scvf); + KeyValue[] expectedGreaterOrEqual = { + 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[0]), + new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), + new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]), + new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]) + }; + verifyScanFull(scan3, expectedGreaterOrEqual); + // 4. Test < + scvf = new SingleColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], + CompareOperator.LESS, VALUES[1]); + scvf.setFetchEntireRow(false); + Scan scan4 = new Scan().setFilter(scvf); + 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(scan4, expectedLess); + // 5. Test <= + scvf = new SingleColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], + CompareOperator.LESS_OR_EQUAL, VALUES[1]); + scvf.setFetchEntireRow(false); + Scan scan5 = new Scan().setFilter(scvf); + 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(scan5, expectedLessOrEqual); + // 6. Test != + scvf = new SingleColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], + CompareOperator.NOT_EQUAL, VALUES[1]); + scvf.setFetchEntireRow(false); + Scan scan6 = new Scan().setFilter(scvf); + 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(scan6, expectedNotEqual); } private void verifyScan(Scan s, long expectedRows, long expectedKeys) 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..24a71aab73 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 @@ -334,7 +334,6 @@ public class TestParseFilter { assertTrue(singleColumnValueFilter.getFilterIfMissing()); assertFalse(singleColumnValueFilter.getLatestVersionOnly()); - filterString = "SingleColumnValueFilter ('family', 'qualifier', >, 'binaryprefix:a')"; singleColumnValueFilter = doTestFilter(filterString, SingleColumnValueFilter.class); assertEquals("family", new String(singleColumnValueFilter.getFamily(), StandardCharsets.UTF_8)); @@ -347,6 +346,19 @@ public class TestParseFilter { assertEquals("a", new String(binaryPrefixComparator.getValue(), StandardCharsets.UTF_8)); assertFalse(singleColumnValueFilter.getFilterIfMissing()); assertTrue(singleColumnValueFilter.getLatestVersionOnly()); + + filterString = "SingleColumnValueFilter " + + "('family', 'qualifier', >=, 'binary:a', true, false, true)"; + singleColumnValueFilter = doTestFilter(filterString, SingleColumnValueFilter.class); + assertEquals("family", new String(singleColumnValueFilter.getFamily(), StandardCharsets.UTF_8)); + assertEquals("qualifier", + new String(singleColumnValueFilter.getQualifier(), StandardCharsets.UTF_8)); + assertEquals(CompareOperator.GREATER_OR_EQUAL, singleColumnValueFilter.getCompareOperator()); + assertTrue(singleColumnValueFilter.getComparator() instanceof BinaryComparator); + assertEquals("a", new String(binaryComparator.getValue(), StandardCharsets.UTF_8)); + assertTrue(singleColumnValueFilter.getFilterIfMissing()); + assertFalse(singleColumnValueFilter.getLatestVersionOnly()); + assertTrue(singleColumnValueFilter.getFetchEntireRow()); } @Test -- 2.15.0