You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2016/12/29 01:48:55 UTC

[2/2] hbase git commit: HBASE-17320 Add inclusive/exclusive support for startRow and endRow of scan

HBASE-17320 Add inclusive/exclusive support for startRow and endRow of scan


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/05b1d918
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/05b1d918
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/05b1d918

Branch: refs/heads/master
Commit: 05b1d918b0a845ced066a66b187823c357ed673d
Parents: a3e0e0d
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 28 21:35:50 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 29 09:43:31 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncClientScanner.java |  22 +-
 .../client/AsyncRpcRetryingCallerFactory.java   |   2 +-
 .../AsyncScanSingleRegionRpcRetryingCaller.java |  65 +---
 .../client/AsyncSmallScanRpcRetryingCaller.java |  71 ++--
 .../hbase/client/AsyncTableResultScanner.java   |  20 +-
 .../hadoop/hbase/client/ConnectionUtils.java    |  42 +++
 .../org/apache/hadoop/hbase/client/Scan.java    | 183 ++++++---
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  21 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  21 +-
 .../org/apache/hadoop/hbase/io/TimeRange.java   |   2 -
 .../shaded/protobuf/generated/ClientProtos.java | 363 +++++++++++++-----
 .../src/main/protobuf/Client.proto              |   2 +
 .../hbase/protobuf/generated/ClientProtos.java  | 367 ++++++++++++++-----
 hbase-protocol/src/main/protobuf/Client.proto   |   2 +
 .../regionserver/DefaultStoreFileManager.java   |   4 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  38 +-
 .../hadoop/hbase/regionserver/HStore.java       |  41 +--
 .../regionserver/ReversedRegionScannerImpl.java |  15 +-
 .../apache/hadoop/hbase/regionserver/Store.java |  83 +++--
 .../hbase/regionserver/StoreFileManager.java    |   6 +-
 .../regionserver/StripeStoreFileManager.java    |   4 +-
 .../CompactionScanQueryMatcher.java             |   5 +-
 .../querymatcher/LegacyScanQueryMatcher.java    |   3 +-
 .../NormalUserScanQueryMatcher.java             |  48 ++-
 .../querymatcher/RawScanQueryMatcher.java       |  48 ++-
 .../querymatcher/ScanQueryMatcher.java          |   8 +-
 .../querymatcher/UserScanQueryMatcher.java      |  15 +-
 .../client/AbstractTestAsyncTableScan.java      |  58 ++-
 .../hbase/client/TestRawAsyncTableScan.java     |  18 +-
 .../TestStripeStoreFileManager.java             |  27 +-
 30 files changed, 1102 insertions(+), 502 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java
index dfffd39..d7a3ed1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.HConstants.EMPTY_END_ROW;
 import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.getLocateType;
 
 import java.io.IOException;
 import java.util.concurrent.CompletableFuture;
