From cdda96fb4709a68ab40aaab106dc03cb421e2a40 Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Mon, 13 Dec 2021 16:31:14 -0500 Subject: [PATCH] HBASE-26603 Cherry pick HBASE-26537 to branch-2.4 HBASE-26537: Make HBASE-15676 backwards compatible, using a flag on the proto (#3931) Signed-off-by: Andrew Purtell --- .../hadoop/hbase/filter/FuzzyRowFilter.java | 51 ++++++++++++++---- .../src/main/protobuf/Filter.proto | 1 + .../hbase/filter/TestFuzzyRowFilter.java | 36 +++++++++++++ .../filter/TestFuzzyRowFilterEndToEnd.java | 53 ++++++++++++------- 4 files changed, 112 insertions(+), 29 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java index 259a6cb09a4..5151d79023d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java @@ -58,6 +58,16 @@ import org.apache.hadoop.hbase.util.UnsafeAvailChecker; @InterfaceAudience.Public public class FuzzyRowFilter extends FilterBase { private static final boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned(); + + // the wildcard byte is 1 on the user side. but the filter converts it internally + // in preprocessMask. This was changed in HBASE-15676 due to a bug with using 0. + // in v1, the 1 byte gets converted to 0 + // in v2, the 1 byte gets converted to 2. + // we support both here to ensure backwards compatibility between client and server + static final byte V1_PROCESSED_WILDCARD_MASK = 0; + static final byte V2_PROCESSED_WILDCARD_MASK = 2; + + private final byte processedWildcardMask; private List> fuzzyKeysData; private boolean done = false; @@ -73,7 +83,18 @@ public class FuzzyRowFilter extends FilterBase { */ private RowTracker tracker; + // this client side constructor ensures that all client-constructed + // FuzzyRowFilters use the new v2 mask. public FuzzyRowFilter(List> fuzzyKeysData) { + this(fuzzyKeysData, V2_PROCESSED_WILDCARD_MASK); + } + + // This constructor is only used internally here, when parsing from protos on the server side. + // It exists to enable seamless migration from v1 to v2. + // Additionally used in tests, but never used on client side. + FuzzyRowFilter(List> fuzzyKeysData, byte processedWildcardMask) { + this.processedWildcardMask = processedWildcardMask; + List> fuzzyKeyDataCopy = new ArrayList<>(fuzzyKeysData.size()); for (Pair aFuzzyKeysData : fuzzyKeysData) { @@ -88,7 +109,7 @@ public class FuzzyRowFilter extends FilterBase { p.setFirst(Arrays.copyOf(aFuzzyKeysData.getFirst(), aFuzzyKeysData.getFirst().length)); p.setSecond(Arrays.copyOf(aFuzzyKeysData.getSecond(), aFuzzyKeysData.getSecond().length)); - // update mask ( 0 -> -1 (0xff), 1 -> 2) + // update mask ( 0 -> -1 (0xff), 1 -> [0 or 2 depending on processedWildcardMask value]) p.setSecond(preprocessMask(p.getSecond())); preprocessSearchKey(p); @@ -107,7 +128,7 @@ public class FuzzyRowFilter extends FilterBase { byte[] mask = p.getSecond(); for (int i = 0; i < mask.length; i++) { // set non-fixed part of a search key to 0. - if (mask[i] == 2) { + if (mask[i] == processedWildcardMask) { key[i] = 0; } } @@ -129,7 +150,7 @@ public class FuzzyRowFilter extends FilterBase { if (mask[i] == 0) { mask[i] = -1; // 0 -> -1 } else if (mask[i] == 1) { - mask[i] = 2;// 1 -> 2 + mask[i] = processedWildcardMask;// 1 -> 0 or 2 depending on mask version } } return mask; @@ -137,7 +158,7 @@ public class FuzzyRowFilter extends FilterBase { private boolean isPreprocessedMask(byte[] mask) { for (int i = 0; i < mask.length; i++) { - if (mask[i] != -1 && mask[i] != 2) { + if (mask[i] != -1 && mask[i] != processedWildcardMask) { return false; } } @@ -157,10 +178,7 @@ public class FuzzyRowFilter extends FilterBase { for (int i = startIndex; i < size + startIndex; i++) { final int index = i % size; Pair fuzzyData = fuzzyKeysData.get(index); - // This shift is idempotent - always end up with 0 and -1 as mask values. - for (int j = 0; j < fuzzyData.getSecond().length; j++) { - fuzzyData.getSecond()[j] >>= 2; - } + idempotentMaskShift(fuzzyData.getSecond()); SatisfiesCode satisfiesCode = satisfies(isReversed(), c.getRowArray(), c.getRowOffset(), c.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond()); @@ -173,7 +191,15 @@ public class FuzzyRowFilter extends FilterBase { lastFoundIndex = -1; return ReturnCode.SEEK_NEXT_USING_HINT; + } + static void idempotentMaskShift(byte[] mask) { + // This shift is idempotent - always end up with 0 and -1 as mask values. + // This works regardless of mask version, because both 0 >> 2 and 2 >> 2 + // result in 0. + for (int j = 0; j < mask.length; j++) { + mask[j] >>= 2; + } } @Override @@ -262,7 +288,9 @@ public class FuzzyRowFilter extends FilterBase { */ @Override public byte[] toByteArray() { - FilterProtos.FuzzyRowFilter.Builder builder = FilterProtos.FuzzyRowFilter.newBuilder(); + FilterProtos.FuzzyRowFilter.Builder builder = FilterProtos.FuzzyRowFilter + .newBuilder() + .setIsMaskV2(processedWildcardMask == V2_PROCESSED_WILDCARD_MASK); for (Pair fuzzyData : fuzzyKeysData) { BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder(); bbpBuilder.setFirst(UnsafeByteOperations.unsafeWrap(fuzzyData.getFirst())); @@ -293,7 +321,10 @@ public class FuzzyRowFilter extends FilterBase { byte[] keyMeta = current.getSecond().toByteArray(); fuzzyKeysData.add(new Pair<>(keyBytes, keyMeta)); } - return new FuzzyRowFilter(fuzzyKeysData); + byte processedWildcardMask = proto.hasIsMaskV2() && proto.getIsMaskV2() + ? V2_PROCESSED_WILDCARD_MASK + : V1_PROCESSED_WILDCARD_MASK; + return new FuzzyRowFilter(fuzzyKeysData, processedWildcardMask); } @Override diff --git a/hbase-protocol-shaded/src/main/protobuf/Filter.proto b/hbase-protocol-shaded/src/main/protobuf/Filter.proto index b0d6da68c3b..adb788e820f 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Filter.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Filter.proto @@ -94,6 +94,7 @@ message FirstKeyValueMatchingQualifiersFilter { message FuzzyRowFilter { repeated BytesBytesPair fuzzy_keys_data = 1; + optional bool is_mask_v2 = 2; } message InclusiveStopFilter { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java index 07548a846e0..f5788e4081f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.filter; +import java.util.Arrays; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; @@ -27,6 +28,7 @@ import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.internal.ArrayComparisonFailure; @Category({FilterTests.class, SmallTests.class}) public class TestFuzzyRowFilter { @@ -35,6 +37,40 @@ public class TestFuzzyRowFilter { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestFuzzyRowFilter.class); + @Test + public void testIdempotentMaskShift() { + byte[] test = new byte[] { + -1, + FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK, + FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK + }; + + byte[] original = Arrays.copyOf(test, test.length); + byte[] expected = new byte[] { -1, 0, 0}; + + Assert.assertArrayEquals(test, original); + assertArrayNotEquals(expected, test); + + // shifting once should equal expected + FuzzyRowFilter.idempotentMaskShift(test); + Assert.assertArrayEquals(expected, test); + assertArrayNotEquals(original, test); + + // shifting again should still equal expected, because it's idempotent + FuzzyRowFilter.idempotentMaskShift(test); + Assert.assertArrayEquals(expected, test); + assertArrayNotEquals(original, test); + } + + private void assertArrayNotEquals(byte[] expected, byte[] testcase) { + try { + Assert.assertArrayEquals(expected, testcase); + Assert.fail("expected arrays to fail equality test"); + } catch (ArrayComparisonFailure e) { + // success + } + } + @Test public void testSatisfiesNoUnsafeForward() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java index cf326311ab0..38562500d83 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.filter; import static org.junit.Assert.assertEquals; - import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -56,7 +55,6 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({ FilterTests.class, LargeTests.class }) @@ -142,7 +140,16 @@ public class TestFuzzyRowFilterEndToEnd { TEST_UTIL.flush(); - List> data = new ArrayList<>(); + // v1 should match all rows, because v2 has the actual fix for this bug + testAllFixedBitsRunScanWithMask(ht, rows.length, FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK); + testAllFixedBitsRunScanWithMask(ht, 2, FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK); + + TEST_UTIL.deleteTable(TableName.valueOf(table)); + } + + private void testAllFixedBitsRunScanWithMask(Table ht, int expectedRows, byte processedRowMask) + throws IOException { + List> data = new ArrayList>(); byte[] fuzzyKey = Bytes.toBytesBinary("\\x9B\\x00\\x044e"); byte[] mask = new byte[] { 0, 0, 0, 0, 0 }; @@ -151,7 +158,7 @@ public class TestFuzzyRowFilterEndToEnd { byte[] copyMask = Arrays.copyOf(mask, mask.length); data.add(new Pair<>(fuzzyKey, mask)); - FuzzyRowFilter filter = new FuzzyRowFilter(data); + FuzzyRowFilter filter = new FuzzyRowFilter(data, processedRowMask); Scan scan = new Scan(); scan.setFilter(filter); @@ -161,12 +168,10 @@ public class TestFuzzyRowFilterEndToEnd { while (scanner.next() != null) { total++; } - assertEquals(2, total); + assertEquals(expectedRows, total); assertEquals(true, Arrays.equals(copyFuzzyKey, fuzzyKey)); assertEquals(true, Arrays.equals(copyMask, mask)); - - TEST_UTIL.deleteTable(TableName.valueOf(name.getMethodName())); } @Test @@ -201,11 +206,20 @@ public class TestFuzzyRowFilterEndToEnd { TEST_UTIL.flush(); - List> data = new ArrayList<>(); + testHBASE14782RunScanWithMask(ht, rows.length, FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK); + testHBASE14782RunScanWithMask(ht, rows.length, FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK); + + TEST_UTIL.deleteTable(TableName.valueOf(name.getMethodName())); + } + + private void testHBASE14782RunScanWithMask(Table ht, int expectedRows, byte processedRowMask) + throws IOException { + List> data = new ArrayList>(); + byte[] fuzzyKey = Bytes.toBytesBinary("\\x00\\x00\\x044"); byte[] mask = new byte[] { 1,0,0,0}; data.add(new Pair<>(fuzzyKey, mask)); - FuzzyRowFilter filter = new FuzzyRowFilter(data); + FuzzyRowFilter filter = new FuzzyRowFilter(data, processedRowMask); Scan scan = new Scan(); scan.setFilter(filter); @@ -215,8 +229,7 @@ public class TestFuzzyRowFilterEndToEnd { while(scanner.next() != null){ total++; } - assertEquals(rows.length, total); - TEST_UTIL.deleteTable(TableName.valueOf(name.getMethodName())); + assertEquals(expectedRows, total); } @Test @@ -258,12 +271,14 @@ public class TestFuzzyRowFilterEndToEnd { TEST_UTIL.flush(); // test passes - runTest1(ht); - runTest2(ht); + runTest1(ht, FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK); + runTest1(ht, FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK); + runTest2(ht, FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK); + runTest2(ht, FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK); } - private void runTest1(Table hTable) throws IOException { + private void runTest1(Table hTable, byte processedWildcardMask) throws IOException { // [0, 2, ?, ?, ?, ?, 0, 0, 0, 1] byte[] mask = new byte[] { 0, 0, 1, 1, 1, 1, 0, 0, 0, 0 }; @@ -284,9 +299,9 @@ public class TestFuzzyRowFilterEndToEnd { } int expectedSize = secondPartCardinality * totalFuzzyKeys * colQualifiersTotal; - FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list); + FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list, processedWildcardMask); // Filters are not stateless - we can't reuse them - FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list); + FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list, processedWildcardMask); // regular test runScanner(hTable, expectedSize, fuzzyRowFilter0); @@ -295,7 +310,7 @@ public class TestFuzzyRowFilterEndToEnd { } - private void runTest2(Table hTable) throws IOException { + private void runTest2(Table hTable, byte processedWildcardMask) throws IOException { // [0, 0, ?, ?, ?, ?, 0, 0, 0, 0] , [0, 1, ?, ?, ?, ?, 0, 0, 0, 1]... byte[] mask = new byte[] { 0, 0, 1, 1, 1, 1, 0, 0, 0, 0 }; @@ -318,9 +333,9 @@ public class TestFuzzyRowFilterEndToEnd { int expectedSize = totalFuzzyKeys * secondPartCardinality * colQualifiersTotal; - FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list); + FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list, processedWildcardMask); // Filters are not stateless - we can't reuse them - FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list); + FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list, processedWildcardMask); // regular test runScanner(hTable, expectedSize, fuzzyRowFilter0);