You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ya...@apache.org on 2017/06/07 03:38:40 UTC

[5/5] hbase git commit: HBASE-15576 Scanning cursor to prevent blocking long time on ResultScanner.next()

HBASE-15576 Scanning cursor to prevent blocking long time on ResultScanner.next()


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

Branch: refs/heads/branch-1
Commit: 381c89b5cc463b58010ab443de9e1c1d40d8bf98
Parents: 9cb57ae
Author: Phil Yang <ya...@apache.org>
Authored: Tue Jun 6 15:39:24 2017 +0800
Committer: Phil Yang <ya...@apache.org>
Committed: Wed Jun 7 11:37:22 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/ClientScanner.java      |   15 +
 .../apache/hadoop/hbase/client/ClientUtil.java  |    4 +
 .../org/apache/hadoop/hbase/client/Cursor.java  |   43 +
 .../org/apache/hadoop/hbase/client/Result.java  |   45 +
 .../org/apache/hadoop/hbase/client/Scan.java    |   42 +
 .../hadoop/hbase/client/ScannerCallable.java    |   14 +-
 .../client/ScannerCallableWithReplicas.java     |    4 +
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   23 +
 .../hbase/protobuf/generated/ClientProtos.java  | 1086 +++++++++++++++---
 hbase-protocol/src/main/protobuf/Client.proto   |   14 +-
 .../hbase/regionserver/RSRpcServices.java       |   18 +-
 .../hbase/regionserver/ScannerContext.java      |   13 +
 .../hadoop/hbase/regionserver/StoreScanner.java |    1 +
 .../hbase/regionserver/TestScannerCursor.java   |  191 +++
 14 files changed, 1375 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index 8e94c7c..d548901 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -505,6 +505,21 @@ public abstract class ClientScanner extends AbstractClientScanner {
           break;
         }
       }
