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

hbase git commit: HBASE-17334 Add locate row before/after support for AsyncRegionLocator

Repository: hbase
Updated Branches:
  refs/heads/master 66781864a -> 09bb42876


HBASE-17334 Add locate row before/after support for AsyncRegionLocator


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

Branch: refs/heads/master
Commit: 09bb4287631563df934cfe88b16fa6dc03e490e6
Parents: 6678186
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 22 14:22:41 2016 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 22 20:39:01 2016 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/AsyncClientScanner.java |  14 +-
 .../client/AsyncMultiGetRpcRetryingCaller.java  |   3 +-
 .../hbase/client/AsyncNonMetaRegionLocator.java | 183 ++++++++-----------
 .../hadoop/hbase/client/AsyncRegionLocator.java |  23 +--
 .../client/AsyncRpcRetryingCallerFactory.java   |  10 +-
 .../AsyncScanSingleRegionRpcRetryingCaller.java |   9 +-
 .../AsyncSingleRequestRpcRetryingCaller.java    |  46 ++---
 .../client/AsyncSmallScanRpcRetryingCaller.java |  17 +-
 .../client/AsyncTableRegionLocatorImpl.java     |   2 +-
 .../hadoop/hbase/client/RegionLocateType.java   |  33 ++++
 .../client/TestAsyncNonMetaRegionLocator.java   | 101 ++++++----
 ...syncNonMetaRegionLocatorConcurrenyLimit.java |  13 +-
 .../client/TestAsyncRegionLocatorTimeout.java   |   8 +-
 ...TestAsyncSingleRequestRpcRetryingCaller.java |  10 +-
 14 files changed, 236 insertions(+), 236 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java
index 74c20de..dfffd39 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java
@@ -118,23 +118,22 @@ class AsyncClientScanner {
         .setScan(scan).consumer(consumer).resultCache(resultCache)
         .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
         .scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).start()
