[GitHub] [hbase] Apache9 commented on a change in pull request #2584: HBASE-25126 Add load balance logic in hbase-client to distribute read…

2020-11-03 Thread GitBox


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



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##
@@ -196,8 +201,44 @@ private boolean tryComplete(LocateRequest req, 
CompletableFuture {
+ConnectionConfiguration connConf = new 
ConnectionConfiguration(conn.getConfiguration());

Review comment:
   There is a `connConf` in AsyncConnectionImpl so we do not need to create 
a new one again.





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




[GitHub] [hbase] Apache9 commented on a change in pull request #2584: HBASE-25126 Add load balance logic in hbase-client to distribute read…

2020-11-03 Thread GitBox


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



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java
##
@@ -0,0 +1,302 @@
+/**
+ * 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.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 java.util.function.IntSupplier;
+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.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+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:
+ *
+ * 
+ *  If there is no stale location entry for rows it looks up, it will 
randomly
+ * pick a replica region to do lookup. 
+ *  If the location from the replica region is stale, client gets 
RegionNotServedException
+ * from region server, in this case, it will create 
StaleLocationCacheEntry in
+ * CatalogReplicaLoadBalanceReplicaSimpleSelector.
+ *  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.
+ *  A chore will periodically run to clean up cache entries in the 
StaleLocationCache.
+ * 
+ */
+class CatalogReplicaLoadBalanceSimpleSelector implements
+  CatalogReplicaLoadBalanceSelector, Stoppable {
+  private static final Logger LOG =
+LoggerFactory.getLogger(CatalogReplicaLoadBalanceSimpleSelector.class);
+  private final long STALE_CACHE_TIMEOUT_IN_MILLISECONDS = 3000; // 3 seconds
+  private final int STALE_CACHE_CLEAN_CHORE_INTERVAL_IN_MILLISECONDS = 1500; 
// 1.5 seconds
+  private final int REFRESH_REPLICA_COUNT_CHORE_INTERVAL_IN_MILLISECONDS = 
6; // 1 minute
+
+  /**
+   * 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 final int fromReplicaId;
+
+// timestamp in milliseconds
+private final long timestamp;
+
+private final byte[] endKey;
+
+StaleLocationCacheEntry(final int metaReplicaId, final byte[] endKey) {
+  this.fromReplicaId = metaReplicaId;
+  this.endKey = endKey;
+  timestamp = EnvironmentEdgeManager.currentTime();
+}
+
+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", 

[GitHub] [hbase] Apache9 commented on a change in pull request #2584: HBASE-25126 Add load balance logic in hbase-client to distribute read…

2020-11-03 Thread GitBox


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



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
##
@@ -38,6 +38,9 @@
 
   private final int numNonNullElements;
 
+  // For Meta Replica LoadBalance mode, this indicates which meta replica the 
locations come from.
+  private int fromMetaReplicaId;

Review comment:
   Do we have other ways to pass this infromation? I do not think this is a 
good way to go as we create RegionLocations everywhere in code, not only at 
client side when locating regions. If there is no other simple way, I suggest 
we just remove the fromReplicaId parameter in the selector interface, as now 
the simple selector does not need it. The interface is IA.Private so we can 
change it at any time.

##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##
@@ -82,14 +84,18 @@
   static String LOCATE_PREFETCH_LIMIT = "hbase.client.locate.prefetch.limit";
 
   private static final int DEFAULT_LOCATE_PREFETCH_LIMIT = 10;
+  private static final long GET_META_LOCATIONS_TIMEOUT = 2000; // 2 seconds

Review comment:
   What's this hard coded timeout here?

##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
##
@@ -182,8 +183,21 @@ public void newDead(ServerName sn) {
   }
 
   private void spawnRenewalChore(final UserGroupInformation user) {
-authService = new ChoreService("Relogin service");
-authService.scheduleChore(AuthUtil.getAuthRenewalChore(user));
+ChoreService service = getChoreService();
+service.scheduleChore(AuthUtil.getAuthRenewalChore(user));
+  }
+
+  /**
+   * If choreService has not been created yet, create the ChoreService.
+   * @return ChoreService
+   */
+  ChoreService getChoreService() {
+synchronized (this) {
+  if (choreService == null) {
+choreService = new ChoreService("AsyncConn Chore Service");
+  }
+}
+return choreService;

Review comment:
   Move the return into the synchronized block? Or if you want to use DCL, 
declare choreService as volatile and test it first before the synchronized.

##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java
##
@@ -0,0 +1,302 @@
+/**
+ * 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.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 java.util.function.IntSupplier;
+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.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+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 

[GitHub] [hbase] Apache9 commented on a change in pull request #2584: HBASE-25126 Add load balance logic in hbase-client to distribute read…

2020-10-29 Thread GitBox


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



##
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 
cache =
+ 

[GitHub] [hbase] Apache9 commented on a change in pull request #2584: HBASE-25126 Add load balance logic in hbase-client to distribute read…

2020-10-29 Thread GitBox


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



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
##
@@ -577,6 +635,15 @@ private void removeLocationFromCache(HRegionLocation loc) {
   if (!canUpdateOnError(loc, oldLoc)) {
 return;
   }
+  // Tell metaReplicaSelector 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) {
+// metaReplicaId is not used in simpleSelector, default to the primary 
one.
+metaReplicaSelector.onError(loc, RegionInfo.DEFAULT_REPLICA_ID);

Review comment:
   This is a bit strange? We are using a CatalogReplicaLoadBalanceSelector, 
not a CatalogReplicaLoadBalanceSimpleSelector, so I do not think we should 
assume it is a simple selector and pass default replica id here? It may spend a 
bunch of time for later developers who implement new selectors to find out why 
the selector does not work as expected.  If you want to do this please just 
declare a CatalogReplicaLoadBalanceSimpleSelector above.





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




[GitHub] [hbase] Apache9 commented on a change in pull request #2584: HBASE-25126 Add load balance logic in hbase-client to distribute read…

2020-10-29 Thread GitBox


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



##
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:
   NotServingRegionException? Anyway, this is an IA.Private class so it can 
be changed at any time. I think we could move the filtering logic down to the 
implementation class.





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




[GitHub] [hbase] Apache9 commented on a change in pull request #2584: HBASE-25126 Add load balance logic in hbase-client to distribute read…

2020-10-27 Thread GitBox


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



##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
##
@@ -181,9 +185,18 @@ public void newDead(ServerName sn) {
 this.clusterStatusListener = listener;
   }
 
+  public void startChoreService() {
+if (this.choreService == null) {
+  choreService = new ChoreService("AsyncConn Chore Service");
+}
+  }
+
   private void spawnRenewalChore(final UserGroupInformation user) {
-authService = new ChoreService("Relogin service");
-authService.scheduleChore(AuthUtil.getAuthRenewalChore(user));
+choreService.scheduleChore(AuthUtil.getAuthRenewalChore(user));
+  }
+
+  public ChoreService getChoreService() {

Review comment:
   Could this be private or at least package private?

##
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
##
@@ -181,9 +185,18 @@ public void newDead(ServerName sn) {
 this.clusterStatusListener = listener;
   }
 
+  public void startChoreService() {

Review comment:
   I do not think we need this method? Just do the initialization work in 
the getChoreService method?

##
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:
   Do we need the exception?

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