diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java new file mode 100644 index 0000000..6de09f1 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java @@ -0,0 +1,412 @@ +/* + * 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.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValueUtil; +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.protobuf.HBaseZeroCopyByteString; +import com.google.protobuf.InvalidProtocolBufferException; + +/** + * Filter to support scan multiple row key ranges. It can construct the row key ranges from the + * passed list which can be accessed by each region server. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class MultiRowRangeFilter extends FilterBase { + + private List rangeList; + + private static final int ROW_BEFORE_FIRST_RANGE = -1; + private boolean done = false; + private boolean initialized = false; + private int index; + private RowKeyRange range; + private ReturnCode currentReturnCode; + + /** + * @param list A list of RowKeyRange + * @throws java.io.IOException + * throw an exception if the range list is not in an natural order or any + * RowKeyRange is invalid + */ + public MultiRowRangeFilter(List list) throws IOException { + this.rangeList = sortAndMerge(list); + } + + @Override + public boolean filterAllRemaining() { + return done; + } + + public List getRowRanges() { + return this.rangeList; + } + + @Override + public boolean filterRowKey(byte[] buffer, int offset, int length) { + // If it is the first time of running, calculate the current range index for + // the row key. If index is out of bound which happens when the start row + // user sets is after the largest stop row of the ranges, stop the scan. + // If row key is after the current range, find the next range and update index. + if (!initialized || !range.contains(buffer, offset, length)) { + byte[] rowkey = new byte[length]; + System.arraycopy(buffer, offset, rowkey, 0, length); + index = getNextRangeIndex(rowkey); + if (index >= rangeList.size()) { + done = true; + currentReturnCode = ReturnCode.NEXT_ROW; + return false; + } + if(index != ROW_BEFORE_FIRST_RANGE) { + range = rangeList.get(index); + } else { + range = rangeList.get(0); + } + if (!initialized) { + if(index != ROW_BEFORE_FIRST_RANGE) { + currentReturnCode = ReturnCode.INCLUDE; + } else { + currentReturnCode = ReturnCode.SEEK_NEXT_USING_HINT; + } + } else { + currentReturnCode = ReturnCode.SEEK_NEXT_USING_HINT; + } + initialized = true; + } else { + currentReturnCode = ReturnCode.INCLUDE; + } + return false; + } + + @Override + public ReturnCode filterKeyValue(Cell ignored) { + return currentReturnCode; + } + + @Override + public Cell getNextCellHint(Cell currentKV) { + // skip to the next range's start row + return KeyValueUtil.createFirstOnRow(range.startRow); + } + + /** + * @return The filter serialized using pb + */ + public byte[] toByteArray() { + FilterProtos.MultiRowRangeFilter.Builder builder = FilterProtos.MultiRowRangeFilter + .newBuilder(); + for (RowKeyRange range : rangeList) { + if (range != null) { + FilterProtos.RowKeyRange.Builder rangebuilder = FilterProtos.RowKeyRange.newBuilder(); + if (range.startRow != null) + rangebuilder.setStartRow(HBaseZeroCopyByteString.wrap(range.startRow)); + if (range.stopRow != null) + rangebuilder.setStopRow(HBaseZeroCopyByteString.wrap(range.stopRow)); + range.isScan = Bytes.equals(range.startRow, range.stopRow) ? 1 : 0; + builder.addRowKeyRangeList(rangebuilder.build()); + } + } + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized instance + * @return An instance of MultiRowRangeFilter + * @throws org.apache.hadoop.hbase.exceptions.DeserializationException + */ + public static MultiRowRangeFilter parseFrom(final byte[] pbBytes) + throws DeserializationException { + FilterProtos.MultiRowRangeFilter proto; + try { + proto = FilterProtos.MultiRowRangeFilter.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + int length = proto.getRowKeyRangeListCount(); + List rangeProtos = proto.getRowKeyRangeListList(); + List rangeList = new ArrayList(length); + for (FilterProtos.RowKeyRange rangeProto : rangeProtos) { + RowKeyRange range = new RowKeyRange(rangeProto.hasStartRow() ? rangeProto.getStartRow() + .toByteArray() : null, rangeProto.hasStopRow() ? rangeProto.getStopRow().toByteArray() + : null); + rangeList.add(range); + } + try { + return new MultiRowRangeFilter(rangeList); + } catch (IOException e) { + throw new DeserializationException("Fail to instantiate the MultiRowRangeFilter", e); + } + } + + /** + * @param o the filter to compare + * @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 MultiRowRangeFilter)) + return false; + + MultiRowRangeFilter other = (MultiRowRangeFilter) o; + if (this.rangeList.size() != other.rangeList.size()) + return false; + for (int i = 0; i < rangeList.size(); ++i) { + RowKeyRange thisRange = this.rangeList.get(i); + RowKeyRange otherRange = other.rangeList.get(i); + if (!(Bytes.equals(thisRange.startRow, otherRange.startRow) && Bytes.equals( + thisRange.stopRow, otherRange.stopRow))) { + return false; + } + } + return true; + } + + /** + * calculate the position where the row key in the ranges list. + * + * @param rowKey the row key to calculate + * @return index the position of the row key + */ + private int getNextRangeIndex(byte[] rowKey) { + RowKeyRange temp = new RowKeyRange(rowKey, null); + int index = Collections.binarySearch(rangeList, temp); + if (index < 0) { + int insertionPosition = -index - 1; + // check if the row key in the range before the insertion position + if (insertionPosition != 0 && rangeList.get(insertionPosition - 1).contains(rowKey)) { + return insertionPosition - 1; + } + // check if the row key is before the first range + if (insertionPosition == 0 && !rangeList.get(insertionPosition).contains(rowKey)) { + return ROW_BEFORE_FIRST_RANGE; + } + return insertionPosition; + } + // the row key equals one of the start keys + return index; + } + + /** + * sort the ranges and if the ranges with overlap, then merge them. + * + * @param ranges the list of ranges to sort and merge. + * @return the ranges after sort and merge. + */ + public static List sortAndMerge(List ranges) { + if (ranges.size() == 0) { + throw new IllegalArgumentException("No ranges found."); + } + List invalidRanges = new ArrayList(); + List newRanges = new ArrayList(ranges.size()); + Collections.sort(ranges); + if(ranges.get(0).isValid()) { + if (ranges.size() == 1) { + newRanges.add(ranges.get(0)); + } + } else { + invalidRanges.add(ranges.get(0)); + } + byte[] lastStartRow = ranges.get(0).startRow; + byte[] lastStopRow = ranges.get(0).stopRow; + int i = 1; + for (; i < ranges.size(); i++) { + RowKeyRange range = ranges.get(i); + if (!range.isValid()) { + invalidRanges.add(range); + } + if(Bytes.equals(lastStopRow, HConstants.EMPTY_BYTE_ARRAY)) { + newRanges.add(new RowKeyRange(lastStartRow, lastStopRow)); + break; + } + // with overlap in the ranges + if (Bytes.compareTo(lastStopRow, range.startRow) >= 0) { + if(Bytes.equals(range.stopRow, HConstants.EMPTY_BYTE_ARRAY)) { + newRanges.add(new RowKeyRange(lastStartRow, range.stopRow)); + break; + } + // if first range contains second range, ignore the second range + if (Bytes.compareTo(lastStopRow, range.stopRow) >= 0) { + if ((i + 1) == ranges.size()) { + newRanges.add(new RowKeyRange(lastStartRow, lastStopRow)); + } + } else { + lastStopRow = range.stopRow; + if ((i + 1) < ranges.size()) { + i++; + range = ranges.get(i); + if (!range.isValid()) { + invalidRanges.add(range); + } + } else { + newRanges.add(new RowKeyRange(lastStartRow, lastStopRow)); + break; + } + while (Bytes.compareTo(lastStopRow, range.startRow) >= 0) { + if(Bytes.equals(range.stopRow, HConstants.EMPTY_BYTE_ARRAY)) { + break; + } + // if this first range contain second range, ignore the second range + if (Bytes.compareTo(lastStopRow, range.stopRow) >= 0) { + i++; + if (i < ranges.size()) { + range = ranges.get(i); + if (!range.isValid()) { + invalidRanges.add(range); + } + } else { + break; + } + } else { + lastStopRow = range.stopRow; + i++; + if (i < ranges.size()) { + range = ranges.get(i); + if (!range.isValid()) { + invalidRanges.add(range); + } + } else { + break; + } + } + } + if(Bytes.equals(range.stopRow, HConstants.EMPTY_BYTE_ARRAY)) { + if(Bytes.compareTo(lastStopRow, range.startRow) < 0) + { + newRanges.add(new RowKeyRange(lastStartRow, lastStopRow)); + newRanges.add(range); + } else { + newRanges.add(new RowKeyRange(lastStartRow, range.stopRow)); + break; + } + } + newRanges.add(new RowKeyRange(lastStartRow, lastStopRow)); + lastStartRow = range.startRow; + lastStopRow = range.stopRow; + if ((i + 1) == ranges.size()) { + newRanges.add(new RowKeyRange(lastStartRow, lastStopRow)); + } + } + } else { + newRanges.add(new RowKeyRange(lastStartRow, lastStopRow)); + lastStartRow = range.startRow; + lastStopRow = range.stopRow; + if ((i + 1) == ranges.size()) { + newRanges.add(new RowKeyRange(lastStartRow, lastStopRow)); + } + } + } + // check the remaining ranges + for(int j=i; j < ranges.size(); j++) { + if(!ranges.get(j).isValid()) { + invalidRanges.add(ranges.get(j)); + } + } + // if invalid range exists, throw the exception + if (invalidRanges.size() != 0) { + throwExceptionForInvalidRanges(invalidRanges, true); + } + // If no valid ranges found, throw the exception + if(newRanges.size() == 0) { + throw new IllegalArgumentException("No valid ranges found."); + } + return newRanges; + } + + private static void throwExceptionForInvalidRanges(List invalidRanges, + boolean details) { + StringBuilder sb = new StringBuilder(); + sb.append(invalidRanges.size()).append(" invaild ranges.\n"); + if (details) { + for (RowKeyRange range : invalidRanges) { + sb.append( + "Invalid range: start row => " + Bytes.toString(range.startRow) + ", stop row => " + + Bytes.toString(range.stopRow)).append('\n'); + } + } + throw new IllegalArgumentException(sb.toString()); + } + + @InterfaceAudience.Public + @InterfaceStability.Evolving + public static class RowKeyRange implements Comparable { + private byte[] startRow; + private byte[] stopRow; + private int isScan = 0; + + /** + * If the startRow is empty or null, set it to HConstants.EMPTY_BYTE_ARRAY, means begin at the + * start row of the table. If the stopRow is empty or null, set it to + * HConstants.EMPTY_BYTE_ARRAY, means end of the last row of table. + */ + public RowKeyRange(String startRow, String stopRow) { + this((startRow == null || startRow.isEmpty()) ? HConstants.EMPTY_BYTE_ARRAY : Bytes + .toBytes(startRow), (stopRow == null || stopRow.isEmpty()) ? HConstants.EMPTY_BYTE_ARRAY + : Bytes.toBytes(stopRow)); + } + + public RowKeyRange(byte[] startRow, byte[] stopRow) { + this.startRow = (startRow == null) ? HConstants.EMPTY_BYTE_ARRAY : startRow; + this.stopRow = (stopRow == null) ? HConstants.EMPTY_BYTE_ARRAY :stopRow; + isScan = Bytes.equals(startRow, stopRow) ? 1 : 0; + } + + public byte[] getStartRow() { + return startRow; + } + + public byte[] getStopRow() { + return stopRow; + } + + public boolean contains(byte[] row) { + return contains(row, 0, row.length); + } + + public boolean contains(byte[] buffer, int offset, int length) { + return Bytes.compareTo(buffer, offset, length, startRow, 0, startRow.length) >= 0 + && (Bytes.equals(stopRow, HConstants.EMPTY_BYTE_ARRAY) || Bytes.compareTo(buffer, offset, + length, stopRow, 0, stopRow.length) < isScan); + } + + @Override + public int compareTo(RowKeyRange other) { + return Bytes.compareTo(startRow, other.startRow); + } + + public boolean isValid() { + return Bytes.equals(startRow, HConstants.EMPTY_BYTE_ARRAY) + || Bytes.equals(stopRow, HConstants.EMPTY_BYTE_ARRAY) + || Bytes.compareTo(startRow, stopRow) <= 0; + } + } +} 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..2a99456 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 @@ -15881,6 +15881,1245 @@ public Builder mergeFrom( // @@protoc_insertion_point(class_scope:FilterAllFilter) } + public interface RowKeyRangeOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional bytes start_row = 1; + /** + * optional bytes start_row = 1; + */ + boolean hasStartRow(); + /** + * optional bytes start_row = 1; + */ + com.google.protobuf.ByteString getStartRow(); + + // optional bytes stop_row = 2; + /** + * optional bytes stop_row = 2; + */ + boolean hasStopRow(); + /** + * optional bytes stop_row = 2; + */ + com.google.protobuf.ByteString getStopRow(); + } + /** + * Protobuf type {@code RowKeyRange} + */ + public static final class RowKeyRange extends + com.google.protobuf.GeneratedMessage + implements RowKeyRangeOrBuilder { + // Use RowKeyRange.newBuilder() to construct. + private RowKeyRange(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private RowKeyRange(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final RowKeyRange defaultInstance; + public static RowKeyRange getDefaultInstance() { + return defaultInstance; + } + + public RowKeyRange getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private RowKeyRange( + 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; + startRow_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + stopRow_ = 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_RowKeyRange_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_RowKeyRange_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public RowKeyRange parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RowKeyRange(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // optional bytes start_row = 1; + public static final int START_ROW_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString startRow_; + /** + * optional bytes start_row = 1; + */ + public boolean hasStartRow() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional bytes start_row = 1; + */ + public com.google.protobuf.ByteString getStartRow() { + return startRow_; + } + + // optional bytes stop_row = 2; + public static final int STOP_ROW_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString stopRow_; + /** + * optional bytes stop_row = 2; + */ + public boolean hasStopRow() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bytes stop_row = 2; + */ + public com.google.protobuf.ByteString getStopRow() { + return stopRow_; + } + + private void initFields() { + startRow_ = com.google.protobuf.ByteString.EMPTY; + stopRow_ = 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, startRow_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, stopRow_); + } + 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, startRow_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, stopRow_); + } + 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.RowKeyRange)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange other = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange) obj; + + boolean result = true; + result = result && (hasStartRow() == other.hasStartRow()); + if (hasStartRow()) { + result = result && getStartRow() + .equals(other.getStartRow()); + } + result = result && (hasStopRow() == other.hasStopRow()); + if (hasStopRow()) { + result = result && getStopRow() + .equals(other.getStopRow()); + } + 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 (hasStartRow()) { + hash = (37 * hash) + START_ROW_FIELD_NUMBER; + hash = (53 * hash) + getStartRow().hashCode(); + } + if (hasStopRow()) { + hash = (37 * hash) + STOP_ROW_FIELD_NUMBER; + hash = (53 * hash) + getStopRow().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange 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.RowKeyRange parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange 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.RowKeyRange parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange 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.RowKeyRange parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange 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.RowKeyRange parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange 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.RowKeyRange 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 RowKeyRange} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRangeOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_RowKeyRange_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_RowKeyRange_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.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(); + startRow_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + stopRow_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + 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_RowKeyRange_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange build() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange result = new org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.startRow_ = startRow_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.stopRow_ = stopRow_; + 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.RowKeyRange) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.getDefaultInstance()) return this; + if (other.hasStartRow()) { + setStartRow(other.getStartRow()); + } + if (other.hasStopRow()) { + setStopRow(other.getStopRow()); + } + 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.RowKeyRange parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // optional bytes start_row = 1; + private com.google.protobuf.ByteString startRow_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes start_row = 1; + */ + public boolean hasStartRow() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional bytes start_row = 1; + */ + public com.google.protobuf.ByteString getStartRow() { + return startRow_; + } + /** + * optional bytes start_row = 1; + */ + public Builder setStartRow(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + startRow_ = value; + onChanged(); + return this; + } + /** + * optional bytes start_row = 1; + */ + public Builder clearStartRow() { + bitField0_ = (bitField0_ & ~0x00000001); + startRow_ = getDefaultInstance().getStartRow(); + onChanged(); + return this; + } + + // optional bytes stop_row = 2; + private com.google.protobuf.ByteString stopRow_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes stop_row = 2; + */ + public boolean hasStopRow() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bytes stop_row = 2; + */ + public com.google.protobuf.ByteString getStopRow() { + return stopRow_; + } + /** + * optional bytes stop_row = 2; + */ + public Builder setStopRow(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + stopRow_ = value; + onChanged(); + return this; + } + /** + * optional bytes stop_row = 2; + */ + public Builder clearStopRow() { + bitField0_ = (bitField0_ & ~0x00000002); + stopRow_ = getDefaultInstance().getStopRow(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:RowKeyRange) + } + + static { + defaultInstance = new RowKeyRange(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RowKeyRange) + } + + public interface MultiRowRangeFilterOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // repeated .RowKeyRange row_key_range_list = 1; + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + java.util.List + getRowKeyRangeListList(); + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange getRowKeyRangeList(int index); + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + int getRowKeyRangeListCount(); + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + java.util.List + getRowKeyRangeListOrBuilderList(); + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRangeOrBuilder getRowKeyRangeListOrBuilder( + int index); + } + /** + * Protobuf type {@code MultiRowRangeFilter} + */ + public static final class MultiRowRangeFilter extends + com.google.protobuf.GeneratedMessage + implements MultiRowRangeFilterOrBuilder { + // Use MultiRowRangeFilter.newBuilder() to construct. + private MultiRowRangeFilter(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private MultiRowRangeFilter(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final MultiRowRangeFilter defaultInstance; + public static MultiRowRangeFilter getDefaultInstance() { + return defaultInstance; + } + + public MultiRowRangeFilter getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private MultiRowRangeFilter( + 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: { + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + rowKeyRangeList_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + rowKeyRangeList_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.PARSER, extensionRegistry)); + 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 { + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + rowKeyRangeList_ = java.util.Collections.unmodifiableList(rowKeyRangeList_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_MultiRowRangeFilter_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_MultiRowRangeFilter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public MultiRowRangeFilter parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new MultiRowRangeFilter(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + // repeated .RowKeyRange row_key_range_list = 1; + public static final int ROW_KEY_RANGE_LIST_FIELD_NUMBER = 1; + private java.util.List rowKeyRangeList_; + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public java.util.List getRowKeyRangeListList() { + return rowKeyRangeList_; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public java.util.List + getRowKeyRangeListOrBuilderList() { + return rowKeyRangeList_; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public int getRowKeyRangeListCount() { + return rowKeyRangeList_.size(); + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange getRowKeyRangeList(int index) { + return rowKeyRangeList_.get(index); + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRangeOrBuilder getRowKeyRangeListOrBuilder( + int index) { + return rowKeyRangeList_.get(index); + } + + private void initFields() { + rowKeyRangeList_ = java.util.Collections.emptyList(); + } + 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(); + for (int i = 0; i < rowKeyRangeList_.size(); i++) { + output.writeMessage(1, rowKeyRangeList_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < rowKeyRangeList_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, rowKeyRangeList_.get(i)); + } + 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.MultiRowRangeFilter)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter other = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter) obj; + + boolean result = true; + result = result && getRowKeyRangeListList() + .equals(other.getRowKeyRangeListList()); + 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 (getRowKeyRangeListCount() > 0) { + hash = (37 * hash) + ROW_KEY_RANGE_LIST_FIELD_NUMBER; + hash = (53 * hash) + getRowKeyRangeListList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter 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.MultiRowRangeFilter parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter 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.MultiRowRangeFilter parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter 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.MultiRowRangeFilter parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter 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.MultiRowRangeFilter parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter 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.MultiRowRangeFilter 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 MultiRowRangeFilter} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilterOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_MultiRowRangeFilter_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_MultiRowRangeFilter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getRowKeyRangeListFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (rowKeyRangeListBuilder_ == null) { + rowKeyRangeList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + rowKeyRangeListBuilder_.clear(); + } + 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_MultiRowRangeFilter_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter build() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter result = new org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter(this); + int from_bitField0_ = bitField0_; + if (rowKeyRangeListBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + rowKeyRangeList_ = java.util.Collections.unmodifiableList(rowKeyRangeList_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.rowKeyRangeList_ = rowKeyRangeList_; + } else { + result.rowKeyRangeList_ = rowKeyRangeListBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter.getDefaultInstance()) return this; + if (rowKeyRangeListBuilder_ == null) { + if (!other.rowKeyRangeList_.isEmpty()) { + if (rowKeyRangeList_.isEmpty()) { + rowKeyRangeList_ = other.rowKeyRangeList_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureRowKeyRangeListIsMutable(); + rowKeyRangeList_.addAll(other.rowKeyRangeList_); + } + onChanged(); + } + } else { + if (!other.rowKeyRangeList_.isEmpty()) { + if (rowKeyRangeListBuilder_.isEmpty()) { + rowKeyRangeListBuilder_.dispose(); + rowKeyRangeListBuilder_ = null; + rowKeyRangeList_ = other.rowKeyRangeList_; + bitField0_ = (bitField0_ & ~0x00000001); + rowKeyRangeListBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getRowKeyRangeListFieldBuilder() : null; + } else { + rowKeyRangeListBuilder_.addAllMessages(other.rowKeyRangeList_); + } + } + } + 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.MultiRowRangeFilter parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultiRowRangeFilter) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // repeated .RowKeyRange row_key_range_list = 1; + private java.util.List rowKeyRangeList_ = + java.util.Collections.emptyList(); + private void ensureRowKeyRangeListIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + rowKeyRangeList_ = new java.util.ArrayList(rowKeyRangeList_); + bitField0_ |= 0x00000001; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRangeOrBuilder> rowKeyRangeListBuilder_; + + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public java.util.List getRowKeyRangeListList() { + if (rowKeyRangeListBuilder_ == null) { + return java.util.Collections.unmodifiableList(rowKeyRangeList_); + } else { + return rowKeyRangeListBuilder_.getMessageList(); + } + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public int getRowKeyRangeListCount() { + if (rowKeyRangeListBuilder_ == null) { + return rowKeyRangeList_.size(); + } else { + return rowKeyRangeListBuilder_.getCount(); + } + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange getRowKeyRangeList(int index) { + if (rowKeyRangeListBuilder_ == null) { + return rowKeyRangeList_.get(index); + } else { + return rowKeyRangeListBuilder_.getMessage(index); + } + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public Builder setRowKeyRangeList( + int index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange value) { + if (rowKeyRangeListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureRowKeyRangeListIsMutable(); + rowKeyRangeList_.set(index, value); + onChanged(); + } else { + rowKeyRangeListBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public Builder setRowKeyRangeList( + int index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder builderForValue) { + if (rowKeyRangeListBuilder_ == null) { + ensureRowKeyRangeListIsMutable(); + rowKeyRangeList_.set(index, builderForValue.build()); + onChanged(); + } else { + rowKeyRangeListBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public Builder addRowKeyRangeList(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange value) { + if (rowKeyRangeListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureRowKeyRangeListIsMutable(); + rowKeyRangeList_.add(value); + onChanged(); + } else { + rowKeyRangeListBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public Builder addRowKeyRangeList( + int index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange value) { + if (rowKeyRangeListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureRowKeyRangeListIsMutable(); + rowKeyRangeList_.add(index, value); + onChanged(); + } else { + rowKeyRangeListBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public Builder addRowKeyRangeList( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder builderForValue) { + if (rowKeyRangeListBuilder_ == null) { + ensureRowKeyRangeListIsMutable(); + rowKeyRangeList_.add(builderForValue.build()); + onChanged(); + } else { + rowKeyRangeListBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public Builder addRowKeyRangeList( + int index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder builderForValue) { + if (rowKeyRangeListBuilder_ == null) { + ensureRowKeyRangeListIsMutable(); + rowKeyRangeList_.add(index, builderForValue.build()); + onChanged(); + } else { + rowKeyRangeListBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public Builder addAllRowKeyRangeList( + java.lang.Iterable values) { + if (rowKeyRangeListBuilder_ == null) { + ensureRowKeyRangeListIsMutable(); + super.addAll(values, rowKeyRangeList_); + onChanged(); + } else { + rowKeyRangeListBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public Builder clearRowKeyRangeList() { + if (rowKeyRangeListBuilder_ == null) { + rowKeyRangeList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + rowKeyRangeListBuilder_.clear(); + } + return this; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public Builder removeRowKeyRangeList(int index) { + if (rowKeyRangeListBuilder_ == null) { + ensureRowKeyRangeListIsMutable(); + rowKeyRangeList_.remove(index); + onChanged(); + } else { + rowKeyRangeListBuilder_.remove(index); + } + return this; + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder getRowKeyRangeListBuilder( + int index) { + return getRowKeyRangeListFieldBuilder().getBuilder(index); + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRangeOrBuilder getRowKeyRangeListOrBuilder( + int index) { + if (rowKeyRangeListBuilder_ == null) { + return rowKeyRangeList_.get(index); } else { + return rowKeyRangeListBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public java.util.List + getRowKeyRangeListOrBuilderList() { + if (rowKeyRangeListBuilder_ != null) { + return rowKeyRangeListBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(rowKeyRangeList_); + } + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder addRowKeyRangeListBuilder() { + return getRowKeyRangeListFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.getDefaultInstance()); + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder addRowKeyRangeListBuilder( + int index) { + return getRowKeyRangeListFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.getDefaultInstance()); + } + /** + * repeated .RowKeyRange row_key_range_list = 1; + */ + public java.util.List + getRowKeyRangeListBuilderList() { + return getRowKeyRangeListFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRangeOrBuilder> + getRowKeyRangeListFieldBuilder() { + if (rowKeyRangeListBuilder_ == null) { + rowKeyRangeListBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowKeyRangeOrBuilder>( + rowKeyRangeList_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + rowKeyRangeList_ = null; + } + return rowKeyRangeListBuilder_; + } + + // @@protoc_insertion_point(builder_scope:MultiRowRangeFilter) + } + + static { + defaultInstance = new MultiRowRangeFilter(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:MultiRowRangeFilter) + } + private static com.google.protobuf.Descriptors.Descriptor internal_static_Filter_descriptor; private static @@ -16021,6 +17260,16 @@ public Builder mergeFrom( private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_FilterAllFilter_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RowKeyRange_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RowKeyRange_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_MultiRowRangeFilter_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_MultiRowRangeFilter_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -16075,9 +17324,12 @@ public Builder mergeFrom( "\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" + "lter\"\021\n\017FilterAllFilter\"2\n\013RowKeyRange\022\021" + + "\n\tstart_row\030\001 \001(\014\022\020\n\010stop_row\030\002 \001(\014\"?\n\023M" + + "ultiRowRangeFilter\022(\n\022row_key_range_list" + + "\030\001 \003(\0132\014.RowKeyRangeBB\n*org.apache.hadoo" + + "p.hbase.protobuf.generatedB\014FilterProtos", + "H\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -16252,6 +17504,18 @@ public Builder mergeFrom( com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_FilterAllFilter_descriptor, new java.lang.String[] { }); + internal_static_RowKeyRange_descriptor = + getDescriptor().getMessageTypes().get(28); + internal_static_RowKeyRange_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_RowKeyRange_descriptor, + new java.lang.String[] { "StartRow", "StopRow", }); + internal_static_MultiRowRangeFilter_descriptor = + getDescriptor().getMessageTypes().get(29); + internal_static_MultiRowRangeFilter_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_MultiRowRangeFilter_descriptor, + new java.lang.String[] { "RowKeyRangeList", }); return null; } }; diff --git a/hbase-protocol/src/main/protobuf/Filter.proto b/hbase-protocol/src/main/protobuf/Filter.proto index d5c51a4..f1b2538 100644 --- a/hbase-protocol/src/main/protobuf/Filter.proto +++ b/hbase-protocol/src/main/protobuf/Filter.proto @@ -155,4 +155,13 @@ message WhileMatchFilter { required Filter filter = 1; } message FilterAllFilter { +} + +message RowKeyRange { + optional bytes start_row = 1; + optional bytes stop_row = 2; +} + +message MultiRowRangeFilter { + repeated RowKeyRange row_key_range_list = 1; } \ No newline at end of file diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java index 2ffdd4f..cc2a930 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java @@ -32,8 +32,8 @@ import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlRootElement; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.BinaryPrefixComparator; @@ -52,6 +52,8 @@ import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.filter.InclusiveStopFilter; import org.apache.hadoop.hbase.filter.KeyOnlyFilter; +import org.apache.hadoop.hbase.filter.MultiRowRangeFilter; +import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowKeyRange; import org.apache.hadoop.hbase.filter.MultipleColumnPrefixFilter; import org.apache.hadoop.hbase.filter.NullComparator; import org.apache.hadoop.hbase.filter.PageFilter; @@ -212,6 +214,7 @@ public ByteArrayComparable build() { @XmlAttribute public Boolean dropDependentColumn; @XmlAttribute public Float chance; @XmlElement public List prefixes; + @XmlElement private List ranges; @XmlElement public List timestamps; static enum FilterType { @@ -226,6 +229,7 @@ public ByteArrayComparable build() { InclusiveStopFilter, KeyOnlyFilter, MultipleColumnPrefixFilter, + MultiRowRangeFilter, PageFilter, PrefixFilter, QualifierFilter, @@ -293,6 +297,13 @@ public FilterModel(Filter filter) { this.prefixes.add(Base64.encodeBytes(prefix)); } break; + case MultiRowRangeFilter: + this.ranges = new ArrayList(); + for(RowKeyRange range : ((MultiRowRangeFilter)filter).getRowRanges()) { + ranges.add(new RowKeyRange(Base64.encodeBytes(range.getStartRow()), + Base64.encodeBytes(range.getStopRow()))); + } + break; case PageFilter: this.value = Long.toString(((PageFilter)filter).getPageSize()); break; @@ -394,6 +405,18 @@ public Filter build() { } filter = new MultipleColumnPrefixFilter(values); } break; + case MultiRowRangeFilter: { + List values = new ArrayList(ranges.size()); + for (int i = 0; i < ranges.size(); i++) { + values.add(new RowKeyRange(Base64.decode(Bytes.toString(ranges.get(i).getStartRow())), + Base64.decode(Bytes.toString(ranges.get(i).getStopRow())))); + } + try { + filter = new MultiRowRangeFilter(values); + } catch (IOException e) { + throw new RuntimeException(e); + } + } break; case PageFilter: filter = new PageFilter(Long.valueOf(value)); break; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java new file mode 100644 index 0000000..095e9e8 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java @@ -0,0 +1,373 @@ +/* + * 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 static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowKeyRange; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(MediumTests.class) +public class TestMultiRowRangeFilter { + + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private final Log LOG = LogFactory.getLog(this.getClass()); + private byte[] family = Bytes.toBytes("family"); + private byte[] qf = Bytes.toBytes("qf"); + private byte[] value = Bytes.toBytes("val"); + private byte[] tableName; + private int numRows = 100; + + /** + * @throws Exception + */ + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL.startMiniCluster(); + } + + /** + * @throws Exception + */ + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testMergeAndSortWithEmptyStartRow() throws IOException { + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(""), Bytes.toBytes(20))); + ranges.add(new RowKeyRange(Bytes.toBytes(15), Bytes.toBytes(40))); + List actualRanges = MultiRowRangeFilter.sortAndMerge(ranges); + List expectedRanges = new ArrayList(); + expectedRanges.add(new RowKeyRange(Bytes.toBytes(""), Bytes.toBytes(40))); + assertRangesEqual(expectedRanges, actualRanges); + } + + @Test + public void testMergeAndSortWithEmptyStopRow() throws IOException { + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + ranges.add(new RowKeyRange(Bytes.toBytes(15), Bytes.toBytes(""))); + ranges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(70))); + List actualRanges = MultiRowRangeFilter.sortAndMerge(ranges); + List expectedRanges = new ArrayList(); + expectedRanges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(""))); + assertRangesEqual(expectedRanges, actualRanges); + } + + @Test + public void testMergeAndSortWithEmptyStartRowAndStopRow() throws IOException { + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + ranges.add(new RowKeyRange(Bytes.toBytes(""), Bytes.toBytes(""))); + ranges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(70))); + List actualRanges = MultiRowRangeFilter.sortAndMerge(ranges); + List expectedRanges = new ArrayList(); + expectedRanges.add(new RowKeyRange(Bytes.toBytes(""), Bytes.toBytes(""))); + assertRangesEqual(expectedRanges, actualRanges); + } + + @Test(expected=IllegalArgumentException.class) + public void testMultiRowRangeWithoutRange() throws IOException { + List ranges = new ArrayList(); + new MultiRowRangeFilter(ranges); + } + + @Test(expected=IllegalArgumentException.class) + public void testMultiRowRangeWithInvalidRange() throws IOException { + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + // the start row larger than the stop row + ranges.add(new RowKeyRange(Bytes.toBytes(80), Bytes.toBytes(20))); + ranges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(70))); + new MultiRowRangeFilter(ranges); + } + + @Test + public void testMergeAndSortWithoutOverlap() throws IOException { + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + ranges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(40))); + ranges.add(new RowKeyRange(Bytes.toBytes(60), Bytes.toBytes(70))); + List actualRanges = MultiRowRangeFilter.sortAndMerge(ranges); + List expectedRanges = new ArrayList(); + expectedRanges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + expectedRanges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(40))); + expectedRanges.add(new RowKeyRange(Bytes.toBytes(60), Bytes.toBytes(70))); + assertRangesEqual(expectedRanges, actualRanges); + } + + @Test + public void testMergeAndSortWithOverlap() throws IOException { + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + ranges.add(new RowKeyRange(Bytes.toBytes(15), Bytes.toBytes(40))); + ranges.add(new RowKeyRange(Bytes.toBytes(20), Bytes.toBytes(30))); + ranges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(50))); + ranges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(70))); + ranges.add(new RowKeyRange(Bytes.toBytes(90), Bytes.toBytes(100))); + ranges.add(new RowKeyRange(Bytes.toBytes(95), Bytes.toBytes(100))); + List actualRanges = MultiRowRangeFilter.sortAndMerge(ranges); + List expectedRanges = new ArrayList(); + expectedRanges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(70))); + expectedRanges.add(new RowKeyRange(Bytes.toBytes(90), Bytes.toBytes(100))); + assertRangesEqual(expectedRanges, actualRanges); + } + + public void assertRangesEqual(List expected, List actual) { + assertEquals(expected.size(), actual.size()); + for(int i = 0; i < expected.size(); i++) { + Assert.assertTrue(Bytes.equals(expected.get(i).getStartRow(), actual.get(i).getStartRow())); + Assert.assertTrue(Bytes.equals(expected.get(i).getStopRow(), actual.get(i).getStopRow())); + } + } + + @Test + public void testMultiRowRangeFilterWithRangeOverlap() throws IOException { + tableName = Bytes.toBytes("testMultiRowRangeFilterWithRangeOverlap"); + HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE); + generateRows(numRows, ht, family, qf, value); + + Scan scan = new Scan(); + scan.setMaxVersions(); + + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + ranges.add(new RowKeyRange(Bytes.toBytes(15), Bytes.toBytes(40))); + ranges.add(new RowKeyRange(Bytes.toBytes(65), Bytes.toBytes(75))); + ranges.add(new RowKeyRange(Bytes.toBytes(60), null)); + ranges.add(new RowKeyRange(Bytes.toBytes(60), Bytes.toBytes(80))); + + MultiRowRangeFilter filter = new MultiRowRangeFilter(ranges); + scan.setFilter(filter); + int resultsSize = getResultsSize(ht, scan); + LOG.info("found " + resultsSize + " results"); + List results1 = getScanResult(Bytes.toBytes(10), Bytes.toBytes(40), ht); + List results2 = getScanResult(Bytes.toBytes(60), Bytes.toBytes(""), ht); + + assertEquals(results1.size() + results2.size(), resultsSize); + + ht.close(); + } + + @Test + public void testMultiRowRangeFilterWithoutRangeOverlap() throws IOException { + tableName = Bytes.toBytes("testMultiRowRangeFilterWithoutRangeOverlap"); + HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE); + generateRows(numRows, ht, family, qf, value); + + Scan scan = new Scan(); + scan.setMaxVersions(); + + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(40))); + ranges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + ranges.add(new RowKeyRange(Bytes.toBytes(60), Bytes.toBytes(70))); + + MultiRowRangeFilter filter = new MultiRowRangeFilter(ranges); + scan.setFilter(filter); + int resultsSize = getResultsSize(ht, scan); + LOG.info("found " + resultsSize + " results"); + List results1 = getScanResult(Bytes.toBytes(10), Bytes.toBytes(20), ht); + List results2 = getScanResult(Bytes.toBytes(30), Bytes.toBytes(40), ht); + List results3 = getScanResult(Bytes.toBytes(60), Bytes.toBytes(70), ht); + + assertEquals(results1.size() + results2.size() + results3.size(), resultsSize); + + ht.close(); + } + + @Test + public void testMultiRowRangeFilterWithEmptyStartRow() throws IOException { + tableName = Bytes.toBytes("testMultiRowRangeFilterWithEmptyStartRow"); + HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE); + generateRows(numRows, ht, family, qf, value); + Scan scan = new Scan(); + scan.setMaxVersions(); + + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(""), Bytes.toBytes(10))); + ranges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(40))); + + MultiRowRangeFilter filter = new MultiRowRangeFilter(ranges); + scan.setFilter(filter); + int resultsSize = getResultsSize(ht, scan); + List results1 = getScanResult(Bytes.toBytes(""), Bytes.toBytes(10), ht); + List results2 = getScanResult(Bytes.toBytes(30), Bytes.toBytes(40), ht); + assertEquals(results1.size() + results2.size(), resultsSize); + + ht.close(); + } + + @Test + public void testMultiRowRangeFilterWithEmptyStopRow() throws IOException { + tableName = Bytes.toBytes("testMultiRowRangeFilterWithEmptyStopRow"); + HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE); + generateRows(numRows, ht, family, qf, value); + Scan scan = new Scan(); + scan.setMaxVersions(); + + List ranges = new ArrayList(); + ranges.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(""))); + ranges.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(40))); + + MultiRowRangeFilter filter = new MultiRowRangeFilter(ranges); + scan.setFilter(filter); + int resultsSize = getResultsSize(ht, scan); + List results1 = getScanResult(Bytes.toBytes(10), Bytes.toBytes(""), ht); + assertEquals(results1.size(), resultsSize); + + ht.close(); + } + + @Test + public void testMultiRowRangeWithFilterListAndOperator() throws IOException { + tableName = Bytes.toBytes("TestMultiRowRangeFilterWithFilterListAndOperator"); + HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE); + generateRows(numRows, ht, family, qf, value); + + Scan scan = new Scan(); + scan.setMaxVersions(); + + List ranges1 = new ArrayList(); + ranges1.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + ranges1.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(40))); + ranges1.add(new RowKeyRange(Bytes.toBytes(60), Bytes.toBytes(70))); + + MultiRowRangeFilter filter1 = new MultiRowRangeFilter(ranges1); + + List ranges2 = new ArrayList(); + ranges2.add(new RowKeyRange(Bytes.toBytes(20), Bytes.toBytes(40))); + ranges2.add(new RowKeyRange(Bytes.toBytes(80), Bytes.toBytes(90))); + + MultiRowRangeFilter filter2 = new MultiRowRangeFilter(ranges2); + + FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL); + filterList.addFilter(filter1); + filterList.addFilter(filter2); + scan.setFilter(filterList); + int resultsSize = getResultsSize(ht, scan); + LOG.info("found " + resultsSize + " results"); + List results1 = getScanResult(Bytes.toBytes(30), Bytes.toBytes(40), ht); + + assertEquals(results1.size(), resultsSize); + + ht.close(); + } + + @Test + public void testMultiRowRangeWithFilterListOrOperator() throws IOException { + tableName = Bytes.toBytes("TestMultiRowRangeFilterWithFilterListOrOperator"); + HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE); + generateRows(numRows, ht, family, qf, value); + + Scan scan = new Scan(); + scan.setMaxVersions(); + + List ranges1 = new ArrayList(); + ranges1.add(new RowKeyRange(Bytes.toBytes(30), Bytes.toBytes(40))); + ranges1.add(new RowKeyRange(Bytes.toBytes(10), Bytes.toBytes(20))); + ranges1.add(new RowKeyRange(Bytes.toBytes(60), Bytes.toBytes(70))); + + MultiRowRangeFilter filter1 = new MultiRowRangeFilter(ranges1); + + List ranges2 = new ArrayList(); + ranges2.add(new RowKeyRange(Bytes.toBytes(20), Bytes.toBytes(40))); + ranges2.add(new RowKeyRange(Bytes.toBytes(80), Bytes.toBytes(90))); + + MultiRowRangeFilter filter2 = new MultiRowRangeFilter(ranges2); + + FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE); + filterList.addFilter(filter1); + filterList.addFilter(filter2); + scan.setFilter(filterList); + int resultsSize = getResultsSize(ht, scan); + LOG.info("found " + resultsSize + " results"); + List results1 = getScanResult(Bytes.toBytes(10), Bytes.toBytes(40), ht); + List results2 = getScanResult(Bytes.toBytes(60), Bytes.toBytes(70), ht); + List results3 = getScanResult(Bytes.toBytes(80), Bytes.toBytes(90), ht); + + assertEquals(results1.size() + results2.size() + results3.size(),resultsSize); + + ht.close(); + } + + private void generateRows(int numberOfRows, HTable ht, byte[] family, byte[] qf, byte[] value) + throws IOException { + for (int i = 0; i < numberOfRows; i++) { + byte[] row = Bytes.toBytes(i); + Put p = new Put(row); + p.add(family, qf, value); + ht.put(p); + } + TEST_UTIL.flush(); + } + + private List getScanResult(byte[] startRow, byte[] stopRow, HTable ht) throws IOException { + Scan scan = new Scan(); + scan.setMaxVersions(); + if(!Bytes.toString(startRow).isEmpty()) { + scan.setStartRow(startRow); + } + if(!Bytes.toString(stopRow).isEmpty()) { + scan.setStopRow(stopRow); + } + ResultScanner scanner = ht.getScanner(scan); + List kvList = new ArrayList(); + Result r; + while ((r = scanner.next()) != null) { + for (Cell kv : r.listCells()) { + kvList.add(kv); + } + } + return kvList; + } + + private int getResultsSize(HTable ht, Scan scan) throws IOException { + ResultScanner scanner = ht.getScanner(scan); + List results = new ArrayList(); + Result r; + while ((r = scanner.next()) != null) { + for (Cell kv : r.listCells()) { + results.add(kv); + } + } + return results.size(); + } +} \ No newline at end of file