+      if (cache.isEmpty() && !closed && scan.isNeedCursorResult()) {
+        if (callable.isHeartbeatMessage() && callable.getCursor() != null) {
+          // Use cursor row key from server
+          cache.add(Result.createCursorResult(callable.getCursor()));
+          break;
+        }
+        if (values.length > 0) {
+          // It is size limit exceed and we need return the last Result's row.
+          // When user setBatch and the scanner is reopened, the server may return Results that
+          // user has seen and the last Result can not be seen because the number is not enough.
+          // So the row keys of results may not be same, we must use the last one.
+          cache.add(Result.createCursorResult(new Cursor(values[values.length - 1].getRow())));
+          break;
+        }
+      }
       if (countdown <= 0) {
         // we have enough result.
         closeScannerIfExhausted(regionExhausted);

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientUtil.java
index e4a84d5..a839080 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientUtil.java
@@ -27,4 +27,8 @@ public class ClientUtil {
   public static boolean areScanStartRowAndStopRowEqual(byte[] startRow, byte[] stopRow) {
     return startRow != null && startRow.length > 0 && Bytes.equals(startRow, stopRow);
   }
+
+  public static Cursor createCursor(byte[] row) {
+    return new Cursor(row);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cursor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cursor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cursor.java
new file mode 100644
index 0000000..374025e
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Cursor.java
@@ -0,0 +1,43 @@
+/**
+ * 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.client;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Scan cursor to tell client where server is scanning
+ * {@link Scan#setNeedCursorResult(boolean)}
+ * {@link Result#isCursor()}
+ * {@link Result#getCursor()}
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class Cursor {
+
+  private final byte[] row;
+
+  Cursor(byte[] row) {
+    this.row = row;
+  }
+
+  public byte[] getRow() {
+    return row;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
index 4c67c50..16b3624 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -107,6 +107,8 @@ public class Result implements CellScannable, CellScanner {
 
   private final boolean readonly;
 
+  private Cursor cursor = null;
+
   /**
    * Creates an empty Result w/ no KeyValue payload; returns null if you call {@link #rawCells()}.
    * Use this to represent no results if <code>null</code> won't do or in old 'mapred' as oppposed to 'mapreduce' package
@@ -188,6 +190,15 @@ public class Result implements CellScannable, CellScanner {
     return new Result(cells, null, stale, mayHaveMoreCellsInRow);
   }
 
+  public static Result createCursorResult(Cursor cursor) {
+    return new Result(cursor);
+  }
+
+  private Result(Cursor cursor) {
+    this.cursor = cursor;
+    this.readonly = false;
+  }
+
   /** Private ctor. Use {@link #create(Cell[])}. */
   private Result(Cell[] cells, Boolean exists, boolean stale, boolean mayHaveMoreCellsInRow) {
     this.cells = cells;
@@ -1030,4 +1041,38 @@ public class Result implements CellScannable, CellScanner {
       throw new UnsupportedOperationException("Attempting to modify readonly EMPTY_RESULT!");
     }
   }
+
+  /**
+   * Return true if this Result is a cursor to tell users where the server has scanned.
+   * In this Result the only meaningful method is {@link #getCursor()}.
+   *
+   * {@code
+   *  while (r = scanner.next() && r != null) {
+   *    if(r.isCursor()){
+   *    // scanning is not end, it is a cursor, save its row key and close scanner if you want, or
+   *    // just continue the loop to call next().
+   *    } else {
+   *    // just like before
+   *    }
+   *  }
+   *  // scanning is end
+   *
+   * }
+   * {@link Scan#setNeedCursorResult(boolean)}
+   * {@link Cursor}
+   * {@link #getCursor()}
+   */
+  public boolean isCursor() {
+    return cursor != null ;
+  }
+
+  /**
+   * Return the cursor if this Result is a cursor result.
+   * {@link Scan#setNeedCursorResult(boolean)}
+   * {@link Cursor}
+   * {@link #isCursor()}
+   */
+  public Cursor getCursor(){
+    return cursor;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/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 64a5787..9b8724c 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
@@ -187,6 +187,8 @@ public class Scan extends Query {
    */
   private ReadType readType = ReadType.DEFAULT;
 
+  private boolean needCursorResult = false;
+
   /**
    * Create a Scan operation across all rows.
    */
@@ -275,6 +277,7 @@ public class Scan extends Query {
     }
     this.mvccReadPoint = scan.getMvccReadPoint();
     this.limit = scan.getLimit();
+    this.needCursorResult = scan.isNeedCursorResult();
   }
 
   /**
@@ -1209,4 +1212,43 @@ public class Scan extends Query {
   Scan resetMvccReadPoint() {
     return setMvccReadPoint(-1L);
   }
+
+  /**
+   * When the server is slow or we scan a table with many deleted data or we use a sparse filter,
+   * the server will response heartbeat to prevent timeout. However the scanner will return a Result
+   * only when client can do it. So if there are many heartbeats, the blocking time on
+   * ResultScanner#next() may be very long, which is not friendly to online services.
+   *
+   * Set this to true then you can get a special Result whose #isCursor() returns true and is not
+   * contains any real data. It only tells you where the server has scanned. You can call next
+   * to continue scanning or open a new scanner with this row key as start row whenever you want.
+   *
+   * Users can get a cursor when and only when there is a response from the server but we can not
+   * return a Result to users, for example, this response is a heartbeat or there are partial cells
+   * but users do not allow partial result.
+   *
+   * Now the cursor is in row level which means the special Result will only contains a row key.
+   * {@link Result#isCursor()}
+   * {@link Result#getCursor()}
+   * {@link Cursor}
+   */
+  public Scan setNeedCursorResult(boolean needCursorResult) {
+    this.needCursorResult = needCursorResult;
+    return this;
+  }
+
+  public boolean isNeedCursorResult() {
+    return needCursorResult;
+  }
+
+  /**
+   * Create a new Scan with a cursor. It only set the position information like start row key.
+   * The others (like cfs, stop row, limit) should still be filled in by the user.
+   * {@link Result#isCursor()}
+   * {@link Result#getCursor()}
+   * {@link Cursor}
+   */
+  public static Scan createScanFromCursor(Cursor cursor) {
+    return new Scan().withStartRow(cursor.getRow());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index ebac361..caa9dec 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -100,6 +100,8 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
     }
   }
 
+  protected Cursor cursor;
+
   // indicate if it is a remote server call
   protected boolean isRegionServerRemote = true;
   private long nextCallSeq = 0;
@@ -168,7 +170,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
       checkIfRegionServerIsRemote();
       instantiated = true;
     }
-
+    cursor = null;
     // check how often we retry.
     // HConnectionManager will call instantiateServer with reload==true
     // if and only if for retries.
@@ -274,7 +276,11 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
       response = next();
     }
     long timestamp = System.currentTimeMillis();
-    setHeartbeatMessage(response.hasHeartbeatMessage() && response.getHeartbeatMessage());
+    boolean isHeartBeat = response.hasHeartbeatMessage() && response.getHeartbeatMessage();
+    setHeartbeatMessage(isHeartBeat);
+    if (isHeartBeat && scan.isNeedCursorResult() && response.hasCursor()) {
+      cursor = ProtobufUtil.toCursor(response.getCursor());
+    }
     Result[] rrs = ResponseConverter.getResults(controller.cellScanner(), response);
     if (logScannerActivity) {
       long now = System.currentTimeMillis();
@@ -320,6 +326,10 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
     return heartbeatMessage;
   }
 
+  public Cursor getCursor() {
+    return cursor;
+  }
+
   private void setHeartbeatMessage(boolean heartbeatMessage) {
     this.heartbeatMessage = heartbeatMessage;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
index 1beb0a9..59d0562 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallableWithReplicas.java
@@ -303,6 +303,10 @@ class ScannerCallableWithReplicas implements RetryingCallable<Result[]> {
     return currentScannerCallable != null && currentScannerCallable.isHeartbeatMessage();
   }
 
+  public Cursor getCursor() {
+    return currentScannerCallable != null ? currentScannerCallable.getCursor() : null;
+  }
+
   private void addCallsForCurrentReplica(
       ResultBoundedCompletionService<Pair<Result[], ScannerCallable>> cs, RegionLocations rl) {
     RetryingRPC retryingOnReplica = new RetryingRPC(currentScannerCallable);

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/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 b561cfd..b704f4a 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
@@ -75,6 +75,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.ClientUtil;
 import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Cursor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
@@ -1058,6 +1059,9 @@ public final class ProtobufUtil {
     if (scan.getReadType() != Scan.ReadType.DEFAULT) {
       scanBuilder.setReadType(toReadType(scan.getReadType()));
     }
+    if (scan.isNeedCursorResult()) {
+      scanBuilder.setNeedCursorResult(true);
+    }
     return scanBuilder.build();
   }
 
@@ -1167,9 +1171,28 @@ public final class ProtobufUtil {
     } else if (proto.hasReadType()) {
       scan.setReadType(toReadType(proto.getReadType()));
     }
+    if (proto.getNeedCursorResult()) {
+      scan.setNeedCursorResult(true);
+    }
     return scan;
   }
 
+  public static ClientProtos.Cursor toCursor(Cursor cursor) {
+    ClientProtos.Cursor.Builder builder = ClientProtos.Cursor.newBuilder();
+    ClientProtos.Cursor.newBuilder().setRow(ByteString.copyFrom(cursor.getRow()));
+    return builder.build();
+  }
+
+  public static ClientProtos.Cursor toCursor(Cell cell) {
+    return ClientProtos.Cursor.newBuilder()
+        .setRow(ByteString.copyFrom(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()))
+        .build();
+  }
+
+  public static Cursor toCursor(ClientProtos.Cursor cursor) {
+    return ClientUtil.createCursor(cursor.getRow().toByteArray());
+  }
+
   /**
    * Create a protocol buffer Get based on a client Get.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
index 9133c19..12950f7 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
@@ -14277,6 +14277,16 @@ public final class ClientProtos {
      * <code>optional .hbase.pb.Scan.ReadType readType = 23 [default = DEFAULT];</code>
      */
     org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType getReadType();
+
+    // optional bool need_cursor_result = 24 [default = false];
+    /**
+     * <code>optional bool need_cursor_result = 24 [default = false];</code>
+     */
+    boolean hasNeedCursorResult();
+    /**
+     * <code>optional bool need_cursor_result = 24 [default = false];</code>
+     */
+    boolean getNeedCursorResult();
   }
   /**
    * Protobuf type {@code hbase.pb.Scan}
@@ -14492,6 +14502,11 @@ public final class ClientProtos {
               }
               break;
             }
+            case 192: {
+              bitField0_ |= 0x00100000;
+              needCursorResult_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -15080,6 +15095,22 @@ public final class ClientProtos {
       return readType_;
     }
 
+    // optional bool need_cursor_result = 24 [default = false];
+    public static final int NEED_CURSOR_RESULT_FIELD_NUMBER = 24;
+    private boolean needCursorResult_;
+    /**
+     * <code>optional bool need_cursor_result = 24 [default = false];</code>
+     */
+    public boolean hasNeedCursorResult() {
+      return ((bitField0_ & 0x00100000) == 0x00100000);
+    }
+    /**
+     * <code>optional bool need_cursor_result = 24 [default = false];</code>
+     */
+    public boolean getNeedCursorResult() {
+      return needCursorResult_;
+    }
+
     private void initFields() {
       column_ = java.util.Collections.emptyList();
       attribute_ = java.util.Collections.emptyList();
@@ -15104,6 +15135,7 @@ public final class ClientProtos {
       includeStartRow_ = true;
       includeStopRow_ = false;
       readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT;
+      needCursorResult_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -15210,6 +15242,9 @@ public final class ClientProtos {
       if (((bitField0_ & 0x00080000) == 0x00080000)) {
         output.writeEnum(23, readType_.getNumber());
       }
+      if (((bitField0_ & 0x00100000) == 0x00100000)) {
+        output.writeBool(24, needCursorResult_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -15311,6 +15346,10 @@ public final class ClientProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeEnumSize(23, readType_.getNumber());
       }
+      if (((bitField0_ & 0x00100000) == 0x00100000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(24, needCursorResult_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -15440,6 +15479,11 @@ public final class ClientProtos {
         result = result &&
             (getReadType() == other.getReadType());
       }
+      result = result && (hasNeedCursorResult() == other.hasNeedCursorResult());
+      if (hasNeedCursorResult()) {
+        result = result && (getNeedCursorResult()
+            == other.getNeedCursorResult());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -15545,6 +15589,10 @@ public final class ClientProtos {
         hash = (37 * hash) + READTYPE_FIELD_NUMBER;
         hash = (53 * hash) + hashEnum(getReadType());
       }
+      if (hasNeedCursorResult()) {
+        hash = (37 * hash) + NEED_CURSOR_RESULT_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getNeedCursorResult());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -15736,6 +15784,8 @@ public final class ClientProtos {
         bitField0_ = (bitField0_ & ~0x00200000);
         readType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.ReadType.DEFAULT;
         bitField0_ = (bitField0_ & ~0x00400000);
+        needCursorResult_ = false;
+        bitField0_ = (bitField0_ & ~0x00800000);
         return this;
       }
 
@@ -15879,6 +15929,10 @@ public final class ClientProtos {
           to_bitField0_ |= 0x00080000;
         }
         result.readType_ = readType_;
+        if (((from_bitField0_ & 0x00800000) == 0x00800000)) {
+          to_bitField0_ |= 0x00100000;
+        }
+        result.needCursorResult_ = needCursorResult_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -16033,6 +16087,9 @@ public final class ClientProtos {
         if (other.hasReadType()) {
           setReadType(other.getReadType());
         }
+        if (other.hasNeedCursorResult()) {
+          setNeedCursorResult(other.getNeedCursorResult());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -17660,6 +17717,39 @@ public final class ClientProtos {
         return this;
       }
 
+      // optional bool need_cursor_result = 24 [default = false];
+      private boolean needCursorResult_ ;
+      /**
+       * <code>optional bool need_cursor_result = 24 [default = false];</code>
+       */
+      public boolean hasNeedCursorResult() {
+        return ((bitField0_ & 0x00800000) == 0x00800000);
+      }
+      /**
+       * <code>optional bool need_cursor_result = 24 [default = false];</code>
+       */
+      public boolean getNeedCursorResult() {
+        return needCursorResult_;
+      }
+      /**
+       * <code>optional bool need_cursor_result = 24 [default = false];</code>
+       */
+      public Builder setNeedCursorResult(boolean value) {
+        bitField0_ |= 0x00800000;
+        needCursorResult_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool need_cursor_result = 24 [default = false];</code>
+       */
+      public Builder clearNeedCursorResult() {
+        bitField0_ = (bitField0_ & ~0x00800000);
+        needCursorResult_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.Scan)
     }
 
@@ -19249,64 +19339,511 @@ public final class ClientProtos {
         return this;
       }
 
-      // optional uint32 limit_of_rows = 11 [default = 0];
-      private int limitOfRows_ ;
+      // optional uint32 limit_of_rows = 11 [default = 0];
+      private int limitOfRows_ ;
+      /**
+       * <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
+       *
+       * <pre>
+       * if we have returned limit_of_rows rows to client, then close the scanner.
+       * </pre>
+       */
+      public boolean hasLimitOfRows() {
+        return ((bitField0_ & 0x00000400) == 0x00000400);
+      }
+      /**
+       * <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
+       *
+       * <pre>
+       * if we have returned limit_of_rows rows to client, then close the scanner.
+       * </pre>
+       */
+      public int getLimitOfRows() {
+        return limitOfRows_;
+      }
+      /**
+       * <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
+       *
+       * <pre>
+       * if we have returned limit_of_rows rows to client, then close the scanner.
+       * </pre>
+       */
+      public Builder setLimitOfRows(int value) {
+        bitField0_ |= 0x00000400;
+        limitOfRows_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
+       *
+       * <pre>
+       * if we have returned limit_of_rows rows to client, then close the scanner.
+       * </pre>
+       */
+      public Builder clearLimitOfRows() {
+        bitField0_ = (bitField0_ & ~0x00000400);
+        limitOfRows_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.ScanRequest)
+    }
+
+    static {
+      defaultInstance = new ScanRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.ScanRequest)
+  }
+
+  public interface CursorOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional bytes row = 1;
+    /**
+     * <code>optional bytes row = 1;</code>
+     */
+    boolean hasRow();
+    /**
+     * <code>optional bytes row = 1;</code>
+     */
+    com.google.protobuf.ByteString getRow();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.Cursor}
+   *
+   * <pre>
+   **
+   * Scan cursor to tell client where we are scanning.
+   * </pre>
+   */
+  public static final class Cursor extends
+      com.google.protobuf.GeneratedMessage
+      implements CursorOrBuilder {
+    // Use Cursor.newBuilder() to construct.
+    private Cursor(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private Cursor(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final Cursor defaultInstance;
+    public static Cursor getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public Cursor getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private Cursor(
+        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;
+              row_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_hbase_pb_Cursor_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_hbase_pb_Cursor_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<Cursor> PARSER =
+        new com.google.protobuf.AbstractParser<Cursor>() {
+      public Cursor parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new Cursor(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<Cursor> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional bytes row = 1;
+    public static final int ROW_FIELD_NUMBER = 1;
+    private com.google.protobuf.ByteString row_;
+    /**
+     * <code>optional bytes row = 1;</code>
+     */
+    public boolean hasRow() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional bytes row = 1;</code>
+     */
+    public com.google.protobuf.ByteString getRow() {
+      return row_;
+    }
+
+    private void initFields() {
+      row_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, row_);
+      }
+      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, row_);
+      }
+      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.ClientProtos.Cursor)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor) obj;
+
+      boolean result = true;
+      result = result && (hasRow() == other.hasRow());
+      if (hasRow()) {
+        result = result && getRow()
+            .equals(other.getRow());
+      }
+      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 (hasRow()) {
+        hash = (37 * hash) + ROW_FIELD_NUMBER;
+        hash = (53 * hash) + getRow().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor 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.ClientProtos.Cursor parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor 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.ClientProtos.Cursor parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor 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.ClientProtos.Cursor parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor 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.ClientProtos.Cursor parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor 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.ClientProtos.Cursor 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 hbase.pb.Cursor}
+     *
+     * <pre>
+     **
+     * Scan cursor to tell client where we are scanning.
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CursorOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_hbase_pb_Cursor_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_hbase_pb_Cursor_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.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();
+        row_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_hbase_pb_Cursor_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor build() {
+        org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.row_ = row_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.getDefaultInstance()) return this;
+        if (other.hasRow()) {
+          setRow(other.getRow());
+        }
+        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.ClientProtos.Cursor parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional bytes row = 1;
+      private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY;
       /**
-       * <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
-       *
-       * <pre>
-       * if we have returned limit_of_rows rows to client, then close the scanner.
-       * </pre>
+       * <code>optional bytes row = 1;</code>
        */
-      public boolean hasLimitOfRows() {
-        return ((bitField0_ & 0x00000400) == 0x00000400);
+      public boolean hasRow() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
-       *
-       * <pre>
-       * if we have returned limit_of_rows rows to client, then close the scanner.
-       * </pre>
+       * <code>optional bytes row = 1;</code>
        */
-      public int getLimitOfRows() {
-        return limitOfRows_;
+      public com.google.protobuf.ByteString getRow() {
+        return row_;
       }
       /**
-       * <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
-       *
-       * <pre>
-       * if we have returned limit_of_rows rows to client, then close the scanner.
-       * </pre>
+       * <code>optional bytes row = 1;</code>
        */
-      public Builder setLimitOfRows(int value) {
-        bitField0_ |= 0x00000400;
-        limitOfRows_ = value;
+      public Builder setRow(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        row_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional uint32 limit_of_rows = 11 [default = 0];</code>
-       *
-       * <pre>
-       * if we have returned limit_of_rows rows to client, then close the scanner.
-       * </pre>
+       * <code>optional bytes row = 1;</code>
        */
-      public Builder clearLimitOfRows() {
-        bitField0_ = (bitField0_ & ~0x00000400);
-        limitOfRows_ = 0;
+      public Builder clearRow() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        row_ = getDefaultInstance().getRow();
         onChanged();
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.ScanRequest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.Cursor)
     }
 
     static {
-      defaultInstance = new ScanRequest(true);
+      defaultInstance = new Cursor(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.ScanRequest)
+    // @@protoc_insertion_point(class_scope:hbase.pb.Cursor)
   }
 
   public interface ScanResponseOrBuilder
@@ -19594,6 +20131,35 @@ public final class ClientProtos {
      * </pre>
      */
     long getMvccReadPoint();
+
+    // optional .hbase.pb.Cursor cursor = 12;
+    /**
+     * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+     *
+     * <pre>
+     * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+     * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+     * </pre>
+     */
+    boolean hasCursor();
+    /**
+     * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+     *
+     * <pre>
+     * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+     * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor getCursor();
+    /**
+     * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+     *
+     * <pre>
+     * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+     * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CursorOrBuilder getCursorOrBuilder();
   }
   /**
    * Protobuf type {@code hbase.pb.ScanResponse}
@@ -19750,6 +20316,19 @@ public final class ClientProtos {
               mvccReadPoint_ = input.readUInt64();
               break;
             }
+            case 98: {
+              org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000100) == 0x00000100)) {
+                subBuilder = cursor_.toBuilder();
+              }
+              cursor_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(cursor_);
+                cursor_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000100;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -20161,6 +20740,43 @@ public final class ClientProtos {
       return mvccReadPoint_;
     }
 
+    // optional .hbase.pb.Cursor cursor = 12;
+    public static final int CURSOR_FIELD_NUMBER = 12;
+    private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor cursor_;
+    /**
+     * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+     *
+     * <pre>
+     * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+     * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+     * </pre>
+     */
+    public boolean hasCursor() {
+      return ((bitField0_ & 0x00000100) == 0x00000100);
+    }
+    /**
+     * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+     *
+     * <pre>
+     * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+     * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor getCursor() {
+      return cursor_;
+    }
+    /**
+     * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+     *
+     * <pre>
+     * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+     * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CursorOrBuilder getCursorOrBuilder() {
+      return cursor_;
+    }
+
     private void initFields() {
       cellsPerResult_ = java.util.Collections.emptyList();
       scannerId_ = 0L;
@@ -20173,6 +20789,7 @@ public final class ClientProtos {
       heartbeatMessage_ = false;
       scanMetrics_ = org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.ScanMetrics.getDefaultInstance();
       mvccReadPoint_ = 0L;
+      cursor_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -20219,6 +20836,9 @@ public final class ClientProtos {
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
         output.writeUInt64(11, mvccReadPoint_);
       }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        output.writeMessage(12, cursor_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -20279,6 +20899,10 @@ public final class ClientProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeUInt64Size(11, mvccReadPoint_);
       }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(12, cursor_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -20348,6 +20972,11 @@ public final class ClientProtos {
         result = result && (getMvccReadPoint()
             == other.getMvccReadPoint());
       }
+      result = result && (hasCursor() == other.hasCursor());
+      if (hasCursor()) {
+        result = result && getCursor()
+            .equals(other.getCursor());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -20405,6 +21034,10 @@ public final class ClientProtos {
         hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getMvccReadPoint());
       }
+      if (hasCursor()) {
+        hash = (37 * hash) + CURSOR_FIELD_NUMBER;
+        hash = (53 * hash) + getCursor().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -20514,6 +21147,7 @@ public final class ClientProtos {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getResultsFieldBuilder();
           getScanMetricsFieldBuilder();
+          getCursorFieldBuilder();
         }
       }
       private static Builder create() {
@@ -20552,6 +21186,12 @@ public final class ClientProtos {
         bitField0_ = (bitField0_ & ~0x00000200);
         mvccReadPoint_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000400);
+        if (cursorBuilder_ == null) {
+          cursor_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.getDefaultInstance();
+        } else {
+          cursorBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000800);
         return this;
       }
 
@@ -20635,6 +21275,14 @@ public final class ClientProtos {
           to_bitField0_ |= 0x00000080;
         }
         result.mvccReadPoint_ = mvccReadPoint_;
+        if (((from_bitField0_ & 0x00000800) == 0x00000800)) {
+          to_bitField0_ |= 0x00000100;
+        }
+        if (cursorBuilder_ == null) {
+          result.cursor_ = cursor_;
+        } else {
+          result.cursor_ = cursorBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -20721,6 +21369,9 @@ public final class ClientProtos {
         if (other.hasMvccReadPoint()) {
           setMvccReadPoint(other.getMvccReadPoint());
         }
+        if (other.hasCursor()) {
+          mergeCursor(other.getCursor());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -21846,6 +22497,168 @@ public final class ClientProtos {
         return this;
       }
 
+      // optional .hbase.pb.Cursor cursor = 12;
+      private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor cursor_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CursorOrBuilder> cursorBuilder_;
+      /**
+       * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+       *
+       * <pre>
+       * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+       * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+       * </pre>
+       */
+      public boolean hasCursor() {
+        return ((bitField0_ & 0x00000800) == 0x00000800);
+      }
+      /**
+       * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+       *
+       * <pre>
+       * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+       * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor getCursor() {
+        if (cursorBuilder_ == null) {
+          return cursor_;
+        } else {
+          return cursorBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+       *
+       * <pre>
+       * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+       * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+       * </pre>
+       */
+      public Builder setCursor(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor value) {
+        if (cursorBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          cursor_ = value;
+          onChanged();
+        } else {
+          cursorBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000800;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+       *
+       * <pre>
+       * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+       * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+       * </pre>
+       */
+      public Builder setCursor(
+          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder builderForValue) {
+        if (cursorBuilder_ == null) {
+          cursor_ = builderForValue.build();
+          onChanged();
+        } else {
+          cursorBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000800;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+       *
+       * <pre>
+       * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+       * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+       * </pre>
+       */
+      public Builder mergeCursor(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor value) {
+        if (cursorBuilder_ == null) {
+          if (((bitField0_ & 0x00000800) == 0x00000800) &&
+              cursor_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.getDefaultInstance()) {
+            cursor_ =
+              org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.newBuilder(cursor_).mergeFrom(value).buildPartial();
+          } else {
+            cursor_ = value;
+          }
+          onChanged();
+        } else {
+          cursorBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000800;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+       *
+       * <pre>
+       * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+       * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+       * </pre>
+       */
+      public Builder clearCursor() {
+        if (cursorBuilder_ == null) {
+          cursor_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.getDefaultInstance();
+          onChanged();
+        } else {
+          cursorBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000800);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+       *
+       * <pre>
+       * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+       * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder getCursorBuilder() {
+        bitField0_ |= 0x00000800;
+        onChanged();
+        return getCursorFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+       *
+       * <pre>
+       * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+       * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CursorOrBuilder getCursorOrBuilder() {
+        if (cursorBuilder_ != null) {
+          return cursorBuilder_.getMessageOrBuilder();
+        } else {
+          return cursor_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.Cursor cursor = 12;</code>
+       *
+       * <pre>
+       * If the Scan need cursor, return the row key we are scanning in heartbeat message.
+       * If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+       * </pre>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CursorOrBuilder>
+          getCursorFieldBuilder() {
+        if (cursorBuilder_ == null) {
+          cursorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CursorOrBuilder>(
+                  cursor_,
+                  getParentForChildren(),
+                  isClean());
+          cursor_ = null;
+        }
+        return cursorBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.ScanResponse)
     }
 
@@ -36257,6 +37070,11 @@ public final class ClientProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_hbase_pb_ScanRequest_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_Cursor_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_hbase_pb_Cursor_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ScanResponse_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -36399,8 +37217,8 @@ public final class ClientProtos {
       "\0132\027.hbase.pb.MutationProto\022&\n\tcondition\030" +
       "\003 \001(\0132\023.hbase.pb.Condition\022\023\n\013nonce_grou" +
       "p\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\"\377" +
-      "\005\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb.Colum" +
+      "\0132\020.hbase.pb.Result\022\021\n\tprocessed\030\002 \001(\010\"\242" +
+      "\006\n\004Scan\022 \n\006column\030\001 \003(\0132\020.hbase.pb.Colum" +
       "n\022*\n\tattribute\030\002 \003(\0132\027.hbase.pb.NameByte" +
       "sPair\022\021\n\tstart_row\030\003 \001(\014\022\020\n\010stop_row\030\004 \001" +
       "(\014\022 \n\006filter\030\005 \001(\0132\020.hbase.pb.Filter\022\'\n\n" +
@@ -36418,83 +37236,85 @@ public final class ClientProtos {
       "\0010\022\037\n\021include_start_row\030\025 \001(\010:\004true\022\037\n\020i" +
       "nclude_stop_row\030\026 \001(\010:\005false\0222\n\010readType" +
       "\030\027 \001(\0162\027.hbase.pb.Scan.ReadType:\007DEFAULT" +
-      "\".\n\010ReadType\022\013\n\007DEFAULT\020\000\022\n\n\006STREAM\020\001\022\t\n" +
-      "\005PREAD\020\002\"\300\002\n\013ScanRequest\022)\n\006region\030\001 \001(\013" +
-      "2\031.hbase.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(" +
-      "\0132\016.hbase.pb.Scan\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_call_seq\030\006 \001(\004\022\037\n\027client_ha" +
-      "ndles_partials\030\007 \001(\010\022!\n\031client_handles_h",
-      "eartbeats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t " +
-      "\001(\010\022\024\n\005renew\030\n \001(\010:\005false\022\030\n\rlimit_of_ro" +
-      "ws\030\013 \001(\r:\0010\"\266\002\n\014ScanResponse\022\030\n\020cells_pe" +
-      "r_result\030\001 \003(\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014mo" +
-      "re_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_me" +
-      "ssage\030\t \001(\010\022+\n\014scan_metrics\030\n \001(\0132\025.hbas" +
-      "e.pb.ScanMetrics\022\032\n\017mvcc_read_point\030\013 \001(",
-      "\004:\0010\"\305\001\n\024BulkLoadHFileRequest\022)\n\006region\030" +
-      "\001 \002(\0132\031.hbase.pb.RegionSpecifier\022>\n\013fami" +
-      "ly_path\030\002 \003(\0132).hbase.pb.BulkLoadHFileRe" +
-      "quest.FamilyPath\022\026\n\016assign_seq_num\030\003 \001(\010" +
-      "\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\"a\n\026CoprocessorServiceCall\022\013\n\003row\030\001" +
-      " \002(\014\022\024\n\014service_name\030\002 \002(\t\022\023\n\013method_nam" +
-      "e\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030CoprocessorS" +
-      "erviceResult\022&\n\005value\030\001 \001(\0132\027.hbase.pb.N",
-      "ameBytesPair\"v\n\031CoprocessorServiceReques" +
+      "\022!\n\022need_cursor_result\030\030 \001(\010:\005false\".\n\010R" +
+      "eadType\022\013\n\007DEFAULT\020\000\022\n\n\006STREAM\020\001\022\t\n\005PREA" +
+      "D\020\002\"\300\002\n\013ScanRequest\022)\n\006region\030\001 \001(\0132\031.hb" +
+      "ase.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.h" +
+      "base.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016numb" +
+      "er_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_handles",
+      "_partials\030\007 \001(\010\022!\n\031client_handles_heartb" +
+      "eats\030\010 \001(\010\022\032\n\022track_scan_metrics\030\t \001(\010\022\024" +
+      "\n\005renew\030\n \001(\010:\005false\022\030\n\rlimit_of_rows\030\013 " +
+      "\001(\r:\0010\"\025\n\006Cursor\022\013\n\003row\030\001 \001(\014\"\330\002\n\014ScanRe" +
+      "sponse\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nscan" +
+      "ner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003tt" +
+      "l\030\004 \001(\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Resu" +
+      "lt\022\r\n\005stale\030\006 \001(\010\022\037\n\027partial_flag_per_re" +
+      "sult\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_me",
+      "trics\030\n \001(\0132\025.hbase.pb.ScanMetrics\022\032\n\017mv" +
+      "cc_read_point\030\013 \001(\004:\0010\022 \n\006cursor\030\014 \001(\0132\020" +
+      ".hbase.pb.Cursor\"\305\001\n\024BulkLoadHFileReques" +
       "t\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpeci" +
-      "fier\022.\n\004call\030\002 \002(\0132 .hbase.pb.Coprocesso" +
-      "rServiceCall\"o\n\032CoprocessorServiceRespon" +
-      "se\022)\n\006region\030\001 \002(\0132\031.hbase.pb.RegionSpec" +
-      "ifier\022&\n\005value\030\002 \002(\0132\027.hbase.pb.NameByte" +
-      "sPair\"\226\001\n\006Action\022\r\n\005index\030\001 \001(\r\022)\n\010mutat" +
-      "ion\030\002 \001(\0132\027.hbase.pb.MutationProto\022\032\n\003ge" +
-      "t\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.p" +
-      "b.RegionSpecifier\022\016\n\006atomic\030\002 \001(\010\022 \n\006act" +
-      "ion\030\003 \003(\0132\020.hbase.pb.Action\"c\n\017RegionLoa" +
-      "dStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022\030\n\rheapO" +
-      "ccupancy\030\002 \001(\005:\0010\022\035\n\022compactionPressure\030" +
-      "\003 \001(\005:\0010\"j\n\024MultiRegionLoadStats\022)\n\006regi" +
-      "on\030\001 \003(\0132\031.hbase.pb.RegionSpecifier\022\'\n\004s" +
-      "tat\030\002 \003(\0132\031.hbase.pb.RegionLoadStats\"\336\001\n" +
-      "\021ResultOrException\022\r\n\005index\030\001 \001(\r\022 \n\006res" +
-      "ult\030\002 \001(\0132\020.hbase.pb.Result\022*\n\texception",
-      "\030\003 \001(\0132\027.hbase.pb.NameBytesPair\022:\n\016servi" +
-      "ce_result\030\004 \001(\0132\".hbase.pb.CoprocessorSe" +
-      "rviceResult\0220\n\tloadStats\030\005 \001(\0132\031.hbase.p" +
-      "b.RegionLoadStatsB\002\030\001\"x\n\022RegionActionRes" +
-      "ult\0226\n\021resultOrException\030\001 \003(\0132\033.hbase.p" +
-      "b.ResultOrException\022*\n\texception\030\002 \001(\0132\027" +
-      ".hbase.pb.NameBytesPair\"x\n\014MultiRequest\022" +
-      ",\n\014regionAction\030\001 \003(\0132\026.hbase.pb.RegionA" +
-      "ction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tcondition\030\003" +
-      " \001(\0132\023.hbase.pb.Condition\"\226\001\n\rMultiRespo",
-      "nse\0228\n\022regionActionResult\030\001 \003(\0132\034.hbase." +
-      "pb.RegionActionResult\022\021\n\tprocessed\030\002 \001(\010" +
-      "\0228\n\020regionStatistics\030\003 \001(\0132\036.hbase.pb.Mu" +
-      "ltiRegionLoadStats*\'\n\013Consistency\022\n\n\006STR" +
-      "ONG\020\000\022\014\n\010TIMELINE\020\0012\203\004\n\rClientService\0222\n" +
-      "\003Get\022\024.hbase.pb.GetRequest\032\025.hbase.pb.Ge" +
-      "tResponse\022;\n\006Mutate\022\027.hbase.pb.MutateReq" +
-      "uest\032\030.hbase.pb.MutateResponse\0225\n\004Scan\022\025" +
-      ".hbase.pb.ScanRequest\032\026.hbase.pb.ScanRes" +
-      "ponse\022P\n\rBulkLoadHFile\022\036.hbase.pb.BulkLo",
-      "adHFileRequest\032\037.hbase.pb.BulkLoadHFileR" +
-      "esponse\022X\n\013ExecService\022#.hbase.pb.Coproc" +
-      "essorServiceRequest\032$.hbase.pb.Coprocess" +
-      "orServiceResponse\022d\n\027ExecRegionServerSer" +
-      "vice\022#.hbase.pb.CoprocessorServiceReques" +
-      "t\032$.hbase.pb.CoprocessorServiceResponse\022" +
-      "8\n\005Multi\022\026.hbase.pb.MultiRequest\032\027.hbase" +
-      ".pb.MultiResponseBB\n*org.apache.hadoop.h" +
-      "base.protobuf.generatedB\014ClientProtosH\001\210" +
-      "\001\001\240\001\001"
+      "fier\022>\n\013family_path\030\002 \003(\0132).hbase.pb.Bul" +
+      "kLoadHFileRequest.FamilyPath\022\026\n\016assign_s" +
+      "eq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006family\030\001 \002" +
+      "(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileRespons" +
+      "e\022\016\n\006loaded\030\001 \002(\010\"a\n\026CoprocessorServiceC" +
+      "all\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\030" +
+      "CoprocessorServiceResult\022&\n\005value\030\001 \001(\0132" +
+      "\027.hbase.pb.NameBytesPair\"v\n\031CoprocessorS" +
+      "erviceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb" +
+      ".RegionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.p" +
+      "b.CoprocessorServiceCall\"o\n\032CoprocessorS" +
+      "erviceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.p" +
+      "b.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.Mutatio",
+      "nProto\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014se" +
+      "rvice_call\030\004 \001(\0132 .hbase.pb.CoprocessorS" +
+      "erviceCall\"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\022compact" +
+      "ionPressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadS" +
+      "tats\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSp" +
+      "ecifier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionL",
+      "oadStats\"\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.NameBytesP" +
+      "air\022:\n\016service_result\030\004 \001(\0132\".hbase.pb.C" +
+      "oprocessorServiceResult\0220\n\tloadStats\030\005 \001" +
+      "(\0132\031.hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022Reg" +
+      "ionActionResult\0226\n\021resultOrException\030\001 \003" +
+      "(\0132\033.hbase.pb.ResultOrException\022*\n\texcep" +
+      "tion\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014M" +
+      "ultiRequest\022,\n\014regionAction\030\001 \003(\0132\026.hbas",
+      "e.pb.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n" +
+      "\tcondition\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\tpro" +
+      "cessed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036" +
+      ".hbase.pb.MultiRegionLoadStats*\'\n\013Consis" +
+      "tency\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\203\004\n\rClie" +
+      "ntService\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025" +
+      ".hbase.pb.GetResponse\022;\n\006Mutate\022\027.hbase." +
+      "pb.MutateRequest\032\030.hbase.pb.MutateRespon",
+      "se\0225\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbas" +
+      "e.pb.ScanResponse\022P\n\rBulkLoadHFile\022\036.hba" +
+      "se.pb.BulkLoadHFileRequest\032\037.hbase.pb.Bu" +
+      "lkLoadHFileResponse\022X\n\013ExecService\022#.hba" +
+      "se.pb.CoprocessorServiceRequest\032$.hbase." +
+      "pb.CoprocessorServiceResponse\022d\n\027ExecReg" +
+      "ionServerService\022#.hbase.pb.CoprocessorS" +
+      "erviceRequest\032$.hbase.pb.CoprocessorServ" +
+      "iceResponse\0228\n\005Multi\022\026.hbase.pb.MultiReq" +
+      "uest\032\027.hbase.pb.MultiResponseBB\n*org.apa",
+      "che.hadoop.hbase.protobuf.generatedB\014Cli" +
+      "entProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -36584,21 +37404,27 @@ public final class ClientProtos {
           internal_static_hbase_pb_Scan_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.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", "IncludeStartRow", "IncludeStopRow", "ReadType", });
+              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", "ReadType", "NeedCursorResult", });
           internal_static_hbase_pb_ScanRequest_descriptor =
             getDescriptor().getMessageTypes().get(12);
           internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ScanRequest_descriptor,
               new java.lang.String[] { "Region", "Scan", "ScannerId", "NumberOfRows", "CloseScanner", "NextCallSeq", "ClientHandlesPartials", "ClientHandlesHeartbeats", "TrackScanMetrics", "Renew", "LimitOfRows", });
-          internal_static_hbase_pb_ScanResponse_descriptor =
+          internal_static_hbase_pb_Cursor_descriptor =
             getDescriptor().getMessageTypes().get(13);
+          internal_static_hbase_pb_Cursor_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_hbase_pb_Cursor_descriptor,
+              new java.lang.String[] { "Row", });
+          internal_static_hbase_pb_ScanResponse_descriptor =
+            getDescriptor().getMessageTypes().get(14);
           internal_static_hbase_pb_ScanResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ScanResponse_descriptor,
-              new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", "PartialFlagPerResult", "MoreResultsInRegion", "HeartbeatMessage", "ScanMetrics", "MvccReadPoint", });
+              new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", "PartialFlagPerResult", "MoreResultsInRegion", "HeartbeatMessage", "ScanMetrics", "MvccReadPoint", "Cursor", });
           internal_static_hbase_pb_BulkLoadHFileRequest_descriptor =
-            getDescriptor().getMessageTypes().get(14);
+            getDescriptor().getMessageTypes().get(15);
           internal_static_hbase_pb_BulkLoadHFileRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_BulkLoadHFileRequest_descriptor,
@@ -36610,79 +37436,79 @@ public final class ClientProtos {
               internal_static_hbase_pb_BulkLoadHFileRequest_FamilyPath_descriptor,
               new java.lang.String[] { "Family", "Path", });
           internal_static_hbase_pb_BulkLoadHFileResponse_descriptor =
-            getDescriptor().getMessageTypes().get(15);
+            getDescriptor().getMessageTypes().get(16);
           internal_static_hbase_pb_BulkLoadHFileResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_BulkLoadHFileResponse_descriptor,
               new java.lang.String[] { "Loaded", });
           internal_static_hbase_pb_CoprocessorServiceCall_descriptor =
-            getDescriptor().getMessageTypes().get(16);
+            getDescriptor().getMessageTypes().get(17);
           internal_static_hbase_pb_CoprocessorServiceCall_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_CoprocessorServiceCall_descriptor,
               new java.lang.String[] { "Row", "ServiceName", "MethodName", "Request", });
           internal_static_hbase_pb_CoprocessorServiceResult_descriptor =
-            getDescriptor().getMessageTypes().get(17);
+            getDescriptor().getMessageTypes().get(18);
           internal_static_hbase_pb_CoprocessorServiceResult_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_CoprocessorServiceResult_descriptor,
               new java.lang.String[] { "Value", });
           internal_static_hbase_pb_CoprocessorServiceRequest_descriptor =
-            getDescriptor().getMessageTypes().get(18);
+            getDescriptor().getMessageTypes().get(19);
           internal_static_hbase_pb_CoprocessorServiceRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_CoprocessorServiceRequest_descriptor,
               new java.lang.String[] { "Region", "Call", });
           internal_static_hbase_pb_CoprocessorServiceResponse_descriptor =
-            getDescriptor().getMessageTypes().get(19);
+            getDescriptor().getMessageTypes().get(20);
           internal_static_hbase_pb_CoprocessorServiceResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_CoprocessorServiceResponse_descriptor,
               new java.lang.String[] { "Region", "Value", });
           internal_static_hbase_pb_Action_descriptor =
-            getDescriptor().getMessageTypes().get(20);
+            getDescriptor().getMessageTypes().get(21);
           internal_static_hbase_pb_Action_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_Action_descriptor,
               new java.lang.String[] { "Index", "Mutation", "Get", "ServiceCall", });
           internal_static_hbase_pb_RegionAction_descriptor =
-            getDescriptor().getMessageTypes().get(21);
+            getDescriptor().getMessageTypes().get(22);
           internal_static_hbase_pb_RegionAction_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_RegionAction_descriptor,
               new java.lang.String[] { "Region", "Atomic", "Action", });
           internal_static_hbase_pb_RegionLoadStats_descriptor =
-            getDescriptor().getMessageTypes().get(22);
+            getDescriptor().getMessageTypes().get(23);
           internal_static_hbase_pb_RegionLoadStats_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_RegionLoadStats_descriptor,
               new java.lang.String[] { "MemstoreLoad", "HeapOccupancy", "CompactionPressure", });
           internal_static_hbase_pb_MultiRegionLoadStats_descriptor =
-            getDescriptor().getMessageTypes().get(23);
+            getDescriptor().getMessageTypes().get(24);
           internal_static_hbase_pb_MultiRegionLoadStats_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_MultiRegionLoadStats_descriptor,
               new java.lang.String[] { "Region", "Stat", });
           internal_static_hbase_pb_ResultOrException_descriptor =
-            getDescriptor().getMessageTypes().get(24);
+            getDescriptor().getMessageTypes().get(25);
           internal_static_hbase_pb_ResultOrException_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_ResultOrException_descriptor,
               new java.lang.String[] { "Index", "Result", "Exception", "ServiceResult", "LoadStats", });
           internal_static_hbase_pb_RegionActionResult_descriptor =
-            getDescriptor().getMessageTypes().get(25);
+            getDescriptor().getMessageTypes().get(26);
           internal_static_hbase_pb_RegionActionResult_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_RegionActionResult_descriptor,
               new java.lang.String[] { "ResultOrException", "Exception", });
           internal_static_hbase_pb_MultiRequest_descriptor =
-            getDescriptor().getMessageTypes().get(26);
+            getDescriptor().getMessageTypes().get(27);
           internal_static_hbase_pb_MultiRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_MultiRequest_descriptor,
               new java.lang.String[] { "RegionAction", "NonceGroup", "Condition", });
           internal_static_hbase_pb_MultiResponse_descriptor =
-            getDescriptor().getMessageTypes().get(27);
+            getDescriptor().getMessageTypes().get(28);
           internal_static_hbase_pb_MultiResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_hbase_pb_MultiResponse_descriptor,

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-protocol/src/main/protobuf/Client.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto
index d55b97a..9a68031 100644
--- a/hbase-protocol/src/main/protobuf/Client.proto
+++ b/hbase-protocol/src/main/protobuf/Client.proto
@@ -263,13 +263,13 @@ message Scan {
   optional uint64 mvcc_read_point = 20 [default = 0];
   optional bool include_start_row = 21 [default = true];
   optional bool include_stop_row = 22 [default = false];
-
   enum ReadType {
     DEFAULT = 0;
     STREAM = 1;
     PREAD = 2;
   }
   optional ReadType readType = 23 [default = DEFAULT];
+  optional bool need_cursor_result = 24 [default = false];
 }
 
 /**
@@ -299,6 +299,14 @@ message ScanRequest {
 }
 
 /**
+* Scan cursor to tell client where we are scanning.
+*
+ */
+message Cursor {
+  optional bytes row = 1;
+}
+
+/**
  * The scan response. If there are no more results, more_results will
  * be false.  If it is not specified, it means there are more.
  */
@@ -350,6 +358,10 @@ message ScanResponse {
   // make use of this mvcc_read_point when restarting a scanner to get a consistent view
   // of a row.
   optional uint64 mvcc_read_point = 11 [default = 0];
+
+  // If the Scan need cursor, return the row key we are scanning in heartbeat message.
+  // If the Scan doesn't need a cursor, don't set this field to reduce network IO.
+  optional Cursor cursor = 12;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 4d43b1e..7b81a99 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -282,11 +282,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     private final RegionScanner s;
     private final Region r;
     private byte[] rowOfLastPartialResult;
+    private boolean needCursor;
 
-    public RegionScannerHolder(String scannerName, RegionScanner s, Region r) {
+    public RegionScannerHolder(String scannerName, RegionScanner s, Region r, boolean needCursor) {
       this.scannerName = scannerName;
       this.s = s;
       this.r = r;
+      this.needCursor = needCursor;
     }
 
     public long getNextCallSeq() {
@@ -1165,11 +1167,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return lastBlock;
   }
 
-  private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r)
-      throws LeaseStillHeldException {
+  private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Region r,
+      boolean needCursor) throws LeaseStillHeldException {
     regionServer.leases.createLease(scannerName, this.scannerLeaseTimeoutPeriod,
       new ScannerListener(scannerName));
-    RegionScannerHolder rsh = new RegionScannerHolder(scannerName, s, r);
+    RegionScannerHolder rsh = new RegionScannerHolder(scannerName, s, r, needCursor);
     RegionScannerHolder existing = scanners.putIfAbsent(scannerName, rsh);
     assert existing == null : "scannerId must be unique within regionserver's whole lifecycle!";
     return rsh;
@@ -2568,7 +2570,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     builder.setMvccReadPoint(scanner.getMvccReadPoint());
     builder.setTtl(scannerLeaseTimeoutPeriod);
     String scannerName = String.valueOf(scannerId);
-    return addScanner(scannerName, scanner, region);
+    return addScanner(scannerName, scanner, region, scan.isNeedCursorResult());
   }
 
   private void checkScanNextCallSeq(ScanRequest request, RegionScannerHolder rsh)
@@ -2764,6 +2766,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
             if (moreRows) {
               // Heartbeat messages occur when the time limit has been reached.
               builder.setHeartbeatMessage(timeLimitReached);
+              if (timeLimitReached && rsh.needCursor) {
+                Cell readingCell = scannerContext.getPeekedCellInHeartbeat();
+                if (readingCell != null) {
+                  builder.setCursor(ProtobufUtil.toCursor(readingCell));
+                }
+              }
             }
             break;
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java
index 2c5fd01..ee53a95 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScannerContext.java
@@ -19,6 +19,9 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -94,6 +97,8 @@ public class ScannerContext {
   boolean keepProgress;
   private static boolean DEFAULT_KEEP_PROGRESS = false;
 
+  private Cell peekedCellInHeartbeat = null;
+
   /**
    * Tracks the relevant server side metrics during scans. null when metrics should not be tracked
    */
@@ -319,6 +324,14 @@ public class ScannerContext {
         || checkTimeLimit(checkerScope);
   }
 
+  public Cell getPeekedCellInHeartbeat() {
+    return peekedCellInHeartbeat;
+  }
+
+  public void setPeekedCellInHeartbeat(Cell peekedCellInHeartbeat) {
+    this.peekedCellInHeartbeat = peekedCellInHeartbeat;
+  }
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/381c89b5/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index b10c37d..d3a1e49 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -539,6 +539,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
       if ((kvsScanned % cellsPerHeartbeatCheck == 0)) {
         scannerContext.updateTimeProgress();
         if (scannerContext.checkTimeLimit(LimitScope.BETWEEN_CELLS)) {
+          scannerContext.setPeekedCellInHeartbeat(prevCell);
           return scannerContext.setScannerState(NextState.TIME_LIMIT_REACHED).hasMoreValues();
         }
       }