Apache9 commented on code in PR #5631:
URL: https://github.com/apache/hbase/pull/5631#discussion_r1477006736


##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryRpcStubHolder.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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 java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+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.hadoop.hbase.util.IOExceptionSupplier;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ClientMetaService;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ConnectionRegistryService;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryRequest;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryResponse;
+
+/**
+ * A class for creating {@link RpcClient} and related stubs used by
+ * {@link AbstractRpcBasedConnectionRegistry}. We need to connect to bootstrap 
nodes to get the
+ * cluster id first, before creating the final {@link RpcClient} and related 
stubs.
+ * <p>
+ * See HBASE-25051 for more details.
+ */
[email protected]
+class ConnectionRegistryRpcStubHolder implements Closeable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ConnectionRegistryRpcStubHolder.class);
+
+  private final Configuration conf;
+
+  // used for getting cluster id
+  private final Configuration noAuthConf;
+
+  private final User user;
+
+  private final RpcControllerFactory rpcControllerFactory;
+
+  private final Set<ServerName> bootstrapNodes;
+
+  private final int rpcTimeoutMs;
+
+  private volatile ImmutableMap<ServerName, ClientMetaService.Interface> 
addr2Stub;
+
+  private volatile RpcClient rpcClient;
+
+  private CompletableFuture<ImmutableMap<ServerName, 
ClientMetaService.Interface>> addr2StubFuture;
+
+  ConnectionRegistryRpcStubHolder(Configuration conf, User user,
+    RpcControllerFactory rpcControllerFactory, Set<ServerName> bootstrapNodes) 
{
+    this.conf = conf;
+    if (User.isHBaseSecurityEnabled(conf)) {
+      this.noAuthConf = new Configuration(conf);
+      this.noAuthConf.set(User.HBASE_SECURITY_CONF_KEY, "simple");
+    } else {
+      this.noAuthConf = conf;
+    }
+    this.user = user;
+    this.rpcControllerFactory = rpcControllerFactory;
+    this.bootstrapNodes = Collections.unmodifiableSet(bootstrapNodes);
+    this.rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+      conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, 
HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+  }
+
+  private ImmutableMap<ServerName, ClientMetaService.Interface> 
createStubs(RpcClient rpcClient,
+    Collection<ServerName> addrs) {
+    LOG.debug("Going to use new servers to create stubs: {}", addrs);
+    Preconditions.checkNotNull(addrs);
+    ImmutableMap.Builder<ServerName, ClientMetaService.Interface> builder =
+      ImmutableMap.builderWithExpectedSize(addrs.size());
+    for (ServerName masterAddr : addrs) {
+      builder.put(masterAddr,
+        ClientMetaService.newStub(rpcClient.createRpcChannel(masterAddr, user, 
rpcTimeoutMs)));
+    }
+    return builder.build();
+  }
+
+  private void createStubsAndComplete(String clusterId,
+    CompletableFuture<ImmutableMap<ServerName, ClientMetaService.Interface>> 
future) {
+    RpcClient c = RpcClientFactory.createClient(conf, clusterId);
+    ImmutableMap<ServerName, ClientMetaService.Interface> m = createStubs(c, 
bootstrapNodes);
+    rpcClient = c;
+    addr2Stub = m;
+    future.complete(m);
+  }
+
+  private void createStubs(List<ServerName> bootstrapServers, int index,
+    CompletableFuture<ImmutableMap<ServerName, ClientMetaService.Interface>> 
future) {
+    // use null cluster id here as we do not know the cluster id yet, we will 
fetch it through this
+    // rpc client and use it to create the final rpc client
+    RpcClient getConnectionRegistryRpcClient = 
RpcClientFactory.createClient(noAuthConf, null);
+    // user and rpcTimeout are both not important here, as we will not 
actually send any rpc calls
+    // out, only a preamble connection header, but if we pass null as user, 
there will be NPE in
+    // some code paths...
+    RpcChannel channel =
+      
getConnectionRegistryRpcClient.createRpcChannel(bootstrapServers.get(index), 
user, 0);

Review Comment:
   This is intentional, I do not want to let developers think the timeout will 
take effect, as not everyone will look at the comment unless there is something 
unusual...



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to