@@ -64,10 +64,10 @@ class AsyncClientScanner {
   public AsyncClientScanner(Scan scan, RawScanResultConsumer consumer, TableName tableName,
       AsyncConnectionImpl conn, long scanTimeoutNs, long rpcTimeoutNs) {
     if (scan.getStartRow() == null) {
-      scan.setStartRow(EMPTY_START_ROW);
+      scan.withStartRow(EMPTY_START_ROW, scan.includeStartRow());
     }
     if (scan.getStopRow() == null) {
-      scan.setStopRow(EMPTY_END_ROW);
+      scan.withStopRow(EMPTY_END_ROW, scan.includeStopRow());
     }
     this.scan = scan;
     this.consumer = consumer;
@@ -117,23 +117,22 @@ class AsyncClientScanner {
     conn.callerFactory.scanSingleRegion().id(resp.scannerId).location(resp.loc).stub(resp.stub)
         .setScan(scan).consumer(consumer).resultCache(resultCache)
         .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
-        .scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).start()
-        .whenComplete((locateType, error) -> {
+        .scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).start().whenComplete((hasMore, error) -> {
           if (error != null) {
             consumer.onError(error);
             return;
           }
-          if (locateType == null) {
-            consumer.onComplete();
+          if (hasMore) {
+            openScanner();
           } else {
-            openScanner(locateType);
+            consumer.onComplete();
           }
         });
   }
 
-  private void openScanner(RegionLocateType locateType) {
+  private void openScanner() {
     conn.callerFactory.<OpenScannerResponse> single().table(tableName).row(scan.getStartRow())
-        .locateType(locateType).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
+        .locateType(getLocateType(scan)).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
         .operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).action(this::callOpenScanner).call()
         .whenComplete((resp, error) -> {
           if (error != null) {
@@ -145,7 +144,6 @@ class AsyncClientScanner {
   }
 
   public void start() {
-    openScanner(scan.isReversed() && isEmptyStartRow(scan.getStartRow()) ? RegionLocateType.BEFORE
-        : RegionLocateType.CURRENT);
+    openScanner();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
index c90bee2..55c56ab 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
@@ -246,7 +246,7 @@ class AsyncRpcRetryingCallerFactory {
     /**
      * Short cut for {@code build().start()}.
      */
-    public CompletableFuture<RegionLocateType> start() {
+    public CompletableFuture<Boolean> start() {
       return build().start();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
index 5bf6195..dae88a7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
@@ -17,11 +17,9 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowBefore;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForReverseScan;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForScan;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
@@ -34,7 +32,6 @@ import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -53,7 +50,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientServ
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.Interface;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
@@ -91,11 +87,9 @@ class AsyncScanSingleRegionRpcRetryingCaller {
 
   private final int startLogErrorsCnt;
 
-  private final Supplier<byte[]> createNextStartRowWhenError;
-
   private final Runnable completeWhenNoMoreResultsInRegion;
 
-  private final CompletableFuture<RegionLocateType> future;
+  private final CompletableFuture<Boolean> future;
 
   private final HBaseRpcController controller;
 
@@ -128,10 +122,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {
     this.rpcTimeoutNs = rpcTimeoutNs;
     this.startLogErrorsCnt = startLogErrorsCnt;
     if (scan.isReversed()) {
-      createNextStartRowWhenError = this::createReversedNextStartRowWhenError;
       completeWhenNoMoreResultsInRegion = this::completeReversedWhenNoMoreResultsInRegion;
     } else {
-      createNextStartRowWhenError = this::createNextStartRowWhenError;
       completeWhenNoMoreResultsInRegion = this::completeWhenNoMoreResultsInRegion;
     }
     this.future = new CompletableFuture<>();
@@ -164,23 +156,13 @@ class AsyncScanSingleRegionRpcRetryingCaller {
     future.completeExceptionally(new RetriesExhaustedException(tries - 1, exceptions));
   }
 
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NONNULL_PARAM_VIOLATION",
-      justification = "https://github.com/findbugsproject/findbugs/issues/79")
   private void completeNoMoreResults() {
-    future.complete(null);
-  }
-
-  private void completeWithNextStartRow(byte[] nextStartRow) {
-    scan.setStartRow(nextStartRow);
-    future.complete(scan.isReversed() ? RegionLocateType.BEFORE : RegionLocateType.CURRENT);
+    future.complete(false);
   }
 
-  private byte[] createNextStartRowWhenError() {
-    return createClosestRowAfter(nextStartRowWhenError);
-  }
-
-  private byte[] createReversedNextStartRowWhenError() {
-    return createClosestRowBefore(nextStartRowWhenError);
+  private void completeWithNextStartRow(byte[] row, boolean inclusive) {
+    scan.withStartRow(row, inclusive);
+    future.complete(true);
   }
 
   private void completeWhenError(boolean closeScanner) {
@@ -189,12 +171,9 @@ class AsyncScanSingleRegionRpcRetryingCaller {
       closeScanner();
     }
     if (nextStartRowWhenError != null) {
-      scan.setStartRow(
-        includeNextStartRowWhenError ? nextStartRowWhenError : createNextStartRowWhenError.get());
+      scan.withStartRow(nextStartRowWhenError, includeNextStartRowWhenError);
     }
-    future.complete(
-      scan.isReversed() && Bytes.equals(scan.getStartRow(), loc.getRegionInfo().getEndKey())
-          ? RegionLocateType.BEFORE : RegionLocateType.CURRENT);
+    future.complete(true);
   }
 
   private void onError(Throwable error) {
@@ -251,29 +230,19 @@ class AsyncScanSingleRegionRpcRetryingCaller {
   }
 
   private void completeWhenNoMoreResultsInRegion() {
-    if (isEmptyStopRow(scan.getStopRow())) {
-      if (isEmptyStopRow(loc.getRegionInfo().getEndKey())) {
-        completeNoMoreResults();
-      }
+    if (noMoreResultsForScan(scan, loc.getRegionInfo())) {
+      completeNoMoreResults();
     } else {
-      if (Bytes.compareTo(loc.getRegionInfo().getEndKey(), scan.getStopRow()) >= 0) {
-        completeNoMoreResults();
-      }
+      completeWithNextStartRow(loc.getRegionInfo().getEndKey(), true);
     }
-    completeWithNextStartRow(loc.getRegionInfo().getEndKey());
   }
 
   private void completeReversedWhenNoMoreResultsInRegion() {
-    if (isEmptyStopRow(scan.getStopRow())) {
-      if (isEmptyStartRow(loc.getRegionInfo().getStartKey())) {
-        completeNoMoreResults();
-      }
+    if (noMoreResultsForReverseScan(scan, loc.getRegionInfo())) {
+      completeNoMoreResults();
     } else {
-      if (Bytes.compareTo(loc.getRegionInfo().getStartKey(), scan.getStopRow()) <= 0) {
-        completeNoMoreResults();
-      }
+      completeWithNextStartRow(loc.getRegionInfo().getStartKey(), false);
     }
-    completeWithNextStartRow(loc.getRegionInfo().getStartKey());
   }
 
   private void onComplete(ScanResponse resp) {
@@ -343,9 +312,9 @@ class AsyncScanSingleRegionRpcRetryingCaller {
   }
 
   /**
-   * @return return locate direction for next open scanner call, or null if we should stop.
+   * @return {@code true} if we should continue, otherwise {@code false}.
    */
-  public CompletableFuture<RegionLocateType> start() {
+  public CompletableFuture<Boolean> start() {
     next();
     return future;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java
index c4c2074..6ffa30a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java
@@ -17,8 +17,9 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.getLocateType;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForReverseScan;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForScan;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -37,7 +38,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
-import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Retry caller for smaller scan.
@@ -57,10 +57,6 @@ class AsyncSmallScanRpcRetryingCaller {
 
   private final long rpcTimeoutNs;
 
-  private final Function<byte[], byte[]> createClosestNextRow;
-
-  private final Runnable firstScan;
-
   private final Function<HRegionInfo, Boolean> nextScan;
 
   private final List<Result> resultList;
@@ -76,12 +72,8 @@ class AsyncSmallScanRpcRetryingCaller {
     this.scanTimeoutNs = scanTimeoutNs;
     this.rpcTimeoutNs = rpcTimeoutNs;
     if (scan.isReversed()) {
-      this.createClosestNextRow = ConnectionUtils::createClosestRowBefore;
-      this.firstScan = this::reversedFirstScan;
       this.nextScan = this::reversedNextScan;
     } else {
-      this.createClosestNextRow = ConnectionUtils::createClosestRowAfter;
-      this.firstScan = this::firstScan;
       this.nextScan = this::nextScan;
     }
     this.resultList = new ArrayList<>();
@@ -141,10 +133,9 @@ class AsyncSmallScanRpcRetryingCaller {
     }
     if (resp.hasMoreResultsInRegion) {
       if (resp.results.length > 0) {
-        scan.setStartRow(
-          createClosestNextRow.apply(resp.results[resp.results.length - 1].getRow()));
+        scan.withStartRow(resp.results[resp.results.length - 1].getRow(), false);
       }
-      scan(RegionLocateType.CURRENT);
+      scan();
       return;
     }
     if (!nextScan.apply(resp.currentRegion)) {
@@ -152,11 +143,11 @@ class AsyncSmallScanRpcRetryingCaller {
     }
   }
 
-  private void scan(RegionLocateType locateType) {
+  private void scan() {
     conn.callerFactory.<SmallScanResponse> single().table(tableName).row(scan.getStartRow())
-        .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
-        .operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).locateType(locateType)
-        .action(this::scan).call().whenComplete((resp, error) -> {
+        .locateType(getLocateType(scan)).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
+        .operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).action(this::scan).call()
+        .whenComplete((resp, error) -> {
           if (error != null) {
             future.completeExceptionally(error);
           } else {
@@ -166,45 +157,27 @@ class AsyncSmallScanRpcRetryingCaller {
   }
 
   public CompletableFuture<List<Result>> call() {
-    firstScan.run();
+    scan();
     return future;
   }
 
-  private void firstScan() {
-    scan(RegionLocateType.CURRENT);
-  }
-
-  private void reversedFirstScan() {
-    scan(isEmptyStartRow(scan.getStartRow()) ? RegionLocateType.BEFORE : RegionLocateType.CURRENT);
-  }
-
-  private boolean nextScan(HRegionInfo region) {
-    if (isEmptyStopRow(scan.getStopRow())) {
-      if (isEmptyStopRow(region.getEndKey())) {
-        return false;
-      }
+  private boolean nextScan(HRegionInfo info) {
+    if (noMoreResultsForScan(scan, info)) {
+      return false;
     } else {
-      if (Bytes.compareTo(region.getEndKey(), scan.getStopRow()) >= 0) {
-        return false;
-      }
+      scan.withStartRow(info.getEndKey());
+      scan();
+      return true;
     }
-    scan.setStartRow(region.getEndKey());
-    scan(RegionLocateType.CURRENT);
-    return true;
   }
 
-  private boolean reversedNextScan(HRegionInfo region) {
-    if (isEmptyStopRow(scan.getStopRow())) {
-      if (isEmptyStartRow(region.getStartKey())) {
-        return false;
-      }
+  private boolean reversedNextScan(HRegionInfo info) {
+    if (noMoreResultsForReverseScan(scan, info)) {
+      return false;
     } else {
-      if (Bytes.compareTo(region.getStartKey(), scan.getStopRow()) <= 0) {
-        return false;
-      }
+      scan.withStartRow(info.getStartKey(), false);
+      scan();
+      return true;
     }
-    scan.setStartRow(region.getStartKey());
-    scan(RegionLocateType.BEFORE);
-    return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
index cb8652e..e2c4ec3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java
@@ -26,7 +26,6 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayDeque;
 import java.util.Queue;
-import java.util.function.Function;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -65,14 +64,10 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
   // used to filter out cells that already returned when we restart a scan
   private Cell lastCell;
 
-  private Function<byte[], byte[]> createClosestRow;
-
   public AsyncTableResultScanner(RawAsyncTable table, Scan scan, long maxCacheSize) {
     this.rawTable = table;
     this.scan = scan;
     this.maxCacheSize = maxCacheSize;
-    this.createClosestRow = scan.isReversed() ? ConnectionUtils::createClosestRowBefore
-        : ConnectionUtils::createClosestRowAfter;
     table.scan(scan, this);
   }
 
@@ -84,16 +79,17 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
   private void stopPrefetch(Result lastResult) {
     prefetchStopped = true;
     if (lastResult.isPartial() || scan.getBatch() > 0) {
-      scan.setStartRow(lastResult.getRow());
+      scan.withStartRow(lastResult.getRow());
       lastCell = lastResult.rawCells()[lastResult.rawCells().length - 1];
     } else {
-      scan.setStartRow(createClosestRow.apply(lastResult.getRow()));
+      scan.withStartRow(lastResult.getRow(), false);
     }
     if (LOG.isDebugEnabled()) {
-      LOG.debug(System.identityHashCode(this) + " stop prefetching when scanning "
-          + rawTable.getName() + " as the cache size " + cacheSize
-          + " is greater than the maxCacheSize + " + maxCacheSize + ", the next start row is "
-          + Bytes.toStringBinary(scan.getStartRow()) + ", lastCell is " + lastCell);
+      LOG.debug(
+        String.format("0x%x", System.identityHashCode(this)) + " stop prefetching when scanning "
+            + rawTable.getName() + " as the cache size " + cacheSize
+            + " is greater than the maxCacheSize " + maxCacheSize + ", the next start row is "
+            + Bytes.toStringBinary(scan.getStartRow()) + ", lastCell is " + lastCell);
     }
     // Ignore an onComplete call as the scan is stopped by us.
     // Here we can not use a simple boolean flag. A scan operation can cross multiple regions and
@@ -166,7 +162,7 @@ class AsyncTableResultScanner implements ResultScanner, RawScanResultConsumer {
 
   private void resumePrefetch() {
     if (LOG.isDebugEnabled()) {
-      LOG.debug(System.identityHashCode(this) + " resume prefetching");
+      LOG.debug(String.format("0x%x", System.identityHashCode(this)) + " resume prefetching");
     }
     prefetchStopped = false;
     rawTable.scan(scan, this);

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index 4355182..6f4a844 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -248,7 +249,10 @@ public final class ConnectionUtils {
 
   /**
    * Create the closest row before the specified row
+   * @deprecated in fact, we do not know the closest row before the given row, the result is only a
+   *             row very close to the current row. Avoid using this method in the future.
    */
+  @Deprecated
   static byte[] createClosestRowBefore(byte[] row) {
     if (row.length == 0) {
       return MAX_BYTE_ARRAY;
@@ -347,4 +351,42 @@ public final class ConnectionUtils {
   static CompletableFuture<Void> voidBatchAll(AsyncTableBase table, List<? extends Row> actions) {
     return table.<Object> batchAll(actions).thenApply(r -> null);
   }
+
+  static RegionLocateType getLocateType(Scan scan) {
+    if (scan.isReversed()) {
+      if (isEmptyStartRow(scan.getStartRow())) {
+        return RegionLocateType.BEFORE;
+      } else {
+        return scan.includeStartRow() ? RegionLocateType.CURRENT : RegionLocateType.BEFORE;
+      }
+    } else {
+      return scan.includeStartRow() ? RegionLocateType.CURRENT : RegionLocateType.AFTER;
+    }
+  }
+
+  static boolean noMoreResultsForScan(Scan scan, HRegionInfo info) {
+    if (isEmptyStopRow(info.getEndKey())) {
+      return true;
+    }
+    if (isEmptyStopRow(scan.getStopRow())) {
+      return false;
+    }
+    int c = Bytes.compareTo(info.getEndKey(), scan.getStopRow());
+    // 1. if our stop row is less than the endKey of the region
+    // 2. if our stop row is equal to the endKey of the region and we do not include the stop row
+    // for scan.
+    return c > 0 || (c == 0 && !scan.includeStopRow());
+  }
+
+  static boolean noMoreResultsForReverseScan(Scan scan, HRegionInfo info) {
+    if (isEmptyStartRow(info.getStartKey())) {
+      return true;
+    }
+    if (isEmptyStopRow(scan.getStopRow())) {
+      return false;
+    }
+    // no need to test the inclusive of the stop row as the start key of a region is included in
+    // the region.
+    return Bytes.compareTo(info.getStartKey(), scan.getStopRow()) <= 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
index 81a8414..2c69924 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
@@ -87,7 +87,9 @@ public class Scan extends Query {
   private static final String RAW_ATTR = "_raw_";
 
   private byte [] startRow = HConstants.EMPTY_START_ROW;
+  private boolean includeStartRow = true;
   private byte [] stopRow  = HConstants.EMPTY_END_ROW;
+  private boolean includeStopRow = false;
   private int maxVersions = 1;
   private int batch = -1;
 
@@ -106,7 +108,6 @@ public class Scan extends Query {
 
   private int storeLimit = -1;
   private int storeOffset = 0;
-  private boolean getScan;
 
   /**
    * @deprecated since 1.0.0. Use {@link #setScanMetricsEnabled(boolean)}
@@ -135,8 +136,8 @@ public class Scan extends Query {
   private long maxResultSize = -1;
   private boolean cacheBlocks = true;
   private boolean reversed = false;
-  private Map<byte [], NavigableSet<byte []>> familyMap =
-    new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
+  private Map<byte[], NavigableSet<byte[]>> familyMap =
+      new TreeMap<byte[], NavigableSet<byte[]>>(Bytes.BYTES_COMPARATOR);
   private Boolean asyncPrefetch = null;
 
   /**
@@ -175,7 +176,11 @@ public class Scan extends Query {
    */
   public Scan() {}
 
-  public Scan(byte [] startRow, Filter filter) {
+  /**
+   * @deprecated use {@code new Scan().withStartRow(startRow).setFilter(filter)} instead.
+   */
+  @Deprecated
+  public Scan(byte[] startRow, Filter filter) {
     this(startRow);
     this.filter = filter;
   }
@@ -183,24 +188,26 @@ public class Scan extends Query {
   /**
    * Create a Scan operation starting at the specified row.
    * <p>
-   * If the specified row does not exist, the Scanner will start from the
-   * next closest row after the specified row.
+   * If the specified row does not exist, the Scanner will start from the next closest row after the
+   * specified row.
    * @param startRow row to start scanner at or after
+   * @deprecated use {@code new Scan().withStartRow(startRow)} instead.
    */
-  public Scan(byte [] startRow) {
-    this.startRow = startRow;
+  @Deprecated
+  public Scan(byte[] startRow) {
+    setStartRow(startRow);
   }
 
   /**
    * Create a Scan operation for the range of rows specified.
    * @param startRow row to start scanner at or after (inclusive)
    * @param stopRow row to stop scanner before (exclusive)
+   * @deprecated use {@code new Scan().withStartRow(startRow).withStopRow(stopRow)} instead.
    */
-  public Scan(byte [] startRow, byte [] stopRow) {
-    this.startRow = startRow;
-    this.stopRow = stopRow;
-    //if the startRow and stopRow both are empty, it is not a Get
-    this.getScan = isStartRowAndEqualsStopRow();
+  @Deprecated
+  public Scan(byte[] startRow, byte[] stopRow) {
+    setStartRow(startRow);
+    setStopRow(stopRow);
   }
 
   /**
@@ -211,7 +218,9 @@ public class Scan extends Query {
    */
   public Scan(Scan scan) throws IOException {
     startRow = scan.getStartRow();
+    includeStartRow = scan.includeStartRow();
     stopRow  = scan.getStopRow();
+    includeStopRow = scan.includeStopRow();
     maxVersions = scan.getMaxVersions();
     batch = scan.getBatch();
     storeLimit = scan.getMaxResultsPerColumnFamily();
@@ -219,7 +228,6 @@ public class Scan extends Query {
     caching = scan.getCaching();
     maxResultSize = scan.getMaxResultSize();
     cacheBlocks = scan.getCacheBlocks();
-    getScan = scan.isGetScan();
     filter = scan.getFilter(); // clone?
     loadColumnFamiliesOnDemand = scan.getLoadColumnFamiliesOnDemandValue();
     consistency = scan.getConsistency();
@@ -228,8 +236,7 @@ public class Scan extends Query {
     asyncPrefetch = scan.isAsyncPrefetch();
     small = scan.isSmall();
     allowPartialResults = scan.getAllowPartialResults();
-    TimeRange ctr = scan.getTimeRange();
-    tr = new TimeRange(ctr.getMin(), ctr.getMax());
+    tr = scan.getTimeRange(); // TimeRange is immutable
     Map<byte[], NavigableSet<byte[]>> fams = scan.getFamilyMap();
     for (Map.Entry<byte[],NavigableSet<byte[]>> entry : fams.entrySet()) {
       byte [] fam = entry.getKey();
@@ -258,7 +265,9 @@ public class Scan extends Query {
    */
   public Scan(Get get) {
     this.startRow = get.getRow();
+    this.includeStartRow = true;
     this.stopRow = get.getRow();
+    this.includeStopRow = true;
     this.filter = get.getFilter();
     this.cacheBlocks = get.getCacheBlocks();
     this.maxVersions = get.getMaxVersions();
@@ -266,7 +275,6 @@ public class Scan extends Query {
     this.storeOffset = get.getRowOffsetPerColumnFamily();
     this.tr = get.getTimeRange();
     this.familyMap = get.getFamilyMap();
-    this.getScan = true;
     this.asyncPrefetch = false;
     this.consistency = get.getConsistency();
     this.setIsolationLevel(get.getIsolationLevel());
@@ -282,13 +290,13 @@ public class Scan extends Query {
   }
 
   public boolean isGetScan() {
-    return this.getScan || isStartRowAndEqualsStopRow();
+    return includeStartRow && includeStopRow && areStartRowAndStopRowEqual(startRow, stopRow);
   }
 
-  private boolean isStartRowAndEqualsStopRow() {
-    return this.startRow != null && this.startRow.length > 0 &&
-        Bytes.equals(this.startRow, this.stopRow);
+  private static boolean areStartRowAndStopRowEqual(byte[] startRow, byte[] stopRow) {
+    return startRow != null && startRow.length > 0 && Bytes.equals(startRow, stopRow);
   }
+
   /**
    * Get all columns from the specified family.
    * <p>
@@ -383,45 +391,120 @@ public class Scan extends Query {
   /**
    * Set the start row of the scan.
    * <p>
-   * If the specified row does not exist, the Scanner will start from the
-   * next closest row after the specified row.
+   * If the specified row does not exist, the Scanner will start from the next closest row after the
+   * specified row.
+   * @param startRow row to start scanner at or after
+   * @return this
+   * @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length
+   *           exceeds {@link HConstants#MAX_ROW_LENGTH})
+   * @deprecated use {@link #withStartRow(byte[])} instead. This method may change the inclusive of
+   *             the stop row to keep compatible with the old behavior.
+   */
+  @Deprecated
+  public Scan setStartRow(byte[] startRow) {
+    withStartRow(startRow);
+    if (areStartRowAndStopRowEqual(startRow, stopRow)) {
+      // for keeping the old behavior that a scan with the same start and stop row is a get scan.
+      this.includeStopRow = true;
+    }
+    return this;
+  }
+
+  /**
+   * Set the start row of the scan.
+   * <p>
+   * If the specified row does not exist, the Scanner will start from the next closest row after the
+   * specified row.
+   * @param startRow row to start scanner at or after
+   * @return this
+   * @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length
+   *           exceeds {@link HConstants#MAX_ROW_LENGTH})
+   */
+  public Scan withStartRow(byte[] startRow) {
+    return withStartRow(startRow, true);
+  }
+
+  /**
+   * Set the start row of the scan.
+   * <p>
+   * If the specified row does not exist, or the {@code inclusive} is {@code false}, the Scanner
+   * will start from the next closest row after the specified row.
    * @param startRow row to start scanner at or after
+   * @param inclusive whether we should include the start row when scan
    * @return this
-   * @throws IllegalArgumentException if startRow does not meet criteria
-   * for a row key (when length exceeds {@link HConstants#MAX_ROW_LENGTH})
+   * @throws IllegalArgumentException if startRow does not meet criteria for a row key (when length
+   *           exceeds {@link HConstants#MAX_ROW_LENGTH})
    */
-  public Scan setStartRow(byte [] startRow) {
+  public Scan withStartRow(byte[] startRow, boolean inclusive) {
     if (Bytes.len(startRow) > HConstants.MAX_ROW_LENGTH) {
-      throw new IllegalArgumentException(
-        "startRow's length must be less than or equal to " +
-        HConstants.MAX_ROW_LENGTH + " to meet the criteria" +
-        " for a row key.");
+      throw new IllegalArgumentException("startRow's length must be less than or equal to "
+          + HConstants.MAX_ROW_LENGTH + " to meet the criteria" + " for a row key.");
     }
     this.startRow = startRow;
+    this.includeStartRow = inclusive;
     return this;
   }
 
   /**
    * Set the stop row of the scan.
+   * <p>
+   * The scan will include rows that are lexicographically less than the provided stopRow.
+   * <p>
+   * <b>Note:</b> When doing a filter for a rowKey <u>Prefix</u> use
+   * {@link #setRowPrefixFilter(byte[])}. The 'trailing 0' will not yield the desired result.
+   * </p>
+   * @param stopRow row to end at (exclusive)
+   * @return this
+   * @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length
+   *           exceeds {@link HConstants#MAX_ROW_LENGTH})
+   * @deprecated use {@link #withStartRow(byte[])} instead. This method may change the inclusive of
+   *             the stop row to keep compatible with the old behavior.
+   */
+  @Deprecated
+  public Scan setStopRow(byte[] stopRow) {
+    withStopRow(stopRow);
+    if (areStartRowAndStopRowEqual(startRow, stopRow)) {
+      // for keeping the old behavior that a scan with the same start and stop row is a get scan.
+      this.includeStopRow = true;
+    }
+    return this;
+  }
+
+  /**
+   * Set the stop row of the scan.
+   * <p>
+   * The scan will include rows that are lexicographically less than the provided stopRow.
+   * <p>
+   * <b>Note:</b> When doing a filter for a rowKey <u>Prefix</u> use
+   * {@link #setRowPrefixFilter(byte[])}. The 'trailing 0' will not yield the desired result.
+   * </p>
    * @param stopRow row to end at (exclusive)
+   * @return this
+   * @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length
+   *           exceeds {@link HConstants#MAX_ROW_LENGTH})
+   */
+  public Scan withStopRow(byte[] stopRow) {
+    return withStopRow(stopRow, false);
+  }
+
+  /**
+   * Set the stop row of the scan.
    * <p>
-   * The scan will include rows that are lexicographically less than
-   * the provided stopRow.
-   * <p><b>Note:</b> When doing a filter for a rowKey <u>Prefix</u>
-   * use {@link #setRowPrefixFilter(byte[])}.
-   * The 'trailing 0' will not yield the desired result.</p>
+   * The scan will include rows that are lexicographically less than (or equal to if
+   * {@code inclusive} is {@code true}) the provided stopRow.
+   * @param stopRow row to end at
+   * @param inclusive whether we should include the stop row when scan
    * @return this
-   * @throws IllegalArgumentException if stopRow does not meet criteria
-   * for a row key (when length exceeds {@link HConstants#MAX_ROW_LENGTH})
+   * @throws IllegalArgumentException if stopRow does not meet criteria for a row key (when length
+   *           exceeds {@link HConstants#MAX_ROW_LENGTH})
    */
-  public Scan setStopRow(byte [] stopRow) {
+  public Scan withStopRow(byte[] stopRow, boolean inclusive) {
     if (Bytes.len(stopRow) > HConstants.MAX_ROW_LENGTH) {
-      throw new IllegalArgumentException(
-        "stopRow's length must be less than or equal to " +
-        HConstants.MAX_ROW_LENGTH + " to meet the criteria" +
-        " for a row key.");
+      throw new IllegalArgumentException("stopRow's length must be less than or equal to "
+          + HConstants.MAX_ROW_LENGTH + " to meet the criteria" + " for a row key.");
     }
     this.stopRow = stopRow;
+    this.includeStopRow = inclusive;
     return this;
   }
 
@@ -636,13 +719,27 @@ public class Scan extends Query {
   }
 
   /**
+   * @return if we should include start row when scan
+   */
+  public boolean includeStartRow() {
+    return includeStartRow;
+  }
+
+  /**
    * @return the stoprow
    */
-  public byte [] getStopRow() {
+  public byte[] getStopRow() {
     return this.stopRow;
   }
 
   /**
+   * @return if we should include stop row when scan
+   */
+  public boolean includeStopRow() {
+    return includeStopRow;
+  }
+
+  /**
    * @return the max number of versions to fetch
    */
   public int getMaxVersions() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index d6dc7e9..d3898d4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -911,6 +911,12 @@ public final class ProtobufUtil {
     if (mvccReadPoint > 0) {
       scanBuilder.setMvccReadPoint(mvccReadPoint);
     }
+    if (!scan.includeStartRow()) {
+      scanBuilder.setIncludeStartRow(false);
+    }
+    if (scan.includeStopRow()) {
+      scanBuilder.setIncludeStopRow(true);
+    }
     return scanBuilder.build();
   }
 
@@ -923,15 +929,24 @@ public final class ProtobufUtil {
    */
   public static Scan toScan(
       final ClientProtos.Scan proto) throws IOException {
-    byte [] startRow = HConstants.EMPTY_START_ROW;
-    byte [] stopRow  = HConstants.EMPTY_END_ROW;
+    byte[] startRow = HConstants.EMPTY_START_ROW;
+    byte[] stopRow = HConstants.EMPTY_END_ROW;
+    boolean includeStartRow = true;
+    boolean includeStopRow = false;
     if (proto.hasStartRow()) {
       startRow = proto.getStartRow().toByteArray();
     }
     if (proto.hasStopRow()) {
       stopRow = proto.getStopRow().toByteArray();
     }
-    Scan scan = new Scan(startRow, stopRow);
+    if (proto.hasIncludeStartRow()) {
+      includeStartRow = proto.getIncludeStartRow();
+    }
+    if (proto.hasIncludeStopRow()) {
+      includeStopRow = proto.getIncludeStopRow();
+    }
+    Scan scan =
+        new Scan().withStartRow(startRow, includeStartRow).withStopRow(stopRow, includeStopRow);
     if (proto.hasCacheBlocks()) {
       scan.setCacheBlocks(proto.getCacheBlocks());
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 0f2cf1d..285e19a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -1024,6 +1024,12 @@ public final class ProtobufUtil {
     if (mvccReadPoint > 0) {
       scanBuilder.setMvccReadPoint(mvccReadPoint);
     }
+    if (!scan.includeStartRow()) {
+      scanBuilder.setIncludeStartRow(false);
+    }
+    if (scan.includeStopRow()) {
+      scanBuilder.setIncludeStopRow(true);
+    }
     return scanBuilder.build();
   }
 
@@ -1036,15 +1042,24 @@ public final class ProtobufUtil {
    */
   public static Scan toScan(
       final ClientProtos.Scan proto) throws IOException {
-    byte [] startRow = HConstants.EMPTY_START_ROW;
-    byte [] stopRow  = HConstants.EMPTY_END_ROW;
+    byte[] startRow = HConstants.EMPTY_START_ROW;
+    byte[] stopRow = HConstants.EMPTY_END_ROW;
+    boolean includeStartRow = true;
+    boolean includeStopRow = false;
     if (proto.hasStartRow()) {
       startRow = proto.getStartRow().toByteArray();
     }
     if (proto.hasStopRow()) {
       stopRow = proto.getStopRow().toByteArray();
     }
-    Scan scan = new Scan(startRow, stopRow);
+    if (proto.hasIncludeStartRow()) {
+      includeStartRow = proto.getIncludeStartRow();
+    }
+    if (proto.hasIncludeStopRow()) {
+      includeStopRow = proto.getIncludeStopRow();
+    }
+    Scan scan =
+        new Scan().withStartRow(startRow, includeStartRow).withStopRow(stopRow, includeStopRow);
     if (proto.hasCacheBlocks()) {
       scan.setCacheBlocks(proto.getCacheBlocks());
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
index 2efcde1..77b9495 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/TimeRange.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase.io;
 
-import java.io.IOException;
-
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
index eab62eb..ef44295 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos.java
@@ -14563,6 +14563,24 @@ public final class ClientProtos {
      * <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
      */
     long getMvccReadPoint();
+
+    /**
+     * <code>optional bool include_start_row = 21 [default = true];</code>
+     */
+    boolean hasIncludeStartRow();
+    /**
+     * <code>optional bool include_start_row = 21 [default = true];</code>
+     */
+    boolean getIncludeStartRow();
+
+    /**
+     * <code>optional bool include_stop_row = 22 [default = false];</code>
+     */
+    boolean hasIncludeStopRow();
+    /**
+     * <code>optional bool include_stop_row = 22 [default = false];</code>
+     */
+    boolean getIncludeStopRow();
   }
   /**
    * <pre>
@@ -14604,6 +14622,8 @@ public final class ClientProtos {
       allowPartialResults_ = false;
       cfTimeRange_ = java.util.Collections.emptyList();
       mvccReadPoint_ = 0L;
+      includeStartRow_ = true;
+      includeStopRow_ = false;
     }
 
     @java.lang.Override
@@ -14768,6 +14788,16 @@ public final class ClientProtos {
               mvccReadPoint_ = input.readUInt64();
               break;
             }
+            case 168: {
+              bitField0_ |= 0x00020000;
+              includeStartRow_ = input.readBool();
+              break;
+            }
+            case 176: {
+              bitField0_ |= 0x00040000;
+              includeStopRow_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@@ -15183,6 +15213,36 @@ public final class ClientProtos {
       return mvccReadPoint_;
     }
 
+    public static final int INCLUDE_START_ROW_FIELD_NUMBER = 21;
+    private boolean includeStartRow_;
+    /**
+     * <code>optional bool include_start_row = 21 [default = true];</code>
+     */
+    public boolean hasIncludeStartRow() {
+      return ((bitField0_ & 0x00020000) == 0x00020000);
+    }
+    /**
+     * <code>optional bool include_start_row = 21 [default = true];</code>
+     */
+    public boolean getIncludeStartRow() {
+      return includeStartRow_;
+    }
+
+    public static final int INCLUDE_STOP_ROW_FIELD_NUMBER = 22;
+    private boolean includeStopRow_;
+    /**
+     * <code>optional bool include_stop_row = 22 [default = false];</code>
+     */
+    public boolean hasIncludeStopRow() {
+      return ((bitField0_ & 0x00040000) == 0x00040000);
+    }
+    /**
+     * <code>optional bool include_stop_row = 22 [default = false];</code>
+     */
+    public boolean getIncludeStopRow() {
+      return includeStopRow_;
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -15279,6 +15339,12 @@ public final class ClientProtos {
       if (((bitField0_ & 0x00010000) == 0x00010000)) {
         output.writeUInt64(20, mvccReadPoint_);
       }
+      if (((bitField0_ & 0x00020000) == 0x00020000)) {
+        output.writeBool(21, includeStartRow_);
+      }
+      if (((bitField0_ & 0x00040000) == 0x00040000)) {
+        output.writeBool(22, includeStopRow_);
+      }
       unknownFields.writeTo(output);
     }
 
@@ -15367,6 +15433,14 @@ public final class ClientProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(20, mvccReadPoint_);
       }
+      if (((bitField0_ & 0x00020000) == 0x00020000)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeBoolSize(21, includeStartRow_);
+      }
+      if (((bitField0_ & 0x00040000) == 0x00040000)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeBoolSize(22, includeStopRow_);
+      }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
       return size;
@@ -15474,6 +15548,16 @@ public final class ClientProtos {
         result = result && (getMvccReadPoint()
             == other.getMvccReadPoint());
       }
+      result = result && (hasIncludeStartRow() == other.hasIncludeStartRow());
+      if (hasIncludeStartRow()) {
+        result = result && (getIncludeStartRow()
+            == other.getIncludeStartRow());
+      }
+      result = result && (hasIncludeStopRow() == other.hasIncludeStopRow());
+      if (hasIncludeStopRow()) {
+        result = result && (getIncludeStopRow()
+            == other.getIncludeStopRow());
+      }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -15572,6 +15656,16 @@ public final class ClientProtos {
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
             getMvccReadPoint());
       }
+      if (hasIncludeStartRow()) {
+        hash = (37 * hash) + INCLUDE_START_ROW_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+            getIncludeStartRow());
+      }
+      if (hasIncludeStopRow()) {
+        hash = (37 * hash) + INCLUDE_STOP_ROW_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+            getIncludeStopRow());
+      }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -15765,6 +15859,10 @@ public final class ClientProtos {
         }
         mvccReadPoint_ = 0L;
         bitField0_ = (bitField0_ & ~0x00080000);
+        includeStartRow_ = true;
+        bitField0_ = (bitField0_ & ~0x00100000);
+        includeStopRow_ = false;
+        bitField0_ = (bitField0_ & ~0x00200000);
         return this;
       }
 
@@ -15892,6 +15990,14 @@ public final class ClientProtos {
           to_bitField0_ |= 0x00010000;
         }
         result.mvccReadPoint_ = mvccReadPoint_;
+        if (((from_bitField0_ & 0x00100000) == 0x00100000)) {
+          to_bitField0_ |= 0x00020000;
+        }
+        result.includeStartRow_ = includeStartRow_;
+        if (((from_bitField0_ & 0x00200000) == 0x00200000)) {
+          to_bitField0_ |= 0x00040000;
+        }
+        result.includeStopRow_ = includeStopRow_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -16063,6 +16169,12 @@ public final class ClientProtos {
         if (other.hasMvccReadPoint()) {
           setMvccReadPoint(other.getMvccReadPoint());
         }
+        if (other.hasIncludeStartRow()) {
+          setIncludeStartRow(other.getIncludeStartRow());
+        }
+        if (other.hasIncludeStopRow()) {
+          setIncludeStopRow(other.getIncludeStopRow());
+        }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
         return this;
@@ -17572,6 +17684,70 @@ public final class ClientProtos {
         onChanged();
         return this;
       }
+
+      private boolean includeStartRow_ = true;
+      /**
+       * <code>optional bool include_start_row = 21 [default = true];</code>
+       */
+      public boolean hasIncludeStartRow() {
+        return ((bitField0_ & 0x00100000) == 0x00100000);
+      }
+      /**
+       * <code>optional bool include_start_row = 21 [default = true];</code>
+       */
+      public boolean getIncludeStartRow() {
+        return includeStartRow_;
+      }
+      /**
+       * <code>optional bool include_start_row = 21 [default = true];</code>
+       */
+      public Builder setIncludeStartRow(boolean value) {
+        bitField0_ |= 0x00100000;
+        includeStartRow_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool include_start_row = 21 [default = true];</code>
+       */
+      public Builder clearIncludeStartRow() {
+        bitField0_ = (bitField0_ & ~0x00100000);
+        includeStartRow_ = true;
+        onChanged();
+        return this;
+      }
+
+      private boolean includeStopRow_ ;
+      /**
+       * <code>optional bool include_stop_row = 22 [default = false];</code>
+       */
+      public boolean hasIncludeStopRow() {
+        return ((bitField0_ & 0x00200000) == 0x00200000);
+      }
+      /**
+       * <code>optional bool include_stop_row = 22 [default = false];</code>
+       */
+      public boolean getIncludeStopRow() {
+        return includeStopRow_;
+      }
+      /**
+       * <code>optional bool include_stop_row = 22 [default = false];</code>
+       */
+      public Builder setIncludeStopRow(boolean value) {
+        bitField0_ |= 0x00200000;
+        includeStopRow_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool include_stop_row = 22 [default = false];</code>
+       */
+      public Builder clearIncludeStopRow() {
+        bitField0_ = (bitField0_ & ~0x00200000);
+        includeStopRow_ = false;
+        onChanged();
+        return this;
+      }
       public final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
@@ -40658,7 +40834,7 @@ public final class ClientProtos {
       "tion\030\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce" +
       "_group\030\004 \001(\004\"E\n\016MutateResponse\022 \n\006result" +
       "\030\001 \001(\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 " +
-      "\001(\010\"\331\004\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
+      "\001(\010\"\233\005\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb." +
       "Column\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.Nam" +
       "eBytesPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_ro" +
       "w\030\004 \001(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filte" +
@@ -40673,97 +40849,98 @@ public final class ClientProtos {
       "aching\030\021 \001(\r\022\035\n\025allow_partial_results\030\022 " +
       "\001(\010\0226\n\rcf_time_range\030\023 \003(\0132\037.hbase.pb.Co" +
       "lumnFamilyTimeRange\022\032\n\017mvcc_read_point\030\024",
-      " \001(\004:\0010\"\246\002\n\013ScanRequest\022)\n\006region\030\001 \001(\0132" +
-      "\031.hbase.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(\013" +
-      "2\016.hbase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016" +
-      "number_of_rows\030\004 \001(\r\022\025\n\rclose_scanner\030\005 " +
-      "\001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027client_han" +
-      "dles_partials\030\007 \001(\010\022!\n\031client_handles_he" +
-      "artbeats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t \001" +
-      "(\010\022\024\n\005renew\030\n \001(\010:\005false\"\266\002\n\014ScanRespons" +
-      "e\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nscanner_i" +
-      "d\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001",
-      "(\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Result\022\r\n" +
-      "\005stale\030\006 \001(\010\022\037\n\027partial_flag_per_result\030" +
-      "\007 \003(\010\022\036\n\026more_results_in_region\030\010 \001(\010\022\031\n" +
-      "\021heartbeat_message\030\t \001(\010\022+\n\014scan_metrics" +
-      "\030\n \001(\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_re" +
-      "ad_point\030\013 \001(\004:\0010\"\240\002\n\024BulkLoadHFileReque" +
-      "st\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpec" +
-      "ifier\022>\n\013family_path\030\002 \003(\0132).hbase.pb.Bu" +
-      "lkLoadHFileRequest.FamilyPath\022\026\n\016assign_" +
-      "seq_num\030\003 \001(\010\022+\n\010fs_token\030\004 \001(\0132\031.hbase.",
-      "pb.DelegationToken\022\022\n\nbulk_token\030\005 \001(\t\022\030" +
-      "\n\tcopy_file\030\006 \001(\010:\005false\032*\n\nFamilyPath\022\016" +
-      "\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoad" +
-      "HFileResponse\022\016\n\006loaded\030\001 \002(\010\"V\n\017Delegat" +
-      "ionToken\022\022\n\nidentifier\030\001 \001(\014\022\020\n\010password" +
-      "\030\002 \001(\014\022\014\n\004kind\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n" +
-      "\026PrepareBulkLoadRequest\022\'\n\ntable_name\030\001 " +
-      "\002(\0132\023.hbase.pb.TableName\022)\n\006region\030\002 \001(\013" +
-      "2\031.hbase.pb.RegionSpecifier\"-\n\027PrepareBu" +
-      "lkLoadResponse\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026Cl",
-      "eanupBulkLoadRequest\022\022\n\nbulk_token\030\001 \002(\t" +
-      "\022)\n\006region\030\002 \001(\0132\031.hbase.pb.RegionSpecif" +
-      "ier\"\031\n\027CleanupBulkLoadResponse\"a\n\026Coproc" +
-      "essorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service" +
-      "_name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007requ" +
-      "est\030\004 \002(\014\"B\n\030CoprocessorServiceResult\022&\n" +
-      "\005value\030\001 \001(\0132\027.hbase.pb.NameBytesPair\"v\n" +
-      "\031CoprocessorServiceRequest\022)\n\006region\030\001 \002" +
-      "(\0132\031.hbase.pb.RegionSpecifier\022.\n\004call\030\002 " +
-      "\002(\0132 .hbase.pb.CoprocessorServiceCall\"o\n",
-      "\032CoprocessorServiceResponse\022)\n\006region\030\001 " +
-      "\002(\0132\031.hbase.pb.RegionSpecifier\022&\n\005value\030" +
-      "\002 \002(\0132\027.hbase.pb.NameBytesPair\"\226\001\n\006Actio" +
-      "n\022\r\n\005index\030\001 \001(\r\022)\n\010mutation\030\002 \001(\0132\027.hba" +
-      "se.pb.MutationProto\022\032\n\003get\030\003 \001(\0132\r.hbase" +
-      ".pb.Get\0226\n\014service_call\030\004 \001(\0132 .hbase.pb" +
-      ".CoprocessorServiceCall\"k\n\014RegionAction\022" +
-      ")\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifi" +
-      "er\022\016\n\006atomic\030\002 \001(\010\022 \n\006action\030\003 \003(\0132\020.hba" +
-      "se.pb.Action\"c\n\017RegionLoadStats\022\027\n\014memst",
-      "oreLoad\030\001 \001(\005:\0010\022\030\n\rheapOccupancy\030\002 \001(\005:" +
-      "\0010\022\035\n\022compactionPressure\030\003 \001(\005:\0010\"j\n\024Mul" +
-      "tiRegionLoadStats\022)\n\006region\030\001 \003(\0132\031.hbas" +
-      "e.pb.RegionSpecifier\022\'\n\004stat\030\002 \003(\0132\031.hba" +
-      "se.pb.RegionLoadStats\"\336\001\n\021ResultOrExcept" +
-      "ion\022\r\n\005index\030\001 \001(\r\022 \n\006result\030\002 \001(\0132\020.hba" +
-      "se.pb.Result\022*\n\texception\030\003 \001(\0132\027.hbase." +
-      "pb.NameBytesPair\022:\n\016service_result\030\004 \001(\013" +
-      "2\".hbase.pb.CoprocessorServiceResult\0220\n\t" +
-      "loadStats\030\005 \001(\0132\031.hbase.pb.RegionLoadSta",
-      "tsB\002\030\001\"x\n\022RegionActionResult\0226\n\021resultOr" +
-      "Exception\030\001 \003(\0132\033.hbase.pb.ResultOrExcep" +
-      "tion\022*\n\texception\030\002 \001(\0132\027.hbase.pb.NameB" +
-      "ytesPair\"x\n\014MultiRequest\022,\n\014regionAction" +
-      "\030\001 \003(\0132\026.hbase.pb.RegionAction\022\022\n\nnonceG" +
-      "roup\030\002 \001(\004\022&\n\tcondition\030\003 \001(\0132\023.hbase.pb" +
-      ".Condition\"\226\001\n\rMultiResponse\0228\n\022regionAc" +
-      "tionResult\030\001 \003(\0132\034.hbase.pb.RegionAction" +
-      "Result\022\021\n\tprocessed\030\002 \001(\010\0228\n\020regionStati" +
-      "stics\030\003 \001(\0132\036.hbase.pb.MultiRegionLoadSt",
-      "ats*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n\010TIMELI" +
-      "NE\020\0012\263\005\n\rClientService\0222\n\003Get\022\024.hbase.pb" +
-      ".GetRequest\032\025.hbase.pb.GetResponse\022;\n\006Mu" +
-      "tate\022\027.hbase.pb.MutateRequest\032\030.hbase.pb" +
-      ".MutateResponse\0225\n\004Scan\022\025.hbase.pb.ScanR" +
-      "equest\032\026.hbase.pb.ScanResponse\022P\n\rBulkLo" +
-      "adHFile\022\036.hbase.pb.BulkLoadHFileRequest\032" +
-      "\037.hbase.pb.BulkLoadHFileResponse\022V\n\017Prep" +
-      "areBulkLoad\022 .hbase.pb.PrepareBulkLoadRe" +
-      "quest\032!.hbase.pb.PrepareBulkLoadResponse",
-      "\022V\n\017CleanupBulkLoad\022 .hbase.pb.CleanupBu" +
-      "lkLoadRequest\032!.hbase.pb.CleanupBulkLoad" +
-      "Response\022X\n\013ExecService\022#.hbase.pb.Copro" +
-      "cessorServiceRequest\032$.hbase.pb.Coproces" +
-      "sorServiceResponse\022d\n\027ExecRegionServerSe" +
-      "rvice\022#.hbase.pb.CoprocessorServiceReque" +
-      "st\032$.hbase.pb.CoprocessorServiceResponse" +
-      "\0228\n\005Multi\022\026.hbase.pb.MultiRequest\032\027.hbas" +
-      "e.pb.MultiResponseBI\n1org.apache.hadoop." +
-      "hbase.shaded.protobuf.generatedB\014ClientP",
-      "rotosH\001\210\001\001\240\001\001"
+      " \001(\004:\0010\022\037\n\021include_start_row\030\025 \001(\010:\004true" +
+      "\022\037\n\020include_stop_row\030\026 \001(\010:\005false\"\246\002\n\013Sc" +
+      "anRequest\022)\n\006region\030\001 \001(\0132\031.hbase.pb.Reg" +
+      "ionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.hbase.pb.Sc" +
+      "an\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016number_of_rows" +
+      "\030\004 \001(\r\022\025\n\rclose_scanner\030\005 \001(\010\022\025\n\rnext_ca" +
+      "ll_seq\030\006 \001(\004\022\037\n\027client_handles_partials\030" +
+      "\007 \001(\010\022!\n\031client_handles_heartbeats\030\010 \001(\010" +
+      "\022\032\n\022track_scan_metrics\030\t \001(\010\022\024\n\005renew\030\n " +
+      "\001(\010:\005false\"\266\002\n\014ScanResponse\022\030\n\020cells_per",
+      "_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014mor" +
+      "e_results\030\003 \001(\010\022\013\n\003ttl\030\004 \001(\r\022!\n\007results\030" +
+      "\005 \003(\0132\020.hbase.pb.Result\022\r\n\005stale\030\006 \001(\010\022\037" +
+      "\n\027partial_flag_per_result\030\007 \003(\010\022\036\n\026more_" +
+      "results_in_region\030\010 \001(\010\022\031\n\021heartbeat_mes" +
+      "sage\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.hbase" +
+      ".pb.ScanMetrics\022\032\n\017mvcc_read_point\030\013 \001(\004" +
+      ":\0010\"\240\002\n\024BulkLoadHFileRequest\022)\n\006region\030\001" +
+      " \002(\0132\031.hbase.pb.RegionSpecifier\022>\n\013famil" +
+      "y_path\030\002 \003(\0132).hbase.pb.BulkLoadHFileReq",
+      "uest.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010\022" +
+      "+\n\010fs_token\030\004 \001(\0132\031.hbase.pb.DelegationT" +
+      "oken\022\022\n\nbulk_token\030\005 \001(\t\022\030\n\tcopy_file\030\006 " +
+      "\001(\010:\005false\032*\n\nFamilyPath\022\016\n\006family\030\001 \002(\014" +
+      "\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileResponse\022" +
+      "\016\n\006loaded\030\001 \002(\010\"V\n\017DelegationToken\022\022\n\nid" +
+      "entifier\030\001 \001(\014\022\020\n\010password\030\002 \001(\014\022\014\n\004kind" +
+      "\030\003 \001(\t\022\017\n\007service\030\004 \001(\t\"l\n\026PrepareBulkLo" +
+      "adRequest\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb" +
+      ".TableName\022)\n\006region\030\002 \001(\0132\031.hbase.pb.Re",
+      "gionSpecifier\"-\n\027PrepareBulkLoadResponse" +
+      "\022\022\n\nbulk_token\030\001 \002(\t\"W\n\026CleanupBulkLoadR" +
+      "equest\022\022\n\nbulk_token\030\001 \002(\t\022)\n\006region\030\002 \001" +
+      "(\0132\031.hbase.pb.RegionSpecifier\"\031\n\027Cleanup" +
+      "BulkLoadResponse\"a\n\026CoprocessorServiceCa" +
+      "ll\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n" +
+      "\013method_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030C" +
+      "oprocessorServiceResult\022&\n\005value\030\001 \001(\0132\027" +
+      ".hbase.pb.NameBytesPair\"v\n\031CoprocessorSe" +
+      "rviceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.",
+      "RegionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb" +
+      ".CoprocessorServiceCall\"o\n\032CoprocessorSe" +
+      "rviceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb" +
+      ".RegionSpecifier\022&\n\005value\030\002 \002(\0132\027.hbase." +
+      "pb.NameBytesPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001" +
+      "(\r\022)\n\010mutation\030\002 \001(\0132\027.hbase.pb.Mutation" +
+      "Proto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014ser" +
+      "vice_call\030\004 \001(\0132 .hbase.pb.CoprocessorSe" +
+      "rviceCall\"k\n\014RegionAction\022)\n\006region\030\001 \002(" +
+      "\0132\031.hbase.pb.RegionSpecifier\022\016\n\006atomic\030\002",
+      " \001(\010\022 \n\006action\030\003 \003(\0132\020.hbase.pb.Action\"c" +
+      "\n\017RegionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:" +
+      "\0010\022\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compacti" +
+      "onPressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadSt" +
+      "ats\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpe" +
+      "cifier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLo" +
+      "adStats\"\336\001\n\021ResultOrException\022\r\n\005index\030\001" +
+      " \001(\r\022 \n\006result\030\002 \001(\0132\020.hbase.pb.Result\022*" +
+      "\n\texception\030\003 \001(\0132\027.hbase.pb.NameBytesPa" +
+      "ir\022:\n\016service_result\030\004 \001(\0132\".hbase.pb.Co",
+      "processorServiceResult\0220\n\tloadStats\030\005 \001(" +
+      "\0132\031.hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022Regi" +
+      "onActionResult\0226\n\021resultOrException\030\001 \003(" +
+      "\0132\033.hbase.pb.ResultOrException\022*\n\texcept" +
+      "ion\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Mu" +
+      "ltiRequest\022,\n\014regionAction\030\001 \003(\0132\026.hbase" +
+      ".pb.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\t" +
+      "condition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n" +
+      "\rMultiResponse\0228\n\022regionActionResult\030\001 \003" +
+      "(\0132\034.hbase.pb.RegionActionResult\022\021\n\tproc",
+      "essed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036." +
+      "hbase.pb.MultiRegionLoadStats*\'\n\013Consist" +
+      "ency\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\263\005\n\rClien" +
+      "tService\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025." +
+      "hbase.pb.GetResponse\022;\n\006Mutate\022\027.hbase.p" +
+      "b.MutateRequest\032\030.hbase.pb.MutateRespons" +
+      "e\0225\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase" +
+      ".pb.ScanResponse\022P\n\rBulkLoadHFile\022\036.hbas" +
+      "e.pb.BulkLoadHFileRequest\032\037.hbase.pb.Bul" +
+      "kLoadHFileResponse\022V\n\017PrepareBulkLoad\022 .",
+      "hbase.pb.PrepareBulkLoadRequest\032!.hbase." +
+      "pb.PrepareBulkLoadResponse\022V\n\017CleanupBul" +
+      "kLoad\022 .hbase.pb.CleanupBulkLoadRequest\032" +
+      "!.hbase.pb.CleanupBulkLoadResponse\022X\n\013Ex" +
+      "ecService\022#.hbase.pb.CoprocessorServiceR" +
+      "equest\032$.hbase.pb.CoprocessorServiceResp" +
+      "onse\022d\n\027ExecRegionServerService\022#.hbase." +
+      "pb.CoprocessorServiceRequest\032$.hbase.pb." +
+      "CoprocessorServiceResponse\0228\n\005Multi\022\026.hb" +
+      "ase.pb.MultiRequest\032\027.hbase.pb.MultiResp",
+      "onseBI\n1org.apache.hadoop.hbase.shaded.p" +
+      "rotobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -40865,7 +41042,7 @@ public final class ClientProtos {
     internal_static_hbase_pb_Scan_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_Scan_descriptor,
-        new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", });
+        new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", "IncludeStartRow", "IncludeStopRow", });
     internal_static_hbase_pb_ScanRequest_descriptor =
       getDescriptor().getMessageTypes().get(12);
     internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/05b1d918/hbase-protocol-shaded/src/main/protobuf/Client.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Client.proto b/hbase-protocol-shaded/src/main/protobuf/Client.proto
index 9a7fea2..2793b89 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Client.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Client.proto
@@ -256,6 +256,8 @@ message Scan {
   optional bool allow_partial_results = 18;
   repeated ColumnFamilyTimeRange cf_time_range = 19;
   optional uint64 mvcc_read_point = 20 [default = 0];
+  optional bool include_start_row = 21 [default = true];
+  optional bool include_stop_row = 22 [default = false];
 }
 
 /**