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



##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaReplicaLoadBalanceReplicaChooser.java
##########
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+/**
+ * There are two modes with meta replica support.
+ *   HighAvailable    - Client sends requests to the primary meta region 
first, within a
+ *                      configured amount of time, if  there is no response 
coming back,
+ *                      client sends requests to all replica regions and takes 
the first
+ *                      response.
+ *
+ *   LoadBalance      - Client sends requests to meta replica regions in a 
round-robin mode,
+ *                      if results from replica regions are stale, next time, 
client sends requests for
+ *                      these stable locations to the primary meta region. In 
this mode, scan
+ *                      requests are load balanced across all replica regions.
+ */
+enum MetaReplicaMode {
+  None,
+  HighAvailable,
+  LoadBalance
+}
+
+/**
+ * A Meta replica chooser decides which meta replica to go for scan requests.
+ */
+@InterfaceAudience.Private
+public interface MetaReplicaLoadBalanceReplicaChooser {

Review comment:
       Is selector better than chooser?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaReplicaLoadBalanceReplicaChooser.java
##########
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+/**
+ * There are two modes with meta replica support.
+ *   HighAvailable    - Client sends requests to the primary meta region 
first, within a
+ *                      configured amount of time, if  there is no response 
coming back,
+ *                      client sends requests to all replica regions and takes 
the first
+ *                      response.
+ *
+ *   LoadBalance      - Client sends requests to meta replica regions in a 
round-robin mode,
+ *                      if results from replica regions are stale, next time, 
client sends requests for
+ *                      these stable locations to the primary meta region. In 
this mode, scan
+ *                      requests are load balanced across all replica regions.
+ */
+enum MetaReplicaMode {
+  None,
+  HighAvailable,

Review comment:
       Does the enum have to be named 'MetaReplicaMode' Can it be named 
ReplicaMode or ReadReplicaClientPolicy ?
   
   When it is named for meta only, it implies this policy only works for meta 
replica? Is this so?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaReplicaLoadBalanceReplicaChooser.java
##########
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+/**
+ * There are two modes with meta replica support.
+ *   HighAvailable    - Client sends requests to the primary meta region 
first, within a
+ *                      configured amount of time, if  there is no response 
coming back,
+ *                      client sends requests to all replica regions and takes 
the first
+ *                      response.
+ *
+ *   LoadBalance      - Client sends requests to meta replica regions in a 
round-robin mode,
+ *                      if results from replica regions are stale, next time, 
client sends requests for
+ *                      these stable locations to the primary meta region. In 
this mode, scan
+ *                      requests are load balanced across all replica regions.
+ */
+enum MetaReplicaMode {
+  None,
+  HighAvailable,

Review comment:
       s/HighAvailable/HighlyAvailable/ or Failover or PrimaryThenReplicas.
   
   s/stable/stale/?
   

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaReplicaLoadBalanceReplicaChooser.java
##########
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+/**
+ * There are two modes with meta replica support.
+ *   HighAvailable    - Client sends requests to the primary meta region 
first, within a
+ *                      configured amount of time, if  there is no response 
coming back,
+ *                      client sends requests to all replica regions and takes 
the first
+ *                      response.
+ *
+ *   LoadBalance      - Client sends requests to meta replica regions in a 
round-robin mode,
+ *                      if results from replica regions are stale, next time, 
client sends requests for
+ *                      these stable locations to the primary meta region. In 
this mode, scan
+ *                      requests are load balanced across all replica regions.
+ */
+enum MetaReplicaMode {
+  None,
+  HighAvailable,
+  LoadBalance
+}
+
+/**
+ * A Meta replica chooser decides which meta replica to go for scan requests.
+ */
+@InterfaceAudience.Private
+public interface MetaReplicaLoadBalanceReplicaChooser {
+
+  void updateCacheOnError(final HRegionLocation loc, final int 
fromMetaReplicaId);

Review comment:
       No need of the 'final' qualifiers in an Interface at least.
   
   Need doc on these methods because in an Interface describing expectations?
   
   You don't pass cache on updateCacheOnError?
   
   s/chooseReplicaToGo/chooseReplica/ ?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaReplicaLoadBalanceReplicaSimpleChooser.java
##########
@@ -0,0 +1,248 @@
+/**
+ * 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.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;
+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 static org.apache.hadoop.hbase.HConstants.DEFAULT_META_REPLICA_NUM;
+import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM;
+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;
+
+/**
+ * MetaReplicaLoadBalanceReplicaSimpleChooser implements a simple meta replica 
load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up meta,
+ * it first check if the row is the stale location cache, if yes, this means 
the the location from
+ * meta replica is stale, it will go to the primary meta to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a meta replica region for meta 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.
+ */
+
+/**
+ * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+ */
+class StaleLocationCacheEntry {
+  // meta replica id where
+  private int metaReplicaId;
+
+  // timestamp in milliseconds
+  private long timestamp;
+
+  private byte[] endKey;
+
+  StaleLocationCacheEntry(final int metaReplicaId, final byte[] endKey) {
+    this.metaReplicaId = metaReplicaId;
+    this.endKey = endKey;
+    timestamp = System.currentTimeMillis();
+  }
+
+  public byte[] getEndKey() {
+    return this.endKey;
+  }
+
+  public int getMetaReplicaId() {
+    return this.metaReplicaId;
+  }
+  public long getTimestamp() {
+    return this.timestamp;
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this)
+      .append("endKey", endKey)
+      .append("metaReplicaId", metaReplicaId)
+      .append("timestamp", timestamp)
+      .toString();
+  }
+}
+
+/**
+ * A simple implementation of MetaReplicaLoadBalanceReplicaChooser.
+ *
+ * It follows a simple algorithm to choose a meta replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a meta replica region to do lookup.
+ *  2. If the location from meta replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     MetaReplicaLoadBalanceReplicaSimpleChooser.
+ *  3. When client tries to do meta 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 MetaReplicaLoadBalanceReplicaSimpleChooser implements 
MetaReplicaLoadBalanceReplicaChooser {
+  private static final Logger LOG =
+    LoggerFactory.getLogger(MetaReplicaLoadBalanceReplicaSimpleChooser.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  private final class StaleTableCache {
+    private final ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> 
cache =
+      new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
+  }
+
+  private final ConcurrentMap<TableName, StaleTableCache> staleCache;
+  private final int numOfMetaReplicas;
+  private final AsyncConnectionImpl conn;
+
+  MetaReplicaLoadBalanceReplicaSimpleChooser(final AsyncConnectionImpl conn) {
+    staleCache = new ConcurrentHashMap<>();

Review comment:
       Declare the data member and assign at same time rather than declare 
above and assign here? nit.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaReplicaLoadBalanceReplicaSimpleChooser.java
##########
@@ -0,0 +1,248 @@
+/**
+ * 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.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;
+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 static org.apache.hadoop.hbase.HConstants.DEFAULT_META_REPLICA_NUM;
+import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM;
+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;
+
+/**
+ * MetaReplicaLoadBalanceReplicaSimpleChooser implements a simple meta replica 
load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up meta,
+ * it first check if the row is the stale location cache, if yes, this means 
the the location from
+ * meta replica is stale, it will go to the primary meta to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a meta replica region for meta 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.

Review comment:
       Is chore heavyweight for a small map like this? Weak references? Or 
guava cache?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
##########
@@ -1128,6 +1128,9 @@
 
   /** Conf key for enabling meta replication */
   public static final String USE_META_REPLICAS = "hbase.meta.replicas.use";
+  public static final String META_REPLICAS_MODE = "hbase.meta.replicas.mode";

Review comment:
       Yeah, does it have to be meta exclusive. Can this go somewhere in client 
package rather than here in the global HConstants? It is a client-only config?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaReplicaLoadBalanceReplicaSimpleChooser.java
##########
@@ -0,0 +1,248 @@
+/**
+ * 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.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;
+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 static org.apache.hadoop.hbase.HConstants.DEFAULT_META_REPLICA_NUM;
+import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM;
+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;
+
+/**
+ * MetaReplicaLoadBalanceReplicaSimpleChooser implements a simple meta replica 
load balancing
+ * algorithm. It maintains a stale location cache for each table. Whenever 
client looks up meta,
+ * it first check if the row is the stale location cache, if yes, this means 
the the location from
+ * meta replica is stale, it will go to the primary meta to look up 
update-to-date location;
+ * otherwise, it will randomly pick up a meta replica region for meta 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.
+ */
+
+/**
+ * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+ */
+class StaleLocationCacheEntry {
+  // meta replica id where
+  private int metaReplicaId;
+
+  // timestamp in milliseconds
+  private long timestamp;
+
+  private byte[] endKey;
+
+  StaleLocationCacheEntry(final int metaReplicaId, final byte[] endKey) {
+    this.metaReplicaId = metaReplicaId;
+    this.endKey = endKey;
+    timestamp = System.currentTimeMillis();
+  }
+
+  public byte[] getEndKey() {
+    return this.endKey;
+  }
+
+  public int getMetaReplicaId() {
+    return this.metaReplicaId;
+  }
+  public long getTimestamp() {
+    return this.timestamp;
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this)
+      .append("endKey", endKey)
+      .append("metaReplicaId", metaReplicaId)
+      .append("timestamp", timestamp)
+      .toString();
+  }
+}
+
+/**
+ * A simple implementation of MetaReplicaLoadBalanceReplicaChooser.
+ *
+ * It follows a simple algorithm to choose a meta replica to go:
+ *
+ *  1. If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a meta replica region to do lookup.
+ *  2. If the location from meta replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     MetaReplicaLoadBalanceReplicaSimpleChooser.
+ *  3. When client tries to do meta 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 MetaReplicaLoadBalanceReplicaSimpleChooser implements 
MetaReplicaLoadBalanceReplicaChooser {
+  private static final Logger LOG =
+    LoggerFactory.getLogger(MetaReplicaLoadBalanceReplicaSimpleChooser.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL = 1500; // 1.5 seconds
+
+  private final class StaleTableCache {
+    private final ConcurrentNavigableMap<byte[], StaleLocationCacheEntry> 
cache =
+      new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
+  }
+
+  private final ConcurrentMap<TableName, StaleTableCache> staleCache;
+  private final int numOfMetaReplicas;
+  private final AsyncConnectionImpl conn;
+
+  MetaReplicaLoadBalanceReplicaSimpleChooser(final AsyncConnectionImpl conn) {
+    staleCache = new ConcurrentHashMap<>();
+    this.numOfMetaReplicas = conn.getConfiguration().getInt(
+      META_REPLICAS_NUM, DEFAULT_META_REPLICA_NUM);
+    this.conn = conn;
+    this.conn.getChoreService().scheduleChore(getCacheCleanupChore(this));

Review comment:
       What you thinking here? You made the chore on client side generic. You 
want to make it so different policy implementations can add chore? The chores 
can do different things dependent on implementation?
   
   Above I wonder if you have to use chore? Is what is needed here too much to 
do in a guava cache say?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaReplicaLoadBalanceReplicaChooser.java
##########
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+/**
+ * There are two modes with meta replica support.
+ *   HighAvailable    - Client sends requests to the primary meta region 
first, within a
+ *                      configured amount of time, if  there is no response 
coming back,
+ *                      client sends requests to all replica regions and takes 
the first
+ *                      response.
+ *
+ *   LoadBalance      - Client sends requests to meta replica regions in a 
round-robin mode,
+ *                      if results from replica regions are stale, next time, 
client sends requests for
+ *                      these stable locations to the primary meta region. In 
this mode, scan
+ *                      requests are load balanced across all replica regions.
+ */
+enum MetaReplicaMode {
+  None,
+  HighAvailable,
+  LoadBalance
+}
+
+/**
+ * A Meta replica chooser decides which meta replica to go for scan requests.
+ */
+@InterfaceAudience.Private
+public interface MetaReplicaLoadBalanceReplicaChooser {
+
+  void updateCacheOnError(final HRegionLocation loc, final int 
fromMetaReplicaId);

Review comment:
       Is this stuff only for meta? Can it be more generic than meta? Drop the 
'meta' prefix?

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetaReplicaLoadBalanceReplicaChooser.java
##########
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+/**
+ * There are two modes with meta replica support.
+ *   HighAvailable    - Client sends requests to the primary meta region 
first, within a
+ *                      configured amount of time, if  there is no response 
coming back,
+ *                      client sends requests to all replica regions and takes 
the first
+ *                      response.
+ *
+ *   LoadBalance      - Client sends requests to meta replica regions in a 
round-robin mode,
+ *                      if results from replica regions are stale, next time, 
client sends requests for
+ *                      these stable locations to the primary meta region. In 
this mode, scan
+ *                      requests are load balanced across all replica regions.
+ */
+enum MetaReplicaMode {
+  None,
+  HighAvailable,

Review comment:
       So, here we 'choose' a policy but then the implementation is elsewhere. 
Could vary? You provide a 'simple' one here. What if I want to do an involved 
one? That would be a config?




----------------------------------------------------------------
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