-        .whenComplete((locateToPreviousRegion, error) -> {
+        .whenComplete((locateType, error) -> {
           if (error != null) {
             consumer.onError(error);
             return;
           }
-          if (locateToPreviousRegion == null) {
+          if (locateType == null) {
             consumer.onComplete();
           } else {
-            openScanner(locateToPreviousRegion.booleanValue());
+            openScanner(locateType);
           }
         });
   }
 
-  private void openScanner(boolean locateToPreviousRegion) {
+  private void openScanner(RegionLocateType locateType) {
     conn.callerFactory.<OpenScannerResponse> single().table(tableName).row(scan.getStartRow())
-        .locateToPreviousRegion(locateToPreviousRegion)
-        .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
+        .locateType(locateType).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
         .operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).action(this::callOpenScanner).call()
         .whenComplete((resp, error) -> {
           if (error != null) {
@@ -146,6 +145,7 @@ class AsyncClientScanner {
   }
 
   public void start() {
-    openScanner(scan.isReversed() && isEmptyStartRow(scan.getStartRow()));
+    openScanner(scan.isReversed() && isEmptyStartRow(scan.getStartRow()) ? RegionLocateType.BEFORE
+        : RegionLocateType.CURRENT);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMultiGetRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMultiGetRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMultiGetRpcRetryingCaller.java
index 8a9b9a8..e1208c2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMultiGetRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMultiGetRpcRetryingCaller.java
@@ -374,7 +374,8 @@ class AsyncMultiGetRpcRetryingCaller {
         new ConcurrentHashMap<>();
     ConcurrentLinkedQueue<Get> locateFailed = new ConcurrentLinkedQueue<>();
     CompletableFuture.allOf(gets.map(get -> conn.getLocator()
-        .getRegionLocation(tableName, get.getRow(), locateTimeoutNs).whenComplete((loc, error) -> {
+        .getRegionLocation(tableName, get.getRow(), RegionLocateType.CURRENT, locateTimeoutNs)
+        .whenComplete((loc, error) -> {
           if (error != null) {
             error = translateException(error);
             if (error instanceof DoNotRetryIOException) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
index c22d210..ae79b65 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.HConstants.ZEROES;
 import static org.apache.hadoop.hbase.HRegionInfo.createRegionName;
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
 import static org.apache.hadoop.hbase.client.AsyncRegionLocator.updateCachedLoation;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
 import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
@@ -77,16 +78,16 @@ class AsyncNonMetaRegionLocator {
 
     public final byte[] row;
 
-    public final boolean locateToPrevious;
+    public final RegionLocateType locateType;
 
-    public LocateRequest(byte[] row, boolean locateToPrevious) {
+    public LocateRequest(byte[] row, RegionLocateType locateType) {
       this.row = row;
-      this.locateToPrevious = locateToPrevious;
+      this.locateType = locateType;
     }
 
     @Override
     public int hashCode() {
-      return Bytes.hashCode(row) ^ Boolean.hashCode(locateToPrevious);
+      return Bytes.hashCode(row) ^ locateType.hashCode();
     }
 
     @Override
@@ -95,7 +96,7 @@ class AsyncNonMetaRegionLocator {
         return false;
       }
       LocateRequest that = (LocateRequest) obj;
-      return locateToPrevious == that.locateToPrevious && Bytes.equals(row, that.row);
+      return locateType.equals(that.locateType) && Bytes.equals(row, that.row);
     }
   }
 
@@ -192,8 +193,14 @@ class AsyncNonMetaRegionLocator {
       return true;
     }
     boolean completed;
-    if (req.locateToPrevious) {
-      completed = Bytes.equals(loc.getRegionInfo().getEndKey(), req.row);
+    if (req.locateType.equals(RegionLocateType.BEFORE)) {
+      // for locating the row before current row, the common case is to find the previous region in
+      // reverse scan, so we check the endKey first. In general, the condition should be startKey <
+      // req.row and endKey >= req.row. Here we split it to endKey == req.row || (endKey > req.row
+      // && startKey < req.row). The two conditions are equal since startKey < endKey.
+      int c = Bytes.compareTo(loc.getRegionInfo().getEndKey(), req.row);
+      completed =
+          c == 0 || (c > 0 && Bytes.compareTo(loc.getRegionInfo().getStartKey(), req.row) < 0);
     } else {
       completed = loc.getRegionInfo().containsRow(req.row);
     }
@@ -206,11 +213,11 @@ class AsyncNonMetaRegionLocator {
   }
 
   private void complete(TableName tableName, LocateRequest req, HRegionLocation loc,
-      Throwable error, String rowNameInErrorMsg) {
+      Throwable error) {
     if (error != null) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Failed to locate region in '" + tableName + "', " + rowNameInErrorMsg + "='"
-            + Bytes.toStringBinary(req.row) + "'",
+        LOG.debug("Failed to locate region in '" + tableName + "', row='"
+            + Bytes.toStringBinary(req.row) + "', locateType=" + req.locateType,
           error);
       }
     }
@@ -254,87 +261,67 @@ class AsyncNonMetaRegionLocator {
       }
     }
     if (toSend != null) {
-      if (toSend.locateToPrevious) {
-        locatePreviousInMeta(tableName, toSend);
-      } else {
-        locateInMeta(tableName, toSend);
-      }
+      locateInMeta(tableName, toSend);
     }
   }
 
   private void onScanComplete(TableName tableName, LocateRequest req, List<Result> results,
-      Throwable error, String rowNameInErrorMsg) {
+      Throwable error) {
     if (error != null) {
-      complete(tableName, req, null, error, rowNameInErrorMsg);
+      complete(tableName, req, null, error);
       return;
     }
     if (results.isEmpty()) {
-      complete(tableName, req, null, new TableNotFoundException(tableName), rowNameInErrorMsg);
+      complete(tableName, req, null, new TableNotFoundException(tableName));
       return;
     }
     RegionLocations locs = MetaTableAccessor.getRegionLocations(results.get(0));
     if (LOG.isDebugEnabled()) {
-      LOG.debug("The fetched location of '" + tableName + "', " + rowNameInErrorMsg + "='"
-          + Bytes.toStringBinary(req.row) + "' is " + locs);
+      LOG.debug("The fetched location of '" + tableName + "', row='" + Bytes.toStringBinary(req.row)
+          + "', locateType=" + req.locateType + " is " + locs);
     }
     if (locs == null || locs.getDefaultRegionLocation() == null) {
       complete(tableName, req, null,
-        new IOException(String.format("No location found for '%s', %s='%s'", tableName,
-          rowNameInErrorMsg, Bytes.toStringBinary(req.row))),
-        rowNameInErrorMsg);
+        new IOException(String.format("No location found for '%s', row='%s', locateType=%s",
+          tableName, Bytes.toStringBinary(req.row), req.locateType)));
       return;
     }
     HRegionLocation loc = locs.getDefaultRegionLocation();
     HRegionInfo info = loc.getRegionInfo();
     if (info == null) {
       complete(tableName, req, null,
-        new IOException(String.format("HRegionInfo is null for '%s', %s='%s'", tableName,
-          rowNameInErrorMsg, Bytes.toStringBinary(req.row))),
-        rowNameInErrorMsg);
+        new IOException(String.format("HRegionInfo is null for '%s', row='%s', locateType=%s",
+          tableName, Bytes.toStringBinary(req.row), req.locateType)));
       return;
     }
     if (!info.getTable().equals(tableName)) {
-      complete(tableName, req, null,
-        new TableNotFoundException(
-            "Table '" + tableName + "' was not found, got: '" + info.getTable() + "'"),
-        rowNameInErrorMsg);
+      complete(tableName, req, null, new TableNotFoundException(
+          "Table '" + tableName + "' was not found, got: '" + info.getTable() + "'"));
       return;
     }
     if (info.isSplit()) {
       complete(tableName, req, null,
         new RegionOfflineException(
             "the only available region for the required row is a split parent,"
-                + " the daughters should be online soon: '" + info.getRegionNameAsString() + "'"),
-        rowNameInErrorMsg);
+                + " the daughters should be online soon: '" + info.getRegionNameAsString() + "'"));
       return;
     }
     if (info.isOffline()) {
-      complete(tableName, req, null,
-        new RegionOfflineException("the region is offline, could"
-            + " be caused by a disable table call: '" + info.getRegionNameAsString() + "'"),
-        rowNameInErrorMsg);
+      complete(tableName, req, null, new RegionOfflineException("the region is offline, could"
+          + " be caused by a disable table call: '" + info.getRegionNameAsString() + "'"));
       return;
     }
     if (loc.getServerName() == null) {
       complete(tableName, req, null,
         new NoServerForRegionException(
-            String.format("No server address listed for region '%s', %s='%s'",
-              info.getRegionNameAsString(), rowNameInErrorMsg, Bytes.toStringBinary(req.row))),
-        rowNameInErrorMsg);
-      return;
-    }
-    if (req.locateToPrevious && !Bytes.equals(info.getEndKey(), req.row)) {
-      complete(tableName, req, null,
-        new DoNotRetryIOException("The end key of '" + info.getRegionNameAsString() + "' is '"
-            + Bytes.toStringBinary(info.getEndKey()) + "', expected '"
-            + Bytes.toStringBinary(req.row) + "'"),
-        rowNameInErrorMsg);
+            String.format("No server address listed for region '%s', row='%s', locateType=%s",
+              info.getRegionNameAsString(), Bytes.toStringBinary(req.row), req.locateType)));
       return;
     }
-    complete(tableName, req, loc, null, rowNameInErrorMsg);
+    complete(tableName, req, loc, null);
   }
 
-  private HRegionLocation locateInCache(TableCache tableCache, TableName tableName, byte[] row) {
+  private HRegionLocation locateRowInCache(TableCache tableCache, TableName tableName, byte[] row) {
     Map.Entry<byte[], HRegionLocation> entry = tableCache.cache.floorEntry(row);
     if (entry == null) {
       return null;
@@ -344,7 +331,7 @@ class AsyncNonMetaRegionLocator {
     if (isEmptyStopRow(endKey) || Bytes.compareTo(row, endKey) < 0) {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='"
-            + Bytes.toStringBinary(row) + "'");
+            + Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.CURRENT);
       }
       return loc;
     } else {
@@ -352,22 +339,19 @@ class AsyncNonMetaRegionLocator {
     }
   }
 
-  private HRegionLocation locatePreviousInCache(TableCache tableCache, TableName tableName,
-      byte[] startRowOfCurrentRegion) {
-    Map.Entry<byte[], HRegionLocation> entry;
-    if (isEmptyStopRow(startRowOfCurrentRegion)) {
-      entry = tableCache.cache.lastEntry();
-    } else {
-      entry = tableCache.cache.lowerEntry(startRowOfCurrentRegion);
-    }
+  private HRegionLocation locateRowBeforeInCache(TableCache tableCache, TableName tableName,
+      byte[] row) {
+    Map.Entry<byte[], HRegionLocation> entry =
+        isEmptyStopRow(row) ? tableCache.cache.lastEntry() : tableCache.cache.lowerEntry(row);
     if (entry == null) {
       return null;
     }
     HRegionLocation loc = entry.getValue();
-    if (Bytes.equals(loc.getRegionInfo().getEndKey(), startRowOfCurrentRegion)) {
+    if (isEmptyStopRow(loc.getRegionInfo().getEndKey())
+        || Bytes.compareTo(loc.getRegionInfo().getEndKey(), row) >= 0) {
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Found " + loc + " in cache for '" + tableName + "', startRowOfCurrentRegion='"
-            + Bytes.toStringBinary(startRowOfCurrentRegion) + "'");
+        LOG.trace("Found " + loc + " in cache for '" + tableName + "', row='"
+            + Bytes.toStringBinary(row) + "', locateType=" + RegionLocateType.BEFORE);
       }
       return loc;
     } else {
@@ -377,46 +361,41 @@ class AsyncNonMetaRegionLocator {
 
   private void locateInMeta(TableName tableName, LocateRequest req) {
     if (LOG.isTraceEnabled()) {
-      LOG.trace(
-        "Try locate '" + tableName + "', row='" + Bytes.toStringBinary(req.row) + "' in meta");
-    }
-    byte[] metaKey = createRegionName(tableName, req.row, NINES, false);
-    conn.getRawTable(META_TABLE_NAME)
-        .smallScan(new Scan(metaKey).setReversed(true).setSmall(true).addFamily(CATALOG_FAMILY), 1)
-        .whenComplete((results, error) -> onScanComplete(tableName, req, results, error, "row"));
-  }
-
-  private void locatePreviousInMeta(TableName tableName, LocateRequest req) {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Try locate '" + tableName + "', startRowOfCurrentRegion='"
-          + Bytes.toStringBinary(req.row) + "' in meta");
+      LOG.trace("Try locate '" + tableName + "', row='" + Bytes.toStringBinary(req.row)
+          + "', locateType=" + req.locateType + " in meta");
     }
     byte[] metaKey;
-    if (isEmptyStopRow(req.row)) {
-      byte[] binaryTableName = tableName.getName();
-      metaKey = Arrays.copyOf(binaryTableName, binaryTableName.length + 1);
+    if (req.locateType.equals(RegionLocateType.BEFORE)) {
+      if (isEmptyStopRow(req.row)) {
+        byte[] binaryTableName = tableName.getName();
+        metaKey = Arrays.copyOf(binaryTableName, binaryTableName.length + 1);
+      } else {
+        metaKey = createRegionName(tableName, req.row, ZEROES, false);
+      }
     } else {
-      metaKey = createRegionName(tableName, req.row, ZEROES, false);
+      metaKey = createRegionName(tableName, req.row, NINES, false);
     }
     conn.getRawTable(META_TABLE_NAME)
         .smallScan(new Scan(metaKey).setReversed(true).setSmall(true).addFamily(CATALOG_FAMILY), 1)
-        .whenComplete((results, error) -> onScanComplete(tableName, req, results, error,
-          "startRowOfCurrentRegion"));
+        .whenComplete((results, error) -> onScanComplete(tableName, req, results, error));
   }
 
   private HRegionLocation locateInCache(TableCache tableCache, TableName tableName, byte[] row,
-      boolean locateToPrevious) {
-    return locateToPrevious ? locatePreviousInCache(tableCache, tableName, row)
-        : locateInCache(tableCache, tableName, row);
+      RegionLocateType locateType) {
+    return locateType.equals(RegionLocateType.BEFORE)
+        ? locateRowBeforeInCache(tableCache, tableName, row)
+        : locateRowInCache(tableCache, tableName, row);
   }
 
   // locateToPrevious is true means we will use the start key of a region to locate the region
   // placed before it. Used for reverse scan. See the comment of
   // AsyncRegionLocator.getPreviousRegionLocation.
-  private CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-      boolean locateToPrevious) {
+  private CompletableFuture<HRegionLocation> getRegionLocationInternal(TableName tableName,
+      byte[] row, RegionLocateType locateType) {
+    // AFTER should be convert to CURRENT before calling this method
+    assert !locateType.equals(RegionLocateType.AFTER);
     TableCache tableCache = getTableCache(tableName);
-    HRegionLocation loc = locateInCache(tableCache, tableName, row, locateToPrevious);
+    HRegionLocation loc = locateInCache(tableCache, tableName, row, locateType);
     if (loc != null) {
       return CompletableFuture.completedFuture(loc);
     }
@@ -425,11 +404,11 @@ class AsyncNonMetaRegionLocator {
     boolean sendRequest = false;
     synchronized (tableCache) {
       // check again
-      loc = locateInCache(tableCache, tableName, row, locateToPrevious);
+      loc = locateInCache(tableCache, tableName, row, locateType);
       if (loc != null) {
         return CompletableFuture.completedFuture(loc);
       }
-      req = new LocateRequest(row, locateToPrevious);
+      req = new LocateRequest(row, locateType);
       future = tableCache.allRequests.get(req);
       if (future == null) {
         future = new CompletableFuture<>();
@@ -441,25 +420,23 @@ class AsyncNonMetaRegionLocator {
       }
     }
     if (sendRequest) {
-      if (locateToPrevious) {
-        locatePreviousInMeta(tableName, req);
-      } else {
-        locateInMeta(tableName, req);
-      }
+      locateInMeta(tableName, req);
     }
     return future;
   }
 
-  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row) {
-    return getRegionLocation(tableName, row, false);
-  }
-
-  // Used for reverse scan. See the comment of AsyncRegionLocator.getPreviousRegionLocation.
-  // TODO: need to deal with region merge where the startRowOfCurrentRegion will not be the endRow
-  // of a region.
-  CompletableFuture<HRegionLocation> getPreviousRegionLocation(TableName tableName,
-      byte[] startRowOfCurrentRegion) {
-    return getRegionLocation(tableName, startRowOfCurrentRegion, true);
+  CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
+      RegionLocateType locateType) {
+    if (locateType.equals(RegionLocateType.BEFORE)) {
+      return getRegionLocationInternal(tableName, row, locateType);
+    } else {
+      // as we know the exact row after us, so we can just create the new row, and use the same
+      // algorithm to locate it.
+      if (locateType.equals(RegionLocateType.AFTER)) {
+        row = createClosestRowAfter(row);
+      }
+      return getRegionLocationInternal(tableName, row, RegionLocateType.CURRENT);
+    }
   }
 
   void updateCachedLocation(HRegionLocation loc, Throwable exception) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 1c3569a..7a45ae3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -79,33 +79,18 @@ class AsyncRegionLocator {
   }
 
   CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-      long timeoutNs) {
+      RegionLocateType type, long timeoutNs) {
+    // meta region can not be split right now so we always call the same method.
+    // Change it later if the meta table can have more than one regions.
     CompletableFuture<HRegionLocation> future =
         tableName.equals(META_TABLE_NAME) ? metaRegionLocator.getRegionLocation()
-            : nonMetaRegionLocator.getRegionLocation(tableName, row);
+            : nonMetaRegionLocator.getRegionLocation(tableName, row, type);
     return withTimeout(future, timeoutNs,
       () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs)
           + "ms) waiting for region location for " + tableName + ", row='"
           + Bytes.toStringBinary(row) + "'");
   }
 
-  /**
-   * Locate the previous region using the current regions start key. Used for reverse scan as the
-   * end key is not included in a region so we need to treat it differently.
-   */
-  CompletableFuture<HRegionLocation> getPreviousRegionLocation(TableName tableName,
-      byte[] startRowOfCurrentRegion, long timeoutNs) {
-    // meta region can not be split right now so we call the same method as getRegionLocation.
-    // Change it later if the meta table can have more than one regions.
-    CompletableFuture<HRegionLocation> future =
-        tableName.equals(META_TABLE_NAME) ? metaRegionLocator.getRegionLocation()
-            : nonMetaRegionLocator.getPreviousRegionLocation(tableName, startRowOfCurrentRegion);
-    return withTimeout(future, timeoutNs,
-      () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs)
-          + "ms) waiting for region location for " + tableName + ", startRowOfCurrentRegion='"
-          + Bytes.toStringBinary(startRowOfCurrentRegion) + "'");
-  }
-
   static boolean canUpdate(HRegionLocation loc, HRegionLocation oldLoc) {
     // Do not need to update if no such location, or the location is newer, or the location is not
     // same with us

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
index f1a4247..d240fab 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java
@@ -58,7 +58,7 @@ class AsyncRpcRetryingCallerFactory {
 
     private long rpcTimeoutNs = -1L;
 
-    private boolean locateToPreviousRegion;
+    private RegionLocateType locateType = RegionLocateType.CURRENT;
 
     public SingleRequestCallerBuilder<T> table(TableName tableName) {
       this.tableName = tableName;
@@ -86,15 +86,15 @@ class AsyncRpcRetryingCallerFactory {
       return this;
     }
 
-    public SingleRequestCallerBuilder<T> locateToPreviousRegion(boolean locateToPreviousRegion) {
-      this.locateToPreviousRegion = locateToPreviousRegion;
+    public SingleRequestCallerBuilder<T> locateType(RegionLocateType locateType) {
+      this.locateType = locateType;
       return this;
     }
 
     public AsyncSingleRequestRpcRetryingCaller<T> build() {
       return new AsyncSingleRequestRpcRetryingCaller<>(retryTimer, conn,
           checkNotNull(tableName, "tableName is null"), checkNotNull(row, "row is null"),
-          locateToPreviousRegion, checkNotNull(callable, "action is null"),
+          checkNotNull(locateType, "locateType is null"), checkNotNull(callable, "action is null"),
           conn.connConf.getPauseNs(), conn.connConf.getMaxRetries(), operationTimeoutNs,
           rpcTimeoutNs, conn.connConf.getStartLogErrorsCnt());
     }
@@ -246,7 +246,7 @@ class AsyncRpcRetryingCallerFactory {
     /**
      * Short cut for {@code build().start()}.
      */
-    public CompletableFuture<Boolean> start() {
+    public CompletableFuture<RegionLocateType> start() {
       return build().start();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
index ca83a51..81c806f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
@@ -95,7 +95,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
 
   private final Runnable completeWhenNoMoreResultsInRegion;
 
-  private final CompletableFuture<Boolean> future;
+  private final CompletableFuture<RegionLocateType> future;
 
   private final HBaseRpcController controller;
 
@@ -172,7 +172,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
 
   private void completeWithNextStartRow(byte[] nextStartRow) {
     scan.setStartRow(nextStartRow);
-    future.complete(scan.isReversed());
+    future.complete(scan.isReversed() ? RegionLocateType.BEFORE : RegionLocateType.CURRENT);
   }
 
   private byte[] createNextStartRowWhenError() {
@@ -193,7 +193,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {
         includeNextStartRowWhenError ? nextStartRowWhenError : createNextStartRowWhenError.get());
     }
     future.complete(
-      scan.isReversed() && Bytes.equals(scan.getStartRow(), loc.getRegionInfo().getEndKey()));
+      scan.isReversed() && Bytes.equals(scan.getStartRow(), loc.getRegionInfo().getEndKey())
+          ? RegionLocateType.BEFORE : RegionLocateType.CURRENT);
   }
 
   private void onError(Throwable error) {
@@ -344,7 +345,7 @@ class AsyncScanSingleRegionRpcRetryingCaller {
   /**
    * @return return locate direction for next open scanner call, or null if we should stop.
    */
-  public CompletableFuture<Boolean> start() {
+  public CompletableFuture<RegionLocateType> start() {
     next();
     return future;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
index d6da131..0b4add1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java
@@ -31,7 +31,6 @@ import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
-import java.util.function.Function;
 import java.util.function.Supplier;
 
 import org.apache.commons.logging.Log;
@@ -67,7 +66,7 @@ class AsyncSingleRequestRpcRetryingCaller<T> {
 
   private final byte[] row;
 
-  private final Function<Long, CompletableFuture<HRegionLocation>> locate;
+  private final RegionLocateType locateType;
 
   private final Callable<T> callable;
 
@@ -90,18 +89,14 @@ class AsyncSingleRequestRpcRetryingCaller<T> {
   private final long startNs;
 
   public AsyncSingleRequestRpcRetryingCaller(HashedWheelTimer retryTimer, AsyncConnectionImpl conn,
-      TableName tableName, byte[] row, boolean locateToPreviousRegion, Callable<T> callable,
+      TableName tableName, byte[] row, RegionLocateType locateType, Callable<T> callable,
       long pauseNs, int maxRetries, long operationTimeoutNs, long rpcTimeoutNs,
       int startLogErrorsCnt) {
     this.retryTimer = retryTimer;
     this.conn = conn;
     this.tableName = tableName;
     this.row = row;
-    if (locateToPreviousRegion) {
-      this.locate = this::locatePrevious;
-    } else {
-      this.locate = this::locate;
-    }
+    this.locateType = locateType;
     this.callable = callable;
     this.pauseNs = pauseNs;
     this.maxAttempts = retries2Attempts(maxRetries);
@@ -210,27 +205,20 @@ class AsyncSingleRequestRpcRetryingCaller<T> {
     } else {
       locateTimeoutNs = -1L;
     }
-    locate.apply(locateTimeoutNs).whenComplete((loc, error) -> {
-      if (error != null) {
-        onError(error,
-          () -> "Locate '" + Bytes.toStringBinary(row) + "' in " + tableName + " failed, tries = "
-              + tries + ", maxAttempts = " + maxAttempts + ", timeout = "
-              + TimeUnit.NANOSECONDS.toMillis(operationTimeoutNs) + " ms, time elapsed = "
-              + elapsedMs() + " ms",
-          err -> {
-          });
-        return;
-      }
-      call(loc);
-    });
-  }
-
-  private CompletableFuture<HRegionLocation> locate(long timeoutNs) {
-    return conn.getLocator().getRegionLocation(tableName, row, timeoutNs);
-  }
-
-  private CompletableFuture<HRegionLocation> locatePrevious(long timeoutNs) {
-    return conn.getLocator().getPreviousRegionLocation(tableName, row, timeoutNs);
+    conn.getLocator().getRegionLocation(tableName, row, locateType, locateTimeoutNs)
+        .whenComplete((loc, error) -> {
+          if (error != null) {
+            onError(error,
+              () -> "Locate '" + Bytes.toStringBinary(row) + "' in " + tableName
+                  + " failed, tries = " + tries + ", maxAttempts = " + maxAttempts + ", timeout = "
+                  + TimeUnit.NANOSECONDS.toMillis(operationTimeoutNs) + " ms, time elapsed = "
+                  + elapsedMs() + " ms",
+              err -> {
+              });
+            return;
+          }
+          call(loc);
+        });
   }
 
   public CompletableFuture<T> call() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java
index af639c0..c4c2074 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSmallScanRpcRetryingCaller.java
@@ -144,7 +144,7 @@ class AsyncSmallScanRpcRetryingCaller {
         scan.setStartRow(
           createClosestNextRow.apply(resp.results[resp.results.length - 1].getRow()));
       }
-      scan(false);
+      scan(RegionLocateType.CURRENT);
       return;
     }
     if (!nextScan.apply(resp.currentRegion)) {
@@ -152,12 +152,11 @@ class AsyncSmallScanRpcRetryingCaller {
     }
   }
 
-  private void scan(boolean locateToPreviousRegion) {
+  private void scan(RegionLocateType locateType) {
     conn.callerFactory.<SmallScanResponse> single().table(tableName).row(scan.getStartRow())
         .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
-        .operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS)
-        .locateToPreviousRegion(locateToPreviousRegion).action(this::scan).call()
-        .whenComplete((resp, error) -> {
+        .operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).locateType(locateType)
+        .action(this::scan).call().whenComplete((resp, error) -> {
           if (error != null) {
             future.completeExceptionally(error);
           } else {
@@ -172,11 +171,11 @@ class AsyncSmallScanRpcRetryingCaller {
   }
 
   private void firstScan() {
-    scan(false);
+    scan(RegionLocateType.CURRENT);
   }
 
   private void reversedFirstScan() {
-    scan(isEmptyStartRow(scan.getStartRow()));
+    scan(isEmptyStartRow(scan.getStartRow()) ? RegionLocateType.BEFORE : RegionLocateType.CURRENT);
   }
 
   private boolean nextScan(HRegionInfo region) {
@@ -190,7 +189,7 @@ class AsyncSmallScanRpcRetryingCaller {
       }
     }
     scan.setStartRow(region.getEndKey());
-    scan(false);
+    scan(RegionLocateType.CURRENT);
     return true;
   }
 
@@ -205,7 +204,7 @@ class AsyncSmallScanRpcRetryingCaller {
       }
     }
     scan.setStartRow(region.getStartKey());
-    scan(true);
+    scan(RegionLocateType.BEFORE);
     return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
index e1f40a7..1986962 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
@@ -45,6 +45,6 @@ class AsyncTableRegionLocatorImpl implements AsyncTableRegionLocator {
 
   @Override
   public CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, boolean reload) {
-    return locator.getRegionLocation(tableName, row, 0L);
+    return locator.getRegionLocation(tableName, row, RegionLocateType.CURRENT, -1L);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocateType.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocateType.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocateType.java
new file mode 100644
index 0000000..57f9498
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLocateType.java
@@ -0,0 +1,33 @@
+/**
+ * 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;
+
+/**
+ * Indicate which row you want to locate.
+ * <ul>
+ * <li>{@link #BEFORE} locate the region which contains the row before the given row.</li>
+ * <li>{@link #CURRENT} locate the region which contains the given row.</li>
+ * <li>{@link #AFTER} locate the region which contains the row after the given row.</li>
+ * </ul>
+ */
+@InterfaceAudience.Private
+enum RegionLocateType {
+  BEFORE, CURRENT, AFTER
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
index f3aa26b..40fca72 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -102,15 +101,12 @@ public class TestAsyncNonMetaRegionLocator {
 
   @Test
   public void testNoTable() throws InterruptedException {
-    try {
-      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW).get();
-    } catch (ExecutionException e) {
-      assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
-    }
-    try {
-      LOCATOR.getPreviousRegionLocation(TABLE_NAME, EMPTY_END_ROW).get();
-    } catch (ExecutionException e) {
-      assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
+    for (RegionLocateType locateType : RegionLocateType.values()) {
+      try {
+        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType).get();
+      } catch (ExecutionException e) {
+        assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
+      }
     }
   }
 
@@ -118,15 +114,12 @@ public class TestAsyncNonMetaRegionLocator {
   public void testDisableTable() throws IOException, InterruptedException {
     createSingleRegionTable();
     TEST_UTIL.getAdmin().disableTable(TABLE_NAME);
-    try {
-      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW).get();
-    } catch (ExecutionException e) {
-      assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
-    }
-    try {
-      LOCATOR.getPreviousRegionLocation(TABLE_NAME, EMPTY_END_ROW).get();
-    } catch (ExecutionException e) {
-      assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
+    for (RegionLocateType locateType : RegionLocateType.values()) {
+      try {
+        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType).get();
+      } catch (ExecutionException e) {
+        assertThat(e.getCause(), instanceOf(TableNotFoundException.class));
+      }
     }
   }
 
@@ -143,21 +136,15 @@ public class TestAsyncNonMetaRegionLocator {
   public void testSingleRegionTable() throws IOException, InterruptedException, ExecutionException {
     createSingleRegionTable();
     ServerName serverName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
-    assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
-      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW).get());
-    assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
-      LOCATOR.getPreviousRegionLocation(TABLE_NAME, EMPTY_START_ROW).get());
+    for (RegionLocateType locateType : RegionLocateType.values()) {
+      assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
+        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, locateType).get());
+    }
     byte[] randKey = new byte[ThreadLocalRandom.current().nextInt(128)];
     ThreadLocalRandom.current().nextBytes(randKey);
-    assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
-      LOCATOR.getRegionLocation(TABLE_NAME, randKey).get());
-    // Use a key which is not the endKey of a region will cause error
-    try {
+    for (RegionLocateType locateType : RegionLocateType.values()) {
       assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName,
-        LOCATOR.getPreviousRegionLocation(TABLE_NAME, new byte[] { 1 }).get());
-    } catch (ExecutionException e) {
-      assertThat(e.getCause(), instanceOf(IOException.class));
-      assertTrue(e.getCause().getMessage().contains("end key of"));
+        LOCATOR.getRegionLocation(TABLE_NAME, randKey, locateType).get());
     }
   }
 
@@ -194,7 +181,19 @@ public class TestAsyncNonMetaRegionLocator {
     IntStream.range(0, 2).forEach(n -> IntStream.range(0, startKeys.length).forEach(i -> {
       try {
         assertLocEquals(startKeys[i], i == startKeys.length - 1 ? EMPTY_END_ROW : startKeys[i + 1],
-          serverNames[i], LOCATOR.getRegionLocation(TABLE_NAME, startKeys[i]).get());
+          serverNames[i],
+          LOCATOR.getRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.CURRENT).get());
+      } catch (InterruptedException | ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    }));
+
+    LOCATOR.clearCache(TABLE_NAME);
+    IntStream.range(0, 2).forEach(n -> IntStream.range(0, startKeys.length).forEach(i -> {
+      try {
+        assertLocEquals(startKeys[i], i == startKeys.length - 1 ? EMPTY_END_ROW : startKeys[i + 1],
+          serverNames[i],
+          LOCATOR.getRegionLocation(TABLE_NAME, startKeys[i], RegionLocateType.AFTER).get());
       } catch (InterruptedException | ExecutionException e) {
         throw new RuntimeException(e);
       }
@@ -205,7 +204,7 @@ public class TestAsyncNonMetaRegionLocator {
       n -> IntStream.range(0, endKeys.length).map(i -> endKeys.length - 1 - i).forEach(i -> {
         try {
           assertLocEquals(i == 0 ? EMPTY_START_ROW : endKeys[i - 1], endKeys[i], serverNames[i],
-            LOCATOR.getPreviousRegionLocation(TABLE_NAME, endKeys[i]).get());
+            LOCATOR.getRegionLocation(TABLE_NAME, endKeys[i], RegionLocateType.BEFORE).get());
         } catch (InterruptedException | ExecutionException e) {
           throw new RuntimeException(e);
         }
@@ -216,7 +215,8 @@ public class TestAsyncNonMetaRegionLocator {
   public void testRegionMove() throws IOException, InterruptedException, ExecutionException {
     createSingleRegionTable();
     ServerName serverName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
-    HRegionLocation loc = LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW).get();
+    HRegionLocation loc =
+        LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT).get();
     assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, serverName, loc);
     ServerName newServerName = TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream()
         .map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName))
@@ -229,12 +229,39 @@ public class TestAsyncNonMetaRegionLocator {
       Thread.sleep(100);
     }
     // Should be same as it is in cache
-    assertSame(loc, LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW).get());
+    assertSame(loc,
+      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT).get());
     LOCATOR.updateCachedLocation(loc, null);
     // null error will not trigger a cache cleanup
-    assertSame(loc, LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW).get());
+    assertSame(loc,
+      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT).get());
     LOCATOR.updateCachedLocation(loc, new NotServingRegionException());
     assertLocEquals(EMPTY_START_ROW, EMPTY_END_ROW, newServerName,
-      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW).get());
+      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT).get());
+  }
+
+  // usually locate after will return the same result, so we add a test to make it return different
+  // result.
+  @Test
+  public void testLocateAfter() throws IOException, InterruptedException, ExecutionException {
+    byte[] row = Bytes.toBytes("1");
+    byte[] splitKey = Arrays.copyOf(row, 2);
+    TEST_UTIL.createTable(TABLE_NAME, FAMILY, new byte[][] { splitKey });
+    TEST_UTIL.waitTableAvailable(TABLE_NAME);
+    HRegionLocation currentLoc =
+        LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.CURRENT).get();
+    ServerName currentServerName = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME).getServerName();
+    assertLocEquals(EMPTY_START_ROW, splitKey, currentServerName, currentLoc);
+
+    HRegionLocation afterLoc =
+        LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER).get();
+    ServerName afterServerName =
+        TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
+            .filter(rs -> rs.getOnlineRegions(TABLE_NAME).stream()
+                .anyMatch(r -> Bytes.equals(splitKey, r.getRegionInfo().getStartKey())))
+            .findAny().get().getServerName();
+    assertLocEquals(splitKey, EMPTY_END_ROW, afterServerName, afterLoc);
+
+    assertSame(afterLoc, LOCATOR.getRegionLocation(TABLE_NAME, row, RegionLocateType.AFTER).get());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
index e82703b..3918dc9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
@@ -17,9 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static java.util.stream.Collectors.toCollection;
 import static java.util.stream.Collectors.toList;
-import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
 import static org.apache.hadoop.hbase.client.AsyncNonMetaRegionLocator.MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
@@ -28,7 +26,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
@@ -147,12 +144,10 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
 
   @Test
   public void test() throws InterruptedException, ExecutionException {
-    List<CompletableFuture<HRegionLocation>> futures = IntStream.range(0, 128)
-        .mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
-        .map(r -> LOCATOR.getRegionLocation(TABLE_NAME, r)).collect(toCollection(ArrayList::new));
-    futures.addAll(IntStream.range(129, 257)
-        .mapToObj(i -> i < 256 ? Bytes.toBytes(String.format("%02x", i)) : EMPTY_START_ROW)
-        .map(r -> LOCATOR.getPreviousRegionLocation(TABLE_NAME, r)).collect(toList()));
+    List<CompletableFuture<HRegionLocation>> futures =
+        IntStream.range(0, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
+            .map(r -> LOCATOR.getRegionLocation(TABLE_NAME, r, RegionLocateType.CURRENT))
+            .collect(toList());
     assertLocs(futures);
     assertTrue(MAX_CONCURRENCY.get() <= MAX_ALLOWED);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
index 40190cb..dfefcc7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTimeout.java
@@ -100,8 +100,8 @@ public class TestAsyncRegionLocatorTimeout {
     SLEEP_MS = 1000;
     long startNs = System.nanoTime();
     try {
-      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, TimeUnit.MILLISECONDS.toNanos(500))
-          .get();
+      LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW, RegionLocateType.CURRENT,
+        TimeUnit.MILLISECONDS.toNanos(500)).get();
       fail();
     } catch (ExecutionException e) {
       e.printStackTrace();
@@ -113,8 +113,8 @@ public class TestAsyncRegionLocatorTimeout {
     // wait for the background task finish
     Thread.sleep(2000);
     // Now the location should be in cache, so we will not visit meta again.
-    HRegionLocation loc = LOCATOR
-        .getRegionLocation(TABLE_NAME, EMPTY_START_ROW, TimeUnit.MILLISECONDS.toNanos(500)).get();
+    HRegionLocation loc = LOCATOR.getRegionLocation(TABLE_NAME, EMPTY_START_ROW,
+      RegionLocateType.CURRENT, TimeUnit.MILLISECONDS.toNanos(500)).get();
     assertEquals(loc.getServerName(),
       TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName());
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/09bb4287/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
index f76e240..4a391e0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java
@@ -154,7 +154,7 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
         new AsyncRegionLocator(asyncConn, AsyncConnectionImpl.RETRY_TIMER) {
           @Override
           CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-              long timeoutNs) {
+              RegionLocateType locateType, long timeoutNs) {
             if (tableName.equals(TABLE_NAME)) {
               CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
               if (count.getAndIncrement() == 0) {
@@ -165,17 +165,11 @@ public class TestAsyncSingleRequestRpcRetryingCaller {
               }
               return future;
             } else {
-              return super.getRegionLocation(tableName, row, timeoutNs);
+              return super.getRegionLocation(tableName, row, locateType, timeoutNs);
             }
           }
 
           @Override
-          CompletableFuture<HRegionLocation> getPreviousRegionLocation(TableName tableName,
-              byte[] startRowOfCurrentRegion, long timeoutNs) {
-            return super.getPreviousRegionLocation(tableName, startRowOfCurrentRegion, timeoutNs);
-          }
-
-          @Override
           void updateCachedLocation(HRegionLocation loc, Throwable exception) {
           }
         };