HBASE-26537: Make HBASE-15676 backwards compatible, using a flag on the proto (#3931)

Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
Bryan Beaudreault 2021-12-13 16:31:14 -05:00 committed by Andrew Purtell
parent 4a929175a5
commit 9cf0af6e79
4 changed files with 112 additions and 29 deletions

View File

@ -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<Pair<byte[], byte[]>> 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<Pair<byte[], byte[]>> 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<Pair<byte[], byte[]>> fuzzyKeysData, byte processedWildcardMask) {
this.processedWildcardMask = processedWildcardMask;
List<Pair<byte[], byte[]>> fuzzyKeyDataCopy = new ArrayList<>(fuzzyKeysData.size());
for (Pair<byte[], byte[]> 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<byte[], byte[]> 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<byte[], byte[]> 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

View File

@ -94,6 +94,7 @@ message FirstKeyValueMatchingQualifiersFilter {
message FuzzyRowFilter {
repeated BytesBytesPair fuzzy_keys_data = 1;
optional bool is_mask_v2 = 2;
}
message InclusiveStopFilter {

View File

@ -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() {

View File

@ -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;
@ -57,7 +56,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 })
@ -143,7 +141,16 @@ public class TestFuzzyRowFilterEndToEnd {
TEST_UTIL.flush();
List<Pair<byte[], byte[]>> 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<Pair<byte[], byte[]>> data = new ArrayList<Pair<byte[], byte[]>>();
byte[] fuzzyKey = Bytes.toBytesBinary("\\x9B\\x00\\x044e");
byte[] mask = new byte[] { 0, 0, 0, 0, 0 };
@ -152,7 +159,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);
@ -162,12 +169,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
@ -202,11 +207,20 @@ public class TestFuzzyRowFilterEndToEnd {
TEST_UTIL.flush();
List<Pair<byte[], byte[]>> 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<Pair<byte[], byte[]>> data = new ArrayList<Pair<byte[], byte[]>>();
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);
@ -216,8 +230,7 @@ public class TestFuzzyRowFilterEndToEnd {
while(scanner.next() != null){
total++;
}
assertEquals(rows.length, total);
TEST_UTIL.deleteTable(TableName.valueOf(name.getMethodName()));
assertEquals(expectedRows, total);
}
@Test
@ -259,12 +272,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 };
@ -285,9 +300,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);
@ -296,7 +311,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 };
@ -319,9 +334,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);