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/11/30 02:35:45 UTC

hbase git commit: HBASE-17167 Pass mvcc to client when scan

Repository: hbase
Updated Branches:
  refs/heads/branch-1 b8da9f83c -> af6978312


HBASE-17167 Pass mvcc to client when scan


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

Branch: refs/heads/branch-1
Commit: af69783128ed8bd8bf321f378adcb50d46a4f2fc
Parents: b8da9f8
Author: zhangduo <zh...@apache.org>
Authored: Tue Nov 29 20:35:34 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Wed Nov 30 10:02:17 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/ClientScanner.java      | 507 +++++++++----------
 .../org/apache/hadoop/hbase/client/HTable.java  |  24 +-
 .../client/PackagePrivateFieldAccessor.java     |  41 ++
 .../org/apache/hadoop/hbase/client/Scan.java    |  30 ++
 .../hadoop/hbase/client/ScannerCallable.java    |   3 +
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   8 +
 .../hbase/protobuf/generated/ClientProtos.java  | 414 +++++++++++----
 hbase-protocol/src/main/protobuf/Client.proto   |  12 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   9 +-
 .../hbase/regionserver/RSRpcServices.java       |   1 +
 .../hbase/TestPartialResultsFromClientSide.java |  13 +-
 .../hbase/client/TestMvccConsistentScanner.java | 139 +++++
 .../hadoop/hbase/regionserver/TestTags.java     |  14 +-
 .../regionserver/TestReplicationSink.java       |  22 +-
 14 files changed, 839 insertions(+), 398 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/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 944f44e..0898385 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
@@ -57,54 +57,54 @@ import com.google.common.annotations.VisibleForTesting;
  */
 @InterfaceAudience.Private
 public class ClientScanner extends AbstractClientScanner {
-    private static final Log LOG = LogFactory.getLog(ClientScanner.class);
-    // A byte array in which all elements are the max byte, and it is used to
-    // construct closest front row
-    static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);
-    protected Scan scan;
-    protected boolean closed = false;
-    // Current region scanner is against.  Gets cleared if current region goes
-    // wonky: e.g. if it splits on us.
-    protected HRegionInfo currentRegion = null;
-    protected ScannerCallableWithReplicas callable = null;
-    protected final LinkedList<Result> cache = new LinkedList<Result>();
-    /**
-     * A list of partial results that have been returned from the server. This list should only
-     * contain results if this scanner does not have enough partial results to form the complete
-     * result.
-     */
-    protected final LinkedList<Result> partialResults = new LinkedList<Result>();
-    /**
-     * The row for which we are accumulating partial Results (i.e. the row of the Results stored
-     * inside partialResults). Changes to partialResultsRow and partialResults are kept in sync
-     * via the methods {@link #addToPartialResults(Result)} and {@link #clearPartialResults()}
-     */
-    protected byte[] partialResultsRow = null;
-    /**
-     * The last cell from a not full Row which is added to cache
-     */
-    protected Cell lastCellLoadedToCache = null;
-    protected final int caching;
-    protected long lastNext;
-    // Keep lastResult returned successfully in case we have to reset scanner.
-    protected Result lastResult = null;
-    protected final long maxScannerResultSize;
-    private final ClusterConnection connection;
-    private final TableName tableName;
-    protected final int scannerTimeout;
-    protected boolean scanMetricsPublished = false;
-    protected RpcRetryingCaller<Result []> caller;
-    protected RpcControllerFactory rpcControllerFactory;
-    protected Configuration conf;
-    //The timeout on the primary. Applicable if there are multiple replicas for a region
-    //In that case, we will only wait for this much timeout on the primary before going
-    //to the replicas and trying the same scan. Note that the retries will still happen
-    //on each replica and the first successful results will be taken. A timeout of 0 is
-    //disallowed.
-    protected final int primaryOperationTimeout;
-    private int retries;
-    protected final ExecutorService pool;
-    private static MetaComparator metaComparator = new MetaComparator();
+  private static final Log LOG = LogFactory.getLog(ClientScanner.class);
+  // A byte array in which all elements are the max byte, and it is used to
+  // construct closest front row
+  static byte[] MAX_BYTE_ARRAY = Bytes.createMaxByteArray(9);
+  protected Scan scan;
+  protected boolean closed = false;
+  // Current region scanner is against. Gets cleared if current region goes
+  // wonky: e.g. if it splits on us.
+  protected HRegionInfo currentRegion = null;
+  protected ScannerCallableWithReplicas callable = null;
+  protected final LinkedList<Result> cache = new LinkedList<Result>();
+  /**
+   * A list of partial results that have been returned from the server. This list should only
+   * contain results if this scanner does not have enough partial results to form the complete
+   * result.
+   */
+  protected final LinkedList<Result> partialResults = new LinkedList<Result>();
+  /**
+   * The row for which we are accumulating partial Results (i.e. the row of the Results stored
+   * inside partialResults). Changes to partialResultsRow and partialResults are kept in sync via
+   * the methods {@link #addToPartialResults(Result)} and {@link #clearPartialResults()}
+   */
+  protected byte[] partialResultsRow = null;
+  /**
+   * The last cell from a not full Row which is added to cache
+   */
+  protected Cell lastCellLoadedToCache = null;
+  protected final int caching;
+  protected long lastNext;
+  // Keep lastResult returned successfully in case we have to reset scanner.
+  protected Result lastResult = null;
+  protected final long maxScannerResultSize;
+  private final ClusterConnection connection;
+  private final TableName tableName;
+  protected final int scannerTimeout;
+  protected boolean scanMetricsPublished = false;
+  protected RpcRetryingCaller<Result[]> caller;
+  protected RpcControllerFactory rpcControllerFactory;
+  protected Configuration conf;
+  // The timeout on the primary. Applicable if there are multiple replicas for a region
+  // In that case, we will only wait for this much timeout on the primary before going
+  // to the replicas and trying the same scan. Note that the retries will still happen
+  // on each replica and the first successful results will be taken. A timeout of 0 is
+  // disallowed.
+  protected final int primaryOperationTimeout;
+  private int retries;
+  protected final ExecutorService pool;
+  private static MetaComparator metaComparator = new MetaComparator();
 
   /**
    * Create a new ClientScanner for the specified table Note that the passed {@link Scan}'s start
@@ -119,197 +119,192 @@ public class ClientScanner extends AbstractClientScanner {
       ClusterConnection connection, RpcRetryingCallerFactory rpcFactory,
       RpcControllerFactory controllerFactory, ExecutorService pool, int primaryOperationTimeout)
       throws IOException {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Scan table=" + tableName
-            + ", startRow=" + Bytes.toStringBinary(scan.getStartRow()));
-      }
-      this.scan = scan;
-      this.tableName = tableName;
-      this.lastNext = System.currentTimeMillis();
-      this.connection = connection;
-      this.pool = pool;
-      this.primaryOperationTimeout = primaryOperationTimeout;
-      this.retries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
-          HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
-      if (scan.getMaxResultSize() > 0) {
-        this.maxScannerResultSize = scan.getMaxResultSize();
-      } else {
-        this.maxScannerResultSize = conf.getLong(
-          HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
-          HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
-      }
-      this.scannerTimeout = HBaseConfiguration.getInt(conf,
-        HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
-        HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
-        HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
-
-      // check if application wants to collect scan metrics
-      initScanMetrics(scan);
-
-      // Use the caching from the Scan.  If not set, use the default cache setting for this table.
-      if (this.scan.getCaching() > 0) {
-        this.caching = this.scan.getCaching();
-      } else {
-        this.caching = conf.getInt(
-            HConstants.HBASE_CLIENT_SCANNER_CACHING,
-            HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
-      }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(
+        "Scan table=" + tableName + ", startRow=" + Bytes.toStringBinary(scan.getStartRow()));
+    }
+    this.scan = scan;
+    this.tableName = tableName;
+    this.lastNext = System.currentTimeMillis();
+    this.connection = connection;
+    this.pool = pool;
+    this.primaryOperationTimeout = primaryOperationTimeout;
+    this.retries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
+      HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
+    if (scan.getMaxResultSize() > 0) {
+      this.maxScannerResultSize = scan.getMaxResultSize();
+    } else {
+      this.maxScannerResultSize = conf.getLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
+        HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
+    }
+    this.scannerTimeout =
+        HBaseConfiguration.getInt(conf, HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD,
+          HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
+          HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD);
 
-      this.caller = rpcFactory.<Result[]> newCaller();
-      this.rpcControllerFactory = controllerFactory;
+    // check if application wants to collect scan metrics
+    initScanMetrics(scan);
 
-      this.conf = conf;
-      initializeScannerInConstruction();
+    // Use the caching from the Scan. If not set, use the default cache setting for this table.
+    if (this.scan.getCaching() > 0) {
+      this.caching = this.scan.getCaching();
+    } else {
+      this.caching = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING,
+        HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
     }
 
-    protected void initializeScannerInConstruction() throws IOException{
-      // initialize the scanner
-      nextScanner(this.caching, false);
-    }
+    this.caller = rpcFactory.<Result[]> newCaller();
+    this.rpcControllerFactory = controllerFactory;
 
-    protected ClusterConnection getConnection() {
-      return this.connection;
-    }
+    this.conf = conf;
+    initializeScannerInConstruction();
+  }
 
-    /**
-     * @return Table name
-     * @deprecated As of release 0.96
-     *             (<a href="https://issues.apache.org/jira/browse/HBASE-9508">HBASE-9508</a>).
-     *             This will be removed in HBase 2.0.0. Use {@link #getTable()}.
-     */
-    @Deprecated
-    protected byte [] getTableName() {
-      return this.tableName.getName();
-    }
+  protected void initializeScannerInConstruction() throws IOException {
+    // initialize the scanner
+    nextScanner(this.caching, false);
+  }
 
