Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C9A7310745 for ; Tue, 13 Jan 2015 15:02:14 +0000 (UTC) Received: (qmail 88545 invoked by uid 500); 13 Jan 2015 15:02:16 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 88412 invoked by uid 500); 13 Jan 2015 15:02:16 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 88293 invoked by uid 99); 13 Jan 2015 15:02:16 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 13 Jan 2015 15:02:16 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 118A892DF2D; Tue, 13 Jan 2015 15:02:16 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: tedyu@apache.org To: commits@hbase.apache.org Date: Tue, 13 Jan 2015 15:02:16 -0000 Message-Id: <98383200bcf34273bfe5f49daaf749ae@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [2/2] hbase git commit: HBASE-11144 Filter to support scanning multiple row key ranges (Jiajia Li) HBASE-11144 Filter to support scanning multiple row key ranges (Jiajia Li) Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e5f3dd68 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e5f3dd68 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e5f3dd68 Branch: refs/heads/master Commit: e5f3dd682fb8884a947b40b4348bd5d1386a6470 Parents: 20a096d Author: tedyu Authored: Tue Jan 13 07:02:06 2015 -0800 Committer: tedyu Committed: Tue Jan 13 07:02:06 2015 -0800 ---------------------------------------------------------------------- .../hbase/filter/MultiRowRangeFilter.java | 508 ++++++ .../hbase/protobuf/generated/FilterProtos.java | 1451 +++++++++++++++++- hbase-protocol/src/main/protobuf/Filter.proto | 11 + .../hadoop/hbase/rest/model/ScannerModel.java | 20 +- .../hbase/filter/TestFilterSerialization.java | 15 + .../hbase/filter/TestMultiRowRangeFilter.java | 465 ++++++ 6 files changed, 2466 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f3dd68/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java ---------------------------------------------------------------------- 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..bd880a0 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java @@ -0,0 +1,508 @@ +/* + * 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. + * + * HBase is quite efficient when scanning only one small row key range. If user needs to specify + * multiple row key ranges in one scan, the typical solutions are: 1. through FilterList which is a + * list of row key Filters, 2. using the SQL layer over HBase to join with two table, such as hive, + * phoenix etc. However, both solutions are inefficient. Both of them can't utilize the range info + * to perform fast forwarding during scan which is quite time consuming. If the number of ranges + * are quite big (e.g. millions), join is a proper solution though it is slow. However, there are + * cases that user wants to specify a small number of ranges to scan (e.g. <1000 ranges). Both + * solutions can't provide satisfactory performance in such case. MultiRowRangeFilter is to support + * such usec ase (scan multiple row key ranges), which can construct the row key ranges from user + * specified list and perform fast-forwarding during scan. Thus, the scan will be quite efficient. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class MultiRowRangeFilter extends FilterBase { + + private List rangeList; + + private static final int ROW_BEFORE_FIRST_RANGE = -1; + private boolean EXCLUSIVE = false; + private boolean done = false; + private boolean initialized = false; + private int index; + private RowRange range; + private ReturnCode currentReturnCode; + + /** + * @param list A list of RowRange + * @throws java.io.IOException + * throw an exception if the range list is not in an natural order or any + * RowRange 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(EXCLUSIVE) { + EXCLUSIVE = false; + currentReturnCode = ReturnCode.NEXT_ROW; + return false; + } + if (!initialized) { + if(index != ROW_BEFORE_FIRST_RANGE) { + currentReturnCode = ReturnCode.INCLUDE; + } else { + currentReturnCode = ReturnCode.SEEK_NEXT_USING_HINT; + } + initialized = true; + } else { + currentReturnCode = ReturnCode.SEEK_NEXT_USING_HINT; + } + } 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 (RowRange range : rangeList) { + if (range != null) { + FilterProtos.RowRange.Builder rangebuilder = FilterProtos.RowRange.newBuilder(); + if (range.startRow != null) + rangebuilder.setStartRow(HBaseZeroCopyByteString.wrap(range.startRow)); + rangebuilder.setStartRowInclusive(range.startRowInclusive); + if (range.stopRow != null) + rangebuilder.setStopRow(HBaseZeroCopyByteString.wrap(range.stopRow)); + rangebuilder.setStopRowInclusive(range.stopRowInclusive); + range.isScan = Bytes.equals(range.startRow, range.stopRow) ? 1 : 0; + builder.addRowRangeList(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.getRowRangeListCount(); + List rangeProtos = proto.getRowRangeListList(); + List rangeList = new ArrayList(length); + for (FilterProtos.RowRange rangeProto : rangeProtos) { + RowRange range = new RowRange(rangeProto.hasStartRow() ? rangeProto.getStartRow() + .toByteArray() : null, rangeProto.getStartRowInclusive(), rangeProto.hasStopRow() ? + rangeProto.getStopRow().toByteArray() : null, rangeProto.getStopRowInclusive()); + 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) { + RowRange thisRange = this.rangeList.get(i); + RowRange otherRange = other.rangeList.get(i); + if (!(Bytes.equals(thisRange.startRow, otherRange.startRow) && Bytes.equals( + thisRange.stopRow, otherRange.stopRow) && (thisRange.startRowInclusive == + otherRange.startRowInclusive) && (thisRange.stopRowInclusive == + otherRange.stopRowInclusive))) { + 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) { + RowRange temp = new RowRange(rowKey, true, null, true); + 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, and the the range exclude the start key + if(rangeList.get(index).startRowInclusive == false) { + EXCLUSIVE = true; + } + 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; + boolean lastStartRowInclusive = ranges.get(0).startRowInclusive; + byte[] lastStopRow = ranges.get(0).stopRow; + boolean lastStopRowInclusive = ranges.get(0).stopRowInclusive; + int i = 1; + for (; i < ranges.size(); i++) { + RowRange range = ranges.get(i); + if (!range.isValid()) { + invalidRanges.add(range); + } + if(Bytes.equals(lastStopRow, HConstants.EMPTY_BYTE_ARRAY)) { + newRanges.add(new RowRange(lastStartRow, lastStartRowInclusive, lastStopRow, + lastStopRowInclusive)); + break; + } + // with overlap in the ranges + if ((Bytes.compareTo(lastStopRow, range.startRow) > 0) || + (Bytes.compareTo(lastStopRow, range.startRow) == 0 && !(lastStopRowInclusive == false && + range.isStartRowInclusive() == false))) { + if(Bytes.equals(range.stopRow, HConstants.EMPTY_BYTE_ARRAY)) { + newRanges.add(new RowRange(lastStartRow, lastStartRowInclusive, range.stopRow, + range.stopRowInclusive)); + break; + } + // if first range contains second range, ignore the second range + if (Bytes.compareTo(lastStopRow, range.stopRow) >= 0) { + if((Bytes.compareTo(lastStopRow, range.stopRow) == 0)) { + if(lastStopRowInclusive == true || range.stopRowInclusive == true) { + lastStopRowInclusive = true; + } + } + if ((i + 1) == ranges.size()) { + newRanges.add(new RowRange(lastStartRow, lastStartRowInclusive, lastStopRow, + lastStopRowInclusive)); + } + } else { + lastStopRow = range.stopRow; + lastStopRowInclusive = range.stopRowInclusive; + if ((i + 1) < ranges.size()) { + i++; + range = ranges.get(i); + if (!range.isValid()) { + invalidRanges.add(range); + } + } else { + newRanges.add(new RowRange(lastStartRow, lastStartRowInclusive, lastStopRow, + lastStopRowInclusive)); + break; + } + while ((Bytes.compareTo(lastStopRow, range.startRow) > 0) || + (Bytes.compareTo(lastStopRow, range.startRow) == 0 && + (lastStopRowInclusive == true || range.startRowInclusive==true))) { + 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) { + if(lastStopRowInclusive == true || range.stopRowInclusive == true) { + lastStopRowInclusive = true; + } + i++; + if (i < ranges.size()) { + range = ranges.get(i); + if (!range.isValid()) { + invalidRanges.add(range); + } + } else { + break; + } + } else { + lastStopRow = range.stopRow; + lastStopRowInclusive = range.stopRowInclusive; + 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) || + (Bytes.compareTo(lastStopRow, range.startRow) == 0 && + lastStopRowInclusive == false && range.startRowInclusive == false)) { + newRanges.add(new RowRange(lastStartRow, lastStartRowInclusive, lastStopRow, + lastStopRowInclusive)); + newRanges.add(range); + } else { + newRanges.add(new RowRange(lastStartRow, lastStartRowInclusive, range.stopRow, + range.stopRowInclusive)); + break; + } + } + newRanges.add(new RowRange(lastStartRow, lastStartRowInclusive, lastStopRow, + lastStopRowInclusive)); + if ((i + 1) == ranges.size()) { + newRanges.add(range); + } + lastStartRow = range.startRow; + lastStartRowInclusive = range.startRowInclusive; + lastStopRow = range.stopRow; + lastStopRowInclusive = range.stopRowInclusive; + } + } else { + newRanges.add(new RowRange(lastStartRow, lastStartRowInclusive, lastStopRow, + lastStopRowInclusive)); + if ((i + 1) == ranges.size()) { + newRanges.add(range); + } + lastStartRow = range.startRow; + lastStartRowInclusive = range.startRowInclusive; + lastStopRow = range.stopRow; + lastStopRowInclusive = range.stopRowInclusive; + } + } + // 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 (RowRange 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 RowRange implements Comparable { + private byte[] startRow; + private boolean startRowInclusive = true; + private byte[] stopRow; + private boolean stopRowInclusive = false; + private int isScan = 0; + + public RowRange() { + } + /** + * 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 RowRange(String startRow, boolean startRowInclusive, String stopRow, + boolean stopRowInclusive) { + this((startRow == null || startRow.isEmpty()) ? HConstants.EMPTY_BYTE_ARRAY : + Bytes.toBytes(startRow), startRowInclusive, + (stopRow == null || stopRow.isEmpty()) ? HConstants.EMPTY_BYTE_ARRAY : + Bytes.toBytes(stopRow), stopRowInclusive); + } + + public RowRange(byte[] startRow, boolean startRowInclusive, byte[] stopRow, + boolean stopRowInclusive) { + this.startRow = (startRow == null) ? HConstants.EMPTY_BYTE_ARRAY : startRow; + this.startRowInclusive = startRowInclusive; + this.stopRow = (stopRow == null) ? HConstants.EMPTY_BYTE_ARRAY :stopRow; + this.stopRowInclusive = stopRowInclusive; + isScan = Bytes.equals(startRow, stopRow) ? 1 : 0; + } + + public byte[] getStartRow() { + return startRow; + } + + public byte[] getStopRow() { + return stopRow; + } + + /** + * @return if start row is inclusive. + */ + public boolean isStartRowInclusive() { + return startRowInclusive; + } + + /** + * @return if stop row is inclusive. + */ + public boolean isStopRowInclusive() { + return stopRowInclusive; + } + + public boolean contains(byte[] row) { + return contains(row, 0, row.length); + } + + public boolean contains(byte[] buffer, int offset, int length) { + if(startRowInclusive) { + if(stopRowInclusive) { + 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); + } else { + 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); + } + } else { + if(stopRowInclusive) { + 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); + } else { + 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(RowRange other) { + return Bytes.compareTo(this.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 + || (Bytes.compareTo(startRow, stopRow) == 0 && stopRowInclusive == true); + } + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f3dd68/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java ---------------------------------------------------------------------- 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..3fb466d 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,1425 @@ public final class FilterProtos { // @@protoc_insertion_point(class_scope:FilterAllFilter) } + public interface RowRangeOrBuilder + 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 bool start_row_inclusive = 2; + /** + * optional bool start_row_inclusive = 2; + */ + boolean hasStartRowInclusive(); + /** + * optional bool start_row_inclusive = 2; + */ + boolean getStartRowInclusive(); + + // optional bytes stop_row = 3; + /** + * optional bytes stop_row = 3; + */ + boolean hasStopRow(); + /** + * optional bytes stop_row = 3; + */ + com.google.protobuf.ByteString getStopRow(); + + // optional bool stop_row_inclusive = 4; + /** + * optional bool stop_row_inclusive = 4; + */ + boolean hasStopRowInclusive(); + /** + * optional bool stop_row_inclusive = 4; + */ + boolean getStopRowInclusive(); + } + /** + * Protobuf type {@code RowRange} + */ + public static final class RowRange extends + com.google.protobuf.GeneratedMessage + implements RowRangeOrBuilder { + // Use RowRange.newBuilder() to construct. + private RowRange(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private RowRange(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final RowRange defaultInstance; + public static RowRange getDefaultInstance() { + return defaultInstance; + } + + public RowRange getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private RowRange( + 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 16: { + bitField0_ |= 0x00000002; + startRowInclusive_ = input.readBool(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + stopRow_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + stopRowInclusive_ = input.readBool(); + 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_RowRange_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_RowRange_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public RowRange parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new RowRange(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 bool start_row_inclusive = 2; + public static final int START_ROW_INCLUSIVE_FIELD_NUMBER = 2; + private boolean startRowInclusive_; + /** + * optional bool start_row_inclusive = 2; + */ + public boolean hasStartRowInclusive() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bool start_row_inclusive = 2; + */ + public boolean getStartRowInclusive() { + return startRowInclusive_; + } + + // optional bytes stop_row = 3; + public static final int STOP_ROW_FIELD_NUMBER = 3; + private com.google.protobuf.ByteString stopRow_; + /** + * optional bytes stop_row = 3; + */ + public boolean hasStopRow() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional bytes stop_row = 3; + */ + public com.google.protobuf.ByteString getStopRow() { + return stopRow_; + } + + // optional bool stop_row_inclusive = 4; + public static final int STOP_ROW_INCLUSIVE_FIELD_NUMBER = 4; + private boolean stopRowInclusive_; + /** + * optional bool stop_row_inclusive = 4; + */ + public boolean hasStopRowInclusive() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional bool stop_row_inclusive = 4; + */ + public boolean getStopRowInclusive() { + return stopRowInclusive_; + } + + private void initFields() { + startRow_ = com.google.protobuf.ByteString.EMPTY; + startRowInclusive_ = false; + stopRow_ = com.google.protobuf.ByteString.EMPTY; + stopRowInclusive_ = false; + } + 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.writeBool(2, startRowInclusive_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, stopRow_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBool(4, stopRowInclusive_); + } + 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 + .computeBoolSize(2, startRowInclusive_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, stopRow_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(4, stopRowInclusive_); + } + 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.RowRange)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange other = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange) obj; + + boolean result = true; + result = result && (hasStartRow() == other.hasStartRow()); + if (hasStartRow()) { + result = result && getStartRow() + .equals(other.getStartRow()); + } + result = result && (hasStartRowInclusive() == other.hasStartRowInclusive()); + if (hasStartRowInclusive()) { + result = result && (getStartRowInclusive() + == other.getStartRowInclusive()); + } + result = result && (hasStopRow() == other.hasStopRow()); + if (hasStopRow()) { + result = result && getStopRow() + .equals(other.getStopRow()); + } + result = result && (hasStopRowInclusive() == other.hasStopRowInclusive()); + if (hasStopRowInclusive()) { + result = result && (getStopRowInclusive() + == other.getStopRowInclusive()); + } + 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 (hasStartRowInclusive()) { + hash = (37 * hash) + START_ROW_INCLUSIVE_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getStartRowInclusive()); + } + if (hasStopRow()) { + hash = (37 * hash) + STOP_ROW_FIELD_NUMBER; + hash = (53 * hash) + getStopRow().hashCode(); + } + if (hasStopRowInclusive()) { + hash = (37 * hash) + STOP_ROW_INCLUSIVE_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getStopRowInclusive()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange 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.RowRange parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange 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.RowRange parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange 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.RowRange parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange 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.RowRange parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange 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.RowRange 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 RowRange} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_RowRange_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_RowRange_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.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); + startRowInclusive_ = false; + bitField0_ = (bitField0_ & ~0x00000002); + stopRow_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); + stopRowInclusive_ = false; + bitField0_ = (bitField0_ & ~0x00000008); + 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_RowRange_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange build() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange result = new org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange(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.startRowInclusive_ = startRowInclusive_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.stopRow_ = stopRow_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.stopRowInclusive_ = stopRowInclusive_; + 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.RowRange) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.getDefaultInstance()) return this; + if (other.hasStartRow()) { + setStartRow(other.getStartRow()); + } + if (other.hasStartRowInclusive()) { + setStartRowInclusive(other.getStartRowInclusive()); + } + if (other.hasStopRow()) { + setStopRow(other.getStopRow()); + } + if (other.hasStopRowInclusive()) { + setStopRowInclusive(other.getStopRowInclusive()); + } + 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.RowRange parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange) 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 bool start_row_inclusive = 2; + private boolean startRowInclusive_ ; + /** + * optional bool start_row_inclusive = 2; + */ + public boolean hasStartRowInclusive() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bool start_row_inclusive = 2; + */ + public boolean getStartRowInclusive() { + return startRowInclusive_; + } + /** + * optional bool start_row_inclusive = 2; + */ + public Builder setStartRowInclusive(boolean value) { + bitField0_ |= 0x00000002; + startRowInclusive_ = value; + onChanged(); + return this; + } + /** + * optional bool start_row_inclusive = 2; + */ + public Builder clearStartRowInclusive() { + bitField0_ = (bitField0_ & ~0x00000002); + startRowInclusive_ = false; + onChanged(); + return this; + } + + // optional bytes stop_row = 3; + private com.google.protobuf.ByteString stopRow_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes stop_row = 3; + */ + public boolean hasStopRow() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional bytes stop_row = 3; + */ + public com.google.protobuf.ByteString getStopRow() { + return stopRow_; + } + /** + * optional bytes stop_row = 3; + */ + public Builder setStopRow(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + stopRow_ = value; + onChanged(); + return this; + } + /** + * optional bytes stop_row = 3; + */ + public Builder clearStopRow() { + bitField0_ = (bitField0_ & ~0x00000004); + stopRow_ = getDefaultInstance().getStopRow(); + onChanged(); + return this; + } + + // optional bool stop_row_inclusive = 4; + private boolean stopRowInclusive_ ; + /** + * optional bool stop_row_inclusive = 4; + */ + public boolean hasStopRowInclusive() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional bool stop_row_inclusive = 4; + */ + public boolean getStopRowInclusive() { + return stopRowInclusive_; + } + /** + * optional bool stop_row_inclusive = 4; + */ + public Builder setStopRowInclusive(boolean value) { + bitField0_ |= 0x00000008; + stopRowInclusive_ = value; + onChanged(); + return this; + } + /** + * optional bool stop_row_inclusive = 4; + */ + public Builder clearStopRowInclusive() { + bitField0_ = (bitField0_ & ~0x00000008); + stopRowInclusive_ = false; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:RowRange) + } + + static { + defaultInstance = new RowRange(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RowRange) + } + + public interface MultiRowRangeFilterOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // repeated .RowRange row_range_list = 1; + /** + * repeated .RowRange row_range_list = 1; + */ + java.util.List + getRowRangeListList(); + /** + * repeated .RowRange row_range_list = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange getRowRangeList(int index); + /** + * repeated .RowRange row_range_list = 1; + */ + int getRowRangeListCount(); + /** + * repeated .RowRange row_range_list = 1; + */ + java.util.List + getRowRangeListOrBuilderList(); + /** + * repeated .RowRange row_range_list = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder getRowRangeListOrBuilder( + 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)) { + rowRangeList_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + rowRangeList_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.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)) { + rowRangeList_ = java.util.Collections.unmodifiableList(rowRangeList_); + } + 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 .RowRange row_range_list = 1; + public static final int ROW_RANGE_LIST_FIELD_NUMBER = 1; + private java.util.List rowRangeList_; + /** + * repeated .RowRange row_range_list = 1; + */ + public java.util.List getRowRangeListList() { + return rowRangeList_; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public java.util.List + getRowRangeListOrBuilderList() { + return rowRangeList_; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public int getRowRangeListCount() { + return rowRangeList_.size(); + } + /** + * repeated .RowRange row_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange getRowRangeList(int index) { + return rowRangeList_.get(index); + } + /** + * repeated .RowRange row_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder getRowRangeListOrBuilder( + int index) { + return rowRangeList_.get(index); + } + + private void initFields() { + rowRangeList_ = 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 < rowRangeList_.size(); i++) { + output.writeMessage(1, rowRangeList_.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 < rowRangeList_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, rowRangeList_.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 && getRowRangeListList() + .equals(other.getRowRangeListList()); + 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 (getRowRangeListCount() > 0) { + hash = (37 * hash) + ROW_RANGE_LIST_FIELD_NUMBER; + hash = (53 * hash) + getRowRangeListList().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) { + getRowRangeListFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (rowRangeListBuilder_ == null) { + rowRangeList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + rowRangeListBuilder_.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 (rowRangeListBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + rowRangeList_ = java.util.Collections.unmodifiableList(rowRangeList_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.rowRangeList_ = rowRangeList_; + } else { + result.rowRangeList_ = rowRangeListBuilder_.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 (rowRangeListBuilder_ == null) { + if (!other.rowRangeList_.isEmpty()) { + if (rowRangeList_.isEmpty()) { + rowRangeList_ = other.rowRangeList_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureRowRangeListIsMutable(); + rowRangeList_.addAll(other.rowRangeList_); + } + onChanged(); + } + } else { + if (!other.rowRangeList_.isEmpty()) { + if (rowRangeListBuilder_.isEmpty()) { + rowRangeListBuilder_.dispose(); + rowRangeListBuilder_ = null; + rowRangeList_ = other.rowRangeList_; + bitField0_ = (bitField0_ & ~0x00000001); + rowRangeListBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getRowRangeListFieldBuilder() : null; + } else { + rowRangeListBuilder_.addAllMessages(other.rowRangeList_); + } + } + } + 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 .RowRange row_range_list = 1; + private java.util.List rowRangeList_ = + java.util.Collections.emptyList(); + private void ensureRowRangeListIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + rowRangeList_ = new java.util.ArrayList(rowRangeList_); + bitField0_ |= 0x00000001; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder> rowRangeListBuilder_; + + /** + * repeated .RowRange row_range_list = 1; + */ + public java.util.List getRowRangeListList() { + if (rowRangeListBuilder_ == null) { + return java.util.Collections.unmodifiableList(rowRangeList_); + } else { + return rowRangeListBuilder_.getMessageList(); + } + } + /** + * repeated .RowRange row_range_list = 1; + */ + public int getRowRangeListCount() { + if (rowRangeListBuilder_ == null) { + return rowRangeList_.size(); + } else { + return rowRangeListBuilder_.getCount(); + } + } + /** + * repeated .RowRange row_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange getRowRangeList(int index) { + if (rowRangeListBuilder_ == null) { + return rowRangeList_.get(index); + } else { + return rowRangeListBuilder_.getMessage(index); + } + } + /** + * repeated .RowRange row_range_list = 1; + */ + public Builder setRowRangeList( + int index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange value) { + if (rowRangeListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureRowRangeListIsMutable(); + rowRangeList_.set(index, value); + onChanged(); + } else { + rowRangeListBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public Builder setRowRangeList( + int index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder builderForValue) { + if (rowRangeListBuilder_ == null) { + ensureRowRangeListIsMutable(); + rowRangeList_.set(index, builderForValue.build()); + onChanged(); + } else { + rowRangeListBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public Builder addRowRangeList(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange value) { + if (rowRangeListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureRowRangeListIsMutable(); + rowRangeList_.add(value); + onChanged(); + } else { + rowRangeListBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public Builder addRowRangeList( + int index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange value) { + if (rowRangeListBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureRowRangeListIsMutable(); + rowRangeList_.add(index, value); + onChanged(); + } else { + rowRangeListBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public Builder addRowRangeList( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder builderForValue) { + if (rowRangeListBuilder_ == null) { + ensureRowRangeListIsMutable(); + rowRangeList_.add(builderForValue.build()); + onChanged(); + } else { + rowRangeListBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public Builder addRowRangeList( + int index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder builderForValue) { + if (rowRangeListBuilder_ == null) { + ensureRowRangeListIsMutable(); + rowRangeList_.add(index, builderForValue.build()); + onChanged(); + } else { + rowRangeListBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public Builder addAllRowRangeList( + java.lang.Iterable values) { + if (rowRangeListBuilder_ == null) { + ensureRowRangeListIsMutable(); + super.addAll(values, rowRangeList_); + onChanged(); + } else { + rowRangeListBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public Builder clearRowRangeList() { + if (rowRangeListBuilder_ == null) { + rowRangeList_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + rowRangeListBuilder_.clear(); + } + return this; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public Builder removeRowRangeList(int index) { + if (rowRangeListBuilder_ == null) { + ensureRowRangeListIsMutable(); + rowRangeList_.remove(index); + onChanged(); + } else { + rowRangeListBuilder_.remove(index); + } + return this; + } + /** + * repeated .RowRange row_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder getRowRangeListBuilder( + int index) { + return getRowRangeListFieldBuilder().getBuilder(index); + } + /** + * repeated .RowRange row_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder getRowRangeListOrBuilder( + int index) { + if (rowRangeListBuilder_ == null) { + return rowRangeList_.get(index); } else { + return rowRangeListBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .RowRange row_range_list = 1; + */ + public java.util.List + getRowRangeListOrBuilderList() { + if (rowRangeListBuilder_ != null) { + return rowRangeListBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(rowRangeList_); + } + } + /** + * repeated .RowRange row_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder addRowRangeListBuilder() { + return getRowRangeListFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.getDefaultInstance()); + } + /** + * repeated .RowRange row_range_list = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder addRowRangeListBuilder( + int index) { + return getRowRangeListFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.getDefaultInstance()); + } + /** + * repeated .RowRange row_range_list = 1; + */ + public java.util.List + getRowRangeListBuilderList() { + return getRowRangeListFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder> + getRowRangeListFieldBuilder() { + if (rowRangeListBuilder_ == null) { + rowRangeListBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>( + rowRangeList_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + rowRangeList_ = null; + } + return rowRangeListBuilder_; + } + + // @@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 +17440,16 @@ public final class FilterProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_FilterAllFilter_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RowRange_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RowRange_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 +17504,13 @@ public final class FilterProtos { "\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\"h\n\010RowRange\022\021\n\ts" + + "tart_row\030\001 \001(\014\022\033\n\023start_row_inclusive\030\002 " + + "\001(\010\022\020\n\010stop_row\030\003 \001(\014\022\032\n\022stop_row_inclus" + + "ive\030\004 \001(\010\"8\n\023MultiRowRangeFilter\022!\n\016row_" + + "range_list\030\001 \003(\0132\t.RowRangeBB\n*org.apach", + "e.hadoop.hbase.protobuf.generatedB\014Filte" + + "rProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -16252,6 +17685,18 @@ public final class FilterProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_FilterAllFilter_descriptor, new java.lang.String[] { }); + internal_static_RowRange_descriptor = + getDescriptor().getMessageTypes().get(28); + internal_static_RowRange_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_RowRange_descriptor, + new java.lang.String[] { "StartRow", "StartRowInclusive", "StopRow", "StopRowInclusive", }); + 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[] { "RowRangeList", }); return null; } }; http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f3dd68/hbase-protocol/src/main/protobuf/Filter.proto ---------------------------------------------------------------------- diff --git a/hbase-protocol/src/main/protobuf/Filter.proto b/hbase-protocol/src/main/protobuf/Filter.proto index d5c51a4..a0bfd54 100644 --- a/hbase-protocol/src/main/protobuf/Filter.proto +++ b/hbase-protocol/src/main/protobuf/Filter.proto @@ -155,4 +155,15 @@ message WhileMatchFilter { required Filter filter = 1; } message FilterAllFilter { +} + +message RowRange { + optional bytes start_row = 1; + optional bool start_row_inclusive = 2; + optional bytes stop_row = 3; + optional bool stop_row_inclusive =4; +} + +message MultiRowRangeFilter { + repeated RowRange row_range_list = 1; } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f3dd68/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java ---------------------------------------------------------------------- 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..784f7e6 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.XmlAttribute; 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.FilterList; 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.RowRange; 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 class ScannerModel implements ProtobufMessageHandler, Serializable { @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 class ScannerModel implements ProtobufMessageHandler, Serializable { InclusiveStopFilter, KeyOnlyFilter, MultipleColumnPrefixFilter, + MultiRowRangeFilter, PageFilter, PrefixFilter, QualifierFilter, @@ -293,6 +297,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable { this.prefixes.add(Base64.encodeBytes(prefix)); } break; + case MultiRowRangeFilter: + this.ranges = new ArrayList(); + for(RowRange range : ((MultiRowRangeFilter)filter).getRowRanges()) { + this.ranges.add(new RowRange(range.getStartRow(), range.isStartRowInclusive(), + range.getStopRow(), range.isStopRowInclusive())); + } + break; case PageFilter: this.value = Long.toString(((PageFilter)filter).getPageSize()); break; @@ -394,6 +405,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable { } filter = new MultipleColumnPrefixFilter(values); } break; + case MultiRowRangeFilter: { + try { + filter = new MultiRowRangeFilter(ranges); + } catch (IOException e) { + throw new RuntimeException(e); + } + } break; case PageFilter: filter = new PageFilter(Long.valueOf(value)); break; http://git-wip-us.apache.org/repos/asf/hbase/blob/e5f3dd68/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java index 08ce3d5..0a8b4bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java @@ -21,12 +21,15 @@ package org.apache.hadoop.hbase.filter; import static org.junit.Assert.assertTrue; +import java.util.ArrayList; import java.util.LinkedList; +import java.util.List; import java.util.TreeSet; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -320,4 +323,16 @@ public class TestFilterSerialization { ProtobufUtil.toFilter(ProtobufUtil.toFilter(whileMatchFilter)))); } + @Test + public void testMultiRowRangeFilter() throws Exception { + List ranges = new ArrayList(); + ranges.add(new RowRange(Bytes.toBytes(30), true, Bytes.toBytes(40), false)); + ranges.add(new RowRange(Bytes.toBytes(10), true, Bytes.toBytes(20), false)); + ranges.add(new RowRange(Bytes.toBytes(60), true, Bytes.toBytes(70), false)); + + MultiRowRangeFilter multiRowRangeFilter = + new MultiRowRangeFilter(ranges); + assertTrue(multiRowRangeFilter.areSerializedFieldsEqual( + ProtobufUtil.toFilter(ProtobufUtil.toFilter(multiRowRangeFilter)))); + } }