bharathv commented on a change in pull request #954: HBASE-23305: Master based
registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235641
##########
File path:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -0,0 +1,364 @@
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
+import static org.apache.hadoop.hbase.HConstants.HBASE_RPC_TIMEOUT_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static
org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT;
+import static
org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_NUM_HEDGED_REQS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException;
+import org.apache.hadoop.hbase.ipc.RpcClient;
+import org.apache.hadoop.hbase.ipc.RpcClientFactory;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import
org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import
org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService;
+import
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterRequest;
+import
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetActiveMasterResponse;
+import
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdRequest;
+import
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterIdResponse;
+import
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsRequest;
+import
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetMetaRegionLocationsResponse;
+
+/**
+ * Master based registry implementation. Makes RPCs to the configured master
addresses from config
+ * {@value HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It has the ability to burst the same RPC to multiple masters as a batch and
returns whatever
+ * comes back first (a.k.a hedged RPCs). Number of target masters in a single
batch is controlled
+ * via {@value HConstants#MASTER_REGISTRY_NUM_HEDGED_REQS_KEY}. If it is set
to 1 (default), it is
+ * equivalent to picking a random master from the configured list.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to
restart the client.
+ */
[email protected]
+public class MasterRegistry implements AsyncRegistry {
+ private static final Logger LOG =
LoggerFactory.getLogger(MasterRegistry.class);
+
+ // Configured list of masters to probe the meta information from.
+ private final List<ServerName> masterServers;
+ // Controls the fan out of the hedged requests. Requests are made in batches
of this number until
+ // all the servers are exhausted. The first returned result is passed back
to the client.
Review comment:
> Once we have a successful answer, do we kill/interrupt the other ongoing
queries? Wondering because 100k clients going against 3 or 5 Masters will be a
bit of a load.
We interrupt the threads. I clarified it in the new design and wrote a
detailed javadoc. Let me know if it is not clear.
> If N connections to the cluster, how many MasterRegistries? As many as
there are Connections? Or is there just a single instance per JVM and it is
shared across?
This is actually a very good point. It is not once per JVM (although, I
think that makes more sense to me). Infact it is multiple instances per
connection (look at the callers of AsyncRegistryFactory#getRegistry()). I think
there is definitely a scope for cleanup here. Can i revisit this as a follow up?
> In my earlier review I suggest we do fan out adaptively by default. If
single requests are performing adequately, fanout is unnecessary load for no
reward.
Totally agree. I think the hedging policy needs to be smart. I clarified
this in the new patch set. Can I implement this in a follow up patch? This is
already too big.
----------------------------------------------------------------
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:
[email protected]
With regards,
Apache Git Services