-    protected TableName getTable() {
-      return this.tableName;
-    }
+  protected ClusterConnection getConnection() {
+    return this.connection;
+  }
 
-    protected int getRetries() {
-      return this.retries;
-    }
+  /**
+   * @return Table name
+   * @deprecated As of release 0.96
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-9508">HBASE-9508</a>). This
+   *             will be removed in HBase 2.0.0. Use {@link #getTable()}.
+   */
+  @Deprecated
+  protected byte[] getTableName() {
+    return this.tableName.getName();
+  }
 
-    protected int getScannerTimeout() {
-      return this.scannerTimeout;
-    }
+  protected TableName getTable() {
+    return this.tableName;
+  }
 
-    protected Configuration getConf() {
-      return this.conf;
-    }
+  protected int getRetries() {
+    return this.retries;
+  }
 
-    protected Scan getScan() {
-      return scan;
-    }
+  protected int getScannerTimeout() {
+    return this.scannerTimeout;
+  }
 
-    protected ExecutorService getPool() {
-      return pool;
-    }
+  protected Configuration getConf() {
+    return this.conf;
+  }
 
-    protected int getPrimaryOperationTimeout() {
-      return primaryOperationTimeout;
-    }
+  protected Scan getScan() {
+    return scan;
+  }
 
-    protected int getCaching() {
-      return caching;
-    }
+  protected ExecutorService getPool() {
+    return pool;
+  }
 
-    protected long getTimestamp() {
-      return lastNext;
-    }
+  protected int getPrimaryOperationTimeout() {
+    return primaryOperationTimeout;
+  }
 
-    @VisibleForTesting
-    protected long getMaxResultSize() {
-      return maxScannerResultSize;
-    }
+  protected int getCaching() {
+    return caching;
+  }
 
-    // returns true if the passed region endKey
-    protected boolean checkScanStopRow(final byte [] endKey) {
-      if (this.scan.getStopRow().length > 0) {
-        // there is a stop row, check to see if we are past it.
-        byte [] stopRow = scan.getStopRow();
-        int cmp = Bytes.compareTo(stopRow, 0, stopRow.length,
-          endKey, 0, endKey.length);
-        if (cmp <= 0) {
-          // stopRow <= endKey (endKey is equals to or larger than stopRow)
-          // This is a stop.
-          return true;
-        }
-      }
-      return false; //unlikely.
-    }
+  protected long getTimestamp() {
+    return lastNext;
+  }
 
-    private boolean possiblyNextScanner(int nbRows, final boolean done) throws IOException {
-      // If we have just switched replica, don't go to the next scanner yet. Rather, try
-      // the scanner operations on the new replica, from the right point in the scan
-      // Note that when we switched to a different replica we left it at a point
-      // where we just did the "openScanner" with the appropriate startrow
-      if (callable != null && callable.switchedToADifferentReplica()) return true;
-      return nextScanner(nbRows, done);
-    }
+  @VisibleForTesting
+  protected long getMaxResultSize() {
+    return maxScannerResultSize;
+  }
 
-    /*
-     * Gets a scanner for the next region.  If this.currentRegion != null, then
-     * we will move to the endrow of this.currentRegion.  Else we will get
-     * scanner at the scan.getStartRow().  We will go no further, just tidy
-     * up outstanding scanners, if <code>currentRegion != null</code> and
-     * <code>done</code> is true.
-     * @param nbRows
-     * @param done Server-side says we're done scanning.
-     */
-  protected boolean nextScanner(int nbRows, final boolean done)
-    throws IOException {
-      // Close the previous scanner if it's open
-      if (this.callable != null) {
-        this.callable.setClose();
-        call(callable, caller, scannerTimeout);
-        this.callable = null;
+  // returns true if the passed region endKey
+  protected boolean checkScanStopRow(final byte[] endKey) {
+    if (this.scan.getStopRow().length > 0) {
+      // there is a stop row, check to see if we are past it.
+      byte[] stopRow = scan.getStopRow();
+      int cmp = Bytes.compareTo(stopRow, 0, stopRow.length, endKey, 0, endKey.length);
+      if (cmp <= 0) {
+        // stopRow <= endKey (endKey is equals to or larger than stopRow)
+        // This is a stop.
+        return true;
       }
+    }
+    return false; // unlikely.
+  }
 
-      // Where to start the next scanner
-      byte [] localStartKey;
-
-      // if we're at end of table, close and return false to stop iterating
-      if (this.currentRegion != null) {
-        byte [] endKey = this.currentRegion.getEndKey();
-        if (endKey == null ||
-            Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY) ||
-            checkScanStopRow(endKey) ||
-            done) {
-          close();
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Finished " + this.currentRegion);
-          }
-          return false;
-        }
-        localStartKey = endKey;
+  private boolean possiblyNextScanner(int nbRows, final boolean done) throws IOException {
+    // If we have just switched replica, don't go to the next scanner yet. Rather, try
+    // the scanner operations on the new replica, from the right point in the scan
+    // Note that when we switched to a different replica we left it at a point
+    // where we just did the "openScanner" with the appropriate startrow
+    if (callable != null && callable.switchedToADifferentReplica()) return true;
+    return nextScanner(nbRows, done);
+  }
+
+  /*
+   * Gets a scanner for the next region. If this.currentRegion != null, then we will move to the
+   * endrow of this.currentRegion. Else we will get scanner at the scan.getStartRow(). We will go no
+   * further, just tidy up outstanding scanners, if <code>currentRegion != null</code> and
+   * <code>done</code> is true.
+   * @param nbRows
+   * @param done Server-side says we're done scanning.
+   */
+  protected boolean nextScanner(int nbRows, final boolean done) throws IOException {
+    // Close the previous scanner if it's open
+    if (this.callable != null) {
+      this.callable.setClose();
+      call(callable, caller, scannerTimeout);
+      this.callable = null;
+    }
+
+    // Where to start the next scanner
+    byte[] localStartKey;
+
+    // if we're at end of table, close and return false to stop iterating
+    if (this.currentRegion != null) {
+      byte[] endKey = this.currentRegion.getEndKey();
+      if (endKey == null || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)
+          || checkScanStopRow(endKey) || done) {
+        close();
         if (LOG.isTraceEnabled()) {
           LOG.trace("Finished " + this.currentRegion);
         }
-      } else {
-        localStartKey = this.scan.getStartRow();
+        return false;
       }
-
-      if (LOG.isDebugEnabled() && this.currentRegion != null) {
-        // Only worth logging if NOT first region in scan.
-        LOG.debug("Advancing internal scanner to startKey at '" +
-          Bytes.toStringBinary(localStartKey) + "'");
+      localStartKey = endKey;
+      // clear mvcc read point if we are going to switch regions
+      scan.resetMvccReadPoint();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Finished " + this.currentRegion);
       }
-      try {
-        callable = getScannerCallable(localStartKey, nbRows);
-        // Open a scanner on the region server starting at the
-        // beginning of the region
-        call(callable, caller, scannerTimeout);
-        this.currentRegion = callable.getHRegionInfo();
-        if (this.scanMetrics != null) {
-          this.scanMetrics.countOfRegions.incrementAndGet();
-        }
-      } catch (IOException e) {
-        close();
-        throw e;
+    } else {
+      localStartKey = this.scan.getStartRow();
+    }
+
+    if (LOG.isDebugEnabled() && this.currentRegion != null) {
+      // Only worth logging if NOT first region in scan.
+      LOG.debug(
+        "Advancing internal scanner to startKey at '" + Bytes.toStringBinary(localStartKey) + "'");
+    }
+    try {
+      callable = getScannerCallable(localStartKey, nbRows);
+      // Open a scanner on the region server starting at the
+      // beginning of the region
+      call(callable, caller, scannerTimeout);
+      this.currentRegion = callable.getHRegionInfo();
+      if (this.scanMetrics != null) {
+        this.scanMetrics.countOfRegions.incrementAndGet();
       }
-      return true;
+    } catch (IOException e) {
+      close();
+      throw e;
     }
+    return true;
+  }
 
   @VisibleForTesting
   boolean isAnyRPCcancelled() {
@@ -327,58 +322,54 @@ public class ClientScanner extends AbstractClientScanner {
     return caller.callWithoutRetries(callable, scannerTimeout);
   }
 
-    @InterfaceAudience.Private
-    protected ScannerCallableWithReplicas getScannerCallable(byte [] localStartKey,
-        int nbRows) {
-      scan.setStartRow(localStartKey);
-      ScannerCallable s =
-          new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics,
-              this.rpcControllerFactory);
-      s.setCaching(nbRows);
-      ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas(tableName, getConnection(),
-       s, pool, primaryOperationTimeout, scan,
-       retries, scannerTimeout, caching, conf, caller);
-      return sr;
-    }
+  @InterfaceAudience.Private
+  protected ScannerCallableWithReplicas getScannerCallable(byte[] localStartKey, int nbRows) {
+    scan.setStartRow(localStartKey);
+    ScannerCallable s = new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics,
+        this.rpcControllerFactory);
+    s.setCaching(nbRows);
+    ScannerCallableWithReplicas sr = new ScannerCallableWithReplicas(tableName, getConnection(), s,
+        pool, primaryOperationTimeout, scan, retries, scannerTimeout, caching, conf, caller);
+    return sr;
+  }
 
