You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2021/12/13 21:32:41 UTC
[hbase] branch branch-1 updated: HBASE-26537: Make HBASE-15676 backwards compatible, using a flag on the proto (#3916)
This is an automated email from the ASF dual-hosted git repository.
apurtell pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-1 by this push:
new 1d1b52c HBASE-26537: Make HBASE-15676 backwards compatible, using a flag on the proto (#3916)
1d1b52c is described below
commit 1d1b52cb559d70024ebdc6212a83ce733893aef6
Author: Bryan Beaudreault <bb...@hubspot.com>
AuthorDate: Mon Dec 13 16:31:25 2021 -0500
HBASE-26537: Make HBASE-15676 backwards compatible, using a flag on the proto (#3916)
Signed-off-by: Andrew Purtell <ap...@apache.org>
---
.../apache/hadoop/hbase/filter/FuzzyRowFilter.java | 51 +++++--
.../hbase/protobuf/generated/FilterProtos.java | 158 ++++++++++++++++-----
hbase-protocol/src/main/protobuf/Filter.proto | 1 +
.../hadoop/hbase/filter/TestFuzzyRowFilter.java | 38 +++++
.../hbase/filter/TestFuzzyRowFilterEndToEnd.java | 57 +++++---
5 files changed, 243 insertions(+), 62 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 79edcab..dc71d76 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;
@InterfaceStability.Evolving
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);
@@ -106,7 +127,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;
}
}
@@ -127,7 +148,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;
@@ -135,7 +156,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;
}
}
@@ -149,10 +170,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());
@@ -165,7 +183,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
@@ -257,7 +283,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(ByteStringer.wrap(fuzzyData.getFirst()));
@@ -288,7 +316,10 @@ public class FuzzyRowFilter extends FilterBase {
byte[] keyMeta = current.getSecond().toByteArray();
fuzzyKeysData.add(new Pair<byte[], byte[]>(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/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
index 5b995a4..ebc34ef 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
@@ -7238,6 +7238,16 @@ public final class FilterProtos {
*/
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getFuzzyKeysDataOrBuilder(
int index);
+
+ // optional bool is_mask_v2 = 2;
+ /**
+ * <code>optional bool is_mask_v2 = 2;</code>
+ */
+ boolean hasIsMaskV2();
+ /**
+ * <code>optional bool is_mask_v2 = 2;</code>
+ */
+ boolean getIsMaskV2();
}
/**
* Protobuf type {@code hbase.pb.FuzzyRowFilter}
@@ -7298,6 +7308,11 @@ public final class FilterProtos {
fuzzyKeysData_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.PARSER, extensionRegistry));
break;
}
+ case 16: {
+ bitField0_ |= 0x00000001;
+ isMaskV2_ = input.readBool();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -7340,6 +7355,7 @@ public final class FilterProtos {
return PARSER;
}
+ private int bitField0_;
// repeated .hbase.pb.BytesBytesPair fuzzy_keys_data = 1;
public static final int FUZZY_KEYS_DATA_FIELD_NUMBER = 1;
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair> fuzzyKeysData_;
@@ -7376,8 +7392,25 @@ public final class FilterProtos {
return fuzzyKeysData_.get(index);
}
+ // optional bool is_mask_v2 = 2;
+ public static final int IS_MASK_V2_FIELD_NUMBER = 2;
+ private boolean isMaskV2_;
+ /**
+ * <code>optional bool is_mask_v2 = 2;</code>
+ */
+ public boolean hasIsMaskV2() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional bool is_mask_v2 = 2;</code>
+ */
+ public boolean getIsMaskV2() {
+ return isMaskV2_;
+ }
+
private void initFields() {
fuzzyKeysData_ = java.util.Collections.emptyList();
+ isMaskV2_ = false;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -7400,6 +7433,9 @@ public final class FilterProtos {
for (int i = 0; i < fuzzyKeysData_.size(); i++) {
output.writeMessage(1, fuzzyKeysData_.get(i));
}
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeBool(2, isMaskV2_);
+ }
getUnknownFields().writeTo(output);
}
@@ -7413,6 +7449,10 @@ public final class FilterProtos {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, fuzzyKeysData_.get(i));
}
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(2, isMaskV2_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -7438,6 +7478,11 @@ public final class FilterProtos {
boolean result = true;
result = result && getFuzzyKeysDataList()
.equals(other.getFuzzyKeysDataList());
+ result = result && (hasIsMaskV2() == other.hasIsMaskV2());
+ if (hasIsMaskV2()) {
+ result = result && (getIsMaskV2()
+ == other.getIsMaskV2());
+ }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -7455,6 +7500,10 @@ public final class FilterProtos {
hash = (37 * hash) + FUZZY_KEYS_DATA_FIELD_NUMBER;
hash = (53 * hash) + getFuzzyKeysDataList().hashCode();
}
+ if (hasIsMaskV2()) {
+ hash = (37 * hash) + IS_MASK_V2_FIELD_NUMBER;
+ hash = (53 * hash) + hashBoolean(getIsMaskV2());
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -7571,6 +7620,8 @@ public final class FilterProtos {
} else {
fuzzyKeysDataBuilder_.clear();
}
+ isMaskV2_ = false;
+ bitField0_ = (bitField0_ & ~0x00000002);
return this;
}
@@ -7598,6 +7649,7 @@ public final class FilterProtos {
public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter buildPartial() {
org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter result = new org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter(this);
int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
if (fuzzyKeysDataBuilder_ == null) {
if (((bitField0_ & 0x00000001) == 0x00000001)) {
fuzzyKeysData_ = java.util.Collections.unmodifiableList(fuzzyKeysData_);
@@ -7607,6 +7659,11 @@ public final class FilterProtos {
} else {
result.fuzzyKeysData_ = fuzzyKeysDataBuilder_.build();
}
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.isMaskV2_ = isMaskV2_;
+ result.bitField0_ = to_bitField0_;
onBuilt();
return result;
}
@@ -7648,6 +7705,9 @@ public final class FilterProtos {
}
}
}
+ if (other.hasIsMaskV2()) {
+ setIsMaskV2(other.getIsMaskV2());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -7921,6 +7981,39 @@ public final class FilterProtos {
return fuzzyKeysDataBuilder_;
}
+ // optional bool is_mask_v2 = 2;
+ private boolean isMaskV2_ ;
+ /**
+ * <code>optional bool is_mask_v2 = 2;</code>
+ */
+ public boolean hasIsMaskV2() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional bool is_mask_v2 = 2;</code>
+ */
+ public boolean getIsMaskV2() {
+ return isMaskV2_;
+ }
+ /**
+ * <code>optional bool is_mask_v2 = 2;</code>
+ */
+ public Builder setIsMaskV2(boolean value) {
+ bitField0_ |= 0x00000002;
+ isMaskV2_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bool is_mask_v2 = 2;</code>
+ */
+ public Builder clearIsMaskV2() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ isMaskV2_ = false;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:hbase.pb.FuzzyRowFilter)
}
@@ -17576,38 +17669,39 @@ public final class FilterProtos {
"MUST_PASS_ONE\020\002\"1\n\rFilterWrapper\022 \n\006filt" +
"er\030\001 \002(\0132\020.hbase.pb.Filter\"\024\n\022FirstKeyOn" +
"lyFilter\";\n%FirstKeyValueMatchingQualifi" +
- "ersFilter\022\022\n\nqualifiers\030\001 \003(\014\"C\n\016FuzzyRo" +
+ "ersFilter\022\022\n\nqualifiers\030\001 \003(\014\"W\n\016FuzzyRo" +
"wFilter\0221\n\017fuzzy_keys_data\030\001 \003(\0132\030.hbase" +
- ".pb.BytesBytesPair\"+\n\023InclusiveStopFilte" +
- "r\022\024\n\014stop_row_key\030\001 \001(\014\"#\n\rKeyOnlyFilter" +
- "\022\022\n\nlen_as_val\030\001 \002(\010\"5\n\032MultipleColumnPr" +
- "efixFilter\022\027\n\017sorted_prefixes\030\001 \003(\014\"\037\n\nP",
- "ageFilter\022\021\n\tpage_size\030\001 \002(\003\"\036\n\014PrefixFi" +
- "lter\022\016\n\006prefix\030\001 \001(\014\"B\n\017QualifierFilter\022" +
- "/\n\016compare_filter\030\001 \002(\0132\027.hbase.pb.Compa" +
- "reFilter\"!\n\017RandomRowFilter\022\016\n\006chance\030\001 " +
- "\002(\002\"<\n\tRowFilter\022/\n\016compare_filter\030\001 \002(\013" +
- "2\027.hbase.pb.CompareFilter\"g\n\036SingleColum" +
- "nValueExcludeFilter\022E\n\032single_column_val" +
- "ue_filter\030\001 \002(\0132!.hbase.pb.SingleColumnV" +
- "alueFilter\"\327\001\n\027SingleColumnValueFilter\022\025" +
- "\n\rcolumn_family\030\001 \001(\014\022\030\n\020column_qualifie",
- "r\030\002 \001(\014\022)\n\ncompare_op\030\003 \002(\0162\025.hbase.pb.C" +
- "ompareType\022(\n\ncomparator\030\004 \002(\0132\024.hbase.p" +
- "b.Comparator\022\031\n\021filter_if_missing\030\005 \001(\010\022" +
- "\033\n\023latest_version_only\030\006 \001(\010\".\n\nSkipFilt" +
- "er\022 \n\006filter\030\001 \002(\0132\020.hbase.pb.Filter\"<\n\020" +
- "TimestampsFilter\022\026\n\ntimestamps\030\001 \003(\003B\002\020\001" +
- "\022\020\n\010can_hint\030\002 \001(\010\">\n\013ValueFilter\022/\n\016com" +
- "pare_filter\030\001 \002(\0132\027.hbase.pb.CompareFilt" +
- "er\"4\n\020WhileMatchFilter\022 \n\006filter\030\001 \002(\0132\020" +
- ".hbase.pb.Filter\"\021\n\017FilterAllFilter\"h\n\010R",
- "owRange\022\021\n\tstart_row\030\001 \001(\014\022\033\n\023start_row_" +
- "inclusive\030\002 \001(\010\022\020\n\010stop_row\030\003 \001(\014\022\032\n\022sto" +
- "p_row_inclusive\030\004 \001(\010\"A\n\023MultiRowRangeFi" +
- "lter\022*\n\016row_range_list\030\001 \003(\0132\022.hbase.pb." +
- "RowRangeBB\n*org.apache.hadoop.hbase.prot" +
- "obuf.generatedB\014FilterProtosH\001\210\001\001\240\001\001"
+ ".pb.BytesBytesPair\022\022\n\nis_mask_v2\030\002 \001(\010\"+" +
+ "\n\023InclusiveStopFilter\022\024\n\014stop_row_key\030\001 " +
+ "\001(\014\"#\n\rKeyOnlyFilter\022\022\n\nlen_as_val\030\001 \002(\010" +
+ "\"5\n\032MultipleColumnPrefixFilter\022\027\n\017sorted",
+ "_prefixes\030\001 \003(\014\"\037\n\nPageFilter\022\021\n\tpage_si" +
+ "ze\030\001 \002(\003\"\036\n\014PrefixFilter\022\016\n\006prefix\030\001 \001(\014" +
+ "\"B\n\017QualifierFilter\022/\n\016compare_filter\030\001 " +
+ "\002(\0132\027.hbase.pb.CompareFilter\"!\n\017RandomRo" +
+ "wFilter\022\016\n\006chance\030\001 \002(\002\"<\n\tRowFilter\022/\n\016" +
+ "compare_filter\030\001 \002(\0132\027.hbase.pb.CompareF" +
+ "ilter\"g\n\036SingleColumnValueExcludeFilter\022" +
+ "E\n\032single_column_value_filter\030\001 \002(\0132!.hb" +
+ "ase.pb.SingleColumnValueFilter\"\327\001\n\027Singl" +
+ "eColumnValueFilter\022\025\n\rcolumn_family\030\001 \001(",
+ "\014\022\030\n\020column_qualifier\030\002 \001(\014\022)\n\ncompare_o" +
+ "p\030\003 \002(\0162\025.hbase.pb.CompareType\022(\n\ncompar" +
+ "ator\030\004 \002(\0132\024.hbase.pb.Comparator\022\031\n\021filt" +
+ "er_if_missing\030\005 \001(\010\022\033\n\023latest_version_on" +
+ "ly\030\006 \001(\010\".\n\nSkipFilter\022 \n\006filter\030\001 \002(\0132\020" +
+ ".hbase.pb.Filter\"<\n\020TimestampsFilter\022\026\n\n" +
+ "timestamps\030\001 \003(\003B\002\020\001\022\020\n\010can_hint\030\002 \001(\010\">" +
+ "\n\013ValueFilter\022/\n\016compare_filter\030\001 \002(\0132\027." +
+ "hbase.pb.CompareFilter\"4\n\020WhileMatchFilt" +
+ "er\022 \n\006filter\030\001 \002(\0132\020.hbase.pb.Filter\"\021\n\017",
+ "FilterAllFilter\"h\n\010RowRange\022\021\n\tstart_row" +
+ "\030\001 \001(\014\022\033\n\023start_row_inclusive\030\002 \001(\010\022\020\n\010s" +
+ "top_row\030\003 \001(\014\022\032\n\022stop_row_inclusive\030\004 \001(" +
+ "\010\"A\n\023MultiRowRangeFilter\022*\n\016row_range_li" +
+ "st\030\001 \003(\0132\022.hbase.pb.RowRangeBB\n*org.apac" +
+ "he.hadoop.hbase.protobuf.generatedB\014Filt" +
+ "erProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -17691,7 +17785,7 @@ public final class FilterProtos {
internal_static_hbase_pb_FuzzyRowFilter_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_hbase_pb_FuzzyRowFilter_descriptor,
- new java.lang.String[] { "FuzzyKeysData", });
+ new java.lang.String[] { "FuzzyKeysData", "IsMaskV2", });
internal_static_hbase_pb_InclusiveStopFilter_descriptor =
getDescriptor().getMessageTypes().get(13);
internal_static_hbase_pb_InclusiveStopFilter_fieldAccessorTable = new
diff --git a/hbase-protocol/src/main/protobuf/Filter.proto b/hbase-protocol/src/main/protobuf/Filter.proto
index 1fa6697..7f6a5e3 100644
--- a/hbase-protocol/src/main/protobuf/Filter.proto
+++ b/hbase-protocol/src/main/protobuf/Filter.proto
@@ -93,6 +93,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 c259979..e55fcdc 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,8 @@
*/
package org.apache.hadoop.hbase.filter;
+import java.util.Arrays;
+
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -24,9 +26,45 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.junit.internal.ArrayComparisonFailure;
@Category(SmallTests.class)
public class TestFuzzyRowFilter {
+
+ @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 fa0b77d..ab30f68 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
@@ -133,6 +133,15 @@ public class TestFuzzyRowFilterEndToEnd {
TEST_UTIL.flush();
+ // 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 };
@@ -142,7 +151,7 @@ public class TestFuzzyRowFilterEndToEnd {
byte[] copyMask = Arrays.copyOf(mask, mask.length);
data.add(new Pair<byte[], byte[]>(fuzzyKey, mask));
- FuzzyRowFilter filter = new FuzzyRowFilter(data);
+ FuzzyRowFilter filter = new FuzzyRowFilter(data, processedRowMask);
Scan scan = new Scan();
scan.setFilter(filter);
@@ -152,12 +161,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(table));
}
@Test
@@ -165,7 +172,7 @@ public class TestFuzzyRowFilterEndToEnd {
{
String cf = "f";
String cq = "q";
- String table = "HBASE14872";
+ String table = "HBASE14782";
Table ht =
TEST_UTIL.createTable(TableName.valueOf(table), Bytes.toBytes(cf), Integer.MAX_VALUE);
@@ -193,22 +200,30 @@ public class TestFuzzyRowFilterEndToEnd {
TEST_UTIL.flush();
- List<Pair<byte[], byte[]>> data = new ArrayList<Pair<byte[], byte[]>>();
+ testHBASE14782RunScanWithMask(ht, rows.length, FuzzyRowFilter.V1_PROCESSED_WILDCARD_MASK);
+ testHBASE14782RunScanWithMask(ht, rows.length, FuzzyRowFilter.V2_PROCESSED_WILDCARD_MASK);
+
+ TEST_UTIL.deleteTable(TableName.valueOf(table));
+ }
+
+ 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<byte[], byte[]>(fuzzyKey, mask));
- FuzzyRowFilter filter = new FuzzyRowFilter(data);
-
+ FuzzyRowFilter filter = new FuzzyRowFilter(data, processedRowMask);
+
Scan scan = new Scan();
scan.setFilter(filter);
-
+
ResultScanner scanner = ht.getScanner(scan);
int total = 0;
while(scanner.next() != null){
total++;
- }
- assertEquals(rows.length, total);
- TEST_UTIL.deleteTable(TableName.valueOf(table));
+ }
+ assertEquals(expectedRows, total);
}
@Test
@@ -250,12 +265,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 };
@@ -276,9 +293,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);
@@ -287,7 +304,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 };
@@ -310,9 +327,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);