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))));
+  }
 }