You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2016/07/22 01:15:58 UTC
[2/7] 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)
Conflicts:
hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b2d883dd
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b2d883dd
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b2d883dd
Branch: refs/heads/0.98
Commit: b2d883ddcf47833786d7ee0eeaa52bee60c00de5
Parents: 33d64e0
Author: tedyu <yu...@gmail.com>
Authored: Tue Jan 13 07:04:49 2015 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Jul 21 17:49:22 2016 -0700
----------------------------------------------------------------------
.../hbase/filter/MultiRowRangeFilter.java | 508 ++++++
.../hbase/protobuf/generated/FilterProtos.java | 1449 +++++++++++++++++-
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, 2465 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/b2d883dd/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<RowRange> 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 <code>RowRange</code>
+ * @throws java.io.IOException
+ * throw an exception if the range list is not in an natural order or any
+ * <code>RowRange</code> is invalid
+ */
+ public MultiRowRangeFilter(List<RowRange> list) throws IOException {
+ this.rangeList = sortAndMerge(list);
+ }
+
+ @Override
+ public boolean filterAllRemaining() {
+ return done;
+ }
+
+ public List<RowRange> 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<FilterProtos.RowRange> rangeProtos = proto.getRowRangeListList();
+ List<RowRange> rangeList = new ArrayList<RowRange>(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<RowRange> sortAndMerge(List<RowRange> ranges) {
+ if (ranges.size() == 0) {
+ throw new IllegalArgumentException("No ranges found.");
+ }
+ List<RowRange> invalidRanges = new ArrayList<RowRange>();
+ List<RowRange> newRanges = new ArrayList<RowRange>(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<RowRange> 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<RowRange> {
+ 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/b2d883dd/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 2ce084c..2f210a5 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
@@ -15974,6 +15974,1425 @@ public final class FilterProtos {
// @@protoc_insertion_point(class_scope:FilterAllFilter)
}
+ public interface RowRangeOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // optional bytes start_row = 1;
+ /**
+ * <code>optional bytes start_row = 1;</code>
+ */
+ boolean hasStartRow();
+ /**
+ * <code>optional bytes start_row = 1;</code>
+ */
+ com.google.protobuf.ByteString getStartRow();
+
+ // optional bool start_row_inclusive = 2;
+ /**
+ * <code>optional bool start_row_inclusive = 2;</code>
+ */
+ boolean hasStartRowInclusive();
+ /**
+ * <code>optional bool start_row_inclusive = 2;</code>
+ */
+ boolean getStartRowInclusive();
+
+ // optional bytes stop_row = 3;
+ /**
+ * <code>optional bytes stop_row = 3;</code>
+ */
+ boolean hasStopRow();
+ /**
+ * <code>optional bytes stop_row = 3;</code>
+ */
+ com.google.protobuf.ByteString getStopRow();
+
+ // optional bool stop_row_inclusive = 4;
+ /**
+ * <code>optional bool stop_row_inclusive = 4;</code>
+ */
+ boolean hasStopRowInclusive();
+ /**
+ * <code>optional bool stop_row_inclusive = 4;</code>
+ */
+ 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<RowRange> PARSER =
+ new com.google.protobuf.AbstractParser<RowRange>() {
+ 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<RowRange> 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_;
+ /**
+ * <code>optional bytes start_row = 1;</code>
+ */
+ public boolean hasStartRow() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional bytes start_row = 1;</code>
+ */
+ 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_;
+ /**
+ * <code>optional bool start_row_inclusive = 2;</code>
+ */
+ public boolean hasStartRowInclusive() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional bool start_row_inclusive = 2;</code>
+ */
+ 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_;
+ /**
+ * <code>optional bytes stop_row = 3;</code>
+ */
+ public boolean hasStopRow() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional bytes stop_row = 3;</code>
+ */
+ 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_;
+ /**
+ * <code>optional bool stop_row_inclusive = 4;</code>
+ */
+ public boolean hasStopRowInclusive() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional bool stop_row_inclusive = 4;</code>
+ */
+ 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<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;
+ /**
+ * <code>optional bytes start_row = 1;</code>
+ */
+ public boolean hasStartRow() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional bytes start_row = 1;</code>
+ */
+ public com.google.protobuf.ByteString getStartRow() {
+ return startRow_;
+ }
+ /**
+ * <code>optional bytes start_row = 1;</code>
+ */
+ public Builder setStartRow(com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ startRow_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bytes start_row = 1;</code>
+ */
+ public Builder clearStartRow() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ startRow_ = getDefaultInstance().getStartRow();
+ onChanged();
+ return this;
+ }
+
+ // optional bool start_row_inclusive = 2;
+ private boolean startRowInclusive_ ;
+ /**
+ * <code>optional bool start_row_inclusive = 2;</code>
+ */
+ public boolean hasStartRowInclusive() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional bool start_row_inclusive = 2;</code>
+ */
+ public boolean getStartRowInclusive() {
+ return startRowInclusive_;
+ }
+ /**
+ * <code>optional bool start_row_inclusive = 2;</code>
+ */
+ public Builder setStartRowInclusive(boolean value) {
+ bitField0_ |= 0x00000002;
+ startRowInclusive_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bool start_row_inclusive = 2;</code>
+ */
+ 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;
+ /**
+ * <code>optional bytes stop_row = 3;</code>
+ */
+ public boolean hasStopRow() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>optional bytes stop_row = 3;</code>
+ */
+ public com.google.protobuf.ByteString getStopRow() {
+ return stopRow_;
+ }
+ /**
+ * <code>optional bytes stop_row = 3;</code>
+ */
+ public Builder setStopRow(com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000004;
+ stopRow_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bytes stop_row = 3;</code>
+ */
+ public Builder clearStopRow() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ stopRow_ = getDefaultInstance().getStopRow();
+ onChanged();
+ return this;
+ }
+
+ // optional bool stop_row_inclusive = 4;
+ private boolean stopRowInclusive_ ;
+ /**
+ * <code>optional bool stop_row_inclusive = 4;</code>
+ */
+ public boolean hasStopRowInclusive() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional bool stop_row_inclusive = 4;</code>
+ */
+ public boolean getStopRowInclusive() {
+ return stopRowInclusive_;
+ }
+ /**
+ * <code>optional bool stop_row_inclusive = 4;</code>
+ */
+ public Builder setStopRowInclusive(boolean value) {
+ bitField0_ |= 0x00000008;
+ stopRowInclusive_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bool stop_row_inclusive = 4;</code>
+ */
+ 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;
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ java.util.List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange>
+ getRowRangeListList();
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange getRowRangeList(int index);
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ int getRowRangeListCount();
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>
+ getRowRangeListOrBuilderList();
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ 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<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange>();
+ 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<MultiRowRangeFilter> PARSER =
+ new com.google.protobuf.AbstractParser<MultiRowRangeFilter>() {
+ 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<MultiRowRangeFilter> getParserForType() {
+ return PARSER;
+ }
+
+ // repeated .RowRange row_range_list = 1;
+ public static final int ROW_RANGE_LIST_FIELD_NUMBER = 1;
+ private java.util.List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange> rowRangeList_;
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange> getRowRangeListList() {
+ return rowRangeList_;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>
+ getRowRangeListOrBuilderList() {
+ return rowRangeList_;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public int getRowRangeListCount() {
+ return rowRangeList_.size();
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange getRowRangeList(int index) {
+ return rowRangeList_.get(index);
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ 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<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<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange> rowRangeList_ =
+ java.util.Collections.emptyList();
+ private void ensureRowRangeListIsMutable() {
+ if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+ rowRangeList_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange>(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_;
+
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange> getRowRangeListList() {
+ if (rowRangeListBuilder_ == null) {
+ return java.util.Collections.unmodifiableList(rowRangeList_);
+ } else {
+ return rowRangeListBuilder_.getMessageList();
+ }
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public int getRowRangeListCount() {
+ if (rowRangeListBuilder_ == null) {
+ return rowRangeList_.size();
+ } else {
+ return rowRangeListBuilder_.getCount();
+ }
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange getRowRangeList(int index) {
+ if (rowRangeListBuilder_ == null) {
+ return rowRangeList_.get(index);
+ } else {
+ return rowRangeListBuilder_.getMessage(index);
+ }
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public Builder addAllRowRangeList(
+ java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange> values) {
+ if (rowRangeListBuilder_ == null) {
+ ensureRowRangeListIsMutable();
+ super.addAll(values, rowRangeList_);
+ onChanged();
+ } else {
+ rowRangeListBuilder_.addAllMessages(values);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public Builder clearRowRangeList() {
+ if (rowRangeListBuilder_ == null) {
+ rowRangeList_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000001);
+ onChanged();
+ } else {
+ rowRangeListBuilder_.clear();
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public Builder removeRowRangeList(int index) {
+ if (rowRangeListBuilder_ == null) {
+ ensureRowRangeListIsMutable();
+ rowRangeList_.remove(index);
+ onChanged();
+ } else {
+ rowRangeListBuilder_.remove(index);
+ }
+ return this;
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder getRowRangeListBuilder(
+ int index) {
+ return getRowRangeListFieldBuilder().getBuilder(index);
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder getRowRangeListOrBuilder(
+ int index) {
+ if (rowRangeListBuilder_ == null) {
+ return rowRangeList_.get(index); } else {
+ return rowRangeListBuilder_.getMessageOrBuilder(index);
+ }
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>
+ getRowRangeListOrBuilderList() {
+ if (rowRangeListBuilder_ != null) {
+ return rowRangeListBuilder_.getMessageOrBuilderList();
+ } else {
+ return java.util.Collections.unmodifiableList(rowRangeList_);
+ }
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder addRowRangeListBuilder() {
+ return getRowRangeListFieldBuilder().addBuilder(
+ org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.getDefaultInstance());
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ 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());
+ }
+ /**
+ * <code>repeated .RowRange row_range_list = 1;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder>
+ 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
@@ -16114,6 +17533,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() {
@@ -16169,8 +17598,12 @@ public final class FilterProtos {
"(\010\"5\n\013ValueFilter\022&\n\016compare_filter\030\001 \002(" +
"\0132\016.CompareFilter\"+\n\020WhileMatchFilter\022\027\n" +
"\006filter\030\001 \002(\0132\007.Filter\"\021\n\017FilterAllFilte" +
- "rBB\n*org.apache.hadoop.hbase.protobuf.ge" +
- "neratedB\014FilterProtosH\001\210\001\001\240\001\001"
+ "r\"h\n\010RowRange\022\021\n\tstart_row\030\001 \001(\014\022\033\n\023star" +
+ "t_row_inclusive\030\002 \001(\010\022\020\n\010stop_row\030\003 \001(\014\022" +
+ "\032\n\022stop_row_inclusive\030\004 \001(\010\"8\n\023MultiRowR" +
+ "angeFilter\022!\n\016row_range_list\030\001 \003(\0132\t.Row",
+ "RangeBB\n*org.apache.hadoop.hbase.protobu" +
+ "f.generatedB\014FilterProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -16345,6 +17778,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/b2d883dd/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 b17f855..7fc7344 100644
--- a/hbase-protocol/src/main/protobuf/Filter.proto
+++ b/hbase-protocol/src/main/protobuf/Filter.proto
@@ -156,4 +156,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/b2d883dd/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 f9eaa22..bb8c001 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;
@@ -213,6 +215,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
@XmlAttribute public Boolean dropDependentColumn;
@XmlAttribute public Float chance;
@XmlElement public List<String> prefixes;
+ @XmlElement private List<RowRange> ranges;
@XmlElement public List<Long> timestamps;
static enum FilterType {
@@ -227,6 +230,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
InclusiveStopFilter,
KeyOnlyFilter,
MultipleColumnPrefixFilter,
+ MultiRowRangeFilter,
PageFilter,
PrefixFilter,
QualifierFilter,
@@ -294,6 +298,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
this.prefixes.add(Base64.encodeBytes(prefix));
}
break;
+ case MultiRowRangeFilter:
+ this.ranges = new ArrayList<RowRange>();
+ 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;
@@ -395,6 +406,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/b2d883dd/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 163b7d1..5576296 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,10 +21,13 @@ 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.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -319,4 +322,16 @@ public class TestFilterSerialization {
ProtobufUtil.toFilter(ProtobufUtil.toFilter(whileMatchFilter))));
}
+ @Test
+ public void testMultiRowRangeFilter() throws Exception {
+ List<RowRange> ranges = new ArrayList<RowRange>();
+ 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))));
+ }
}