Index: hbase-protocol/src/main/protobuf/hbase.proto =================================================================== --- hbase-protocol/src/main/protobuf/hbase.proto (revision 1427210) +++ hbase-protocol/src/main/protobuf/hbase.proto (working copy) @@ -268,3 +268,9 @@ optional string name = 1; optional int64 value = 2; } + +message BytesBytesBytesTriple { + required bytes first = 1; + required bytes second = 2; + optional bytes third = 3; +} Index: hbase-protocol/src/main/protobuf/Filter.proto =================================================================== --- hbase-protocol/src/main/protobuf/Filter.proto (revision 1427210) +++ hbase-protocol/src/main/protobuf/Filter.proto (working copy) @@ -85,7 +85,7 @@ } message FuzzyRowFilter { - repeated BytesBytesPair fuzzyKeysData = 1; + repeated BytesBytesBytesTriple fuzzyRule = 1; } message InclusiveStopFilter { Index: hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java =================================================================== --- hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java (revision 1427210) +++ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java (working copy) @@ -11303,6 +11303,534 @@ // @@protoc_insertion_point(class_scope:NameInt64Pair) } + public interface BytesBytesBytesTripleOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required bytes first = 1; + boolean hasFirst(); + com.google.protobuf.ByteString getFirst(); + + // required bytes second = 2; + boolean hasSecond(); + com.google.protobuf.ByteString getSecond(); + + // optional bytes third = 3; + boolean hasThird(); + com.google.protobuf.ByteString getThird(); + } + public static final class BytesBytesBytesTriple extends + com.google.protobuf.GeneratedMessage + implements BytesBytesBytesTripleOrBuilder { + // Use BytesBytesBytesTriple.newBuilder() to construct. + private BytesBytesBytesTriple(Builder builder) { + super(builder); + } + private BytesBytesBytesTriple(boolean noInit) {} + + private static final BytesBytesBytesTriple defaultInstance; + public static BytesBytesBytesTriple getDefaultInstance() { + return defaultInstance; + } + + public BytesBytesBytesTriple getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_BytesBytesBytesTriple_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_BytesBytesBytesTriple_fieldAccessorTable; + } + + private int bitField0_; + // required bytes first = 1; + public static final int FIRST_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString first_; + public boolean hasFirst() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public com.google.protobuf.ByteString getFirst() { + return first_; + } + + // required bytes second = 2; + public static final int SECOND_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString second_; + public boolean hasSecond() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public com.google.protobuf.ByteString getSecond() { + return second_; + } + + // optional bytes third = 3; + public static final int THIRD_FIELD_NUMBER = 3; + private com.google.protobuf.ByteString third_; + public boolean hasThird() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public com.google.protobuf.ByteString getThird() { + return third_; + } + + private void initFields() { + first_ = com.google.protobuf.ByteString.EMPTY; + second_ = com.google.protobuf.ByteString.EMPTY; + third_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasFirst()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSecond()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, first_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, second_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, third_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, first_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, second_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, third_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple) obj; + + boolean result = true; + result = result && (hasFirst() == other.hasFirst()); + if (hasFirst()) { + result = result && getFirst() + .equals(other.getFirst()); + } + result = result && (hasSecond() == other.hasSecond()); + if (hasSecond()) { + result = result && getSecond() + .equals(other.getSecond()); + } + result = result && (hasThird() == other.hasThird()); + if (hasThird()) { + result = result && getThird() + .equals(other.getThird()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasFirst()) { + hash = (37 * hash) + FIRST_FIELD_NUMBER; + hash = (53 * hash) + getFirst().hashCode(); + } + if (hasSecond()) { + hash = (37 * hash) + SECOND_FIELD_NUMBER; + hash = (53 * hash) + getSecond().hashCode(); + } + if (hasThird()) { + hash = (37 * hash) + THIRD_FIELD_NUMBER; + hash = (53 * hash) + getThird().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTripleOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_BytesBytesBytesTriple_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_BytesBytesBytesTriple_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + first_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + second_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + third_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple build() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.first_ = first_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.second_ = second_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.third_ = third_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.getDefaultInstance()) return this; + if (other.hasFirst()) { + setFirst(other.getFirst()); + } + if (other.hasSecond()) { + setSecond(other.getSecond()); + } + if (other.hasThird()) { + setThird(other.getThird()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasFirst()) { + + return false; + } + if (!hasSecond()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + first_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + second_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + third_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required bytes first = 1; + private com.google.protobuf.ByteString first_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasFirst() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public com.google.protobuf.ByteString getFirst() { + return first_; + } + public Builder setFirst(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + first_ = value; + onChanged(); + return this; + } + public Builder clearFirst() { + bitField0_ = (bitField0_ & ~0x00000001); + first_ = getDefaultInstance().getFirst(); + onChanged(); + return this; + } + + // required bytes second = 2; + private com.google.protobuf.ByteString second_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasSecond() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public com.google.protobuf.ByteString getSecond() { + return second_; + } + public Builder setSecond(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + second_ = value; + onChanged(); + return this; + } + public Builder clearSecond() { + bitField0_ = (bitField0_ & ~0x00000002); + second_ = getDefaultInstance().getSecond(); + onChanged(); + return this; + } + + // optional bytes third = 3; + private com.google.protobuf.ByteString third_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasThird() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public com.google.protobuf.ByteString getThird() { + return third_; + } + public Builder setThird(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + third_ = value; + onChanged(); + return this; + } + public Builder clearThird() { + bitField0_ = (bitField0_ & ~0x00000004); + third_ = getDefaultInstance().getThird(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:BytesBytesBytesTriple) + } + + static { + defaultInstance = new BytesBytesBytesTriple(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:BytesBytesBytesTriple) + } + private static com.google.protobuf.Descriptors.Descriptor internal_static_TableSchema_descriptor; private static @@ -11388,6 +11916,11 @@ private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_NameInt64Pair_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_BytesBytesBytesTriple_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_BytesBytesBytesTriple_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -11440,14 +11973,16 @@ "value\030\002 \002(\t\",\n\rNameBytesPair\022\014\n\004name\030\001 \002" + "(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesBytesPair\022\r\n\005f" + "irst\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\",\n\rNameInt64P" + - "air\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 \001(\003*r\n\013Comp" + - "areType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005" + - "EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQU" + - "AL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*_\n\007KeyType\022" + - "\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021\n\rDEL" + - "ETE_COLUMN\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIM" + - "UM\020\377\001B>\n*org.apache.hadoop.hbase.protobu", - "f.generatedB\013HBaseProtosH\001\240\001\001" + "air\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"E\n\025Byte" + + "sBytesBytesTriple\022\r\n\005first\030\001 \002(\014\022\016\n\006seco" + + "nd\030\002 \002(\014\022\r\n\005third\030\003 \001(\014*r\n\013CompareType\022\010" + + "\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r" + + "\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007G" + + "REATER\020\005\022\t\n\005NO_OP\020\006*_\n\007KeyType\022\013\n\007MINIMU" + + "M\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021\n\rDELETE_COLUM", + "N\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIMUM\020\377\001B>\n*" + + "org.apache.hadoop.hbase.protobuf.generat" + + "edB\013HBaseProtosH\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -11590,6 +12125,14 @@ new java.lang.String[] { "Name", "Value", }, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameInt64Pair.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameInt64Pair.Builder.class); + internal_static_BytesBytesBytesTriple_descriptor = + getDescriptor().getMessageTypes().get(15); + internal_static_BytesBytesBytesTriple_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_BytesBytesBytesTriple_descriptor, + new java.lang.String[] { "First", "Second", "Third", }, + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.class, + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder.class); return null; } }; Index: hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java =================================================================== --- hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java (revision 1427210) +++ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java (working copy) @@ -5428,14 +5428,14 @@ public interface FuzzyRowFilterOrBuilder extends com.google.protobuf.MessageOrBuilder { - // repeated .BytesBytesPair fuzzyKeysData = 1; - java.util.List + // repeated .BytesBytesBytesTriple fuzzyKeysData = 1; + java.util.List getFuzzyKeysDataList(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getFuzzyKeysData(int index); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple getFuzzyKeysData(int index); int getFuzzyKeysDataCount(); - java.util.List + java.util.List getFuzzyKeysDataOrBuilderList(); - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getFuzzyKeysDataOrBuilder( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTripleOrBuilder getFuzzyKeysDataOrBuilder( int index); } public static final class FuzzyRowFilter extends @@ -5466,23 +5466,23 @@ return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FuzzyRowFilter_fieldAccessorTable; } - // repeated .BytesBytesPair fuzzyKeysData = 1; + // repeated .BytesBytesBytesTriple fuzzyKeysData = 1; public static final int FUZZYKEYSDATA_FIELD_NUMBER = 1; - private java.util.List fuzzyKeysData_; - public java.util.List getFuzzyKeysDataList() { + private java.util.List fuzzyKeysData_; + public java.util.List getFuzzyKeysDataList() { return fuzzyKeysData_; } - public java.util.List + public java.util.List getFuzzyKeysDataOrBuilderList() { return fuzzyKeysData_; } public int getFuzzyKeysDataCount() { return fuzzyKeysData_.size(); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getFuzzyKeysData(int index) { + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple getFuzzyKeysData(int index) { return fuzzyKeysData_.get(index); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getFuzzyKeysDataOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTripleOrBuilder getFuzzyKeysDataOrBuilder( int index) { return fuzzyKeysData_.get(index); } @@ -5810,7 +5810,7 @@ break; } case 10: { - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.newBuilder(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.newBuilder(); input.readMessage(subBuilder, extensionRegistry); addFuzzyKeysData(subBuilder.buildPartial()); break; @@ -5821,20 +5821,20 @@ private int bitField0_; - // repeated .BytesBytesPair fuzzyKeysData = 1; - private java.util.List fuzzyKeysData_ = + // repeated .BytesBytesBytesTriple fuzzyKeysData = 1; + private java.util.List fuzzyKeysData_ = java.util.Collections.emptyList(); private void ensureFuzzyKeysDataIsMutable() { if (!((bitField0_ & 0x00000001) == 0x00000001)) { - fuzzyKeysData_ = new java.util.ArrayList(fuzzyKeysData_); + fuzzyKeysData_ = new java.util.ArrayList(fuzzyKeysData_); bitField0_ |= 0x00000001; } } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> fuzzyKeysDataBuilder_; + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTripleOrBuilder> fuzzyKeysDataBuilder_; - public java.util.List getFuzzyKeysDataList() { + public java.util.List getFuzzyKeysDataList() { if (fuzzyKeysDataBuilder_ == null) { return java.util.Collections.unmodifiableList(fuzzyKeysData_); } else { @@ -5848,7 +5848,7 @@ return fuzzyKeysDataBuilder_.getCount(); } } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getFuzzyKeysData(int index) { + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple getFuzzyKeysData(int index) { if (fuzzyKeysDataBuilder_ == null) { return fuzzyKeysData_.get(index); } else { @@ -5856,7 +5856,7 @@ } } public Builder setFuzzyKeysData( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair value) { + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple value) { if (fuzzyKeysDataBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -5870,7 +5870,7 @@ return this; } public Builder setFuzzyKeysData( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) { + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder builderForValue) { if (fuzzyKeysDataBuilder_ == null) { ensureFuzzyKeysDataIsMutable(); fuzzyKeysData_.set(index, builderForValue.build()); @@ -5880,7 +5880,7 @@ } return this; } - public Builder addFuzzyKeysData(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair value) { + public Builder addFuzzyKeysData(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple value) { if (fuzzyKeysDataBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -5894,7 +5894,7 @@ return this; } public Builder addFuzzyKeysData( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair value) { + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple value) { if (fuzzyKeysDataBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -5908,7 +5908,7 @@ return this; } public Builder addFuzzyKeysData( - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder builderForValue) { if (fuzzyKeysDataBuilder_ == null) { ensureFuzzyKeysDataIsMutable(); fuzzyKeysData_.add(builderForValue.build()); @@ -5919,7 +5919,7 @@ return this; } public Builder addFuzzyKeysData( - int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) { + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder builderForValue) { if (fuzzyKeysDataBuilder_ == null) { ensureFuzzyKeysDataIsMutable(); fuzzyKeysData_.add(index, builderForValue.build()); @@ -5930,7 +5930,7 @@ return this; } public Builder addAllFuzzyKeysData( - java.lang.Iterable values) { + java.lang.Iterable values) { if (fuzzyKeysDataBuilder_ == null) { ensureFuzzyKeysDataIsMutable(); super.addAll(values, fuzzyKeysData_); @@ -5960,18 +5960,18 @@ } return this; } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder getFuzzyKeysDataBuilder( + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder getFuzzyKeysDataBuilder( int index) { return getFuzzyKeysDataFieldBuilder().getBuilder(index); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getFuzzyKeysDataOrBuilder( + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTripleOrBuilder getFuzzyKeysDataOrBuilder( int index) { if (fuzzyKeysDataBuilder_ == null) { return fuzzyKeysData_.get(index); } else { return fuzzyKeysDataBuilder_.getMessageOrBuilder(index); } } - public java.util.List + public java.util.List getFuzzyKeysDataOrBuilderList() { if (fuzzyKeysDataBuilder_ != null) { return fuzzyKeysDataBuilder_.getMessageOrBuilderList(); @@ -5979,25 +5979,25 @@ return java.util.Collections.unmodifiableList(fuzzyKeysData_); } } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder addFuzzyKeysDataBuilder() { + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder addFuzzyKeysDataBuilder() { return getFuzzyKeysDataFieldBuilder().addBuilder( - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.getDefaultInstance()); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.getDefaultInstance()); } - public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder addFuzzyKeysDataBuilder( + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder addFuzzyKeysDataBuilder( int index) { return getFuzzyKeysDataFieldBuilder().addBuilder( - index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.getDefaultInstance()); + index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.getDefaultInstance()); } - public java.util.List + public java.util.List getFuzzyKeysDataBuilderList() { return getFuzzyKeysDataFieldBuilder().getBuilderList(); } private com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTripleOrBuilder> getFuzzyKeysDataFieldBuilder() { if (fuzzyKeysDataBuilder_ == null) { fuzzyKeysDataBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTriple.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesBytesTripleOrBuilder>( fuzzyKeysData_, ((bitField0_ & 0x00000001) == 0x00000001), getParentForChildren(), @@ -12687,32 +12687,32 @@ "ALL\020\001\022\021\n\rMUST_PASS_ONE\020\002\"(\n\rFilterWrappe" + "r\022\027\n\006filter\030\001 \002(\0132\007.Filter\"\024\n\022FirstKeyOn" + "lyFilter\";\n%FirstKeyValueMatchingQualifi", - "ersFilter\022\022\n\nqualifiers\030\001 \003(\014\"8\n\016FuzzyRo" + - "wFilter\022&\n\rfuzzyKeysData\030\001 \003(\0132\017.BytesBy" + - "tesPair\")\n\023InclusiveStopFilter\022\022\n\nstopRo" + - "wKey\030\001 \001(\014\"!\n\rKeyOnlyFilter\022\020\n\010lenAsVal\030" + - "\001 \002(\010\"4\n\032MultipleColumnPrefixFilter\022\026\n\016s" + - "ortedPrefixes\030\001 \003(\014\"\036\n\nPageFilter\022\020\n\010pag" + - "eSize\030\001 \002(\003\"\036\n\014PrefixFilter\022\016\n\006prefix\030\001 " + - "\001(\014\"8\n\017QualifierFilter\022%\n\rcompareFilter\030" + - "\001 \002(\0132\016.CompareFilter\"!\n\017RandomRowFilter" + - "\022\016\n\006chance\030\001 \002(\002\"2\n\tRowFilter\022%\n\rcompare", - "Filter\030\001 \002(\0132\016.CompareFilter\"[\n\036SingleCo" + - "lumnValueExcludeFilter\0229\n\027singleColumnVa" + - "lueFilter\030\001 \002(\0132\030.SingleColumnValueFilte" + - "r\"\352\001\n\027SingleColumnValueFilter\022\024\n\014columnF" + - "amily\030\001 \001(\014\022\027\n\017columnQualifier\030\002 \001(\014\022\037\n\t" + - "compareOp\030\003 \002(\0162\014.CompareType\022\037\n\ncompara" + - "tor\030\004 \002(\0132\013.Comparator\022\023\n\013foundColumn\030\005 " + - "\001(\010\022\025\n\rmatchedColumn\030\006 \001(\010\022\027\n\017filterIfMi" + - "ssing\030\007 \001(\010\022\031\n\021latestVersionOnly\030\010 \001(\010\"%" + - "\n\nSkipFilter\022\027\n\006filter\030\001 \002(\0132\007.Filter\"&\n", - "\020TimestampsFilter\022\022\n\ntimestamps\030\001 \003(\003\"4\n" + - "\013ValueFilter\022%\n\rcompareFilter\030\001 \002(\0132\016.Co" + - "mpareFilter\"+\n\020WhileMatchFilter\022\027\n\006filte" + - "r\030\001 \002(\0132\007.FilterBB\n*org.apache.hadoop.hb" + - "ase.protobuf.generatedB\014FilterProtosH\001\210\001" + - "\001\240\001\001" + "ersFilter\022\022\n\nqualifiers\030\001 \003(\014\"?\n\016FuzzyRo" + + "wFilter\022-\n\rfuzzyKeysData\030\001 \003(\0132\026.BytesBy" + + "tesBytesTriple\")\n\023InclusiveStopFilter\022\022\n" + + "\nstopRowKey\030\001 \001(\014\"!\n\rKeyOnlyFilter\022\020\n\010le" + + "nAsVal\030\001 \002(\010\"4\n\032MultipleColumnPrefixFilt" + + "er\022\026\n\016sortedPrefixes\030\001 \003(\014\"\036\n\nPageFilter" + + "\022\020\n\010pageSize\030\001 \002(\003\"\036\n\014PrefixFilter\022\016\n\006pr" + + "efix\030\001 \001(\014\"8\n\017QualifierFilter\022%\n\rcompare" + + "Filter\030\001 \002(\0132\016.CompareFilter\"!\n\017RandomRo" + + "wFilter\022\016\n\006chance\030\001 \002(\002\"2\n\tRowFilter\022%\n\r", + "compareFilter\030\001 \002(\0132\016.CompareFilter\"[\n\036S" + + "ingleColumnValueExcludeFilter\0229\n\027singleC" + + "olumnValueFilter\030\001 \002(\0132\030.SingleColumnVal" + + "ueFilter\"\352\001\n\027SingleColumnValueFilter\022\024\n\014" + + "columnFamily\030\001 \001(\014\022\027\n\017columnQualifier\030\002 " + + "\001(\014\022\037\n\tcompareOp\030\003 \002(\0162\014.CompareType\022\037\n\n" + + "comparator\030\004 \002(\0132\013.Comparator\022\023\n\013foundCo" + + "lumn\030\005 \001(\010\022\025\n\rmatchedColumn\030\006 \001(\010\022\027\n\017fil" + + "terIfMissing\030\007 \001(\010\022\031\n\021latestVersionOnly\030" + + "\010 \001(\010\"%\n\nSkipFilter\022\027\n\006filter\030\001 \002(\0132\007.Fi", + "lter\"&\n\020TimestampsFilter\022\022\n\ntimestamps\030\001" + + " \003(\003\"4\n\013ValueFilter\022%\n\rcompareFilter\030\001 \002" + + "(\0132\016.CompareFilter\"+\n\020WhileMatchFilter\022\027" + + "\n\006filter\030\001 \002(\0132\007.FilterBB\n*org.apache.ha" + + "doop.hbase.protobuf.generatedB\014FilterPro" + + "tosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { Index: hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java (revision 1427210) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java (working copy) @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.Triple; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -150,9 +150,12 @@ @Test public void testFuzzyRowFilter() throws Exception { - LinkedList> fuzzyList = new LinkedList>(); - fuzzyList.add(new Pair(Bytes.toBytes("999"),new byte[] {0, 0, 1})); - fuzzyList.add(new Pair(Bytes.toBytes("abcd"),new byte[] {1, 0, 1, 1})); + LinkedList> fuzzyList = + new LinkedList>(); + fuzzyList.add(new Triple( + new byte[] {0, 0, 1}, Bytes.toBytes("999"), null)); + fuzzyList.add(new Triple( + new byte[] {1, 0, 1, 1}, Bytes.toBytes("abcd"), Bytes.toBytes("bcde"))); FuzzyRowFilter fuzzyRowFilter = new FuzzyRowFilter(fuzzyList); assertTrue(fuzzyRowFilter.areSerializedFieldsEqual( ProtobufUtil.toFilter(ProtobufUtil.toFilter(fuzzyRowFilter)))); Index: hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java (revision 1427210) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java (working copy) @@ -17,86 +17,146 @@ */ package org.apache.hadoop.hbase.filter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Triple; import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; @Category(SmallTests.class) public class TestFuzzyRowFilter { + // max byte value (lexographically) + private static final byte M = (byte) 255; + @Test - public void testSatisfies() { + public void testSatisfiesNoRange() { Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS, - FuzzyRowFilter.satisfies(new byte[]{1, (byte) -128, 0, 0, 1}, // row to check + satisfies(new byte[]{1, (byte) -128, 0, 0, 1}, // row to check new byte[]{1, 0, 1}, // fuzzy row new byte[]{0, 1, 0})); // mask Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.YES, - FuzzyRowFilter.satisfies(new byte[]{1, (byte) -128, 1, 0, 1}, + satisfies(new byte[]{1, (byte) -128, 1, 0, 1}, new byte[]{1, 0, 1}, new byte[]{0, 1, 0})); Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS, - FuzzyRowFilter.satisfies(new byte[]{1, (byte) -128, 2, 0, 1}, + satisfies(new byte[]{1, (byte) -128, 2, 0, 1}, new byte[]{1, 0, 1}, new byte[]{0, 1, 0})); Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NO_NEXT, - FuzzyRowFilter.satisfies(new byte[]{2, 3, 1, 1, 1}, + satisfies(new byte[]{2, 3, 1, 1, 1}, new byte[]{1, 0, 1}, new byte[]{0, 1, 0})); Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.YES, - FuzzyRowFilter.satisfies(new byte[]{1, 2, 1, 3, 3}, + satisfies(new byte[]{1, 2, 1, 3, 3}, new byte[]{1, 2, 0, 3}, new byte[]{0, 0, 1, 0})); Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS, - FuzzyRowFilter.satisfies(new byte[]{1, 1, 1, 3, 0}, // row to check + satisfies(new byte[]{1, 1, 1, 3, 0}, // row to check new byte[]{1, 2, 0, 3}, // fuzzy row new byte[]{0, 0, 1, 0})); // mask Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS, - FuzzyRowFilter.satisfies(new byte[]{1, 1, 1, 3, 0}, + satisfies(new byte[]{1, 1, 1, 3, 0}, new byte[]{1, (byte) 245, 0, 3}, new byte[]{0, 0, 1, 0})); Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NO_NEXT, - FuzzyRowFilter.satisfies(new byte[]{1, (byte) 245, 1, 3, 0}, + satisfies(new byte[]{1, (byte) 245, 1, 3, 0}, new byte[]{1, 1, 0, 3}, new byte[]{0, 0, 1, 0})); Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NO_NEXT, - FuzzyRowFilter.satisfies(new byte[]{1, 3, 1, 3, 0}, + satisfies(new byte[]{1, 3, 1, 3, 0}, new byte[]{1, 2, 0, 3}, new byte[]{0, 0, 1, 0})); Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NO_NEXT, - FuzzyRowFilter.satisfies(new byte[]{2, 1, 1, 1, 0}, + satisfies(new byte[]{2, 1, 1, 1, 0}, new byte[]{1, 2, 0, 3}, new byte[]{0, 0, 1, 0})); Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS, - FuzzyRowFilter.satisfies(new byte[]{1, 2, 1, 0, 1}, + satisfies(new byte[]{1, 2, 1, 0, 1}, new byte[]{0, 1, 2}, new byte[]{1, 0, 0})); + + Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NO_NEXT, + satisfies(new byte[]{5, M, 6}, + new byte[]{5, 1, 5}, + new byte[]{0, 1, 0})); + + Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.YES, + satisfies( + new byte[]{1, 1, 0, 0}, + new byte[]{0, 0, 1, 1}, + new byte[]{1, 1, M, M})); + + // Exists: {1, 1, 2, 0} + Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS, + satisfies( + new byte[]{1, 1}, + new byte[]{0, 0, 0, 1}, + new byte[]{1, 1, 2, 0})); + + // Exists: {1, 1, 0, 0} + Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS, + satisfies( + new byte[]{1, 1}, + new byte[]{0, 0, 1, 1}, + new byte[]{1, 1, 0, 0})); + } + private static FuzzyRowFilter.SatisfiesCode satisfies(byte[] row, + byte[] lowerBytes, byte[] mask) { + return FuzzyRowFilter.satisfies(row, + new Triple (mask, lowerBytes, null)); + } + @Test public void testGetNextForFuzzyRule() { assertNext( new byte[]{0, 1, 2}, // fuzzy row new byte[]{1, 0, 0}, // mask new byte[]{1, 2, 1, 0, 1}, // current - new byte[]{2, 1, 2, 0, 0}); // expected next + new byte[]{2, 1, 2}); // expected next assertNext( - new byte[]{0, 1, 2}, // fuzzy row - new byte[]{1, 0, 0}, // mask - new byte[]{1, 1, 2, 0, 1}, // current - new byte[]{1, 1, 2, 0, 2}); // expected next + new byte[]{1, 1, 0, 0}, + new byte[]{0, 0, 1, 1}, + new byte[]{1, 1}, + new byte[]{1, 1, 0, 0}); assertNext( + new byte[]{1, 1, 2, 0}, + new byte[]{0, 0, 0, 1}, + new byte[]{1, 1}, + new byte[]{1, 1, 2, 0}); + + assertNext( + new byte[]{0, 1, 2}, + new byte[]{1, 0, 0}, + new byte[]{1, 1, 2, 0, 1}, + new byte[]{1, 1, 2, 0, 2}); + + assertNext( new byte[]{0, 1, 0, 2, 0}, // fuzzy row new byte[]{1, 0, 1, 0, 1}, // mask new byte[]{1, 0, 2, 0, 1}, // current @@ -106,7 +166,7 @@ new byte[]{1, 0, 1}, new byte[]{0, 1, 0}, new byte[]{1, (byte) 128, 2, 0, 1}, - new byte[]{1, (byte) 129, 1, 0, 0}); + new byte[]{1, (byte) 129, 1}); assertNext( new byte[]{0, 1, 0, 1}, @@ -123,26 +183,26 @@ assertNext( new byte[]{0, 1, 0, 0}, // fuzzy row new byte[]{1, 0, 1, 1}, // mask - new byte[]{5, 1, (byte) 255, 1}, // current - new byte[]{5, 1, (byte) 255, 2}); // expected next + new byte[]{5, 1, M, 1}, // current + new byte[]{5, 1, M, 2}); // expected next assertNext( new byte[]{0, 1, 0, 1}, // fuzzy row new byte[]{1, 0, 1, 0}, // mask - new byte[]{5, 1, (byte) 255, 1}, // current + new byte[]{5, 1, M, 1}, // current new byte[]{6, 1, 0, 1}); // expected next assertNext( new byte[]{0, 1, 0, 1}, // fuzzy row new byte[]{1, 0, 1, 0}, // mask - new byte[]{5, 1, (byte) 255, 0}, // current - new byte[]{5, 1, (byte) 255, 1}); // expected next + new byte[]{5, 1, M, 0}, // current + new byte[]{5, 1, M, 1}); // expected next assertNext( new byte[]{5, 1, 1, 0}, new byte[]{0, 0, 1, 1}, - new byte[]{5, 1, (byte) 255, 1}, - new byte[]{5, 1, (byte) 255, 2}); + new byte[]{5, 1, M, 1}, + new byte[]{5, 1, M, 2}); assertNext( new byte[]{1, 1, 1, 1}, @@ -175,27 +235,183 @@ new byte[]{1, 1, 0, 0}); // No next for this one - Assert.assertNull(FuzzyRowFilter.getNextForFuzzyRule( + Assert.assertNull(getNextForFuzzyRule( new byte[]{2, 3, 1, 1, 1}, // row to check - new byte[]{1, 0, 1}, // fuzzy row + new byte[]{1, 0, 1}, // fuzzy row bytes (lower) new byte[]{0, 1, 0})); // mask - Assert.assertNull(FuzzyRowFilter.getNextForFuzzyRule( + Assert.assertNull(getNextForFuzzyRule( new byte[]{1, (byte) 245, 1, 3, 0}, new byte[]{1, 1, 0, 3}, new byte[]{0, 0, 1, 0})); - Assert.assertNull(FuzzyRowFilter.getNextForFuzzyRule( + Assert.assertNull(getNextForFuzzyRule( new byte[]{1, 3, 1, 3, 0}, new byte[]{1, 2, 0, 3}, new byte[]{0, 0, 1, 0})); - Assert.assertNull(FuzzyRowFilter.getNextForFuzzyRule( + Assert.assertNull(getNextForFuzzyRule( new byte[]{2, 1, 1, 1, 0}, new byte[]{1, 2, 0, 3}, new byte[]{0, 0, 1, 0})); } - private void assertNext(byte[] fuzzyRow, byte[] mask, byte[] current, byte[] expected) { - byte[] nextForFuzzyRule = FuzzyRowFilter.getNextForFuzzyRule(current, fuzzyRow, mask); + private byte[] getNextForFuzzyRule(byte[] row, byte[] lowerBytes, byte[] mask) { + return FuzzyRowFilter.getNextForFuzzyRule( + row, new Triple(mask, lowerBytes, null)); + } + + private void assertNext(byte[] lowerBytes, byte[] mask, byte[] current, byte[] expected) { + byte[] nextForFuzzyRule = + FuzzyRowFilter.getNextForFuzzyRule( + current, new Triple(mask, lowerBytes, null)); + System.out.println(Arrays.toString(nextForFuzzyRule)); Assert.assertArrayEquals(expected, nextForFuzzyRule); } + @Test + public void testGetNextForFuzzyRangeRule() { + assertNextForRange( + new byte[]{0, 0, 0, 0, 1, 0, 0, 0, 0, 1}, + new byte[]{0, 0, 0, 0, 9, 9, 0, 0, 0, 9}, + new byte[]{1, 1, 1, 0, 0, 0, 1, 1, 0, 0}, + new byte[]{1, 2, 3, 0, 0, 9, 4, 5, 0, 6}, + new byte[]{1, 2, 3, 0, 1, 0, 0, 0, 0, 1}); + + assertNextForRange( + new byte[]{0, 0, 0, 0, 0, 1, 0, 0}, + new byte[]{0, 0, 0, 0, 9, 9, 0, 0}, + new byte[]{1, 1, 1, 0, 0, 0, 1, 1}, + new byte[]{1, 2, 3, 0, 9, 9}, + new byte[]{1, 2, 3, 0, 9, 9, 0, 0}); + + assertNextForRange( + new byte[]{0, 0, 0, 0, 0, 1, 2}, + new byte[]{0, 0, 0, 0, 9, 9, 3}, + new byte[]{1, 1, 1, 0, 0, 0, 0}, + new byte[]{1, 2, 3, 0, 9, 9}, + new byte[]{1, 2, 3, 0, 9, 9, 2}); + + assertNextForRange( + new byte[]{0, 0, 0, 0, 0, 1, 0, 0, 0, 1}, + new byte[]{0, 0, 0, 0, 9, 9, 0, 0, 0, 9}, + new byte[]{1, 1, 1, 0, 0, 0, 1, 1, 0, 0}, + new byte[]{1, 2, 3, 0, 1, 7, 4, 5, 0, 6}, + new byte[]{1, 2, 3, 0, 1, 7, 4, 5, 0, 7}); + + assertNextForRange( + new byte[]{0, 0, 0, 0, 0, 1, 0, 0, 0, 1}, + new byte[]{0, 0, 0, 0, 9, 9, 0, 0, 0, 9}, + new byte[]{1, 1, 1, 0, 0, 0, 1, 1, 0, 0}, + new byte[]{1, 2, 3, 0, 1, 7, 4, 5, 0, 9}, + new byte[]{1, 2, 3, 0, 1, 7, 4, 6, 0, 1}); + + assertNextForRange( + new byte[]{0, 0, 0, 0, 0, 1, 0, 0, 0, 1}, + new byte[]{0, 0, 0, 0, 9, 9, 0, 0, 0, 9}, + new byte[]{1, 1, 1, 0, 0, 0, 1, 1, 0, 0}, + new byte[]{1, 2, 3, 0, 1, 7, M, M, 0, 9}, + new byte[]{1, 2, 3, 0, 1, 8, 0, 0, 0, 1}); + + assertNextForRange( + new byte[]{0, 0, 0, 0, 0, 1, 0, 0, 0, 1}, + new byte[]{0, 0, 0, 0, 9, 9, 0, 0, 0, 9}, + new byte[]{1, 1, 1, 0, 0, 0, 1, 1, 0, 0}, + new byte[]{1, 2, 3, 0, 1, M, M, M, 0, 9}, + new byte[]{1, 2, 3, 0, 2, 0, 0, 0, 0, 1}); + + assertNextForRange( + new byte[]{0, 0, 0, 0, 0, 1, 0, 0, 0, 1}, + new byte[]{0, 0, 0, 0, 9, 9, 0, 0, 0, 9}, + new byte[]{1, 1, 1, 0, 0, 0, 1, 1, 0, 0}, + new byte[]{1, 2, 3, 0, 8, M-1, M, M, 0, 9}, + new byte[]{1, 2, 3, 0, 8, M, 0, 0, 0, 1}); + + assertNextForRange( + new byte[]{0, 0, 0, 0, 0, 1, 0, 0, 0, 1}, + new byte[]{0, 0, 0, 0, 9, 9, 0, 0, 0, 9}, + new byte[]{1, 1, 1, 0, 0, 0, 1, 1, 0, 0}, + new byte[]{1, 2, 3, 0, 9, 9, M, M, 0, 9}, + new byte[]{1, 2, 4, 0, 0, 1, 0, 0, 0, 1}); + } + + private void assertNextForRange(byte[] lowerBytes, byte[] upperBytes, byte[] mask, + byte[] current, byte[] expected) { + byte[] nextForFuzzyRule = + FuzzyRowFilter.getNextForFuzzyRule( + current, new Triple(mask, lowerBytes, upperBytes)); + System.out.println(Arrays.toString(nextForFuzzyRule)); + + Assert.assertArrayEquals(expected, nextForFuzzyRule); + } + + @Test + @Category(MediumTests.class) + public void testScan() throws Exception { + // simplest, acts as "integration" test + HBaseTestingUtility testUtil = new HBaseTestingUtility(); + testUtil.startMiniCluster(); + try { + byte[] cf = Bytes.toBytes("colfam"); + HTable ht = testUtil.createTable(Bytes.toBytes("table"), cf); + + byte[] q = Bytes.toBytes("qual"); + insertRow(ht, cf, q, "01A1A", "1"); + insertRow(ht, cf, q, "01A1B", "1"); + insertRow(ht, cf, q, "01A2A", "1"); + insertRow(ht, cf, q, "01A2B", "1"); + insertRow(ht, cf, q, "01A3C", "1"); + insertRow(ht, cf, q, "01B1A", "1"); + insertRow(ht, cf, q, "01B2B", "1"); + insertRow(ht, cf, q, "02A1A", "1"); + insertRow(ht, cf, q, "02B1A", "1"); + insertRow(ht, cf, q, "02C1A", "1"); + insertRow(ht, cf, q, "03C1A", "1"); + insertRow(ht, cf, q, "03C1B", "1"); + insertRow(ht, cf, q, "03C1C", "1"); + + // looking for: + // ??A?[B..C] and ??C?[B..C] + List> fuzzyRules = + new ArrayList>(); + // ??A?[B..C] + fuzzyRules.add(new Triple( + new byte[] {1, 1, 0, 1, 0}, + new byte[] {0, 0, (byte) 'A', 0, (byte) 'B'}, + new byte[] {0, 0, (byte) 'A', 0, (byte) 'C'} + )); + // ??C?[B..C] + fuzzyRules.add(new Triple( + new byte[] {1, 1, 0, 1, 0}, + new byte[] {0, 0, (byte) 'C', 0, (byte) 'B'}, + new byte[] {0, 0, (byte) 'C', 0, (byte) 'C'} + )); + FuzzyRowFilter fuzzyRowFilter = new FuzzyRowFilter(fuzzyRules); + Scan scan = new Scan(); + scan.setFilter(fuzzyRowFilter); + ResultScanner rs = ht.getScanner(scan); + + Assert.assertEquals("01A1B", Bytes.toString(rs.next().getRow())); + Assert.assertEquals("01A2B", Bytes.toString(rs.next().getRow())); + Assert.assertEquals("01A3C", Bytes.toString(rs.next().getRow())); + Assert.assertEquals("03C1B", Bytes.toString(rs.next().getRow())); + Assert.assertEquals("03C1C", Bytes.toString(rs.next().getRow())); + + Assert.assertNull(rs.next()); + + rs.close(); + + } finally { + testUtil.shutdownMiniCluster(); + } + + } + + private static void insertRow(HTable hTable, byte[] colfam, byte[] qual, + String rowKey, String value) throws IOException { + Put put = new Put(Bytes.toBytes(rowKey)); + put.add(colfam, qual, Bytes.toBytes(value)); + hTable.put(put); + } + +// @org.junit.Rule +// public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = +// new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); } Index: hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java (revision 1427210) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java (working copy) @@ -26,17 +26,23 @@ import org.apache.hadoop.hbase.DeserializationException; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; import com.google.protobuf.ByteString; import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.hbase.util.Triple; /** * Filters data based on fuzzy row key. Performs fast-forwards during scanning. - * It takes pairs (row key, fuzzy info) to match row keys. Where fuzzy info is - * a byte array with 0 or 1 as its values: + *

+ * It takes triples (mask, ranges lower bytes, ranges upper bytes) to match row keys. Each triple + * defines independent fuzzy rule. Multiple fuzzy rules are applied with "at least one should match" + * operator. + *

+ *

+ * "mask" defines fixed and non-fixed positions in a row key. It is a byte array with 0 or 1 as its + * values: *

    *
  • * 0 - means that this byte in provided row key is fixed, i.e. row key's byte at same position @@ -47,28 +53,100 @@ * position can be different from the one in provided row key *
  • *
+ *

+ *

+ * "ranges lower bytes" and "ranges upper bytes" defines bytes to match against. Zero bytes should + * be at non-fixed positions (defined by mask). Sequence of fixed bytes defines acceptable range of + * values with lower value from "lower range bytes" and upper value from "upper range bytes". Every + * range is independent. See examples below for more details. + *

+ *

+ * "ranges upper bytes" value is optional: when absent (i.e. null) exact match with + * ranges lower bytes is used (see examples below). + *

* - * - * Example: + *

+ * Examples: + *

+ *

Example 1. Exact match at fixed positions

+ *

* Let's assume row key format is userId_actionId_year_month. Length of userId is fixed - * and is 4, length of actionId is 2 and year and month are 4 and 2 bytes long respectively. - * + * and is 4, length of actionId is 2 and year and months are 4 and 2 bytes long respectively.
* Let's assume that we need to fetch all users that performed certain action (encoded as "99") - * in Jan of any year. Then the pair (row key, fuzzy info) would be the following: - * row key = "????_99_????_01" (one can use any value instead of "?") - * fuzzy info = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00" + * in Jan of any year. Then the triple that defines fuzzy rule would be the following: + *

    + *
  • + * mask = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00" + * ("\x00" is (byte) 0, "\x01" is (byte) 1) + *
  • + *
  • ranges lower bytes = "\x00\x00\x00\x00_99_\x00\x00\x00\x00_01"
  • + *
  • ranges upper bytes = null
  • + *
+ * I.e. fuzzy rule tells the matching mask is "????_99_????_01", where at ? can be any single byte. + *

* - * I.e. fuzzy info tells the matching mask is "????_99_????_01", where at ? can be any value. + *

Example 2. Defining ranges

+ *

+ * Let's change the problem of the first example: action codes that we are looking for are in the + * range "55".."99" and month we are interested in are Jan, Feb, Mar. In this case we'd define + * fuzzy rule the following way: + *

    + *
  • mask = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00" (same)
  • + *
  • ranges lower bytes = "\x00\x00\x00\x00_55_\x00\x00\x00\x00_01" ("????_55_????_01")
  • + *
  • ranges upper bytes = "\x00\x00\x00\x00_99_\x00\x00\x00\x00_03" ("????_99_????_03")
  • + *
+ *

* + *

Example 3. Multiple fuzzy rules

+ *

+ * Let's change the problem of the second example: months we are interested in are Jan and Aug. In + * this case it may be beneficial to define two fuzzy rules (one for each month):
+ * First: + *

    + *
  • mask = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00" (same)
  • + *
  • ranges lower bytes = "\x00\x00\x00\x00_55_\x00\x00\x00\x00_01" ("????_55_????_01")
  • + *
  • ranges upper bytes = "\x00\x00\x00\x00_99_\x00\x00\x00\x00_01" ("????_99_????_01")
  • + *
+ * Second: + *
    + *
  • mask = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00" (same)
  • + *
  • ranges lower bytes = "\x00\x00\x00\x00_55_\x00\x00\x00\x00_08" ("????_55_????_08")
  • + *
  • ranges upper bytes = "\x00\x00\x00\x00_99_\x00\x00\x00\x00_08" ("????_99_????_08")
  • + *
+ *

+ * + *

+ * NOTE that non-fixed position ("?" in the examples above) is treated as "at least + * (byte) 0 should present at this position in the row key. I.e. if mask ends with + * non-fixed position the length of the matching row key should be not smaller than the length of + * the mask. + *

+ * + *

+ * NOTE that currently no checks are performed to ensure that length of ranges lower bytes and + * ranges upper bytes match mask length. Filter may work incorrectly or fail (with runtime + * exceptions) if this is broken. + *

+ * + *

+ * NOTE that currently no checks are performed to ensure that ranges are defined correctly (i.e. + * lower value of each range is not greater than upper value). Filter may work incorrectly or fail + * (with runtime exceptions) if this is broken. + *

+ * + *

+ * NOTE that currently no checks are performed to ensure that at non-fixed positions in + * ranges lower bytes and ranges upper bytes zeroes are set, but implementation may rely on this. + *

*/ @InterfaceAudience.Public @InterfaceStability.Evolving public class FuzzyRowFilter extends FilterBase { - private List> fuzzyKeysData; + private List> fuzzyRules; private boolean done = false; - public FuzzyRowFilter(List> fuzzyKeysData) { - this.fuzzyKeysData = fuzzyKeysData; + public FuzzyRowFilter(List> fuzzyRules) { + this.fuzzyRules = fuzzyRules; } // TODO: possible improvement: save which fuzzy row key to use when providing a hint @@ -77,9 +155,8 @@ byte[] rowKey = kv.getRow(); // assigning "worst" result first and looking for better options SatisfiesCode bestOption = SatisfiesCode.NO_NEXT; - for (Pair fuzzyData : fuzzyKeysData) { - SatisfiesCode satisfiesCode = - satisfies(rowKey, fuzzyData.getFirst(), fuzzyData.getSecond()); + for (Triple fuzzyRule : fuzzyRules) { + SatisfiesCode satisfiesCode = satisfies(rowKey, fuzzyRule); if (satisfiesCode == SatisfiesCode.YES) { return ReturnCode.INCLUDE; } @@ -103,9 +180,8 @@ byte[] rowKey = currentKV.getRow(); byte[] nextRowKey = null; // Searching for the "smallest" row key that satisfies at least one fuzzy row key - for (Pair fuzzyData : fuzzyKeysData) { - byte[] nextRowKeyCandidate = getNextForFuzzyRule(rowKey, - fuzzyData.getFirst(), fuzzyData.getSecond()); + for (Triple fuzzyRule : fuzzyRules) { + byte[] nextRowKeyCandidate = getNextForFuzzyRule(rowKey, fuzzyRule); if (nextRowKeyCandidate == null) { continue; } @@ -117,7 +193,7 @@ if (nextRowKey == null) { // SHOULD NEVER happen // TODO: is there a better way than throw exception? (stop the scanner?) - throw new IllegalStateException("No next row key that satisfies fuzzy exists when" + + throw new IllegalStateException("No next row key that satisfies fuzzy rules exists when" + " getNextKeyHint() is invoked." + " Filter: " + this.toString() + " currentKV: " + currentKV.toString()); @@ -134,13 +210,19 @@ /** * @return The filter serialized using pb */ + @Override public byte [] toByteArray() { FilterProtos.FuzzyRowFilter.Builder builder = FilterProtos.FuzzyRowFilter.newBuilder(); - for (Pair fuzzyData : fuzzyKeysData) { - BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder(); - bbpBuilder.setFirst(ByteString.copyFrom(fuzzyData.getFirst())); - bbpBuilder.setSecond(ByteString.copyFrom(fuzzyData.getSecond())); + for (Triple fuzzyRule : fuzzyRules) { + HBaseProtos.BytesBytesBytesTriple.Builder bbpBuilder = + HBaseProtos.BytesBytesBytesTriple.newBuilder(); + bbpBuilder.setFirst(ByteString.copyFrom(fuzzyRule.getFirst())); + bbpBuilder.setSecond(ByteString.copyFrom(fuzzyRule.getSecond())); + byte[] third = fuzzyRule.getThird(); + if (third != null) { + bbpBuilder.setThird(ByteString.copyFrom(third)); + } builder.addFuzzyKeysData(bbpBuilder); } return builder.build().toByteArray(); @@ -161,12 +243,16 @@ throw new DeserializationException(e); } int count = proto.getFuzzyKeysDataCount(); - ArrayList> fuzzyKeysData= new ArrayList>(count); + ArrayList> fuzzyKeysData = + new ArrayList>(count); for (int i = 0; i < count; ++i) { - BytesBytesPair current = proto.getFuzzyKeysData(i); - byte[] keyBytes = current.getFirst().toByteArray(); - byte[] keyMeta = current.getSecond().toByteArray(); - fuzzyKeysData.add(new Pair(keyBytes, keyMeta)); + HBaseProtos.BytesBytesBytesTriple current = proto.getFuzzyKeysData(i); + byte[] mask = current.getFirst().toByteArray(); + byte[] lowerRangesBytes = current.getSecond().toByteArray(); + byte[] third = current.getThird().toByteArray(); + // NOTE: "(byte[]) null" serialized as "byte[0]", hence this check + byte[] upperRangesBytes = third.length == 0 ? null : third; + fuzzyKeysData.add(new Triple(mask, lowerRangesBytes, upperRangesBytes)); } return new FuzzyRowFilter(fuzzyKeysData); } @@ -175,15 +261,49 @@ public String toString() { final StringBuilder sb = new StringBuilder(); sb.append("FuzzyRowFilter"); - sb.append("{fuzzyKeysData="); - for (Pair fuzzyData : fuzzyKeysData) { - sb.append('{').append(Bytes.toStringBinary(fuzzyData.getFirst())).append(":"); - sb.append(Bytes.toStringBinary(fuzzyData.getSecond())).append('}'); + sb.append("{fuzzyRule="); + for (Triple fuzzyRule : fuzzyRules) { + sb.append('{'); + sb.append(Bytes.toStringBinary(fuzzyRule.getFirst())).append(":"); + sb.append(Bytes.toStringBinary(fuzzyRule.getSecond())).append(':'); + sb.append(Bytes.toStringBinary(fuzzyRule.getThird())); + sb.append('}'); } sb.append("}, "); return sb.toString(); } + /** + * @param o value to compare with + * @return true if and only if the fields of the filter that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + @Override + boolean areSerializedFieldsEqual(Filter o) { + if (o == this) return true; + if (!(o instanceof FuzzyRowFilter)) return false; + + FuzzyRowFilter other = (FuzzyRowFilter)o; + if (this.fuzzyRules.size() != other.fuzzyRules.size()) return false; + for (int i = 0; i < fuzzyRules.size(); ++i) { + Triple thisData = this.fuzzyRules.get(i); + Triple otherData = other.fuzzyRules.get(i); + boolean equals = Bytes.equals(thisData.getFirst(), otherData.getFirst()) + && Bytes.equals(thisData.getSecond(), otherData.getSecond()) + && Bytes.equals(thisData.getThird(), otherData.getThird()); + // NOTE: Bytes.equals(null, ) = Bytes.equals(, null) = true. Hence extra check for + // optional field + equals = equals && ( + (thisData.getThird() == null && otherData.getThird() == null) || + (thisData.getThird() != null && otherData.getThird() != null) + ); + if (!equals) { + return false; + } + } + return true; + } + // Utility methods static enum SatisfiesCode { @@ -195,13 +315,18 @@ NO_NEXT } - static SatisfiesCode satisfies(byte[] row, - byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { - return satisfies(row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta); + static SatisfiesCode satisfies(byte[] rowKey, Triple fuzzyRule) { + return satisfies(rowKey, + fuzzyRule.getFirst(), + fuzzyRule.getSecond(), + // using lower bytes as upper for exact match + fuzzyRule.getThird() == null ? fuzzyRule.getSecond() : fuzzyRule.getThird()); } - private static SatisfiesCode satisfies(byte[] row, int offset, int length, - byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { + private static SatisfiesCode satisfies(byte[] row, + byte[] mask, + byte[] lowerBytes, + byte[] upperBytes) { if (row == null) { // do nothing, let scan to proceed return SatisfiesCode.YES; @@ -209,125 +334,210 @@ boolean nextRowKeyCandidateExists = false; - for (int i = 0; i < fuzzyKeyMeta.length && i < length; i++) { + for (int i = 0; i < mask.length && i < row.length; i++) { // First, checking if this position is fixed and not equals the given one - boolean byteAtPositionFixed = fuzzyKeyMeta[i] == 0; - boolean fixedByteIncorrect = byteAtPositionFixed && fuzzyKeyBytes[i] != row[i + offset]; - if (fixedByteIncorrect) { - // in this case there's another row that satisfies fuzzy rule and bigger than this row - if (nextRowKeyCandidateExists) { - return SatisfiesCode.NEXT_EXISTS; + boolean byteAtPositionFixed = mask[i] == 0; + if (byteAtPositionFixed) { + boolean fixedByteIncorrect = + !byteInRangeInclusive(row[i], lowerBytes[i], upperBytes[i]); + + if (fixedByteIncorrect) { + // in this case there's another row that satisfies fuzzy rule and greater than this row + if (nextRowKeyCandidateExists) { + return SatisfiesCode.NEXT_EXISTS; + } + + // If this row byte is less than fixed then there's a byte array greater than + // this row and which satisfies the fuzzy rule (we can simply inc this byte to the fixed + // byte value). + // Otherwise there's no such byte array: this row is simply greater than any byte array + // that satisfies the fuzzy rule + boolean rowByteLessThanFixed = + (row[i] & 0xFF) < (upperBytes[i] & 0xFF); + return rowByteLessThanFixed ? SatisfiesCode.NEXT_EXISTS : SatisfiesCode.NO_NEXT; } - - // If this row byte is less than fixed then there's a byte array bigger than - // this row and which satisfies the fuzzy rule. Otherwise there's no such byte array: - // this row is simply bigger than any byte array that satisfies the fuzzy rule - boolean rowByteLessThanFixed = (row[i + offset] & 0xFF) < (fuzzyKeyBytes[i] & 0xFF); - return rowByteLessThanFixed ? SatisfiesCode.NEXT_EXISTS : SatisfiesCode.NO_NEXT; } - // Second, checking if this position is not fixed and byte value is not the biggest. In this - // case there's a byte array bigger than this row and which satisfies the fuzzy rule. To get - // bigger byte array that satisfies the rule we need to just increase this byte - // (see the code of getNextForFuzzyRule below) by one. - // Note: if non-fixed byte is already at biggest value, this doesn't allow us to say there's - // bigger one that satisfies the rule as it can't be increased. - if (fuzzyKeyMeta[i] == 1 && !isMax(fuzzyKeyBytes[i])) { + // Second, checking if this position is not fixed and byte value is not the biggest possible. + // If this is the case + // there's a byte array greater than this row and which satisfies the fuzzy rule. To build + // greater byte array that satisfies the rule we need e.g. to just increase this byte by one. + // Note: if non-fixed byte is already at biggest value, this + // doesn't allow us to say that there's greater one that satisfies the rule. Because + // current row first i bytes cannot be changed (increased) to satisfy the fuzzy rule + boolean byteAtPositionNotFixed = mask[i] == 1; + if (byteAtPositionNotFixed && !isMax(row[i])) { nextRowKeyCandidateExists = true; } } + if (row.length < lowerBytes.length) { + // If the row length is less than fuzzy rule, it does NOT satisfies. + // At the same time, next exists, because current row satisfies first part of fuzzy rule (we + // did not exit in the loop above) and + // by appending rest of the bytes from fuzzy rule we will get next row that satisfies and is + // greater. + return SatisfiesCode.NEXT_EXISTS; + } + return SatisfiesCode.YES; } - private static boolean isMax(byte fuzzyKeyByte) { - return (fuzzyKeyByte & 0xFF) == 255; - } - - static byte[] getNextForFuzzyRule(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { - return getNextForFuzzyRule(row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta); - } - /** * @return greater byte array than given (row) which satisfies the fuzzy rule if it exists, * null otherwise */ - private static byte[] getNextForFuzzyRule(byte[] row, int offset, int length, - byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { - // To find out the next "smallest" byte array that satisfies fuzzy rule and "greater" than - // the given one we do the following: - // 1. setting values on all "fixed" positions to the values from fuzzyKeyBytes - // 2. if during the first step given row did not increase, then we increase the value at - // the first "non-fixed" position (where it is not maximum already) + static byte[] getNextForFuzzyRule(byte[] rowKey, + Triple fuzzyRule) { + return getNextForFuzzyRule(rowKey, + fuzzyRule.getFirst(), + fuzzyRule.getSecond(), + // use lower fuzzy row bytes as upper to get exact match + fuzzyRule.getThird() == null ? fuzzyRule.getFirst() : fuzzyRule.getThird()); + } - // It is easier to perform this by using fuzzyKeyBytes copy and setting "non-fixed" position - // values than otherwise. - byte[] result = Arrays.copyOf(fuzzyKeyBytes, - length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length); + private static byte[] getNextForFuzzyRule(byte[] row, + byte[] mask, + byte[] lowerBytes, byte[] upperBytes) { + // Some description for algorithm can be found in HBASE-6618. Here's a high-level explanation. + // Basically we need to find a way to build the smallest possible byte array that satisfies + // fuzzy rule but which greater than given row. For + // that we traverse row from left to right and looking for a byte that can be increased. + // 1. When we look at fixed positions, we compare bytes with those in lowerBytes and upperBytes. + // 1a. The first byte we find that is less than the fixed byte should be considered to be + // _the_ byte to increase and build the next row key. We have to inc it as otherwise the row key + // doesn't satisfy the fuzzy rule. + // 1b. If we don't meet the byte from 1a, we are looking at those bytes that can be (but not + // "necessarily have to be" as in 1a) increased based on provided range. We would increase the + // most right such byte so that the built row key is the smallest possible. + // 2. The same way as in 1a, we consider every byte at non-fixed position as the one that can be + // increased (if it is not at max value already). We also looking for the most right such byte. + // So, we either inc byte we found in 1a or the most right of those find in 1b or 2. + + + // We are going to modify byte array in place, so we make copy to not break values passed + // as params. + byte[] result; + if (row.length >= lowerBytes.length) { + result = Arrays.copyOf(row, row.length); + } else { + // what actually happens here is appending bytes from lowerBytes to row key so that + // result is as long as lowerBytes and satisfies + result = Arrays.copyOf(row, lowerBytes.length); + System.arraycopy(lowerBytes, row.length, result, row.length, + lowerBytes.length - row.length); + } + + // Looking for a way to build greater byte array that satisfies fuzzy rule. int toInc = -1; - - boolean increased = false; + // canChangeFixed used to track if we are inside a group of fixed bytes that can be increased + // based on range lower & upper values + boolean canChangeFixed = false; + boolean increasedBySettingFixed = false; for (int i = 0; i < result.length; i++) { - if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) { - result[i] = row[offset + i]; - if (!isMax(row[i])) { - // this is "non-fixed" position and is not at max value, hence we can increase it + // a) not fixed byte case (point #2 in algo description above) + if (i > mask.length - 1 || mask[i] == 1) { + canChangeFixed = false; + if (!isMax(result[i])) { toInc = i; } - } else if (i < fuzzyKeyMeta.length && fuzzyKeyMeta[i] == 0) { - if ((row[i + offset] & 0xFF) < (fuzzyKeyBytes[i] & 0xFF)) { - // if setting value for any fixed position increased the original array, - // we are OK - increased = true; + continue; + } + + // b) fixed byte case (point #1 in algo description above) + if ((result[i] & 0xFF) < (lowerBytes[i] & 0xFF)) { + // We encountered smaller byte than smallest suggested by fuzzy rules. Increasing it and + // breaking the loop. This is point #1a in algo description above. + result[i] = lowerBytes[i]; + increasedBySettingFixed = true; + toInc = i; + break; + } + + if (!canChangeFixed) { + if ((result[i] & 0xFF) > (upperBytes[i] & 0xFF)) { + // We encountered greater byte than fixed and we cannot change (increase) any of the + // previous bytes to make it satisfy the fuzzy rule. + // Now it depends on toInc: if it is -1, then our row is greater than any which satisfies + // the fuzzy rule break; } - if ((row[i + offset] & 0xFF) > (fuzzyKeyBytes[i] & 0xFF)) { - // if setting value for any fixed position makes array "smaller", then just stop: - // in case we found some non-fixed position to increase we will do it, otherwise - // there's no "next" row key that satisfies fuzzy rule and "greater" than given row - break; + if ((result[i] & 0xFF) < (upperBytes[i] & 0xFF)) { + canChangeFixed = true; + toInc = i; } + } else { + if (!isMax(result[i])) { + toInc = i; + } } } - if (!increased) { - if (toInc < 0) { - return null; - } + if (toInc < 0) { + // our row is greater than any which satisfies the fuzzy rule + return null; + } + + if (toInc > row.length - 1) { + // do NOT increase *appended* bytes: by appending we already increased the row + return result; + } + + if (!increasedBySettingFixed) { + // we increase only if we really need to increase (#1b and #2 points in algo description), + // otherwise we *set* required value (#1a in algo description). result[toInc]++; + } - // Setting all "non-fixed" positions to zeroes to the right of the one we increased so - // that found "next" row key is the smallest possible - for (int i = toInc + 1; i < result.length; i++) { - if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) { - result[i] = 0; - } + if (toInc > lowerBytes.length - 1) { + // We increased byte which is OUT OF fuzzy rule bytes range + // we need to return result byte array which ends with this byte to make it next smallest byte + // array in lexographical sense (note: e.g. {0,0,1} is less than {0,0,1,0} lexographically). + return result.length - 1 == toInc ? result : Bytes.head(result, toInc + 1); + + } else if (result.length > lowerBytes.length) { + // We increased byte which is IN fuzzy rule bytes range. We throw away all other bytes which + // are in positions after fuzzy row rule (to get next minimal row key that satisfies) + result = Arrays.copyOf(result, lowerBytes.length); + } + + // Setting bytes to the right of the one we increased (so that found "next" row key is the + // smallest possible) depending on toInc: + // a) toInc position was non-fixed: + // * all "non-fixed" positions to zeroes and + // * fixed bytes to lower bytes value + // b) toInc position was fixed: + // * all "non-fixed" positions to zeroes and + // * all fixed from this group to zeroes and all fixed in other groups to the smallest values + // from fuzzyKeyBytesFirst + boolean increasedFixedPosition = mask[toInc] == 0; + boolean setBytesOfThisFixedGroupToZeroes = increasedFixedPosition && + // was it set to minimum? + result[toInc] > lowerBytes[toInc]; + for (int i = toInc + 1; i < result.length; i++) { + if (i > mask.length - 1 || mask[i] == 1) { + result[i] = 0; + // resetting, so that next groups are set to their minimum values, but not zeroes + setBytesOfThisFixedGroupToZeroes = false; + continue; } + + if (setBytesOfThisFixedGroupToZeroes) { + result[i] = 0; + } else { + result[i] = lowerBytes[i]; + } } return result; } - /** - * @param other - * @return true if and only if the fields of the filter that are serialized - * are equal to the corresponding fields in other. Used for testing. - */ - boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof FuzzyRowFilter)) return false; + private static boolean byteInRangeInclusive(byte value, + byte valueRangeStart, byte valueRangeEnd) { + return (value & 0xFF) >= (valueRangeStart & 0xFF) && (value & 0xFF) <= (valueRangeEnd & 0xFF); + } - FuzzyRowFilter other = (FuzzyRowFilter)o; - if (this.fuzzyKeysData.size() != other.fuzzyKeysData.size()) return false; - for (int i = 0; i < fuzzyKeysData.size(); ++i) { - Pair thisData = this.fuzzyKeysData.get(i); - Pair otherData = other.fuzzyKeysData.get(i); - if (!(Bytes.equals(thisData.getFirst(), otherData.getFirst()) - && Bytes.equals(thisData.getSecond(), otherData.getSecond()))) { - return false; - } - } - return true; + private static boolean isMax(byte fuzzyKeyByte) { + return (fuzzyKeyByte & 0xFF) == 255; } - }