saintstack commented on a change in pull request #2584:
URL: https://github.com/apache/hbase/pull/2584#discussion_r512903424



##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -89,7 +90,10 @@
 
   private final int locatePrefetchLimit;
 
-  private final boolean useMetaReplicas;
+  // When useMetaReplicas is true, the mode tells if HedgedRead, LoadBalance 
mode is supported.

Review comment:
       Comment is right? There is no useMetaReplicas here anymore.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -196,8 +200,33 @@ private boolean tryComplete(LocateRequest req, 
CompletableFuture<RegionLocations
       MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, 
DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE);
     this.locatePrefetchLimit =
       conn.getConfiguration().getInt(LOCATE_PREFETCH_LIMIT, 
DEFAULT_LOCATE_PREFETCH_LIMIT);
-    this.useMetaReplicas =
-      conn.getConfiguration().getBoolean(USE_META_REPLICAS, 
DEFAULT_USE_META_REPLICAS);
+
+    // Get the region locator's meta replica mode.
+    this.metaReplicaMode = CatalogReplicaMode.valueOf(conn.getConfiguration()
+      .get(LOCATOR_META_REPLICAS_MODE, CatalogReplicaMode.None.toString()));
+
+    switch (this.metaReplicaMode) {
+      case LoadBalance:
+        String replicaSelectorClass = conn.getConfiguration().
+          get(RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR,
+          CatalogReplicaLoadBalanceReplicaSimpleSelector.class.getName());
+
+        this.metaReplicaSelector = 
CatalogReplicaLoadBalanceReplicaSelectorFactory.createSelector(
+          replicaSelectorClass, META_TABLE_NAME, conn);
+        break;
+      case None:
+        // If user does not configure LOCATOR_META_REPLICAS_MODE, let's check 
the legacy config.
+        if (this.metaReplicaMode == CatalogReplicaMode.None) {
+          boolean userMetaReplicas = 
conn.getConfiguration().getBoolean(USE_META_REPLICAS,

Review comment:
       Use old name? s/userMetaReplicas/useMetaReplicas/

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -196,8 +200,33 @@ private boolean tryComplete(LocateRequest req, 
CompletableFuture<RegionLocations
       MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, 
DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE);
     this.locatePrefetchLimit =
       conn.getConfiguration().getInt(LOCATE_PREFETCH_LIMIT, 
DEFAULT_LOCATE_PREFETCH_LIMIT);
-    this.useMetaReplicas =
-      conn.getConfiguration().getBoolean(USE_META_REPLICAS, 
DEFAULT_USE_META_REPLICAS);
+
+    // Get the region locator's meta replica mode.
+    this.metaReplicaMode = CatalogReplicaMode.valueOf(conn.getConfiguration()
+      .get(LOCATOR_META_REPLICAS_MODE, CatalogReplicaMode.None.toString()));
+
+    switch (this.metaReplicaMode) {
+      case LoadBalance:
+        String replicaSelectorClass = conn.getConfiguration().
+          get(RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR,
+          CatalogReplicaLoadBalanceReplicaSimpleSelector.class.getName());
+
+        this.metaReplicaSelector = 
CatalogReplicaLoadBalanceReplicaSelectorFactory.createSelector(
+          replicaSelectorClass, META_TABLE_NAME, conn);
+        break;
+      case None:
+        // If user does not configure LOCATOR_META_REPLICAS_MODE, let's check 
the legacy config.
+        if (this.metaReplicaMode == CatalogReplicaMode.None) {
+          boolean userMetaReplicas = 
conn.getConfiguration().getBoolean(USE_META_REPLICAS,
+            DEFAULT_USE_META_REPLICAS);
+          if (userMetaReplicas) {
+            this.metaReplicaMode = CatalogReplicaMode.HedgedRead;
+          }
+        }
+        break;
+      default:
+        // Doing nothing

Review comment:
       Should at least log here if we cannot do anything w/ the config?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -196,8 +200,33 @@ private boolean tryComplete(LocateRequest req, 
CompletableFuture<RegionLocations
       MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, 
DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE);
     this.locatePrefetchLimit =
       conn.getConfiguration().getInt(LOCATE_PREFETCH_LIMIT, 
DEFAULT_LOCATE_PREFETCH_LIMIT);
-    this.useMetaReplicas =
-      conn.getConfiguration().getBoolean(USE_META_REPLICAS, 
DEFAULT_USE_META_REPLICAS);
+
+    // Get the region locator's meta replica mode.
+    this.metaReplicaMode = CatalogReplicaMode.valueOf(conn.getConfiguration()
+      .get(LOCATOR_META_REPLICAS_MODE, CatalogReplicaMode.None.toString()));
+
+    switch (this.metaReplicaMode) {
+      case LoadBalance:
+        String replicaSelectorClass = conn.getConfiguration().
+          get(RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR,
+          CatalogReplicaLoadBalanceReplicaSimpleSelector.class.getName());
+
+        this.metaReplicaSelector = 
CatalogReplicaLoadBalanceReplicaSelectorFactory.createSelector(
+          replicaSelectorClass, META_TABLE_NAME, conn);
+        break;
+      case None:
+        // If user does not configure LOCATOR_META_REPLICAS_MODE, let's check 
the legacy config.
+        if (this.metaReplicaMode == CatalogReplicaMode.None) {
+          boolean userMetaReplicas = 
conn.getConfiguration().getBoolean(USE_META_REPLICAS,
+            DEFAULT_USE_META_REPLICAS);
+          if (userMetaReplicas) {
+            this.metaReplicaMode = CatalogReplicaMode.HedgedRead;
+          }
+        }
+        break;
+      default:
+        // Doing nothing
+    }

Review comment:
       Should the mode implementation be internal to the CatalogReplicaMode 
class and not exposed in here in the locator?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSelectorFactory.java
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Factory to create a {@link CatalogReplicaLoadBalanceReplicaSelector}
+ */
+@InterfaceAudience.Private
+public final class CatalogReplicaLoadBalanceReplicaSelectorFactory {

Review comment:
       Yeah, don't need second 'Replica' in the name.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements
+  CatalogReplicaLoadBalanceReplicaSelector {
+  private static final Logger LOG =
+    
LoggerFactory.getLogger(CatalogReplicaLoadBalanceReplicaSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {

Review comment:
       Do these instances compare ? Do we need to impl hash and equals?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements
+  CatalogReplicaLoadBalanceReplicaSelector {
+  private static final Logger LOG =
+    
LoggerFactory.getLogger(CatalogReplicaLoadBalanceReplicaSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {
+    // replica id where the stale location comes from.
+    private int fromReplicaId;
+
+    // timestamp in milliseconds
+    private long timestamp;

Review comment:
       This gets updated? If not final?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -196,8 +200,33 @@ private boolean tryComplete(LocateRequest req, 
CompletableFuture<RegionLocations
       MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, 
DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE);
     this.locatePrefetchLimit =
       conn.getConfiguration().getInt(LOCATE_PREFETCH_LIMIT, 
DEFAULT_LOCATE_PREFETCH_LIMIT);
-    this.useMetaReplicas =
-      conn.getConfiguration().getBoolean(USE_META_REPLICAS, 
DEFAULT_USE_META_REPLICAS);
+
+    // Get the region locator's meta replica mode.
+    this.metaReplicaMode = CatalogReplicaMode.valueOf(conn.getConfiguration()
+      .get(LOCATOR_META_REPLICAS_MODE, CatalogReplicaMode.None.toString()));
+
+    switch (this.metaReplicaMode) {
+      case LoadBalance:
+        String replicaSelectorClass = conn.getConfiguration().
+          get(RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR,
+          CatalogReplicaLoadBalanceReplicaSimpleSelector.class.getName());
+
+        this.metaReplicaSelector = 
CatalogReplicaLoadBalanceReplicaSelectorFactory.createSelector(
+          replicaSelectorClass, META_TABLE_NAME, conn);
+        break;
+      case None:
+        // If user does not configure LOCATOR_META_REPLICAS_MODE, let's check 
the legacy config.
+        if (this.metaReplicaMode == CatalogReplicaMode.None) {
+          boolean userMetaReplicas = 
conn.getConfiguration().getBoolean(USE_META_REPLICAS,
+            DEFAULT_USE_META_REPLICAS);
+          if (userMetaReplicas) {
+            this.metaReplicaMode = CatalogReplicaMode.HedgedRead;

Review comment:
       Should this be CatalogReplicaMode.DEFAULT? Rather than refer here 
directly to HedgedRead?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements
+  CatalogReplicaLoadBalanceReplicaSelector {
+  private static final Logger LOG =
+    
LoggerFactory.getLogger(CatalogReplicaLoadBalanceReplicaSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {
+    // replica id where the stale location comes from.
+    private int fromReplicaId;
+
+    // timestamp in milliseconds
+    private long timestamp;
+
+    private byte[] endKey;
+
+    StaleLocationCacheEntry(final int metaReplicaId, final byte[] endKey) {
+      this.fromReplicaId = metaReplicaId;
+      this.endKey = endKey;
+      timestamp = System.currentTimeMillis();
+    }
+
+    public byte[] getEndKey() {
+      return this.endKey;
+    }
+
+    public int getFromReplicaId() {
+      return this.fromReplicaId;
+    }
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+
+    @Override
+    public String toString() {
+      return new ToStringBuilder(this)
+        .append("endKey", endKey)
+        .append("fromReplicaId", fromReplicaId)
+        .append("timestamp", timestamp)
+        .toString();
+    }
+  }
+
+  private static final class StaleTableCache {
+    private final ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> 
cache =
+      new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
+  }
+
+  private final ConcurrentMap<TableName, StaleTableCache> staleCache;
+  private int numOfReplicas;
+  private final AsyncConnectionImpl conn;
+  private TableName tableName;
+
+  CatalogReplicaLoadBalanceReplicaSimpleSelector(TableName tableName, 
AsyncConnectionImpl conn) {
+    staleCache = new ConcurrentHashMap<>();
+    this.conn = conn;
+    this.tableName = tableName;
+
+    // This numOfReplicas is going to be lazy initialized.
+    this.numOfReplicas = -1;
+    // Start connection's chore service in case.
+    this.conn.startChoreService();
+    this.conn.getChoreService().scheduleChore(getCacheCleanupChore(this));

Review comment:
       Any chance of our double starting or making many instances of this 
chore? Any protection?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements
+  CatalogReplicaLoadBalanceReplicaSelector {
+  private static final Logger LOG =
+    
LoggerFactory.getLogger(CatalogReplicaLoadBalanceReplicaSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {
+    // replica id where the stale location comes from.
+    private int fromReplicaId;

Review comment:
       final?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -577,6 +624,15 @@ private void removeLocationFromCache(HRegionLocation loc) {
       if (!canUpdateOnError(loc, oldLoc)) {
         return;
       }
+      // Tell metaReplicaChooser that the location is stale. It will create a 
stale entry
+      // with timestamp internally. Next time the client looks up the same 
location,
+      // it will pick a different meta replica region. For the current 
implementation,
+      // the metaReplicaId is not used, so the primary one is passed in.
+      if (this.metaReplicaMode == CatalogReplicaMode.LoadBalance) {

Review comment:
       This sort of detail could be hidden inside the CatalogReplicaMode? 

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements
+  CatalogReplicaLoadBalanceReplicaSelector {
+  private static final Logger LOG =
+    
LoggerFactory.getLogger(CatalogReplicaLoadBalanceReplicaSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {
+    // replica id where the stale location comes from.
+    private int fromReplicaId;
+
+    // timestamp in milliseconds
+    private long timestamp;
+
+    private byte[] endKey;
+
+    StaleLocationCacheEntry(final int metaReplicaId, final byte[] endKey) {
+      this.fromReplicaId = metaReplicaId;
+      this.endKey = endKey;
+      timestamp = System.currentTimeMillis();
+    }
+
+    public byte[] getEndKey() {
+      return this.endKey;
+    }
+
+    public int getFromReplicaId() {
+      return this.fromReplicaId;
+    }
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+
+    @Override
+    public String toString() {
+      return new ToStringBuilder(this)
+        .append("endKey", endKey)
+        .append("fromReplicaId", fromReplicaId)
+        .append("timestamp", timestamp)
+        .toString();
+    }
+  }
+
+  private static final class StaleTableCache {
+    private final ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> 
cache =
+      new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
+  }
+
+  private final ConcurrentMap<TableName, StaleTableCache> staleCache;
+  private int numOfReplicas;
+  private final AsyncConnectionImpl conn;
+  private TableName tableName;

Review comment:
       Do these change? If not, set final

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -577,6 +624,15 @@ private void removeLocationFromCache(HRegionLocation loc) {
       if (!canUpdateOnError(loc, oldLoc)) {
         return;
       }
+      // Tell metaReplicaChooser that the location is stale. It will create a 
stale entry

Review comment:
       s/metaReplicaChooser/metaReplicaSelector/

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSelector.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A Catalog replica selector decides which catalog replica to go for read 
requests.
+ */
+@InterfaceAudience.Private
+public interface CatalogReplicaLoadBalanceReplicaSelector {

Review comment:
       Make it package private? We use 'Replica' twice in the name? Can we drop 
the second one?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSelector.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A Catalog replica selector decides which catalog replica to go for read 
requests.
+ */
+@InterfaceAudience.Private
+public interface CatalogReplicaLoadBalanceReplicaSelector {
+
+  /**
+   *  This method is called upon when input location is stale.
+   * @param loc location which is stale.
+   * @param fromReplicaId which replica the stale location comes from.
+   */
+  void notifyOnError(HRegionLocation loc, int fromReplicaId);
+
+  /**
+   * Select a catalog replica region to loop up the input row.
+   *
+   * @param tablename table name.
+   * @param row  key to look up
+   * @param locateType  locate type.
+   * @return
+   */
+  int selectReplica(TableName tablename, byte[] row, RegionLocateType 
locateType);

Review comment:
       Could name this 'select'. The 'replica' seems redundant in this class. 
The @return could say returns replica id.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements
+  CatalogReplicaLoadBalanceReplicaSelector {
+  private static final Logger LOG =
+    
LoggerFactory.getLogger(CatalogReplicaLoadBalanceReplicaSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {
+    // replica id where the stale location comes from.
+    private int fromReplicaId;
+
+    // timestamp in milliseconds
+    private long timestamp;
+
+    private byte[] endKey;

Review comment:
       final? These don't change post-construction?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements

Review comment:
       SimpleCatalogReplicaLoadalanceSelector ?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -196,8 +200,33 @@ private boolean tryComplete(LocateRequest req, 
CompletableFuture<RegionLocations
       MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, 
DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE);
     this.locatePrefetchLimit =
       conn.getConfiguration().getInt(LOCATE_PREFETCH_LIMIT, 
DEFAULT_LOCATE_PREFETCH_LIMIT);
-    this.useMetaReplicas =
-      conn.getConfiguration().getBoolean(USE_META_REPLICAS, 
DEFAULT_USE_META_REPLICAS);
+
+    // Get the region locator's meta replica mode.
+    this.metaReplicaMode = CatalogReplicaMode.valueOf(conn.getConfiguration()
+      .get(LOCATOR_META_REPLICAS_MODE, CatalogReplicaMode.None.toString()));
+
+    switch (this.metaReplicaMode) {
+      case LoadBalance:
+        String replicaSelectorClass = conn.getConfiguration().
+          get(RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR,
+          CatalogReplicaLoadBalanceReplicaSimpleSelector.class.getName());
+
+        this.metaReplicaSelector = 
CatalogReplicaLoadBalanceReplicaSelectorFactory.createSelector(
+          replicaSelectorClass, META_TABLE_NAME, conn);
+        break;
+      case None:
+        // If user does not configure LOCATOR_META_REPLICAS_MODE, let's check 
the legacy config.
+        if (this.metaReplicaMode == CatalogReplicaMode.None) {
+          boolean userMetaReplicas = 
conn.getConfiguration().getBoolean(USE_META_REPLICAS,
+            DEFAULT_USE_META_REPLICAS);
+          if (userMetaReplicas) {
+            this.metaReplicaMode = CatalogReplicaMode.HedgedRead;
+          }
+        }
+        break;
+      default:
+        // Doing nothing

Review comment:
       If user misspells the config?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -577,6 +624,15 @@ private void removeLocationFromCache(HRegionLocation loc) {
       if (!canUpdateOnError(loc, oldLoc)) {
         return;
       }
+      // Tell metaReplicaChooser that the location is stale. It will create a 
stale entry
+      // with timestamp internally. Next time the client looks up the same 
location,
+      // it will pick a different meta replica region. For the current 
implementation,
+      // the metaReplicaId is not used, so the primary one is passed in.
+      if (this.metaReplicaMode == CatalogReplicaMode.LoadBalance) {

Review comment:
       hmmm... now I see CRM is just an enum.... but a CatalogReplica class 
that internally implements mode rather than scatter the processing here around 
locator? Can do later.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements
+  CatalogReplicaLoadBalanceReplicaSelector {
+  private static final Logger LOG =
+    
LoggerFactory.getLogger(CatalogReplicaLoadBalanceReplicaSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {
+    // replica id where the stale location comes from.
+    private int fromReplicaId;
+
+    // timestamp in milliseconds
+    private long timestamp;
+
+    private byte[] endKey;
+
+    StaleLocationCacheEntry(final int metaReplicaId, final byte[] endKey) {
+      this.fromReplicaId = metaReplicaId;
+      this.endKey = endKey;
+      timestamp = System.currentTimeMillis();
+    }
+
+    public byte[] getEndKey() {
+      return this.endKey;
+    }
+
+    public int getFromReplicaId() {
+      return this.fromReplicaId;
+    }
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+
+    @Override
+    public String toString() {
+      return new ToStringBuilder(this)
+        .append("endKey", endKey)
+        .append("fromReplicaId", fromReplicaId)
+        .append("timestamp", timestamp)
+        .toString();
+    }
+  }
+
+  private static final class StaleTableCache {
+    private final ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> 
cache =
+      new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
+  }
+
+  private final ConcurrentMap<TableName, StaleTableCache> staleCache;
+  private int numOfReplicas;
+  private final AsyncConnectionImpl conn;
+  private TableName tableName;
+
+  CatalogReplicaLoadBalanceReplicaSimpleSelector(TableName tableName, 
AsyncConnectionImpl conn) {
+    staleCache = new ConcurrentHashMap<>();
+    this.conn = conn;
+    this.tableName = tableName;
+
+    // This numOfReplicas is going to be lazy initialized.
+    this.numOfReplicas = -1;
+    // Start connection's chore service in case.
+    this.conn.startChoreService();
+    this.conn.getChoreService().scheduleChore(getCacheCleanupChore(this));
+  }
+
+  /**
+   * When a client runs into RegionNotServingException, it will call this 
method to
+   * update Selector's internal state.
+   * @param loc the location which causes exception.
+   * @param fromReplicaId the replica id where the stale location comes from.
+   */
+  public void notifyOnError(HRegionLocation loc, int fromReplicaId) {
+    StaleTableCache tableCache =
+      computeIfAbsent(staleCache, loc.getRegion().getTable(), 
StaleTableCache::new);
+    byte[] startKey = loc.getRegion().getStartKey();
+    tableCache.cache.putIfAbsent(startKey,
+      new StaleLocationCacheEntry(fromReplicaId, loc.getRegion().getEndKey()));
+    LOG.debug("Add entry to stale cache for table {} with startKey {}, {}",

Review comment:
       Just do toString on the created StaleLocationCacheEntry?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements
+  CatalogReplicaLoadBalanceReplicaSelector {
+  private static final Logger LOG =
+    
LoggerFactory.getLogger(CatalogReplicaLoadBalanceReplicaSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {
+    // replica id where the stale location comes from.
+    private int fromReplicaId;
+
+    // timestamp in milliseconds
+    private long timestamp;
+
+    private byte[] endKey;
+
+    StaleLocationCacheEntry(final int metaReplicaId, final byte[] endKey) {
+      this.fromReplicaId = metaReplicaId;
+      this.endKey = endKey;
+      timestamp = System.currentTimeMillis();
+    }
+
+    public byte[] getEndKey() {
+      return this.endKey;
+    }
+
+    public int getFromReplicaId() {
+      return this.fromReplicaId;
+    }
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+
+    @Override
+    public String toString() {
+      return new ToStringBuilder(this)
+        .append("endKey", endKey)
+        .append("fromReplicaId", fromReplicaId)
+        .append("timestamp", timestamp)
+        .toString();
+    }
+  }
+
+  private static final class StaleTableCache {
+    private final ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> 
cache =
+      new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
+  }
+
+  private final ConcurrentMap<TableName, StaleTableCache> staleCache;

Review comment:
       Declare and init here rather than declare here and init in constructor?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##########
@@ -196,8 +200,33 @@ private boolean tryComplete(LocateRequest req, 
CompletableFuture<RegionLocations
       MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, 
DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE);
     this.locatePrefetchLimit =
       conn.getConfiguration().getInt(LOCATE_PREFETCH_LIMIT, 
DEFAULT_LOCATE_PREFETCH_LIMIT);
-    this.useMetaReplicas =
-      conn.getConfiguration().getBoolean(USE_META_REPLICAS, 
DEFAULT_USE_META_REPLICAS);
+
+    // Get the region locator's meta replica mode.
+    this.metaReplicaMode = CatalogReplicaMode.valueOf(conn.getConfiguration()
+      .get(LOCATOR_META_REPLICAS_MODE, CatalogReplicaMode.None.toString()));
+
+    switch (this.metaReplicaMode) {
+      case LoadBalance:
+        String replicaSelectorClass = conn.getConfiguration().
+          get(RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR,
+          CatalogReplicaLoadBalanceReplicaSimpleSelector.class.getName());
+
+        this.metaReplicaSelector = 
CatalogReplicaLoadBalanceReplicaSelectorFactory.createSelector(
+          replicaSelectorClass, META_TABLE_NAME, conn);
+        break;
+      case None:
+        // If user does not configure LOCATOR_META_REPLICAS_MODE, let's check 
the legacy config.
+        if (this.metaReplicaMode == CatalogReplicaMode.None) {
+          boolean userMetaReplicas = 
conn.getConfiguration().getBoolean(USE_META_REPLICAS,
+            DEFAULT_USE_META_REPLICAS);
+          if (userMetaReplicas) {
+            this.metaReplicaMode = CatalogReplicaMode.HedgedRead;
+          }
+        }
+        break;
+      default:
+        // Doing nothing
+    }

Review comment:
       Looking below, you'd have to pass a Scan into the CatalogReplicaMode for 
it to configure it? Might be too much?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSelector.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A Catalog replica selector decides which catalog replica to go for read 
requests.
+ */
+@InterfaceAudience.Private
+public interface CatalogReplicaLoadBalanceReplicaSelector {

Review comment:
       Class comment doesn't mention load balance... this is for load balance 
mode only.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSimpleSelector.java
##########
@@ -0,0 +1,273 @@
+/**
+ * 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.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
+import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
+import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector implements a simple catalog 
replica load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up location,
+ * it first check if the row is the stale location cache. If yes, the location 
from
+ * catalog replica is stale, it will go to the primary region to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a replica region for lookup. When 
clients receive
+ * RegionNotServedException from region servers, it will add these region 
locations to the stale
+ * location cache. The stale cache will be cleaned up periodically by a chore.
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup.
+ *  2. If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  3. When client tries to do location lookup, it checks StaleLocationCache 
first for rows it
+ *     tries to lookup, if entry exists, it will go with primary meta region 
to do lookup;
+ *     otherwise, it will follow step 1.
+ *  4. A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ */
+class CatalogReplicaLoadBalanceReplicaSimpleSelector implements
+  CatalogReplicaLoadBalanceReplicaSelector {
+  private static final Logger LOG =
+    
LoggerFactory.getLogger(CatalogReplicaLoadBalanceReplicaSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {
+    // replica id where the stale location comes from.
+    private int fromReplicaId;
+
+    // timestamp in milliseconds
+    private long timestamp;
+
+    private byte[] endKey;
+
+    StaleLocationCacheEntry(final int metaReplicaId, final byte[] endKey) {
+      this.fromReplicaId = metaReplicaId;
+      this.endKey = endKey;
+      timestamp = System.currentTimeMillis();
+    }
+
+    public byte[] getEndKey() {
+      return this.endKey;
+    }
+
+    public int getFromReplicaId() {
+      return this.fromReplicaId;
+    }
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+
+    @Override
+    public String toString() {
+      return new ToStringBuilder(this)
+        .append("endKey", endKey)
+        .append("fromReplicaId", fromReplicaId)
+        .append("timestamp", timestamp)
+        .toString();
+    }
+  }
+
+  private static final class StaleTableCache {
+    private final ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> 
cache =
+      new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
+  }
+
+  private final ConcurrentMap<TableName, StaleTableCache> staleCache;
+  private int numOfReplicas;
+  private final AsyncConnectionImpl conn;
+  private TableName tableName;
+
+  CatalogReplicaLoadBalanceReplicaSimpleSelector(TableName tableName, 
AsyncConnectionImpl conn) {
+    staleCache = new ConcurrentHashMap<>();
+    this.conn = conn;
+    this.tableName = tableName;
+
+    // This numOfReplicas is going to be lazy initialized.
+    this.numOfReplicas = -1;
+    // Start connection's chore service in case.
+    this.conn.startChoreService();
+    this.conn.getChoreService().scheduleChore(getCacheCleanupChore(this));
+  }
+
+  /**
+   * When a client runs into RegionNotServingException, it will call this 
method to
+   * update Selector's internal state.
+   * @param loc the location which causes exception.
+   * @param fromReplicaId the replica id where the stale location comes from.
+   */
+  public void notifyOnError(HRegionLocation loc, int fromReplicaId) {
+    StaleTableCache tableCache =
+      computeIfAbsent(staleCache, loc.getRegion().getTable(), 
StaleTableCache::new);
+    byte[] startKey = loc.getRegion().getStartKey();
+    tableCache.cache.putIfAbsent(startKey,
+      new StaleLocationCacheEntry(fromReplicaId, loc.getRegion().getEndKey()));
+    LOG.debug("Add entry to stale cache for table {} with startKey {}, {}",
+      loc.getRegion().getTable(), startKey, loc.getRegion().getEndKey());
+  }
+
+  private int getRandomNonPrimaryReplicaId() {
+    if (numOfReplicas < 0) {
+      try {
+        Admin admin = conn.toConnection().getAdmin();
+        TableDescriptor tableDescriptor = admin.getDescriptor(tableName);
+        this.numOfReplicas = tableDescriptor.getRegionReplication();
+      } catch (IOException ioe) {
+        LOG.error("Failed to get table {}'s region replication, exception ", 
tableName, ioe);
+        this.numOfReplicas = 1;
+      }
+    }
+    // In case of no replica configured, return the primary region id.
+    if (this.numOfReplicas <= 1) {
+      return RegionInfo.DEFAULT_REPLICA_ID;
+    }
+    return 1 + ThreadLocalRandom.current().nextInt(this.numOfReplicas - 1);
+  }
+
+  /**
+   * When it looks up a location, it will call this method to find a replica 
replica region to go.

Review comment:
       double replica in this sentence.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSelector.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A Catalog replica selector decides which catalog replica to go for read 
requests.
+ */
+@InterfaceAudience.Private
+public interface CatalogReplicaLoadBalanceReplicaSelector {
+
+  /**
+   *  This method is called upon when input location is stale.
+   * @param loc location which is stale.
+   * @param fromReplicaId which replica the stale location comes from.
+   */
+  void notifyOnError(HRegionLocation loc, int fromReplicaId);

Review comment:
       Yeah, do we need to pass in the exception?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceReplicaSelector.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A Catalog replica selector decides which catalog replica to go for read 
requests.
+ */
+@InterfaceAudience.Private
+public interface CatalogReplicaLoadBalanceReplicaSelector {
+
+  /**
+   *  This method is called upon when input location is stale.
+   * @param loc location which is stale.
+   * @param fromReplicaId which replica the stale location comes from.
+   */
+  void notifyOnError(HRegionLocation loc, int fromReplicaId);

Review comment:
       s/notifyOnError/onError/ ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to