Skip to content

Commit ba2a734

Browse files
bbeaudreaultapurtell
authored andcommitted
HBASE-26603 Cherry pick HBASE-26537 to branch-2.4
HBASE-26537: Make HBASE-15676 backwards compatible, using a flag on the proto (apache#3931) Signed-off-by: Andrew Purtell <apurtell@apache.org> (cherry picked from commit cdda96f) Change-Id: I3edc419f10ad2e3d7211710c18c9cca1a07a7587
1 parent 91a0ac8 commit ba2a734

File tree

4 files changed

+112
-29
lines changed

4 files changed

+112
-29
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java

Lines changed: 41 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,16 @@
5858
@InterfaceAudience.Public
5959
public class FuzzyRowFilter extends FilterBase {
6060
private static final boolean UNSAFE_UNALIGNED = UnsafeAvailChecker.unaligned();
61+
62+
// the wildcard byte is 1 on the user side. but the filter converts it internally
63+
// in preprocessMask. This was changed in HBASE-15676 due to a bug with using 0.
64+
// in v1, the 1 byte gets converted to 0
65+
// in v2, the 1 byte gets converted to 2.
66+
// we support both here to ensure backwards compatibility between client and server
67+
static final byte V1_PROCESSED_WILDCARD_MASK = 0;
68+
static final byte V2_PROCESSED_WILDCARD_MASK = 2;
69+
70+
private final byte processedWildcardMask;
6171
private List<Pair<byte[], byte[]>> fuzzyKeysData;
6272
private boolean done = false;
6373

@@ -73,7 +83,18 @@ public class FuzzyRowFilter extends FilterBase {
7383
*/
7484
private RowTracker tracker;
7585

86+
// this client side constructor ensures that all client-constructed
87+
// FuzzyRowFilters use the new v2 mask.
7688
public FuzzyRowFilter(List<Pair<byte[], byte[]>> fuzzyKeysData) {
89+
this(fuzzyKeysData, V2_PROCESSED_WILDCARD_MASK);
90+
}
91+
92+
// This constructor is only used internally here, when parsing from protos on the server side.
93+
// It exists to enable seamless migration from v1 to v2.
94+
// Additionally used in tests, but never used on client side.
95+
FuzzyRowFilter(List<Pair<byte[], byte[]>> fuzzyKeysData, byte processedWildcardMask) {
96+
this.processedWildcardMask = processedWildcardMask;
97+
7798
List<Pair<byte[], byte[]>> fuzzyKeyDataCopy = new ArrayList<>(fuzzyKeysData.size());
7899

79100
for (Pair<byte[], byte[]> aFuzzyKeysData : fuzzyKeysData) {
@@ -88,7 +109,7 @@ public FuzzyRowFilter(List<Pair<byte[], byte[]>> fuzzyKeysData) {
88109
p.setFirst(Arrays.copyOf(aFuzzyKeysData.getFirst(), aFuzzyKeysData.getFirst().length));
89110
p.setSecond(Arrays.copyOf(aFuzzyKeysData.getSecond(), aFuzzyKeysData.getSecond().length));
90111

91-
// update mask ( 0 -> -1 (0xff), 1 -> 2)
112+
// update mask ( 0 -> -1 (0xff), 1 -> [0 or 2 depending on processedWildcardMask value])
92113
p.setSecond(preprocessMask(p.getSecond()));
93114
preprocessSearchKey(p);
94115

@@ -107,7 +128,7 @@ private void preprocessSearchKey(Pair<byte[], byte[]> p) {
107128
byte[] mask = p.getSecond();
108129
for (int i = 0; i < mask.length; i++) {
109130
// set non-fixed part of a search key to 0.
110-
if (mask[i] == 2) {
131+
if (mask[i] == processedWildcardMask) {
111132
key[i] = 0;
112133
}
113134
}
@@ -129,15 +150,15 @@ private byte[] preprocessMask(byte[] mask) {
129150
if (mask[i] == 0) {
130151
mask[i] = -1; // 0 -> -1
131152
} else if (mask[i] == 1) {
132-
mask[i] = 2;// 1 -> 2
153+
mask[i] = processedWildcardMask;// 1 -> 0 or 2 depending on mask version
133154
}
134155
}
135156
return mask;
136157
}
137158

138159
private boolean isPreprocessedMask(byte[] mask) {
139160
for (int i = 0; i < mask.length; i++) {
140-
if (mask[i] != -1 && mask[i] != 2) {
161+
if (mask[i] != -1 && mask[i] != processedWildcardMask) {
141162
return false;
142163
}
143164
}
@@ -157,10 +178,7 @@ public ReturnCode filterCell(final Cell c) {
157178
for (int i = startIndex; i < size + startIndex; i++) {
158179
final int index = i % size;
159180
Pair<byte[], byte[]> fuzzyData = fuzzyKeysData.get(index);
160-
// This shift is idempotent - always end up with 0 and -1 as mask values.
161-
for (int j = 0; j < fuzzyData.getSecond().length; j++) {
162-
fuzzyData.getSecond()[j] >>= 2;
163-
}
181+
idempotentMaskShift(fuzzyData.getSecond());
164182
SatisfiesCode satisfiesCode =
165183
satisfies(isReversed(), c.getRowArray(), c.getRowOffset(), c.getRowLength(),
166184
fuzzyData.getFirst(), fuzzyData.getSecond());
@@ -173,7 +191,15 @@ public ReturnCode filterCell(final Cell c) {
173191
lastFoundIndex = -1;
174192

175193
return ReturnCode.SEEK_NEXT_USING_HINT;
194+
}
176195

196+
static void idempotentMaskShift(byte[] mask) {
197+
// This shift is idempotent - always end up with 0 and -1 as mask values.
198+
// This works regardless of mask version, because both 0 >> 2 and 2 >> 2
199+
// result in 0.
200+
for (int j = 0; j < mask.length; j++) {
201+
mask[j] >>= 2;
202+
}
177203
}
178204

179205
@Override
@@ -262,7 +288,9 @@ public boolean filterAllRemaining() {
262288
*/
263289
@Override
264290
public byte[] toByteArray() {
265-
FilterProtos.FuzzyRowFilter.Builder builder = FilterProtos.FuzzyRowFilter.newBuilder();
291+
FilterProtos.FuzzyRowFilter.Builder builder = FilterProtos.FuzzyRowFilter
292+
.newBuilder()
293+
.setIsMaskV2(processedWildcardMask == V2_PROCESSED_WILDCARD_MASK);
266294
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
267295
BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
268296
bbpBuilder.setFirst(UnsafeByteOperations.unsafeWrap(fuzzyData.getFirst()));
@@ -293,7 +321,10 @@ public static FuzzyRowFilter parseFrom(final byte[] pbBytes) throws Deserializat
293321
byte[] keyMeta = current.getSecond().toByteArray();
294322
fuzzyKeysData.add(new Pair<>(keyBytes, keyMeta));
295323
}
296-
return new FuzzyRowFilter(fuzzyKeysData);
324+
byte processedWildcardMask = proto.hasIsMaskV2() && proto.getIsMaskV2()
325+
? V2_PROCESSED_WILDCARD_MASK
326+
: V1_PROCESSED_WILDCARD_MASK;
327+
return new FuzzyRowFilter(fuzzyKeysData, processedWildcardMask);
297328
}
298329

299330
@Override

hbase-protocol-shaded/src/main/protobuf/Filter.proto

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -94,6 +94,7 @@ message FirstKeyValueMatchingQualifiersFilter {
9494

9595
message FuzzyRowFilter {
9696
repeated BytesBytesPair fuzzy_keys_data = 1;
97+
optional bool is_mask_v2 = 2;
9798
}
9899

99100
message InclusiveStopFilter {

hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
*/
1818
package org.apache.hadoop.hbase.filter;
1919

20+
import java.util.Arrays;
2021
import org.apache.hadoop.hbase.HBaseClassTestRule;
2122
import org.apache.hadoop.hbase.KeyValue;
2223
import org.apache.hadoop.hbase.KeyValueUtil;
@@ -27,6 +28,7 @@
2728
import org.junit.ClassRule;
2829
import org.junit.Test;
2930
import org.junit.experimental.categories.Category;
31+
import org.junit.internal.ArrayComparisonFailure;
3032

3133
@Category({FilterTests.class, SmallTests.class})
3234
public class TestFuzzyRowFilter {
@@ -35,6 +37,40 @@ public class TestFuzzyRowFilter {
3537
public static final HBaseClassTestRule CLASS_RULE =
3638
HBaseClassTestRule.forClass(TestFuzzyRowFilter.class);
3739

40+
@Test
41+
public void testIdempotentMaskShift() {
42+
byte[] test = new byte[] {
43+
-1,
44+
FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK,
45+
FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK
46+
};
47+
48+
byte[] original = Arrays.copyOf(test, test.length);
49+
byte[] expected = new byte[] { -1, 0, 0};
50+
51+
Assert.assertArrayEquals(test, original);
52+
assertArrayNotEquals(expected, test);
53+
54+
// shifting once should equal expected
55+
FuzzyRowFilter.idempotentMaskShift(test);
56+
Assert.assertArrayEquals(expected, test);
57+
assertArrayNotEquals(original, test);
58+
59+
// shifting again should still equal expected, because it's idempotent
60+
FuzzyRowFilter.idempotentMaskShift(test);
61+
Assert.assertArrayEquals(expected, test);
62+
assertArrayNotEquals(original, test);
63+
}
64+
65+
private void assertArrayNotEquals(byte[] expected, byte[] testcase) {
66+
try {
67+
Assert.assertArrayEquals(expected, testcase);
68+
Assert.fail("expected arrays to fail equality test");
69+
} catch (ArrayComparisonFailure e) {
70+
// success
71+
}
72+
}
73+
3874
@Test
3975
public void testSatisfiesNoUnsafeForward() {
4076

hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java

Lines changed: 34 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.hadoop.hbase.filter;
1919

2020
import static org.junit.Assert.assertEquals;
21-
2221
import java.io.IOException;
2322
import java.nio.ByteBuffer;
2423
import java.util.ArrayList;
@@ -56,7 +55,6 @@
5655
import org.junit.rules.TestName;
5756
import org.slf4j.Logger;
5857
import org.slf4j.LoggerFactory;
59-
6058
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
6159

6260
@Category({ FilterTests.class, LargeTests.class })
@@ -142,7 +140,16 @@ public void testAllFixedBits() throws IOException {
142140

143141
TEST_UTIL.flush();
144142

145-
List<Pair<byte[], byte[]>> data = new ArrayList<>();
143+
// v1 should match all rows, because v2 has the actual fix for this bug
144+
testAllFixedBitsRunScanWithMask(ht, rows.length, FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK);
145+
testAllFixedBitsRunScanWithMask(ht, 2, FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK);
146+
147+
TEST_UTIL.deleteTable(TableName.valueOf(table));
148+
}
149+
150+
private void testAllFixedBitsRunScanWithMask(Table ht, int expectedRows, byte processedRowMask)
151+
throws IOException {
152+
List<Pair<byte[], byte[]>> data = new ArrayList<Pair<byte[], byte[]>>();
146153
byte[] fuzzyKey = Bytes.toBytesBinary("\\x9B\\x00\\x044e");
147154
byte[] mask = new byte[] { 0, 0, 0, 0, 0 };
148155

@@ -151,7 +158,7 @@ public void testAllFixedBits() throws IOException {
151158
byte[] copyMask = Arrays.copyOf(mask, mask.length);
152159

153160
data.add(new Pair<>(fuzzyKey, mask));
154-
FuzzyRowFilter filter = new FuzzyRowFilter(data);
161+
FuzzyRowFilter filter = new FuzzyRowFilter(data, processedRowMask);
155162

156163
Scan scan = new Scan();
157164
scan.setFilter(filter);
@@ -161,12 +168,10 @@ public void testAllFixedBits() throws IOException {
161168
while (scanner.next() != null) {
162169
total++;
163170
}
164-
assertEquals(2, total);
171+
assertEquals(expectedRows, total);
165172

166173
assertEquals(true, Arrays.equals(copyFuzzyKey, fuzzyKey));
167174
assertEquals(true, Arrays.equals(copyMask, mask));
168-
169-
TEST_UTIL.deleteTable(TableName.valueOf(name.getMethodName()));
170175
}
171176

172177
@Test
@@ -201,11 +206,20 @@ public void testHBASE14782() throws IOException
201206

202207
TEST_UTIL.flush();
203208

204-
List<Pair<byte[], byte[]>> data = new ArrayList<>();
209+
testHBASE14782RunScanWithMask(ht, rows.length, FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK);
210+
testHBASE14782RunScanWithMask(ht, rows.length, FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK);
211+
212+
TEST_UTIL.deleteTable(TableName.valueOf(name.getMethodName()));
213+
}
214+
215+
private void testHBASE14782RunScanWithMask(Table ht, int expectedRows, byte processedRowMask)
216+
throws IOException {
217+
List<Pair<byte[], byte[]>> data = new ArrayList<Pair<byte[], byte[]>>();
218+
205219
byte[] fuzzyKey = Bytes.toBytesBinary("\\x00\\x00\\x044");
206220
byte[] mask = new byte[] { 1,0,0,0};
207221
data.add(new Pair<>(fuzzyKey, mask));
208-
FuzzyRowFilter filter = new FuzzyRowFilter(data);
222+
FuzzyRowFilter filter = new FuzzyRowFilter(data, processedRowMask);
209223

210224
Scan scan = new Scan();
211225
scan.setFilter(filter);
@@ -215,8 +229,7 @@ public void testHBASE14782() throws IOException
215229
while(scanner.next() != null){
216230
total++;
217231
}
218-
assertEquals(rows.length, total);
219-
TEST_UTIL.deleteTable(TableName.valueOf(name.getMethodName()));
232+
assertEquals(expectedRows, total);
220233
}
221234

222235
@Test
@@ -258,12 +271,14 @@ public void testEndToEnd() throws Exception {
258271
TEST_UTIL.flush();
259272

260273
// test passes
261-
runTest1(ht);
262-
runTest2(ht);
274+
runTest1(ht, FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK);
275+
runTest1(ht, FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK);
276+
runTest2(ht, FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK);
277+
runTest2(ht, FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK);
263278

264279
}
265280

266-
private void runTest1(Table hTable) throws IOException {
281+
private void runTest1(Table hTable, byte processedWildcardMask) throws IOException {
267282
// [0, 2, ?, ?, ?, ?, 0, 0, 0, 1]
268283

269284
byte[] mask = new byte[] { 0, 0, 1, 1, 1, 1, 0, 0, 0, 0 };
@@ -284,9 +299,9 @@ private void runTest1(Table hTable) throws IOException {
284299
}
285300

286301
int expectedSize = secondPartCardinality * totalFuzzyKeys * colQualifiersTotal;
287-
FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list);
302+
FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list, processedWildcardMask);
288303
// Filters are not stateless - we can't reuse them
289-
FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list);
304+
FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list, processedWildcardMask);
290305

291306
// regular test
292307
runScanner(hTable, expectedSize, fuzzyRowFilter0);
@@ -295,7 +310,7 @@ private void runTest1(Table hTable) throws IOException {
295310

296311
}
297312

298-
private void runTest2(Table hTable) throws IOException {
313+
private void runTest2(Table hTable, byte processedWildcardMask) throws IOException {
299314
// [0, 0, ?, ?, ?, ?, 0, 0, 0, 0] , [0, 1, ?, ?, ?, ?, 0, 0, 0, 1]...
300315

301316
byte[] mask = new byte[] { 0, 0, 1, 1, 1, 1, 0, 0, 0, 0 };
@@ -318,9 +333,9 @@ private void runTest2(Table hTable) throws IOException {
318333

319334
int expectedSize = totalFuzzyKeys * secondPartCardinality * colQualifiersTotal;
320335

321-
FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list);
336+
FuzzyRowFilter fuzzyRowFilter0 = new FuzzyRowFilter(list, processedWildcardMask);
322337
// Filters are not stateless - we can't reuse them
323-
FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list);
338+
FuzzyRowFilter fuzzyRowFilter1 = new FuzzyRowFilter(list, processedWildcardMask);
324339

325340
// regular test
326341
runScanner(hTable, expectedSize, fuzzyRowFilter0);

0 commit comments

Comments
 (0)