You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2019/12/25 01:06:37 UTC

[GitHub] [hbase] bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation

bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235693
 
 

 ##########
 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.
+ */
+@InterfaceAudience.Private
+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.
+  private final int requestFanOut;
+  private ExecutorService masterRpcPool;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutNs;
 
 Review comment:
   Cleaned this up. Does not exist any more. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services