-    /**
-     * Publish the scan metrics. For now, we use scan.setAttribute to pass the metrics back to the
-     * application or TableInputFormat.Later, we could push it to other systems. We don't use
-     * metrics framework because it doesn't support multi-instances of the same metrics on the same
-     * machine; for scan/map reduce scenarios, we will have multiple scans running at the same time.
-     *
-     * By default, scan metrics are disabled; if the application wants to collect them, this
-     * behavior can be turned on by calling calling {@link Scan#setScanMetricsEnabled(boolean)}
-     * 
-     * <p>This invocation clears the scan metrics. Metrics are aggregated in the Scan instance.
-     */
-    protected void writeScanMetrics() {
-      if (this.scanMetrics == null || scanMetricsPublished) {
-        return;
-      }
-      MapReduceProtos.ScanMetrics pScanMetrics = ProtobufUtil.toScanMetrics(scanMetrics);
-      scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA, pScanMetrics.toByteArray());
-      scanMetricsPublished = true;
+  /**
+   * Publish the scan metrics. For now, we use scan.setAttribute to pass the metrics back to the
+   * application or TableInputFormat.Later, we could push it to other systems. We don't use metrics
+   * framework because it doesn't support multi-instances of the same metrics on the same machine;
+   * for scan/map reduce scenarios, we will have multiple scans running at the same time. By
+   * default, scan metrics are disabled; if the application wants to collect them, this behavior can
+   * be turned on by calling calling {@link Scan#setScanMetricsEnabled(boolean)}
+   * <p>
+   * This invocation clears the scan metrics. Metrics are aggregated in the Scan instance.
+   */
+  protected void writeScanMetrics() {
+    if (this.scanMetrics == null || scanMetricsPublished) {
+      return;
     }
+    MapReduceProtos.ScanMetrics pScanMetrics = ProtobufUtil.toScanMetrics(scanMetrics);
+    scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA, pScanMetrics.toByteArray());
+    scanMetricsPublished = true;
+  }
 
