diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java index d42433d..e97f1a3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java @@ -33,7 +33,11 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.filter.ExclusiveStartFilter; import org.apache.hadoop.hbase.filter.Filter; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.filter.InclusiveStopFilter; import org.apache.hadoop.hbase.filter.IncompatibleFilterException; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.util.Bytes; @@ -592,6 +596,71 @@ public class Scan extends Query { } /** + * Set up a Scan instance for reversed scanning + * Adjust start row / stop row so that + * 1. the start row for scan is greater than stop row + * 2. start row is excluded from scan result + * 3. stop row is included in scan result + *

+ * Filter(s) is used to achieve the above. Any filter involved has reversed flag set + * + * @param Scan Scan object to be prepared for reversed scanning + * @return Scan object ready for reversed scanning + */ + public static Scan makeReversedExcludingStartRow(Scan original) throws IOException { + Scan scan = new Scan(original); + scan.reversed = true; + byte[] startRow = scan.getStartRow(); + byte[] stopRow = scan.getStopRow(); + + // early out for full scan + if (startRow.length == 0 && stopRow.length == 0) return scan; + + if (startRow.length > 0 && stopRow.length > 0 && + Bytes.compareTo(startRow, stopRow) < 0) { + // swap start row with stop row + scan.setStartRow(stopRow); + scan.setStopRow(startRow); + } + // Filter instance to be applied to scan + Filter filter; + ExclusiveStartFilter esFilter = null; + if (scan.getStartRow().length > 0) { + esFilter = new ExclusiveStartFilter(scan.getStartRow()); + esFilter.setReversed(true); + } + if (scan.getStopRow().length > 0) { + InclusiveStopFilter isFilter = new InclusiveStopFilter(scan.getStopRow()); + isFilter.setReversed(true); + if (esFilter != null) { + FilterList fl = new FilterList(Operator.MUST_PASS_ALL); + fl.setReversed(true); + fl.addFilter(isFilter); + fl.addFilter(esFilter); + filter = fl; + } else { + filter = isFilter; + } + // we don't need the stop row - InclusiveStopFilter accomplishes its role + scan.setStopRow(HConstants.EMPTY_BYTE_ARRAY); + } else { + filter = esFilter; + } + if (scan.getFilter() == null) { + scan.setFilter(filter); + } else { + FilterList fl = new FilterList(Operator.MUST_PASS_ALL); + fl.setReversed(true); + // all involved filters should be reversed + scan.getFilter().setReversed(true); + fl.addFilter(scan.getFilter()); + fl.addFilter(filter); + scan.setFilter(fl); + } + return scan; + } + + /** * Get whether this scan is a reversed one. * @return true if backward scan, false if forward(default) scan */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java index c96ca11..2f40a5f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java @@ -70,7 +70,7 @@ public class InclusiveStopFilter extends FilterBase { int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length, buffer, offset, length); - if(cmp < 0) { + if (!isReversed() && cmp < 0 || isReversed() && cmp > 0) { done = true; } return done; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java index dd8eb62..6def8ab 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.filter.ColumnPrefixFilter; import org.apache.hadoop.hbase.filter.ColumnRangeFilter; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.DependentColumnFilter; +import org.apache.hadoop.hbase.filter.ExclusiveStartFilter; import org.apache.hadoop.hbase.filter.FamilyFilter; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; @@ -122,6 +123,12 @@ public class TestOperation { private static String STR_IS_FILTER = IS_FILTER.getClass().getSimpleName() + " " + STOP_ROW_KEY; + private static String START_ROW_KEY = "start"; + private static ExclusiveStartFilter ES_FILTER = + new ExclusiveStartFilter(START_ROW_KEY.getBytes()); + private static String STR_ES_FILTER = + IS_FILTER.getClass().getSimpleName() + " " + START_ROW_KEY; + private static String PREFIX = "prefix"; private static PrefixFilter PREFIX_FILTER = new PrefixFilter(PREFIX.getBytes()); 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 9b2ce5e..95c001f 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 @@ -8369,6 +8369,443 @@ public final class FilterProtos { // @@protoc_insertion_point(class_scope:InclusiveStopFilter) } + public interface ExclusiveStartFilterOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional bytes start_row_key = 1; + /** + * optional bytes start_row_key = 1; + */ + boolean hasStartRowKey(); + /** + * optional bytes start_row_key = 1; + */ + com.google.protobuf.ByteString getStartRowKey(); + } + /** + * Protobuf type {@code ExclusiveStartFilter} + */ + public static final class ExclusiveStartFilter extends + com.google.protobuf.GeneratedMessage + implements ExclusiveStartFilterOrBuilder { + // Use ExclusiveStartFilter.newBuilder() to construct. + private ExclusiveStartFilter(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private ExclusiveStartFilter(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final ExclusiveStartFilter defaultInstance; + public static ExclusiveStartFilter getDefaultInstance() { + return defaultInstance; + } + + public ExclusiveStartFilter getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ExclusiveStartFilter( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + startRowKey_ = input.readBytes(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_ExclusiveStartFilter_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_ExclusiveStartFilter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public ExclusiveStartFilter parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ExclusiveStartFilter(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // optional bytes start_row_key = 1; + public static final int START_ROW_KEY_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString startRowKey_; + /** + * optional bytes start_row_key = 1; + */ + public boolean hasStartRowKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional bytes start_row_key = 1; + */ + public com.google.protobuf.ByteString getStartRowKey() { + return startRowKey_; + } + + private void initFields() { + startRowKey_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, startRowKey_); + } + 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, startRowKey_); + } + 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.FilterProtos.ExclusiveStartFilter)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter other = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter) obj; + + boolean result = true; + result = result && (hasStartRowKey() == other.hasStartRowKey()); + if (hasStartRowKey()) { + result = result && getStartRowKey() + .equals(other.getStartRowKey()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasStartRowKey()) { + hash = (37 * hash) + START_ROW_KEY_FIELD_NUMBER; + hash = (53 * hash) + getStartRowKey().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter 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; + } + /** + * Protobuf type {@code ExclusiveStartFilter} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilterOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_ExclusiveStartFilter_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_ExclusiveStartFilter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.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(); + startRowKey_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_ExclusiveStartFilter_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter build() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter result = new org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.startRowKey_ = startRowKey_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter.getDefaultInstance()) return this; + if (other.hasStartRowKey()) { + setStartRowKey(other.getStartRowKey()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ExclusiveStartFilter) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // optional bytes start_row_key = 1; + private com.google.protobuf.ByteString startRowKey_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes start_row_key = 1; + */ + public boolean hasStartRowKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional bytes start_row_key = 1; + */ + public com.google.protobuf.ByteString getStartRowKey() { + return startRowKey_; + } + /** + * optional bytes start_row_key = 1; + */ + public Builder setStartRowKey(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + startRowKey_ = value; + onChanged(); + return this; + } + /** + * optional bytes start_row_key = 1; + */ + public Builder clearStartRowKey() { + bitField0_ = (bitField0_ & ~0x00000001); + startRowKey_ = getDefaultInstance().getStartRowKey(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:ExclusiveStartFilter) + } + + static { + defaultInstance = new ExclusiveStartFilter(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:ExclusiveStartFilter) + } + public interface KeyOnlyFilterOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -15952,6 +16389,11 @@ public final class FilterProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_InclusiveStopFilter_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor + internal_static_ExclusiveStartFilter_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_ExclusiveStartFilter_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor internal_static_KeyOnlyFilter_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable @@ -16055,29 +16497,30 @@ public final class FilterProtos { "QualifiersFilter\022\022\n\nqualifiers\030\001 \003(\014\":\n\016" + "FuzzyRowFilter\022(\n\017fuzzy_keys_data\030\001 \003(\0132" + "\017.BytesBytesPair\"+\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\032MultipleColumnPref" + - "ixFilter\022\027\n\017sorted_prefixes\030\001 \003(\014\"\037\n\nPag" + - "eFilter\022\021\n\tpage_size\030\001 \002(\003\"\036\n\014PrefixFilt" + - "er\022\016\n\006prefix\030\001 \001(\014\"9\n\017QualifierFilter\022&\n", - "\016compare_filter\030\001 \002(\0132\016.CompareFilter\"!\n" + - "\017RandomRowFilter\022\016\n\006chance\030\001 \002(\002\"3\n\tRowF" + - "ilter\022&\n\016compare_filter\030\001 \002(\0132\016.CompareF" + - "ilter\"^\n\036SingleColumnValueExcludeFilter\022" + - "<\n\032single_column_value_filter\030\001 \002(\0132\030.Si" + - "ngleColumnValueFilter\"\305\001\n\027SingleColumnVa" + - "lueFilter\022\025\n\rcolumn_family\030\001 \001(\014\022\030\n\020colu" + - "mn_qualifier\030\002 \001(\014\022 \n\ncompare_op\030\003 \002(\0162\014" + - ".CompareType\022\037\n\ncomparator\030\004 \002(\0132\013.Compa" + - "rator\022\031\n\021filter_if_missing\030\005 \001(\010\022\033\n\023late", - "st_version_only\030\006 \001(\010\"%\n\nSkipFilter\022\027\n\006f" + - "ilter\030\001 \002(\0132\007.Filter\"*\n\020TimestampsFilter" + - "\022\026\n\ntimestamps\030\001 \003(\003B\002\020\001\"5\n\013ValueFilter\022" + - "&\n\016compare_filter\030\001 \002(\0132\016.CompareFilter\"" + - "+\n\020WhileMatchFilter\022\027\n\006filter\030\001 \002(\0132\007.Fi" + - "lter\"\021\n\017FilterAllFilterBB\n*org.apache.ha" + - "doop.hbase.protobuf.generatedB\014FilterPro" + - "tosH\001\210\001\001\240\001\001" + "\024\n\014stop_row_key\030\001 \001(\014\"-\n\024ExclusiveStartF" + + "ilter\022\025\n\rstart_row_key\030\001 \001(\014\"#\n\rKeyOnlyF" + + "ilter\022\022\n\nlen_as_val\030\001 \002(\010\"5\n\032MultipleCol" + + "umnPrefixFilter\022\027\n\017sorted_prefixes\030\001 \003(\014" + + "\"\037\n\nPageFilter\022\021\n\tpage_size\030\001 \002(\003\"\036\n\014Pre", + "fixFilter\022\016\n\006prefix\030\001 \001(\014\"9\n\017QualifierFi" + + "lter\022&\n\016compare_filter\030\001 \002(\0132\016.CompareFi" + + "lter\"!\n\017RandomRowFilter\022\016\n\006chance\030\001 \002(\002\"" + + "3\n\tRowFilter\022&\n\016compare_filter\030\001 \002(\0132\016.C" + + "ompareFilter\"^\n\036SingleColumnValueExclude" + + "Filter\022<\n\032single_column_value_filter\030\001 \002" + + "(\0132\030.SingleColumnValueFilter\"\305\001\n\027SingleC" + + "olumnValueFilter\022\025\n\rcolumn_family\030\001 \001(\014\022" + + "\030\n\020column_qualifier\030\002 \001(\014\022 \n\ncompare_op\030" + + "\003 \002(\0162\014.CompareType\022\037\n\ncomparator\030\004 \002(\0132", + "\013.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\027\n\006filter\030\001 \002(\0132\007.Filter\"*\n\020Timestamp" + + "sFilter\022\026\n\ntimestamps\030\001 \003(\003B\002\020\001\"5\n\013Value" + + "Filter\022&\n\016compare_filter\030\001 \002(\0132\016.Compare" + + "Filter\"+\n\020WhileMatchFilter\022\027\n\006filter\030\001 \002" + + "(\0132\007.Filter\"\021\n\017FilterAllFilterBB\n*org.ap" + + "ache.hadoop.hbase.protobuf.generatedB\014Fi" + + "lterProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -16168,86 +16611,92 @@ public final class FilterProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_InclusiveStopFilter_descriptor, new java.lang.String[] { "StopRowKey", }); - internal_static_KeyOnlyFilter_descriptor = + internal_static_ExclusiveStartFilter_descriptor = getDescriptor().getMessageTypes().get(14); + internal_static_ExclusiveStartFilter_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_ExclusiveStartFilter_descriptor, + new java.lang.String[] { "StartRowKey", }); + internal_static_KeyOnlyFilter_descriptor = + getDescriptor().getMessageTypes().get(15); internal_static_KeyOnlyFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_KeyOnlyFilter_descriptor, new java.lang.String[] { "LenAsVal", }); internal_static_MultipleColumnPrefixFilter_descriptor = - getDescriptor().getMessageTypes().get(15); + getDescriptor().getMessageTypes().get(16); internal_static_MultipleColumnPrefixFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultipleColumnPrefixFilter_descriptor, new java.lang.String[] { "SortedPrefixes", }); internal_static_PageFilter_descriptor = - getDescriptor().getMessageTypes().get(16); + getDescriptor().getMessageTypes().get(17); internal_static_PageFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_PageFilter_descriptor, new java.lang.String[] { "PageSize", }); internal_static_PrefixFilter_descriptor = - getDescriptor().getMessageTypes().get(17); + getDescriptor().getMessageTypes().get(18); internal_static_PrefixFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_PrefixFilter_descriptor, new java.lang.String[] { "Prefix", }); internal_static_QualifierFilter_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(19); internal_static_QualifierFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_QualifierFilter_descriptor, new java.lang.String[] { "CompareFilter", }); internal_static_RandomRowFilter_descriptor = - getDescriptor().getMessageTypes().get(19); + getDescriptor().getMessageTypes().get(20); internal_static_RandomRowFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RandomRowFilter_descriptor, new java.lang.String[] { "Chance", }); internal_static_RowFilter_descriptor = - getDescriptor().getMessageTypes().get(20); + getDescriptor().getMessageTypes().get(21); internal_static_RowFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RowFilter_descriptor, new java.lang.String[] { "CompareFilter", }); internal_static_SingleColumnValueExcludeFilter_descriptor = - getDescriptor().getMessageTypes().get(21); + getDescriptor().getMessageTypes().get(22); internal_static_SingleColumnValueExcludeFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SingleColumnValueExcludeFilter_descriptor, new java.lang.String[] { "SingleColumnValueFilter", }); internal_static_SingleColumnValueFilter_descriptor = - getDescriptor().getMessageTypes().get(22); + getDescriptor().getMessageTypes().get(23); internal_static_SingleColumnValueFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SingleColumnValueFilter_descriptor, new java.lang.String[] { "ColumnFamily", "ColumnQualifier", "CompareOp", "Comparator", "FilterIfMissing", "LatestVersionOnly", }); internal_static_SkipFilter_descriptor = - getDescriptor().getMessageTypes().get(23); + getDescriptor().getMessageTypes().get(24); internal_static_SkipFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SkipFilter_descriptor, new java.lang.String[] { "Filter", }); internal_static_TimestampsFilter_descriptor = - getDescriptor().getMessageTypes().get(24); + getDescriptor().getMessageTypes().get(25); internal_static_TimestampsFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TimestampsFilter_descriptor, new java.lang.String[] { "Timestamps", }); internal_static_ValueFilter_descriptor = - getDescriptor().getMessageTypes().get(25); + getDescriptor().getMessageTypes().get(26); internal_static_ValueFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ValueFilter_descriptor, new java.lang.String[] { "CompareFilter", }); internal_static_WhileMatchFilter_descriptor = - getDescriptor().getMessageTypes().get(26); + getDescriptor().getMessageTypes().get(27); internal_static_WhileMatchFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_WhileMatchFilter_descriptor, new java.lang.String[] { "Filter", }); internal_static_FilterAllFilter_descriptor = - getDescriptor().getMessageTypes().get(27); + getDescriptor().getMessageTypes().get(28); internal_static_FilterAllFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_FilterAllFilter_descriptor, diff --git a/hbase-protocol/src/main/protobuf/Filter.proto b/hbase-protocol/src/main/protobuf/Filter.proto index d5c51a4..7d0ea6d 100644 --- a/hbase-protocol/src/main/protobuf/Filter.proto +++ b/hbase-protocol/src/main/protobuf/Filter.proto @@ -98,6 +98,10 @@ message InclusiveStopFilter { optional bytes stop_row_key = 1; } +message ExclusiveStartFilter { + optional bytes start_row_key = 1; +} + message KeyOnlyFilter { required bool len_as_val = 1; } @@ -155,4 +159,4 @@ message WhileMatchFilter { required Filter filter = 1; } message FilterAllFilter { -} \ No newline at end of file +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java index 1fc0a5d..987fb12 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.DependentColumnFilter; import org.apache.hadoop.hbase.filter.FamilyFilter; +import org.apache.hadoop.hbase.filter.ExclusiveStartFilter; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; @@ -220,6 +221,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable { ColumnPrefixFilter, ColumnRangeFilter, DependentColumnFilter, + ExclusiveStartFilter, FamilyFilter, FilterList, FirstKeyOnlyFilter, @@ -287,6 +289,10 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable { this.value = Base64.encodeBytes(((InclusiveStopFilter)filter).getStopRowKey()); break; + case ExclusiveStartFilter: + this.value = + Base64.encodeBytes(((ExclusiveStartFilter)filter).getStartRowKey()); + break; case MultipleColumnPrefixFilter: this.prefixes = new ArrayList(); for (byte[] prefix: ((MultipleColumnPrefixFilter)filter).getPrefix()) { @@ -384,6 +390,9 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable { case InclusiveStopFilter: filter = new InclusiveStopFilter(Base64.decode(value)); break; + case ExclusiveStartFilter: + filter = new ExclusiveStartFilter(Base64.decode(value)); + break; case KeyOnlyFilter: filter = new KeyOnlyFilter(); break; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java index c71f4f9..a9d6109 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java @@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -407,6 +408,115 @@ public class TestReversibleScanners { scanner = region.getScanner(scan); verifyCountAndOrder(scanner, expectedRowNum * 2 * 2, expectedRowNum, false); } + + @Test + public void testReversibleRegionScannerExcludingStartRow() throws IOException { + byte[] tableName = Bytes.toBytes("testtable"); + byte[] FAMILYNAME2 = Bytes.toBytes("testCf2"); + Configuration conf = HBaseConfiguration.create(); + HRegion region = TEST_UTIL.createLocalHRegion(tableName, null, null, + "testReversibleRegionScannerExcludingStartRow", conf, false, Durability.SYNC_WAL, null, + FAMILYNAME, FAMILYNAME2); + loadDataToRegion(region, FAMILYNAME2); + + Scan scan = new Scan(); + + // Case1:Full reversed scan + scan = Scan.makeReversedExcludingStartRow(scan); + InternalScanner scanner = region.getScanner(scan); + verifyCountAndOrder(scanner, ROWSIZE * QUALSIZE * 2, ROWSIZE, false); + + // Case2:Full reversed scan with one family + scan = new Scan(); + scan.addFamily(FAMILYNAME); + scan = Scan.makeReversedExcludingStartRow(scan); + scanner = region.getScanner(scan); + verifyCountAndOrder(scanner, ROWSIZE * QUALSIZE, ROWSIZE, false); + + // Case3: Specify qualifiers + One family + scan = new Scan(); + scan.addFamily(FAMILYNAME); + byte[][] specifiedQualifiers = { QUALS[1], QUALS[2] }; + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME, specifiedQualifier); + scan = Scan.makeReversedExcludingStartRow(scan); + scanner = region.getScanner(scan); + verifyCountAndOrder(scanner, ROWSIZE * 2, ROWSIZE, false); + + // Case4:Specify qualifiers + Two families + scan = new Scan(); + scan.addFamily(FAMILYNAME); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME, specifiedQualifier); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME2, specifiedQualifier); + scan = Scan.makeReversedExcludingStartRow(scan); + scanner = region.getScanner(scan); + verifyCountAndOrder(scanner, ROWSIZE * 2 * 2, ROWSIZE, false); + + // Case5: Case4 + specify start row + scan = new Scan(); + scan.addFamily(FAMILYNAME); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME, specifiedQualifier); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME2, specifiedQualifier); + int startRowNum = ROWSIZE * 3 / 4; + scan.setStartRow(ROWS[startRowNum]); + scan = Scan.makeReversedExcludingStartRow(scan); + scanner = region.getScanner(scan); + verifyCountAndOrder(scanner, (startRowNum) * 2 * 2, (startRowNum), + false, ROWS[startRowNum], null); + + // Case6: Case4 + specify stop row + scan = new Scan(); + scan.addFamily(FAMILYNAME); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME, specifiedQualifier); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME2, specifiedQualifier); + int stopRowNum = ROWSIZE / 4; + scan.setStartRow(HConstants.EMPTY_BYTE_ARRAY); + scan.setStopRow(ROWS[stopRowNum]); + scan = Scan.makeReversedExcludingStartRow(scan); + scanner = region.getScanner(scan); + verifyCountAndOrder(scanner, (ROWSIZE - stopRowNum) * 2 * 2, (ROWSIZE + - stopRowNum), false, null, ROWS[stopRowNum]); + + // Case7: Case4 + specify start row + specify stop row + scan = new Scan(); + scan.addFamily(FAMILYNAME); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME, specifiedQualifier); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME2, specifiedQualifier); + scan.setStartRow(ROWS[startRowNum]); + scan.setStopRow(ROWS[stopRowNum]); + scan = Scan.makeReversedExcludingStartRow(scan); + scanner = region.getScanner(scan); + verifyCountAndOrder(scanner, (startRowNum - stopRowNum) * 2 * 2, + (startRowNum - stopRowNum), false, ROWS[startRowNum], ROWS[stopRowNum]); + + // Case8: Case7 + SingleColumnValueFilter + scan = new Scan(); + scan.addFamily(FAMILYNAME); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME, specifiedQualifier); + for (byte[] specifiedQualifier : specifiedQualifiers) + scan.addColumn(FAMILYNAME2, specifiedQualifier); + scan.setStartRow(ROWS[startRowNum]); + scan.setStopRow(ROWS[stopRowNum]); + int valueNum = startRowNum % VALUESIZE; + Filter filter = new SingleColumnValueFilter(FAMILYNAME, + specifiedQualifiers[0], CompareOp.EQUAL, VALUES[valueNum]); + scan.setFilter(filter); + scan = Scan.makeReversedExcludingStartRow(scan); + scanner = region.getScanner(scan); + int unfilteredRowNum = (startRowNum - stopRowNum) / VALUESIZE + + (stopRowNum / VALUESIZE == valueNum ? 0 : 1); + verifyCountAndOrder(scanner, (unfilteredRowNum-1) * 2 * 2, unfilteredRowNum-1, + false, ROWS[startRowNum], null); + } private StoreScanner getReversibleStoreScanner(MemStore memstore, StoreFile sf1, StoreFile sf2, Scan scan, ScanType scanType, @@ -425,16 +535,41 @@ public class TestReversibleScanners { } private void verifyCountAndOrder(InternalScanner scanner, - int expectedKVCount, int expectedRowCount, boolean forward) + int expectedKVCount, int expectedRowCount, boolean forward) throws IOException { + verifyCountAndOrder(scanner, expectedKVCount, expectedRowCount, forward, null, null); + } + + /* + * @param scanner + * @param expectedKVCount + * @param expectedRowCount + * @param forward + * @param rowToExclude if not null, this row shouldn't appear in scan result + * @param rowToInclude if not null, this row should appear in scan result + */ + private void verifyCountAndOrder(InternalScanner scanner, int expectedKVCount, + int expectedRowCount, boolean forward, byte[] rowToExclude, byte[] rowToInclude) throws IOException { List kvList = new ArrayList(); Result lastResult = null; int rowCount = 0; int kvCount = 0; + boolean rowFound = rowToInclude == null ? true : false; try { while (scanner.next(kvList)) { if (kvList.isEmpty()) continue; rowCount++; + byte[] row = kvList.get(0).getRowArray(); + if (rowToExclude != null) { + int res = Bytes.compareTo(rowToExclude, 0, rowToExclude.length, + row, kvList.get(0).getRowOffset(), kvList.get(0).getRowLength()); + assertTrue(res != 0); + } + if (rowToInclude != null) { + int res = Bytes.compareTo(rowToInclude, 0, rowToInclude.length, + row, kvList.get(0).getRowOffset(), kvList.get(0).getRowLength()); + if (res == 0) rowFound = true; + } kvCount += kvList.size(); if (lastResult != null) { Result curResult = Result.create(kvList); @@ -455,6 +590,7 @@ public class TestReversibleScanners { } assertEquals(expectedKVCount, kvCount); assertEquals(expectedRowCount, rowCount); + assertTrue(Bytes.toString(rowToInclude) + " not found", rowFound); } private void internalTestSeekAndNextForReversibleKeyValueHeap( diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ExclusiveStartFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ExclusiveStartFilter.java new file mode 100644 index 0000000..bacfcce --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ExclusiveStartFilter.java @@ -0,0 +1,128 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.filter; + +import java.util.ArrayList; + +import org.apache.hadoop.hbase.util.ByteStringer; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; +import org.apache.hadoop.hbase.util.Bytes; + +import com.google.common.base.Preconditions; +import com.google.protobuf.InvalidProtocolBufferException; + +/** + * A Filter that starts after the given row. + * + * Use this filter to exclude the start row, eg: (A,Z]. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class ExclusiveStartFilter extends FilterBase { + private byte [] startRowKey; + + public ExclusiveStartFilter(final byte [] startRowKey) { + this.startRowKey = startRowKey; + } + + public byte[] getStartRowKey() { + return this.startRowKey; + } + + @Override + public ReturnCode filterKeyValue(Cell v) { + return ReturnCode.INCLUDE; + } + + public boolean filterRowKey(byte[] buffer, int offset, int length) { + if (buffer == null) { + //noinspection RedundantIfStatement + if (this.startRowKey == null) { + return true; //filter... + } + return false; + } + // if startRowKey is == buffer, then true, filter row. + int cmp = Bytes.compareTo(startRowKey, 0, startRowKey.length, + buffer, offset, length); + + if (cmp == 0) { + return true; + } + return false; + } + + public static Filter createFilterFromArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.size() == 1, + "Expected 1 but got: %s", filterArguments.size()); + byte [] startRowKey = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); + return new ExclusiveStartFilter(startRowKey); + } + + /** + * @return The filter serialized using pb + */ + public byte [] toByteArray() { + FilterProtos.ExclusiveStartFilter.Builder builder = + FilterProtos.ExclusiveStartFilter.newBuilder(); + if (this.startRowKey != null) builder.setStartRowKey(ByteStringer.wrap(this.startRowKey)); + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized {@link ExclusiveStartFilter} instance + * @return An instance of {@link ExclusiveStartFilter} made from bytes + * @throws DeserializationException + * @see #toByteArray + */ + public static ExclusiveStartFilter parseFrom(final byte [] pbBytes) + throws DeserializationException { + FilterProtos.ExclusiveStartFilter proto; + try { + proto = FilterProtos.ExclusiveStartFilter.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + return new ExclusiveStartFilter(proto.hasStartRowKey() ? + proto.getStartRowKey().toByteArray() : null); + } + + /** + * @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 ExclusiveStartFilter)) return false; + + ExclusiveStartFilter other = (ExclusiveStartFilter)o; + return Bytes.equals(this.getStartRowKey(), other.getStartRowKey()); + } + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.startRowKey); + } +}