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



##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java
##########
@@ -0,0 +1,293 @@
+/**
+ * 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;
+
+/**
+ * <p>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.</p>
+ *
+ * It follows a simple algorithm to choose a replica to go:
+ *
+ * <ol>
+ *  <li>If there is no stale location entry for rows it looks up, it will 
randomly
+ *     pick a replica region to do lookup. </li>
+ *  <li>If the location from the replica region is stale, client gets 
RegionNotServedException
+ *     from region server, in this case, it will create 
StaleLocationCacheEntry in
+ *     CatalogReplicaLoadBalanceReplicaSimpleSelector.</li>
+ *  <li>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.</li>
+ *  <li>A chore will periodically run to clean up cache entries in the 
StaleLocationCache.</li>
+ * </ol>
+ */
+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 = 
60000; // 1 minute
+
+  /**
+   * StaleLocationCacheEntry is the entry when a stale location is reported by 
an client.
+   */
+  private static final class StaleLocationCacheEntry {
+    // timestamp in milliseconds
+    private final long timestamp;
+
+    private final byte[] endKey;
+
+    StaleLocationCacheEntry(final byte[] endKey) {
+      this.endKey = endKey;
+      timestamp = EnvironmentEdgeManager.currentTime();
+    }
+
+    public byte[] getEndKey() {
+      return this.endKey;
+    }
+
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+
+    @Override
+    public String toString() {
+      return new ToStringBuilder(this)
+        .append("endKey", endKey)
+        .append("timestamp", timestamp)
+        .toString();
+    }
+  }
+
+  private final ConcurrentMap<TableName, ConcurrentNavigableMap<byte[], 
StaleLocationCacheEntry>>
+    staleCache = new ConcurrentHashMap<>();
+  private volatile int numOfReplicas;
+  private final AsyncConnectionImpl conn;
+  private final TableName tableName;
+  private final IntSupplier getNumOfReplicas;
+  private volatile boolean isStopped = false;
+
+  CatalogReplicaLoadBalanceSimpleSelector(TableName tableName, 
AsyncConnectionImpl conn,
+    IntSupplier getNumOfReplicas) {
+    this.conn = conn;
+    this.tableName = tableName;
+    this.getNumOfReplicas = getNumOfReplicas;
+
+    // This numOfReplicas is going to be lazy initialized.
+    this.numOfReplicas = -1;
+    // Start chores
+    this.conn.getChoreService().scheduleChore(getCacheCleanupChore(this));
+    
this.conn.getChoreService().scheduleChore(getRefreshReplicaCountChore(this));
+  }
+
+  /**
+   * When a client runs into RegionNotServingException, it will call this 
method to

Review comment:
       The doc in the interface method is good enough. Here there is more 
details about implementation, will keep it is as for now.




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