-    @Override
-    public Result next() throws IOException {
-      // If the scanner is closed and there's nothing left in the cache, next is a no-op.
-      if (cache.size() == 0 && this.closed) {
-        return null;
-      }
-      if (cache.size() == 0) {
-        loadCache();
-      }
-
-      if (cache.size() > 0) {
-        return cache.poll();
-      }
-
-      // if we exhausted this scanner before calling close, write out the scan metrics
-      writeScanMetrics();
+  @Override
+  public Result next() throws IOException {
+    // If the scanner is closed and there's nothing left in the cache, next is a no-op.
+    if (cache.size() == 0 && this.closed) {
       return null;
     }
+    if (cache.size() == 0) {
+      loadCache();
+    }
+
+    if (cache.size() > 0) {
+      return cache.poll();
+    }
+
+    // if we exhausted this scanner before calling close, write out the scan metrics
+    writeScanMetrics();
+    return null;
+  }
 
   @VisibleForTesting
   public int getCacheSize() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index e8a969f..6b5b9a8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -18,6 +18,12 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Message;
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -36,12 +42,10 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -49,6 +53,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
@@ -72,12 +78,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Threads;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
 /**
  * An implementation of {@link Table}. Used to communicate with a single HBase table.
  * Lightweight. Get as needed and just close when done.
@@ -775,7 +775,7 @@ public class HTable implements HTableInterface, RegionLocator {
    * {@link HTableInterface#getScanner(Scan)} has other usage details.
    */
   @Override
-  public ResultScanner getScanner(final Scan scan) throws IOException {
+  public ResultScanner getScanner(Scan scan) throws IOException {
     if (scan.getBatch() > 0 && scan.isSmall()) {
       throw new IllegalArgumentException("Small scan should not be used with batching");
     }
@@ -786,6 +786,10 @@ public class HTable implements HTableInterface, RegionLocator {
     if (scan.getMaxResultSize() <= 0) {
       scan.setMaxResultSize(scannerMaxResultSize);
     }
+    if (scan.getMvccReadPoint() > 0) {
+      // it is not supposed to be set by user, clear
+      scan.resetMvccReadPoint();
+    }
 
     if (scan.isReversed()) {
       if (scan.isSmall()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PackagePrivateFieldAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PackagePrivateFieldAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PackagePrivateFieldAccessor.java
new file mode 100644
index 0000000..6a3ac18
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PackagePrivateFieldAccessor.java
@@ -0,0 +1,41 @@
+/**
+ * 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;
+
+/**
+ * A helper class used to access the package private field in o.a.h.h.client package.
+ * <p>
+ * This is because we share some data structures between client and server and the data structures
+ * are marked as {@code InterfaceAudience.Public}, but we do not want to expose some of the fields
+ * to end user.
+ * <p>
+ * TODO: A better solution is to separate the data structures used in client and server.
+ */
+@InterfaceAudience.Private
+public class PackagePrivateFieldAccessor {
+
+  public static void setMvccReadPoint(Scan scan, long mvccReadPoint) {
+    scan.setMvccReadPoint(mvccReadPoint);
+  }
+
+  public static long getMvccReadPoint(Scan scan) {
+    return scan.getMvccReadPoint();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/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 ac7e190..128e7e1 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
@@ -169,6 +169,12 @@ public class Scan extends Query {
   private boolean small = false;
 
   /**
+   * The mvcc read point to use when open a scanner. Remember to clear it after switching regions as
+   * the mvcc is only valid within region scope.
+   */
+  private long mvccReadPoint = -1L;
+
+  /**
    * Create a Scan operation across all rows.
    */
   public Scan() {}
@@ -246,6 +252,7 @@ public class Scan extends Query {
       TimeRange tr = entry.getValue();
       setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
     }
+    this.mvccReadPoint = scan.getMvccReadPoint();
   }
 
   /**
@@ -272,6 +279,7 @@ public class Scan extends Query {
       TimeRange tr = entry.getValue();
       setColumnFamilyTimeRange(entry.getKey(), tr.getMin(), tr.getMax());
     }
+    this.mvccReadPoint = -1L;
   }
 
   public boolean isGetScan() {
@@ -1004,4 +1012,26 @@ public class Scan extends Query {
     if (bytes == null) return null;
     return ProtobufUtil.toScanMetrics(bytes);
   }
+
+  /**
+   * Get the mvcc read point used to open a scanner.
+   */
+  long getMvccReadPoint() {
+    return mvccReadPoint;
+  }
+
+  /**
+   * Set the mvcc read point used to open a scanner.
+   */
+  Scan setMvccReadPoint(long mvccReadPoint) {
+    this.mvccReadPoint = mvccReadPoint;
+    return this;
+  }
+
+  /**
+   * Set the mvcc read point to -1 which means do not use it.
+   */
+  Scan resetMvccReadPoint() {
+    return setMvccReadPoint(-1L);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/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 7d3b9e9..bde2863 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
@@ -407,6 +407,9 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
         LOG.info("Open scanner=" + id + " for scan=" + scan.toString()
           + " on region " + getLocation().toString());
       }
+      if (response.hasMvccReadPoint()) {
+        this.scan.setMvccReadPoint(response.getMvccReadPoint());
+      }
       return id;
     } catch (ServiceException se) {
       throw ProtobufUtil.getRemoteException(se);

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/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 6a5eec5..ca2dd04 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
@@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
@@ -1000,6 +1001,10 @@ public final class ProtobufUtil {
     if (scan.getCaching() > 0) {
       scanBuilder.setCaching(scan.getCaching());
     }
+    long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan);
+    if (mvccReadPoint > 0) {
+      scanBuilder.setMvccReadPoint(mvccReadPoint);
+    }
     return scanBuilder.build();
   }
 
@@ -1087,6 +1092,9 @@ public final class ProtobufUtil {
     if (proto.hasCaching()) {
       scan.setCaching(proto.getCaching());
     }
+    if (proto.hasMvccReadPoint()) {
+      PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint());
+    }
     return scan;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/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 1964140..d1f28ba 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
@@ -14229,6 +14229,16 @@ public final class ClientProtos {
      */
     org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRangeOrBuilder getCfTimeRangeOrBuilder(
         int index);
+
+    // optional uint64 mvcc_read_point = 20 [default = 0];
+    /**
+     * <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
+     */
+    boolean hasMvccReadPoint();
+    /**
+     * <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
+     */
+    long getMvccReadPoint();
   }
   /**
    * Protobuf type {@code hbase.pb.Scan}
@@ -14418,6 +14428,11 @@ public final class ClientProtos {
               cfTimeRange_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilyTimeRange.PARSER, extensionRegistry));
               break;
             }
+            case 160: {
+              bitField0_ |= 0x00010000;
+              mvccReadPoint_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -14851,6 +14866,22 @@ public final class ClientProtos {
       return cfTimeRange_.get(index);
     }
 
+    // optional uint64 mvcc_read_point = 20 [default = 0];
+    public static final int MVCC_READ_POINT_FIELD_NUMBER = 20;
+    private long mvccReadPoint_;
+    /**
+     * <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
+     */
+    public boolean hasMvccReadPoint() {
+      return ((bitField0_ & 0x00010000) == 0x00010000);
+    }
+    /**
+     * <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
+     */
+    public long getMvccReadPoint() {
+      return mvccReadPoint_;
+    }
+
     private void initFields() {
       column_ = java.util.Collections.emptyList();
       attribute_ = java.util.Collections.emptyList();
@@ -14871,6 +14902,7 @@ public final class ClientProtos {
       caching_ = 0;
       allowPartialResults_ = false;
       cfTimeRange_ = java.util.Collections.emptyList();
+      mvccReadPoint_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -14965,6 +14997,9 @@ public final class ClientProtos {
       for (int i = 0; i < cfTimeRange_.size(); i++) {
         output.writeMessage(19, cfTimeRange_.get(i));
       }
+      if (((bitField0_ & 0x00010000) == 0x00010000)) {
+        output.writeUInt64(20, mvccReadPoint_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -15050,6 +15085,10 @@ public final class ClientProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(19, cfTimeRange_.get(i));
       }
+      if (((bitField0_ & 0x00010000) == 0x00010000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(20, mvccReadPoint_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -15159,6 +15198,11 @@ public final class ClientProtos {
       }
       result = result && getCfTimeRangeList()
           .equals(other.getCfTimeRangeList());
+      result = result && (hasMvccReadPoint() == other.hasMvccReadPoint());
+      if (hasMvccReadPoint()) {
+        result = result && (getMvccReadPoint()
+            == other.getMvccReadPoint());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -15248,6 +15292,10 @@ public final class ClientProtos {
         hash = (37 * hash) + CF_TIME_RANGE_FIELD_NUMBER;
         hash = (53 * hash) + getCfTimeRangeList().hashCode();
       }
+      if (hasMvccReadPoint()) {
+        hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getMvccReadPoint());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -15431,6 +15479,8 @@ public final class ClientProtos {
         } else {
           cfTimeRangeBuilder_.clear();
         }
+        mvccReadPoint_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00080000);
         return this;
       }
 
@@ -15558,6 +15608,10 @@ public final class ClientProtos {
         } else {
           result.cfTimeRange_ = cfTimeRangeBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00080000) == 0x00080000)) {
+          to_bitField0_ |= 0x00010000;
+        }
+        result.mvccReadPoint_ = mvccReadPoint_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -15700,6 +15754,9 @@ public final class ClientProtos {
             }
           }
         }
+        if (other.hasMvccReadPoint()) {
+          setMvccReadPoint(other.getMvccReadPoint());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -17192,6 +17249,39 @@ public final class ClientProtos {
         return cfTimeRangeBuilder_;
       }
 
+      // optional uint64 mvcc_read_point = 20 [default = 0];
+      private long mvccReadPoint_ ;
+      /**
+       * <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
+       */
+      public boolean hasMvccReadPoint() {
+        return ((bitField0_ & 0x00080000) == 0x00080000);
+      }
+      /**
+       * <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
+       */
+      public long getMvccReadPoint() {
+        return mvccReadPoint_;
+      }
+      /**
+       * <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
+       */
+      public Builder setMvccReadPoint(long value) {
+        bitField0_ |= 0x00080000;
+        mvccReadPoint_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 mvcc_read_point = 20 [default = 0];</code>
+       */
+      public Builder clearMvccReadPoint() {
+        bitField0_ = (bitField0_ & ~0x00080000);
+        mvccReadPoint_ = 0L;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.Scan)
     }
 
@@ -18957,7 +19047,7 @@ public final class ClientProtos {
      *
      * <pre>
      * This field is filled in if the client has requested that scan metrics be tracked.
-     * The metrics tracked here are sent back to the client to be tracked together with 
+     * The metrics tracked here are sent back to the client to be tracked together with
      * the existing client side metrics.
      * </pre>
      */
@@ -18967,7 +19057,7 @@ public final class ClientProtos {
      *
      * <pre>
      * This field is filled in if the client has requested that scan metrics be tracked.
-     * The metrics tracked here are sent back to the client to be tracked together with 
+     * The metrics tracked here are sent back to the client to be tracked together with
      * the existing client side metrics.
      * </pre>
      */
@@ -18977,11 +19067,33 @@ public final class ClientProtos {
      *
      * <pre>
      * This field is filled in if the client has requested that scan metrics be tracked.
-     * The metrics tracked here are sent back to the client to be tracked together with 
+     * The metrics tracked here are sent back to the client to be tracked together with
      * the existing client side metrics.
      * </pre>
      */
     org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.ScanMetricsOrBuilder getScanMetricsOrBuilder();
+
+    // optional uint64 mvcc_read_point = 11 [default = 0];
+    /**
+     * <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
+     *
+     * <pre>
+     * The mvcc read point which is used to open the scanner at server side. Client can
+     * make use of this mvcc_read_point when restarting a scanner to get a consistent view
+     * of a row.
+     * </pre>
+     */
+    boolean hasMvccReadPoint();
+    /**
+     * <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
+     *
+     * <pre>
+     * The mvcc read point which is used to open the scanner at server side. Client can
+     * make use of this mvcc_read_point when restarting a scanner to get a consistent view
+     * of a row.
+     * </pre>
+     */
+    long getMvccReadPoint();
   }
   /**
    * Protobuf type {@code hbase.pb.ScanResponse}
@@ -19133,6 +19245,11 @@ public final class ClientProtos {
               bitField0_ |= 0x00000040;
               break;
             }
+            case 88: {
+              bitField0_ |= 0x00000080;
+              mvccReadPoint_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -19484,7 +19601,7 @@ public final class ClientProtos {
      *
      * <pre>
      * This field is filled in if the client has requested that scan metrics be tracked.
-     * The metrics tracked here are sent back to the client to be tracked together with 
+     * The metrics tracked here are sent back to the client to be tracked together with
      * the existing client side metrics.
      * </pre>
      */
@@ -19496,7 +19613,7 @@ public final class ClientProtos {
      *
      * <pre>
      * This field is filled in if the client has requested that scan metrics be tracked.
-     * The metrics tracked here are sent back to the client to be tracked together with 
+     * The metrics tracked here are sent back to the client to be tracked together with
      * the existing client side metrics.
      * </pre>
      */
@@ -19508,7 +19625,7 @@ public final class ClientProtos {
      *
      * <pre>
      * This field is filled in if the client has requested that scan metrics be tracked.
-     * The metrics tracked here are sent back to the client to be tracked together with 
+     * The metrics tracked here are sent back to the client to be tracked together with
      * the existing client side metrics.
      * </pre>
      */
@@ -19516,6 +19633,34 @@ public final class ClientProtos {
       return scanMetrics_;
     }
 
+    // optional uint64 mvcc_read_point = 11 [default = 0];
+    public static final int MVCC_READ_POINT_FIELD_NUMBER = 11;
+    private long mvccReadPoint_;
+    /**
+     * <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
+     *
+     * <pre>
+     * The mvcc read point which is used to open the scanner at server side. Client can
+     * make use of this mvcc_read_point when restarting a scanner to get a consistent view
+     * of a row.
+     * </pre>
+     */
+    public boolean hasMvccReadPoint() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
+     *
+     * <pre>
+     * The mvcc read point which is used to open the scanner at server side. Client can
+     * make use of this mvcc_read_point when restarting a scanner to get a consistent view
+     * of a row.
+     * </pre>
+     */
+    public long getMvccReadPoint() {
+      return mvccReadPoint_;
+    }
+
     private void initFields() {
       cellsPerResult_ = java.util.Collections.emptyList();
       scannerId_ = 0L;
@@ -19527,6 +19672,7 @@ public final class ClientProtos {
       moreResultsInRegion_ = false;
       heartbeatMessage_ = false;
       scanMetrics_ = org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.ScanMetrics.getDefaultInstance();
+      mvccReadPoint_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -19570,6 +19716,9 @@ public final class ClientProtos {
       if (((bitField0_ & 0x00000040) == 0x00000040)) {
         output.writeMessage(10, scanMetrics_);
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeUInt64(11, mvccReadPoint_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -19626,6 +19775,10 @@ public final class ClientProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(10, scanMetrics_);
       }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(11, mvccReadPoint_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -19690,6 +19843,11 @@ public final class ClientProtos {
         result = result && getScanMetrics()
             .equals(other.getScanMetrics());
       }
+      result = result && (hasMvccReadPoint() == other.hasMvccReadPoint());
+      if (hasMvccReadPoint()) {
+        result = result && (getMvccReadPoint()
+            == other.getMvccReadPoint());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -19743,6 +19901,10 @@ public final class ClientProtos {
         hash = (37 * hash) + SCAN_METRICS_FIELD_NUMBER;
         hash = (53 * hash) + getScanMetrics().hashCode();
       }
+      if (hasMvccReadPoint()) {
+        hash = (37 * hash) + MVCC_READ_POINT_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getMvccReadPoint());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -19888,6 +20050,8 @@ public final class ClientProtos {
           scanMetricsBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000200);
+        mvccReadPoint_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000400);
         return this;
       }
 
@@ -19967,6 +20131,10 @@ public final class ClientProtos {
         } else {
           result.scanMetrics_ = scanMetricsBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.mvccReadPoint_ = mvccReadPoint_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -20050,6 +20218,9 @@ public final class ClientProtos {
         if (other.hasScanMetrics()) {
           mergeScanMetrics(other.getScanMetrics());
         }
+        if (other.hasMvccReadPoint()) {
+          setMvccReadPoint(other.getMvccReadPoint());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -20956,7 +21127,7 @@ public final class ClientProtos {
        *
        * <pre>
        * This field is filled in if the client has requested that scan metrics be tracked.
-       * The metrics tracked here are sent back to the client to be tracked together with 
+       * The metrics tracked here are sent back to the client to be tracked together with
        * the existing client side metrics.
        * </pre>
        */
@@ -20968,7 +21139,7 @@ public final class ClientProtos {
        *
        * <pre>
        * This field is filled in if the client has requested that scan metrics be tracked.
-       * The metrics tracked here are sent back to the client to be tracked together with 
+       * The metrics tracked here are sent back to the client to be tracked together with
        * the existing client side metrics.
        * </pre>
        */
@@ -20984,7 +21155,7 @@ public final class ClientProtos {
        *
        * <pre>
        * This field is filled in if the client has requested that scan metrics be tracked.
-       * The metrics tracked here are sent back to the client to be tracked together with 
+       * The metrics tracked here are sent back to the client to be tracked together with
        * the existing client side metrics.
        * </pre>
        */
@@ -21006,7 +21177,7 @@ public final class ClientProtos {
        *
        * <pre>
        * This field is filled in if the client has requested that scan metrics be tracked.
-       * The metrics tracked here are sent back to the client to be tracked together with 
+       * The metrics tracked here are sent back to the client to be tracked together with
        * the existing client side metrics.
        * </pre>
        */
@@ -21026,7 +21197,7 @@ public final class ClientProtos {
        *
        * <pre>
        * This field is filled in if the client has requested that scan metrics be tracked.
-       * The metrics tracked here are sent back to the client to be tracked together with 
+       * The metrics tracked here are sent back to the client to be tracked together with
        * the existing client side metrics.
        * </pre>
        */
@@ -21051,7 +21222,7 @@ public final class ClientProtos {
        *
        * <pre>
        * This field is filled in if the client has requested that scan metrics be tracked.
-       * The metrics tracked here are sent back to the client to be tracked together with 
+       * The metrics tracked here are sent back to the client to be tracked together with
        * the existing client side metrics.
        * </pre>
        */
@@ -21070,7 +21241,7 @@ public final class ClientProtos {
        *
        * <pre>
        * This field is filled in if the client has requested that scan metrics be tracked.
-       * The metrics tracked here are sent back to the client to be tracked together with 
+       * The metrics tracked here are sent back to the client to be tracked together with
        * the existing client side metrics.
        * </pre>
        */
@@ -21084,7 +21255,7 @@ public final class ClientProtos {
        *
        * <pre>
        * This field is filled in if the client has requested that scan metrics be tracked.
-       * The metrics tracked here are sent back to the client to be tracked together with 
+       * The metrics tracked here are sent back to the client to be tracked together with
        * the existing client side metrics.
        * </pre>
        */
@@ -21100,7 +21271,7 @@ public final class ClientProtos {
        *
        * <pre>
        * This field is filled in if the client has requested that scan metrics be tracked.
-       * The metrics tracked here are sent back to the client to be tracked together with 
+       * The metrics tracked here are sent back to the client to be tracked together with
        * the existing client side metrics.
        * </pre>
        */
@@ -21118,6 +21289,63 @@ public final class ClientProtos {
         return scanMetricsBuilder_;
       }
 
+      // optional uint64 mvcc_read_point = 11 [default = 0];
+      private long mvccReadPoint_ ;
+      /**
+       * <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
+       *
+       * <pre>
+       * The mvcc read point which is used to open the scanner at server side. Client can
+       * make use of this mvcc_read_point when restarting a scanner to get a consistent view
+       * of a row.
+       * </pre>
+       */
+      public boolean hasMvccReadPoint() {
+        return ((bitField0_ & 0x00000400) == 0x00000400);
+      }
+      /**
+       * <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
+       *
+       * <pre>
+       * The mvcc read point which is used to open the scanner at server side. Client can
+       * make use of this mvcc_read_point when restarting a scanner to get a consistent view
+       * of a row.
+       * </pre>
+       */
+      public long getMvccReadPoint() {
+        return mvccReadPoint_;
+      }
+      /**
+       * <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
+       *
+       * <pre>
+       * The mvcc read point which is used to open the scanner at server side. Client can
+       * make use of this mvcc_read_point when restarting a scanner to get a consistent view
+       * of a row.
+       * </pre>
+       */
+      public Builder setMvccReadPoint(long value) {
+        bitField0_ |= 0x00000400;
+        mvccReadPoint_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 mvcc_read_point = 11 [default = 0];</code>
+       *
+       * <pre>
+       * The mvcc read point which is used to open the scanner at server side. Client can
+       * make use of this mvcc_read_point when restarting a scanner to get a consistent view
+       * of a row.
+       * </pre>
+       */
+      public Builder clearMvccReadPoint() {
+        bitField0_ = (bitField0_ & ~0x00000400);
+        mvccReadPoint_ = 0L;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:hbase.pb.ScanResponse)
     }
 
@@ -35671,7 +35899,7 @@ 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\"\275" +
+      "\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.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" +
@@ -35686,80 +35914,82 @@ public final class ClientProtos {
       "\025.hbase.pb.Consistency:\006STRONG\022\017\n\007cachin" +
       "g\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.ColumnF" +
-      "amilyTimeRange\"\246\002\n\013ScanRequest\022)\n\006region",
-      "\030\001 \001(\0132\031.hbase.pb.RegionSpecifier\022\034\n\004sca" +
-      "n\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_sca" +
-      "nner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\022\037\n\027cli" +
-      "ent_handles_partials\030\007 \001(\010\022!\n\031client_han" +
-      "dles_heartbeats\030\010 \001(\010\022\032\n\022track_scan_metr" +
-      "ics\030\t \001(\010\022\024\n\005renew\030\n \001(\010:\005false\"\232\002\n\014Scan" +
-      "Response\022\030\n\020cells_per_result\030\001 \003(\r\022\022\n\nsc" +
-      "anner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010\022\013\n\003" +
-      "ttl\030\004 \001(\r\022!\n\007results\030\005 \003(\0132\020.hbase.pb.Re",
-      "sult\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\"\305\001\n" +
-      "\024BulkLoadHFileRequest\022)\n\006region\030\001 \002(\0132\031." +
-      "hbase.pb.RegionSpecifier\022>\n\013family_path\030" +
-      "\002 \003(\0132).hbase.pb.BulkLoadHFileRequest.Fa" +
-      "milyPath\022\026\n\016assign_seq_num\030\003 \001(\010\032*\n\nFami" +
-      "lyPath\022\016\n\006family\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025" +
-      "BulkLoadHFileResponse\022\016\n\006loaded\030\001 \002(\010\"a\n",
-      "\026CoprocessorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014" +
-      "service_name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022" +
-      "\017\n\007request\030\004 \002(\014\"B\n\030CoprocessorServiceRe" +
-      "sult\022&\n\005value\030\001 \001(\0132\027.hbase.pb.NameBytes" +
-      "Pair\"v\n\031CoprocessorServiceRequest\022)\n\006reg" +
-      "ion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022.\n\004" +
-      "call\030\002 \002(\0132 .hbase.pb.CoprocessorService" +
-      "Call\"o\n\032CoprocessorServiceResponse\022)\n\006re" +
-      "gion\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.MutationProto\022\032\n\003get\030\003 \001(\0132" +
-      "\r.hbase.pb.Get\0226\n\014service_call\030\004 \001(\0132 .h" +
-      "base.pb.CoprocessorServiceCall\"k\n\014Region" +
-      "Action\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Region" +
-      "Specifier\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\022compactionPressure\030\003 \001(\005:\0010" +
-      "\"j\n\024MultiRegionLoadStats\022)\n\006region\030\001 \003(\013",
-      "2\031.hbase.pb.RegionSpecifier\022\'\n\004stat\030\002 \003(" +
-      "\0132\031.hbase.pb.RegionLoadStats\"\336\001\n\021ResultO" +
-      "rException\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.NameBytesPair\022:\n\016service_resul" +
-      "t\030\004 \001(\0132\".hbase.pb.CoprocessorServiceRes" +
-      "ult\0220\n\tloadStats\030\005 \001(\0132\031.hbase.pb.Region" +
-      "LoadStatsB\002\030\001\"x\n\022RegionActionResult\0226\n\021r" +
-      "esultOrException\030\001 \003(\0132\033.hbase.pb.Result" +
-      "OrException\022*\n\texception\030\002 \001(\0132\027.hbase.p",
-      "b.NameBytesPair\"x\n\014MultiRequest\022,\n\014regio" +
-      "nAction\030\001 \003(\0132\026.hbase.pb.RegionAction\022\022\n" +
-      "\nnonceGroup\030\002 \001(\004\022&\n\tcondition\030\003 \001(\0132\023.h" +
-      "base.pb.Condition\"\226\001\n\rMultiResponse\0228\n\022r" +
-      "egionActionResult\030\001 \003(\0132\034.hbase.pb.Regio" +
-      "nActionResult\022\021\n\tprocessed\030\002 \001(\010\0228\n\020regi" +
-      "onStatistics\030\003 \001(\0132\036.hbase.pb.MultiRegio" +
-      "nLoadStats*\'\n\013Consistency\022\n\n\006STRONG\020\000\022\014\n" +
-      "\010TIMELINE\020\0012\203\004\n\rClientService\0222\n\003Get\022\024.h" +
-      "base.pb.GetRequest\032\025.hbase.pb.GetRespons",
-      "e\022;\n\006Mutate\022\027.hbase.pb.MutateRequest\032\030.h" +
-      "base.pb.MutateResponse\0225\n\004Scan\022\025.hbase.p" +
-      "b.ScanRequest\032\026.hbase.pb.ScanResponse\022P\n" +
-      "\rBulkLoadHFile\022\036.hbase.pb.BulkLoadHFileR" +
-      "equest\032\037.hbase.pb.BulkLoadHFileResponse\022" +
-      "X\n\013ExecService\022#.hbase.pb.CoprocessorSer" +
-      "viceRequest\032$.hbase.pb.CoprocessorServic" +
-      "eResponse\022d\n\027ExecRegionServerService\022#.h" +
-      "base.pb.CoprocessorServiceRequest\032$.hbas" +
-      "e.pb.CoprocessorServiceResponse\0228\n\005Multi",
-      "\022\026.hbase.pb.MultiRequest\032\027.hbase.pb.Mult" +
-      "iResponseBB\n*org.apache.hadoop.hbase.pro" +
-      "tobuf.generatedB\014ClientProtosH\001\210\001\001\240\001\001"
+      "amilyTimeRange\022\032\n\017mvcc_read_point\030\024 \001(\004:",
+      "\0010\"\246\002\n\013ScanRequest\022)\n\006region\030\001 \001(\0132\031.hba" +
+      "se.pb.RegionSpecifier\022\034\n\004scan\030\002 \001(\0132\016.hb" +
+      "ase.pb.Scan\022\022\n\nscanner_id\030\003 \001(\004\022\026\n\016numbe" +
+      "r_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_heartbe" +
+      "ats\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\020" +
+      "cells_per_result\030\001 \003(\r\022\022\n\nscanner_id\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\005stal" +
+      "e\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\021hear" +
+      "tbeat_message\030\t \001(\010\022+\n\014scan_metrics\030\n \001(" +
+      "\0132\025.hbase.pb.ScanMetrics\022\032\n\017mvcc_read_po" +
+      "int\030\013 \001(\004:\0010\"\305\001\n\024BulkLoadHFileRequest\022)\n" +
+      "\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifier" +
+      "\022>\n\013family_path\030\002 \003(\0132).hbase.pb.BulkLoa" +
+      "dHFileRequest.FamilyPath\022\026\n\016assign_seq_n" +
+      "um\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\013me" +
+      "thod_name\030\003 \002(\t\022\017\n\007request\030\004 \002(\014\"B\n\030Copr" +
+      "ocessorServiceResult\022&\n\005value\030\001 \001(\0132\027.hb" +
+      "ase.pb.NameBytesPair\"v\n\031CoprocessorServi" +
+      "ceRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Reg" +
+      "ionSpecifier\022.\n\004call\030\002 \002(\0132 .hbase.pb.Co" +
+      "processorServiceCall\"o\n\032CoprocessorServi" +
+      "ceResponse\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Re",
+      "gionSpecifier\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.MutationPro" +
+      "to\022\032\n\003get\030\003 \001(\0132\r.hbase.pb.Get\0226\n\014servic" +
+      "e_call\030\004 \001(\0132 .hbase.pb.CoprocessorServi" +
+      "ceCall\"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\017R" +
+      "egionLoadStats\022\027\n\014memstoreLoad\030\001 \001(\005:\0010\022" +
+      "\030\n\rheapOccupancy\030\002 \001(\005:\0010\022\035\n\022compactionP",
+      "ressure\030\003 \001(\005:\0010\"j\n\024MultiRegionLoadStats" +
+      "\022)\n\006region\030\001 \003(\0132\031.hbase.pb.RegionSpecif" +
+      "ier\022\'\n\004stat\030\002 \003(\0132\031.hbase.pb.RegionLoadS" +
+      "tats\"\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\te" +
+      "xception\030\003 \001(\0132\027.hbase.pb.NameBytesPair\022" +
+      ":\n\016service_result\030\004 \001(\0132\".hbase.pb.Copro" +
+      "cessorServiceResult\0220\n\tloadStats\030\005 \001(\0132\031" +
+      ".hbase.pb.RegionLoadStatsB\002\030\001\"x\n\022RegionA" +
+      "ctionResult\0226\n\021resultOrException\030\001 \003(\0132\033",
+      ".hbase.pb.ResultOrException\022*\n\texception" +
+      "\030\002 \001(\0132\027.hbase.pb.NameBytesPair\"x\n\014Multi" +
+      "Request\022,\n\014regionAction\030\001 \003(\0132\026.hbase.pb" +
+      ".RegionAction\022\022\n\nnonceGroup\030\002 \001(\004\022&\n\tcon" +
+      "dition\030\003 \001(\0132\023.hbase.pb.Condition\"\226\001\n\rMu" +
+      "ltiResponse\0228\n\022regionActionResult\030\001 \003(\0132" +
+      "\034.hbase.pb.RegionActionResult\022\021\n\tprocess" +
+      "ed\030\002 \001(\010\0228\n\020regionStatistics\030\003 \001(\0132\036.hba" +
+      "se.pb.MultiRegionLoadStats*\'\n\013Consistenc" +
+      "y\022\n\n\006STRONG\020\000\022\014\n\010TIMELINE\020\0012\203\004\n\rClientSe",
+      "rvice\0222\n\003Get\022\024.hbase.pb.GetRequest\032\025.hba" +
+      "se.pb.GetResponse\022;\n\006Mutate\022\027.hbase.pb.M" +
+      "utateRequest\032\030.hbase.pb.MutateResponse\0225" +
+      "\n\004Scan\022\025.hbase.pb.ScanRequest\032\026.hbase.pb" +
+      ".ScanResponse\022P\n\rBulkLoadHFile\022\036.hbase.p" +
+      "b.BulkLoadHFileRequest\032\037.hbase.pb.BulkLo" +
+      "adHFileResponse\022X\n\013ExecService\022#.hbase.p" +
+      "b.CoprocessorServiceRequest\032$.hbase.pb.C" +
+      "oprocessorServiceResponse\022d\n\027ExecRegionS" +
+      "erverService\022#.hbase.pb.CoprocessorServi",
+      "ceRequest\032$.hbase.pb.CoprocessorServiceR" +
+      "esponse\0228\n\005Multi\022\026.hbase.pb.MultiRequest" +
+      "\032\027.hbase.pb.MultiResponseBB\n*org.apache." +
+      "hadoop.hbase.protobuf.generatedB\014ClientP" +
+      "rotosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -35849,7 +36079,7 @@ 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", });
+              new java.lang.String[] { "Column", "Attribute", "StartRow", "StopRow", "Filter", "TimeRange", "MaxVersions", "CacheBlocks", "BatchSize", "MaxResultSize", "StoreLimit", "StoreOffset", "LoadColumnFamiliesOnDemand", "Small", "Reversed", "Consistency", "Caching", "AllowPartialResults", "CfTimeRange", "MvccReadPoint", });
           internal_static_hbase_pb_ScanRequest_descriptor =
             getDescriptor().getMessageTypes().get(12);
           internal_static_hbase_pb_ScanRequest_fieldAccessorTable = new
@@ -35861,7 +36091,7 @@ public final class ClientProtos {
           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", });
+              new java.lang.String[] { "CellsPerResult", "ScannerId", "MoreResults", "Ttl", "Results", "Stale", "PartialFlagPerResult", "MoreResultsInRegion", "HeartbeatMessage", "ScanMetrics", "MvccReadPoint", });
           internal_static_hbase_pb_BulkLoadHFileRequest_descriptor =
             getDescriptor().getMessageTypes().get(14);
           internal_static_hbase_pb_BulkLoadHFileRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/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 9cb1555..f330ea4 100644
--- a/hbase-protocol/src/main/protobuf/Client.proto
+++ b/hbase-protocol/src/main/protobuf/Client.proto
@@ -259,6 +259,7 @@ message Scan {
   optional uint32 caching = 17;
   optional bool allow_partial_results = 18;
   repeated ColumnFamilyTimeRange cf_time_range = 19;
+  optional uint64 mvcc_read_point = 20 [default = 0];
 }
 
 /**
@@ -321,17 +322,22 @@ message ScanResponse {
   // reasons such as the size in bytes or quantity of results accumulated. This field
   // will true when more results exist in the current region.
   optional bool more_results_in_region = 8;
-  
+
   // This field is filled in if the server is sending back a heartbeat message.
   // Heartbeat messages are sent back to the client to prevent the scanner from
   // timing out. Seeing a heartbeat message communicates to the Client that the
   // server would have continued to scan had the time limit not been reached.
   optional bool heartbeat_message = 9;
-  
+
   // This field is filled in if the client has requested that scan metrics be tracked.
-  // The metrics tracked here are sent back to the client to be tracked together with 
+  // The metrics tracked here are sent back to the client to be tracked together with
   // the existing client side metrics.
   optional ScanMetrics scan_metrics = 10;
+
+  // The mvcc read point which is used to open the scanner at server side. Client can
+  // 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];
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 8f8add8..5e98f66 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -103,6 +103,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.IsolationLevel;
 import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
@@ -5744,8 +5745,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       // synchronize on scannerReadPoints so that nobody calculates
       // getSmallestReadPoint, before scannerReadPoints is updated.
       IsolationLevel isolationLevel = scan.getIsolationLevel();
-      synchronized(scannerReadPoints) {
-        if (nonce == HConstants.NO_NONCE || rsServices == null
+      long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan);
+      synchronized (scannerReadPoints) {
+        if (mvccReadPoint > 0) {
+          this.readPt = mvccReadPoint;
+        } else if (nonce == HConstants.NO_NONCE || rsServices == null
             || rsServices.getNonceManager() == null) {
           this.readPt = getReadpoint(isolationLevel);
         } else {
@@ -5753,7 +5757,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
         scannerReadPoints.put(this, this.readPt);
       }
-
       initializeScanners(scan, additionalScanners);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/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 b92dd28..fc77c2c 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
@@ -2510,6 +2510,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         scannerId = addScanner(scanner, region);
         scannerName = String.valueOf(scannerId);
         ttl = this.scannerLeaseTimeoutPeriod;
+        builder.setMvccReadPoint(scanner.getMvccReadPoint());
       }
       if (request.hasRenew() && request.getRenew()) {
         rsh = scanners.get(scannerName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
index 01e6270..ed8e848 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -732,12 +732,9 @@ public class TestPartialResultsFromClientSide {
 
     Table tmpTable = createTestTable(testName, rows, families, qualifiers, value);
 
-    Scan scan = new Scan();
-    scan.setMaxResultSize(1);
-    scan.setAllowPartialResults(true);
-
     // Open scanner before deletes
-    ResultScanner scanner = tmpTable.getScanner(scan);
+    ResultScanner scanner =
+        tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
 
     Delete delete1 = new Delete(rows[0]);
     delete1.addColumn(families[0], qualifiers[0], 0);
@@ -754,13 +751,13 @@ public class TestPartialResultsFromClientSide {
         scannerCount == expectedCount);
 
     // Minus 2 for the two cells that were deleted
-    scanner = tmpTable.getScanner(scan);
+    scanner = tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
     scannerCount = countCellsFromScanner(scanner);
     expectedCount = numRows * numFamilies * numQualifiers - 2;
     assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
         scannerCount == expectedCount);
 
-    scanner = tmpTable.getScanner(scan);
+    scanner = tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
     // Put in 2 new rows. The timestamps differ from the deleted rows
     Put put1 = new Put(rows[0]);
     put1.add(new KeyValue(rows[0], families[0], qualifiers[0], 1, value));
@@ -777,7 +774,7 @@ public class TestPartialResultsFromClientSide {
         scannerCount == expectedCount);
 
     // Now the scanner should see the cells that were added by puts
-    scanner = tmpTable.getScanner(scan);
+    scanner = tmpTable.getScanner(new Scan().setMaxResultSize(1).setAllowPartialResults(true));
     scannerCount = countCellsFromScanner(scanner);
     expectedCount = numRows * numFamilies * numQualifiers;
     assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java
new file mode 100644
index 0000000..41b5a9c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMvccConsistentScanner.java
@@ -0,0 +1,139 @@
+/**
+ * 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 static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+@Category({ LargeTests.class, ClientTests.class })
+public class TestMvccConsistentScanner {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static Connection CONN;
+
+  private static final byte[] CF = Bytes.toBytes("cf");
+
+  private static final byte[] CQ1 = Bytes.toBytes("cq1");
+
+  private static final byte[] CQ2 = Bytes.toBytes("cq2");
+
+  private static final byte[] CQ3 = Bytes.toBytes("cq3");
+  @Rule
+  public TestName testName = new TestName();
+
+  private TableName tableName;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.startMiniCluster(2);
+    CONN = ConnectionFactory.createConnection(UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    CONN.close();
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws IOException, InterruptedException {
+    tableName = TableName.valueOf(testName.getMethodName().replaceAll("[^0-9a-zA-Z]", "_"));
+    UTIL.createTable(tableName, CF);
+    UTIL.waitTableAvailable(tableName);
+  }
+
+  private void put(byte[] row, byte[] cq, byte[] value) throws IOException {
+    try (Table table = CONN.getTable(tableName)) {
+      table.put(new Put(row).addColumn(CF, cq, value));
+    }
+  }
+
+  private void move() throws IOException, InterruptedException {
+    HRegionInfo region = UTIL.getHBaseCluster().getRegions(tableName).get(0).getRegionInfo();
+    HRegionServer rs = null;
+    for (JVMClusterUtil.RegionServerThread thread : UTIL.getHBaseCluster()
+        .getRegionServerThreads()) {
+      if (!thread.getRegionServer().getOnlineTables().contains(tableName)) {
+        rs = thread.getRegionServer();
+        break;
+      }
+    }
+    UTIL.getHBaseAdmin().move(region.getEncodedNameAsBytes(),
+      Bytes.toBytes(rs.getServerName().getServerName()));
+    while (UTIL.getRSForFirstRegionInTable(tableName) != rs) {
+      Thread.sleep(100);
+    }
+  }
+
+  @Test
+  public void testRowAtomic() throws IOException, InterruptedException {
+    byte[] row = Bytes.toBytes("row");
+    put(row, CQ1, Bytes.toBytes(1));
+    put(row, CQ2, Bytes.toBytes(2));
+    try (Table table = CONN.getTable(tableName);
+        ResultScanner scanner = table.getScanner(new Scan().setBatch(1).setCaching(1))) {
+      Result result = scanner.next();
+      assertEquals(1, result.rawCells().length);
+      assertEquals(1, Bytes.toInt(result.getValue(CF, CQ1)));
+      move();
+      put(row, CQ3, Bytes.toBytes(3));
+      result = scanner.next();
+      assertEquals(1, result.rawCells().length);
+      assertEquals(2, Bytes.toInt(result.getValue(CF, CQ2)));
+      assertNull(scanner.next());
+    }
+  }
+
+  @Test
+  public void testCrossRowAtomicInRegion() throws IOException, InterruptedException {
+    put(Bytes.toBytes("row1"), CQ1, Bytes.toBytes(1));
+    put(Bytes.toBytes("row2"), CQ1, Bytes.toBytes(2));
+    try (Table table = CONN.getTable(tableName);
+        ResultScanner scanner = table.getScanner(new Scan().setCaching(1))) {
+      Result result = scanner.next();
+      assertArrayEquals(Bytes.toBytes("row1"), result.getRow());
+      assertEquals(1, Bytes.toInt(result.getValue(CF, CQ1)));
+      move();
+      put(Bytes.toBytes("row3"), CQ1, Bytes.toBytes(3));
+      result = scanner.next();
+      assertArrayEquals(Bytes.toBytes("row2"), result.getRow());
+      assertEquals(2, Bytes.toInt(result.getValue(CF, CQ1)));
+      assertNull(scanner.next());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
index 4c7a204..2de98bd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
@@ -436,10 +436,8 @@ public class TestTags {
       increment.add(new KeyValue(row2, f, q, 1234L, v));
       increment.setAttribute("visibility", Bytes.toBytes("tag2"));
       table.increment(increment);
-      Scan scan = new Scan();
-      scan.setStartRow(row2);
       TestCoprocessorForTags.checkTagPresence = true;
-      scanner = table.getScanner(scan);
+      scanner = table.getScanner(new Scan().setStartRow(row2));
       result = scanner.next();
       kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
       tags = TestCoprocessorForTags.tags;
@@ -458,10 +456,8 @@ public class TestTags {
       Append append = new Append(row3);
       append.add(f, q, Bytes.toBytes("b"));
       table.append(append);
-      scan = new Scan();
-      scan.setStartRow(row3);
       TestCoprocessorForTags.checkTagPresence = true;
-      scanner = table.getScanner(scan);
+      scanner = table.getScanner(new Scan().setStartRow(row3));
       result = scanner.next();
       kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
       tags = TestCoprocessorForTags.tags;
@@ -475,7 +471,7 @@ public class TestTags {
       append.setAttribute("visibility", Bytes.toBytes("tag2"));
       table.append(append);
       TestCoprocessorForTags.checkTagPresence = true;
-      scanner = table.getScanner(scan);
+      scanner = table.getScanner(new Scan().setStartRow(row3));
       result = scanner.next();
       kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
       tags = TestCoprocessorForTags.tags;
@@ -498,10 +494,8 @@ public class TestTags {
       append.add(new KeyValue(row4, f, q, 1234L, v));
       append.setAttribute("visibility", Bytes.toBytes("tag2"));
       table.append(append);
-      scan = new Scan();
-      scan.setStartRow(row4);
       TestCoprocessorForTags.checkTagPresence = true;
-      scanner = table.getScanner(scan);
+      scanner = table.getScanner(new Scan().setStartRow(row4));
       result = scanner.next();
       kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
       tags = TestCoprocessorForTags.tags;

http://git-wip-us.apache.org/repos/asf/hbase/blob/af697831/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index 79c5bec..be0617a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -333,22 +333,16 @@ public class TestReplicationSink {
     }
 
     entries.add(builder.build());
-    ResultScanner scanRes = null;
-    try {
-      Scan scan = new Scan();
-      scanRes = table1.getScanner(scan);
+    try (ResultScanner scanner = table1.getScanner(new Scan())) {
       // 6. Assert no existing data in table
-      assertEquals(0, scanRes.next(numRows).length);
-      // 7. Replicate the bulk loaded entry
-      SINK.replicateEntries(entries, CellUtil.createCellScanner(edit.getCells().iterator()),
-        replicationClusterId, baseNamespaceDir, hfileArchiveDir);
-      scanRes = table1.getScanner(scan);
+      assertEquals(0, scanner.next(numRows).length);
+    }
+    // 7. Replicate the bulk loaded entry
+    SINK.replicateEntries(entries, CellUtil.createCellScanner(edit.getCells().iterator()),
+      replicationClusterId, baseNamespaceDir, hfileArchiveDir);
+    try (ResultScanner scanner = table1.getScanner(new Scan())) {
       // 8. Assert data is replicated
-      assertEquals(numRows, scanRes.next(numRows).length);
-    } finally {
-      if (scanRes != null) {
-        scanRes.close();
-      }
+      assertEquals(numRows, scanner.next(numRows).length);
     }
   }