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/20 03:13:54 UTC

[GitHub] [hbase] bharathv opened a new pull request #954: HBASE-23305: Master based registry implementation

bharathv opened a new pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954
 
 
   Implements a master based registry for clients.
   
   - Supports hedged RPCs (fan out configured via configs).
   - Parameterized existing client tests to run with multiple
     registry combinations.
   - Added unit-test coverage for the new registry implenenation.

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360543287
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
 
 Review comment:
   We don't need a new one, right? Use existing RPC/Call facilities and set a retry policy where there are no retries using the factory methods for that. 

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360542110
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,381 @@
+/*
+ * 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.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 org.apache.hadoop.hbase.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 org.apache.hadoop.hbase.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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
+      // This is the most general (and default) case. We want to avoid the thread creation and
+      // synchronization overhead and hence a special optimization for this case that just loops
+      // through the available masters in that order.
+      return doSequentialRPCs(rpcCall, isvalidResp, debug);
+    }
+    return doHedgedRPCs(rpcCall, isvalidResp, debug);
+  }
+
+  /**
+   * Makes RPCs in batches of {HConstants#MASTER_REGISTRY_NUM_HEDGED_REQS_KEY}. Currently should
+   * only be used if the request fan out is > 1. For the default case, refer to doSequentialRPCs().
+   */
+  private <RESP> Optional<RESP> doHedgedRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    Preconditions.checkState(requestFanOut > 1);
+    Preconditions.checkNotNull(masterRpcPool);
+    int i = 0;
+    while (i < masterServers.size()){
 
 Review comment:
   See above comment regarding adaptive behavior. I'd prefer if we have fan out, to not blindly do it if single requests are performing adequately. 

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362693708
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
 
 Review comment:
   There is a trim() in the next line,  you mean somewhere else? (there is also test coverage for spaces).

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

[GitHub] [hbase] ndimiduk commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-571276468
 
 
   Something broke fund bugs on master perhaps?
   
   ```
   [INFO] --- findbugs-maven-plugin:3.0.4:findbugs (default-cli) @ hbase-server ---
   [INFO] Fork Value is true
        [java] SLF4J: No SLF4J providers were found.
        [java] SLF4J: Defaulting to no-operation (NOP) logger implementation
        [java] SLF4J: See http://www.slf4j.org/codes.html#noProviders for further details.
   ```

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

[GitHub] [hbase] bharathv commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
bharathv commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-572699482
 
 
   I was mentioning this to @ndimiduk offline but just wanted to update here too, for the record. JUnit BeforeParam does not work for us as expected because it runs **before** the test class constructor is called and does not have access to the test parameters. I've clarified this in the test comments. Also rebased and squashed commits. Thanks for the reviews. At this point we are waiting for @apurtell to take another look.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362661475
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
 
 Review comment:
   Excellent point. I think you're right, this should be gated for use with asynchronous connections only. This could be made clear by a change to the class name, or enforced statically if we had a `NonBlockingRrpClient` as a sibling to the `BlockingRpcClient`. For the time being, have this constructor to only accept an instance of `NettyRpcClient`?

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235414
 
 

 ##########
 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.
 
 Review comment:
   Done. Please take a look at the next PR. Re-did the whole thing.

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360543947
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,381 @@
+/*
+ * 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.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 org.apache.hadoop.hbase.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 org.apache.hadoop.hbase.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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
 
 Review comment:
   Let's generalize this and apply it to and reuse the existing RPC controller, RPC client, Callable, Call, Caller hierarchy. 

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

[GitHub] [hbase] ndimiduk commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-572309996
 
 
   Running tests locally I get
   
   ```
   Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 312.454 s <<< FAILURE! - in org.apache.hadoop.hbase.client.TestFromClientSideWithCoprocessor
   org.apache.hadoop.hbase.client.TestFromClientSideWithCoprocessor.testScanMetrics[0]  Time elapsed: 33.15 s  <<< ERROR!
   java.io.IOException: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/hbaseid
   Caused by: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/hbaseid
   ```

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-570571287
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 12s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 31 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 50s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   1m 45s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 36s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   5m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 24s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   5m 17s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   7m 21s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 37s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 44s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 44s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | -1 :x: |  checkstyle  |   0m 33s |  hbase-client: The patch generated 2 new + 14 unchanged - 10 fixed = 16 total (was 24)  |
   | +1 :green_heart: |  checkstyle  |   1m 31s |  hbase-server: The patch generated 0 new + 226 unchanged - 48 fixed = 226 total (was 274)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  1s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 33s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   7m 19s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m 12s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 54s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 171m 59s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m 16s |  The patch does not generate ASF License warnings.  |
   |  |   | 252m 59s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestScannersFromClientSide |
   |   | hadoop.hbase.client.TestFromClientSide |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 265a7760b785 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / dffa9be899 |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/7/artifact/out/diff-checkstyle-hbase-client.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/7/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/7/testReport/ |
   | Max. process+thread count | 6669 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/7/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-570764147
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 17s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 10 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 35s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   6m  1s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   1m 50s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 37s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   5m  5s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 20s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   5m 22s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   7m 22s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 57s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m  2s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m  2s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | +1 :green_heart: |  checkstyle  |   0m 35s |  hbase-client: The patch generated 0 new + 0 unchanged - 8 fixed = 0 total (was 8)  |
   | +1 :green_heart: |  checkstyle  |   1m 36s |  hbase-server: The patch generated 0 new + 223 unchanged - 50 fixed = 223 total (was 273)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m 14s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 59s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 43s |  the patch passed  |
   | -1 :x: |  findbugs  |  11m 30s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m 47s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m 19s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 193m 28s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m 15s |  The patch does not generate ASF License warnings.  |
   |  |   | 284m 46s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestFromClientSideWithCoprocessor |
   |   | hadoop.hbase.client.TestFromClientSide |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux eff02e1348bf 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / d016cd9962 |
   | Default Java | 1.8.0_181 |
   | findbugs | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/9/artifact/out/patch-findbugs-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/9/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/9/testReport/ |
   | Max. process+thread count | 6672 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/9/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362714670
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
 
 Review comment:
   Makes sense.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362731006
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
 ##########
 @@ -0,0 +1,120 @@
+/*
+ * 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.HConstants.META_REPLICAS_NUM;
+import static org.junit.Assert.assertEquals;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestMasterRegistry {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterRegistry.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    builder.numMasters(3).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Generates a string of dummy master addresses in host:port format. Every other hostname won't
+   * have a port number.
+   */
+  private static String generateDummyMastersList(int size) {
+    List<String> masters = new ArrayList<>();
+    for (int i = 0; i < size; i++) {
+      masters.add(" localhost" + (i % 2 == 0 ? ":" + (1000 + i) : ""));
+    }
+    return String.join(",", masters);
+  }
+
+  /**
+   * Makes sure the master registry parses the master end points in the configuration correctly.
+   */
+  @Test public void TestMasterAddressParsing() {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    int numMasters = 10;
+    conf.set(HConstants.MASTER_ADDRS_KEY, generateDummyMastersList(numMasters));
+    try (MasterRegistry registry = new MasterRegistry(conf)) {
+      List<ServerName> parsedMasters = registry.getParsedMasterServers();
+      assertEquals(numMasters, parsedMasters.size());
+      for (int i = 0; i < numMasters; i++) {
+        ServerName sn = parsedMasters.get(i);
+        assertEquals("localhost", sn.getHostname());
+        if (i % 2 == 0) {
+          assertEquals(1000 + i, sn.getPort());
+        } else {
+          assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort());
+        }
+      }
+    }
+  }
+
+  @Test public void TestRegistryRPCs() throws Exception {
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362908992
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public List<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableList(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Function<T, Boolean> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc) {
+    return rpcResult -> {
+      if (rpcResult == null) {
+        future.completeExceptionally(
+            new MasterRegistryFetchException(masterServers, hrc.getFailed()));
+      }
+      if (!isValidResp.apply(rpcResult)) {
+        // Rpc returned ok, but result was malformed.
 
 Review comment:
   The scenario I outline is not a failing RPC, but a lack of data. So in that case, what's the stack trace? The user will see the `Call` and thus the method name?

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-572800179
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 21s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 10 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 49s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   1m 50s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 42s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   5m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 21s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   5m 20s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   7m 20s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 49s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 50s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 50s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | +1 :green_heart: |  checkstyle  |   0m 33s |  hbase-client: The patch generated 0 new + 0 unchanged - 12 fixed = 0 total (was 12)  |
   | +1 :green_heart: |  checkstyle  |   1m 32s |  hbase-server: The patch generated 0 new + 223 unchanged - 50 fixed = 223 total (was 273)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  19m 16s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   8m 27s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m 24s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   2m  5s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 188m  0s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m 35s |  The patch does not generate ASF License warnings.  |
   |  |   | 273m 39s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.quotas.TestClusterScopeQuotaThrottle |
   |   | hadoop.hbase.security.access.TestSnapshotScannerHDFSAclController |
   |   | hadoop.hbase.client.TestFromClientSide |
   |   | hadoop.hbase.client.TestAsyncTableGetMultiThreaded |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/12/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux f67fc951fca9 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / d9c17c3681 |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/12/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/12/testReport/ |
   | Max. process+thread count | 5531 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/12/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362908253
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
 
 Review comment:
   Buried deep in there, yes there is. thanks.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362671865
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##########
 @@ -0,0 +1,249 @@
+/*
+ * 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.master;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ThreadFactory;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
+/**
+ * A cache of meta region location metadata. Registers a listener on ZK to track changes to the
+ * meta table znodes. Clients are expected to retry if the meta information is stale. This class
+ * is thread-safe (a single instance of this class can be shared by multiple threads without race
+ * conditions).
+ */
+@InterfaceAudience.Private
+public class MetaRegionLocationCache extends ZKListener {
 
 Review comment:
   I think we've reviewed this class before...

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362657957
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
 
 Review comment:
   Thank you for the explanation.

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

[GitHub] [hbase] ndimiduk commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-571664717
 
 
   Surprising.
   ```
   [ERROR] org.apache.hadoop.hbase.client.TestFromClientSideWithCoprocessor.testCheckAndDeleteWithCompareOp[0]  Time elapsed: 0.142 s  <<< ERROR!
   java.lang.OutOfMemoryError: Java heap space
   	at org.apache.hadoop.hbase.client.TestFromClientSideWithCoprocessor.<init>(TestFromClientSideWithCoprocessor.java:50)
   ```

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362667826
 
 

 ##########
 File path: hbase-protocol-shaded/src/main/protobuf/Master.proto
 ##########
 @@ -1200,3 +1200,47 @@ service HbckService {
   rpc FixMeta(FixMetaRequest)
     returns(FixMetaResponse);
 }
+
+/** Request and response to get the clusterID for this cluster */
+message GetClusterIdRequest {
+}
+message GetClusterIdResponse {
+  /** Not set if cluster ID could not be determined. */
+  optional string cluster_id = 1;
+}
+
+/** Request and response to get the currently active master name for this cluster */
+message GetActiveMasterRequest {
+}
+message GetActiveMasterResponse {
+  /** Not set if an active master could not be determined. */
+  optional ServerName server_name = 1;
+}
+
+/** Request and response to get the current list of meta region locations */
+message GetMetaRegionLocationsRequest {
+}
+message GetMetaRegionLocationsResponse {
+  /** Not set if meta region locations could not be determined. */
+  repeated RegionLocation meta_locations = 1;
+}
+
+/**
+ * Implements all the RPCs needed by clients to look up cluster meta information needed for connection establishment.
+ */
+service ClientMetaService {
 
 Review comment:
   These methods, in aggregate, allow a client to locate any host in the cluster according to the regions it serves. On its face, this sounds a lot like what the industry is currently calling "service discovery". How about `service ServiceDiscoveryService`?

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364359606
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -64,7 +64,9 @@
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TestTableName;
 
 Review comment:
   haha ya, noticed that. Weird.

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360542320
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,381 @@
+/*
+ * 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.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 org.apache.hadoop.hbase.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 org.apache.hadoop.hbase.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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
+      // This is the most general (and default) case. We want to avoid the thread creation and
+      // synchronization overhead and hence a special optimization for this case that just loops
+      // through the available masters in that order.
+      return doSequentialRPCs(rpcCall, isvalidResp, debug);
+    }
+    return doHedgedRPCs(rpcCall, isvalidResp, debug);
+  }
+
+  /**
+   * Makes RPCs in batches of {HConstants#MASTER_REGISTRY_NUM_HEDGED_REQS_KEY}. Currently should
+   * only be used if the request fan out is > 1. For the default case, refer to doSequentialRPCs().
+   */
+  private <RESP> Optional<RESP> doHedgedRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    Preconditions.checkState(requestFanOut > 1);
+    Preconditions.checkNotNull(masterRpcPool);
+    int i = 0;
+    while (i < masterServers.size()){
+      // Each iteration of loop picks requestFanOut masters
+      int subListSize = Math.min(masterServers.size(), i + requestFanOut);
+      List<ServerName> masterSubList = masterServers.subList(i, subListSize);
 
 Review comment:
   Shouldn't the list be randomized again? Or we're hitting the sublists deterministically. Make a private list at top of function and shuffle?

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360508134
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
 
 Review comment:
   Should it return CompleteableFuture since we in async land?

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

[GitHub] [hbase] ndimiduk commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-570415214
 
 
   FYI, I rebased the feature branch onto the tip of master.

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360541305
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,381 @@
+/*
+ * 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.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 org.apache.hadoop.hbase.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 org.apache.hadoop.hbase.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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
 
 Review comment:
   I was expecting for the first cut we'd do just one request to a random host on the list at a time, and retry with another random choice. (So above randomization of list is good and important.) This is what the zookeeper client does now is no different from current state of play. 
   
   Hedged reading is ahead of the game.
   
   Good that it is off by default, though.
   
   Also, it's nice that fan out factor is configurable, but I would want an adaptive policy by default. Only if single requests are failing at some threshold of unacceptable probability (i suppose controlled by a config param) would you want to start loading up more than one per request in trade for faster response, hopefully, on average.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362671378
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
 ##########
 @@ -0,0 +1,249 @@
+/*
+ * 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.master;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ThreadFactory;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKListener;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+
+/**
+ * A cache of meta region location metadata. Registers a listener on ZK to track changes to the
+ * meta table znodes. Clients are expected to retry if the meta information is stale. This class
+ * is thread-safe (a single instance of this class can be shared by multiple threads without race
+ * conditions).
+ */
+@InterfaceAudience.Private
+public class MetaRegionLocationCache extends ZKListener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MetaRegionLocationCache.class);
+
+  /**
+   * Maximum number of times we retry when ZK operation times out.
+   */
+  private static final int MAX_ZK_META_FETCH_RETRIES = 10;
+  /**
+   * Sleep interval ms between ZK operation retries.
+   */
+  private static final int SLEEP_INTERVAL_MS_BETWEEN_RETRIES = 1000;
+  private static final int SLEEP_INTERVAL_MS_MAX = 10000;
+  private final RetryCounterFactory retryCounterFactory =
+      new RetryCounterFactory(MAX_ZK_META_FETCH_RETRIES, SLEEP_INTERVAL_MS_BETWEEN_RETRIES);
+
+  /**
+   * Cached meta region locations indexed by replica ID.
+   * CopyOnWriteArrayMap ensures synchronization during updates and a consistent snapshot during
+   * client requests. Even though CopyOnWriteArrayMap copies the data structure for every write,
+   * that should be OK since the size of the list is often small and mutations are not too often
+   * and we do not need to block client requests while mutations are in progress.
+   */
+  private final CopyOnWriteArrayMap<Integer, HRegionLocation> cachedMetaLocations;
+
+  private enum ZNodeOpType {
+    INIT,
+    CREATED,
+    CHANGED,
+    DELETED
+  }
+
+  public MetaRegionLocationCache(ZKWatcher zkWatcher) {
+    super(zkWatcher);
+    cachedMetaLocations = new CopyOnWriteArrayMap<>();
+    watcher.registerListener(this);
+    // Populate the initial snapshot of data from meta znodes.
+    // This is needed because stand-by masters can potentially start after the initial znode
+    // creation. It blocks forever until the initial meta locations are loaded from ZK and watchers
+    // are established. Subsequent updates are handled by the registered listener. Also, this runs
+    // in a separate thread in the background to not block master init.
+    ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true).build();
 
 Review comment:
   Please also set a thread name prefix. Think of the operators and poor folks debugging through thread dumps.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362694783
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
 
 Review comment:
   Thanks.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362668122
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
 ##########
 @@ -57,12 +56,18 @@
   final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
   final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
 
+  // This server's information.
   private final ServerName sn;
   private int infoPort;
   private final Server master;
 
+  // Active master's server name. Invalidated anytime active master changes (based on ZK
+  // notifications) and lazily fetched on-demand.
+  // ServerName is immutable, so we don't need heavy synchronization around it.
+  private volatile ServerName activeMasterServerName;
+
   /**
-   * @param watcher
+   * @param watcher ZK watcher
 
 Review comment:
   These `@param` javadocs are not providing any new information; just delete them.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362663256
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    ((HBaseRpcController) controller).setFailed(lastBatchCtx.getLastFailedRpcReason());
+    done.run(null);
+  }
+
+  @Override
+  public void callMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    // To make the channel non-blocking, we run the actual doCalMethod() async. The call back is
+    // called once the hedging finishes.
+    CompletableFuture.runAsync(
 
 Review comment:
   Is it okay to use the common pool for this IO operation? I think this will introduce the potential for deadlocks when other code invoked by the client makes implicit use of the same pool. I think it would be better to have a dedicated, configureable, monitored thread pool for this purpose. It can probably be configured, at least partially, using the configuration already present for hedging (i.e., `fanOutSize` can be used as the pool's size).

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360538339
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
 ##########
 @@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Exception thrown when an master registry RPC fails in client. The exception includes the list of
+ * masters to which RPC was attempted.
+ */
+@InterfaceAudience.Private
+public class MasterRegistryFetchException extends HBaseIOException {
+  public MasterRegistryFetchException(List<ServerName> masters, String failedRPC) {
+    super(String.format("Exception making rpc %s to masters %s", failedRPC,
+        masters.stream().map(Objects::toString).collect(Collectors.toList())));
 
 Review comment:
   Unnecessary Java 8 idioms

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362730928
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
 ##########
 @@ -0,0 +1,120 @@
+/*
+ * 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.HConstants.META_REPLICAS_NUM;
+import static org.junit.Assert.assertEquals;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestMasterRegistry {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterRegistry.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    builder.numMasters(3).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Generates a string of dummy master addresses in host:port format. Every other hostname won't
+   * have a port number.
+   */
+  private static String generateDummyMastersList(int size) {
+    List<String> masters = new ArrayList<>();
+    for (int i = 0; i < size; i++) {
+      masters.add(" localhost" + (i % 2 == 0 ? ":" + (1000 + i) : ""));
+    }
+    return String.join(",", masters);
+  }
+
+  /**
+   * Makes sure the master registry parses the master end points in the configuration correctly.
+   */
+  @Test public void TestMasterAddressParsing() {
 
 Review comment:
   Other tests uses the mini cluster?

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360513295
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
 
 Review comment:
   TODO: delay query of second and third masters?

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362695241
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360599136
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
+      // This is the most general (and default) case. We want to avoid the thread creation and
+      // synchronization overhead and hence a special optimization for this case that just loops
+      // through the available masters in that order.
+      return doSequentialRPCs(rpcCall, isvalidResp, debug);
+    }
+    return doHedgedRPCs(rpcCall, isvalidResp, debug);
+  }
+
+  /**
+   * Makes RPCs in batches of {HConstants#MASTER_REGISTRY_NUM_HEDGED_REQS_KEY}. Currently should
+   * only be used if the request fan out is > 1. For the default case, refer to doSequentialRPCs().
+   */
+  private <RESP> Optional<RESP> doHedgedRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    Preconditions.checkState(requestFanOut > 1);
+    Preconditions.checkNotNull(masterRpcPool);
+    int i = 0;
+    while (i < masterServers.size()){
+      // Each iteration of loop picks requestFanOut masters
+      int subListSize = Math.min(masterServers.size(), i + requestFanOut);
+      List<ServerName> masterSubList = masterServers.subList(i, subListSize);
+      // Create a new RPC context for this batch of RPCs and submit the RPCs to the pool.
+      BatchRpcCtx<RESP> batchRpcCtx = new BatchRpcCtx<>();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Attempting to make rpc {} to batch {}.", debug,
+            masterSubList.stream().map(Objects::toString).collect(Collectors.toList()));
+      }
+      for (ServerName master: masterSubList) {
+        masterRpcPool.submit(new MasterRpc<>(batchRpcCtx, master, rpcCall, isvalidResp, debug));
+      }
+      if (batchRpcCtx.waitForResults(rpcTimeoutNs)) {
+        // Results set by some RPC, no point in doing rest of the calls.
+        return Optional.of(batchRpcCtx.getResult());
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Failed to make rpc {} to batch {}. Trying others.", debug,
+            masterSubList.stream().map(Objects::toString).collect(Collectors.toList()));
+      }
+      i = subListSize;
+    }
+    return Optional.empty();
+  }
+
+
+  /**
+   * Sequentially calls the masters to make an RPC in random order. This removes the unnecessary
+   * thread overhead and synchronization when hedged RPCs are disabled (maxRequestFanOut == 1).
+   */
+  private <RESP> Optional<RESP> doSequentialRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    Preconditions.checkState(requestFanOut == 1, "Invalid request fan out.");
+    for (ServerName master: masterServers) {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (!isvalidResp.apply(resp)) {
+          continue;
+        }
+        return Optional.of(resp);
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debug, master, e);
+      }
+    }
+    // Failed on all the masters.
+    return Optional.empty();
+  }
+
+  /**
+   * Util that generates a master stub for a given ServerName.
+   */
+  private ClientMetaService.BlockingInterface getMasterStub(ServerName server) throws IOException {
+    return ClientMetaService.newBlockingStub(
+        rpcClient.createBlockingRpcChannel(server, User.getCurrent(), rpcTimeoutNs));
+  }
+
+  @Override
+  public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+    CompletableFuture<RegionLocations> result = new CompletableFuture<>();
+    CompletableFuture.runAsync(() -> {
+      Optional<GetMetaRegionLocationsResponse> resp = doRPCs(stub -> stub.getMetaRegionLocations(
+          rpcControllerFactory.newController(), GetMetaRegionLocationsRequest.getDefaultInstance()),
+        (rpcResp) -> rpcResp.getMetaLocationsCount() != 0,"GetMetaRegionLocations()");
+      if (!resp.isPresent()) {
+        result.completeExceptionally(new MasterRegistryFetchException(masterServers,
+            "GetMetaRegionLocations()"));
+      }
+      List<HRegionLocation> regionLocations = new ArrayList<>();
+      resp.get().getMetaLocationsList().forEach(
+        location -> regionLocations.add(ProtobufUtil.toRegionLocation(location)));
+      result.complete(new RegionLocations(regionLocations));
+    });
+    return result;
+  }
+
+  @Override
+  public CompletableFuture<String> getClusterId() {
+    CompletableFuture<String> result = new CompletableFuture<>();
+    CompletableFuture.runAsync(() -> {
+      Optional<GetClusterIdResponse> resp = doRPCs(stub -> stub.getClusterId(
+          rpcControllerFactory.newController(), GetClusterIdRequest.getDefaultInstance()),
+          GetClusterIdResponse::hasClusterId, "GetClusterId()");
+      if (!resp.isPresent()) {
+        result.completeExceptionally(new MasterRegistryFetchException(masterServers,
+            "GetClusterId()"));
+      }
+      result.complete(resp.get().getClusterId());
+    });
+    return result;
+  }
+
+  @Override
+  public CompletableFuture<ServerName> getMasterAddress() {
 
 Review comment:
   https://github.com/apache/hbase/pull/957

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235875
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
 
 Review comment:
   Re-did the whole thing. Now the hedging happens in the RPC layer and is not specific to master registry anymore.

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360538683
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
 ##########
 @@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Exception thrown when an master registry RPC fails in client. The exception includes the list of
+ * masters to which RPC was attempted.
+ */
+@InterfaceAudience.Private
+public class MasterRegistryFetchException extends HBaseIOException {
+  public MasterRegistryFetchException(List<ServerName> masters, String failedRPC) {
+    super(String.format("Exception making rpc %s to masters %s", failedRPC,
+        masters.stream().map(Objects::toString).collect(Collectors.toList())));
 
 Review comment:
   Much of this work has these, all presenting issues for backport (not insurmountable, though), but this one is less readable than a simple string join using Collections. 

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361236125
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,381 @@
+/*
+ * 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.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 org.apache.hadoop.hbase.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 org.apache.hadoop.hbase.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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
+      // This is the most general (and default) case. We want to avoid the thread creation and
+      // synchronization overhead and hence a special optimization for this case that just loops
+      // through the available masters in that order.
+      return doSequentialRPCs(rpcCall, isvalidResp, debug);
+    }
+    return doHedgedRPCs(rpcCall, isvalidResp, debug);
+  }
+
+  /**
+   * Makes RPCs in batches of {HConstants#MASTER_REGISTRY_NUM_HEDGED_REQS_KEY}. Currently should
+   * only be used if the request fan out is > 1. For the default case, refer to doSequentialRPCs().
+   */
+  private <RESP> Optional<RESP> doHedgedRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    Preconditions.checkState(requestFanOut > 1);
+    Preconditions.checkNotNull(masterRpcPool);
+    int i = 0;
+    while (i < masterServers.size()){
+      // Each iteration of loop picks requestFanOut masters
+      int subListSize = Math.min(masterServers.size(), i + requestFanOut);
+      List<ServerName> masterSubList = masterServers.subList(i, subListSize);
 
 Review comment:
   I was thinking it'd be non-deterministic at the registry level. (different connections have different registries anyway, so wondering if we need to shuffle 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:
users@infra.apache.org


With regards,
Apache Git Services

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362674344
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
 ##########
 @@ -44,35 +45,38 @@
 import org.apache.hadoop.hbase.HTestConst;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.TestTableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
 import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
+import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
- * A client-side test, mostly testing scanners with various parameters.
+ * A client-side test, mostly testing scanners with various parameters. Parameterized on different
+ * registry implementations.
  */
 @Category({MediumTests.class, ClientTests.class})
+@RunWith(Parameterized.class)
 
 Review comment:
   Same question about timeout and the parameterized test.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362662261
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
 
 Review comment:
   method should be private.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362674026
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
 ##########
 @@ -0,0 +1,120 @@
+/*
+ * 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.HConstants.META_REPLICAS_NUM;
+import static org.junit.Assert.assertEquals;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestMasterRegistry {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterRegistry.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    builder.numMasters(3).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Generates a string of dummy master addresses in host:port format. Every other hostname won't
+   * have a port number.
+   */
+  private static String generateDummyMastersList(int size) {
+    List<String> masters = new ArrayList<>();
+    for (int i = 0; i < size; i++) {
+      masters.add(" localhost" + (i % 2 == 0 ? ":" + (1000 + i) : ""));
+    }
+    return String.join(",", masters);
+  }
+
+  /**
+   * Makes sure the master registry parses the master end points in the configuration correctly.
+   */
+  @Test public void TestMasterAddressParsing() {
 
 Review comment:
   There's no real reason for this test to use a mini cluster, is there? Could it be in a `SmallTests` unit test instead?

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364348720
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -131,61 +138,115 @@
 
   // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide.
   private static final Logger LOG = LoggerFactory.getLogger(TestFromClientSide.class);
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static HBaseTestingUtility TEST_UTIL;
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static final byte[] INVALID_FAMILY = Bytes.toBytes("invalidTestFamily");
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static byte [] VALUE = Bytes.toBytes("testValue");
   protected static int SLAVES = 3;
 
-  @Rule
-  public TestName name = new TestName();
+  @Rule public TestTableName name = new TestTableName();
+
+  // To keep the child classes happy.
+  TestFromClientSide() {}
 
-  protected static final void initialize(Class<?>... cps) throws Exception {
+  public TestFromClientSide(Class registry, int numHedgedReqs) throws Exception {
+    initialize(registry, numHedgedReqs, MultiRowMutationEndpoint.class);
+  }
+
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { MasterRegistry.class, 1},
+        { MasterRegistry.class, 2},
+        { ZKConnectionRegistry.class, 1}
+    });
+  }
+
+  /**
+   * JUnit does not provide an easy way to run a method after each parameterized run. Without that
 
 Review comment:
   No fun, this solution.
   
   Looks like junit-4.13 adds `@{Before,After}Param` method annotations, via junit-team/junit4#1435 . Does this feature work for you?

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364939843
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
 ##########
 @@ -20,41 +20,23 @@
 
 import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException;
-
-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.cache.CacheBuilder;
-import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
-import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
-import org.apache.hbase.thirdparty.com.google.protobuf.Message;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-
+import org.apache.commons.lang3.NotImplementedException;
 
 Review comment:
   Good catch.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362926824
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public List<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableList(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Function<T, Boolean> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc) {
+    return rpcResult -> {
+      if (rpcResult == null) {
+        future.completeExceptionally(
+            new MasterRegistryFetchException(masterServers, hrc.getFailed()));
+      }
+      if (!isValidResp.apply(rpcResult)) {
+        // Rpc returned ok, but result was malformed.
 
 Review comment:
   Something like this, also plumbed the debug message to make it more clearer.
   
   > Caused by: java.io.IOException: Invalid result for request getClusterId(). Will be retried
   > 	at org.apache.hadoop.hbase.client.MasterRegistry.lambda$getRpcCallBack$0(MasterRegistry.java:152)
   > 	at org.apache.hbase.thirdparty.com.google.protobuf.RpcUtil$1.run(RpcUtil.java:79)
   > 	at org.apache.hbase.thirdparty.com.google.protobuf.RpcUtil$1.run(RpcUtil.java:70)
   > 	at org.apache.hadoop.hbase.ipc.HedgedRpcChannel$BatchRpcCtx.waitForResults(HedgedRpcChannel.java:172)
   > 	at org.apache.hadoop.hbase.ipc.HedgedRpcChannel.doCallMethod(HedgedRpcChannel.java:246)
   > 	at org.apache.hadoop.hbase.ipc.HedgedRpcChannel.lambda$callMethod$0(HedgedRpcChannel.java:270)

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362631300
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
 
 Review comment:
   should strip whitespace from substrings.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364942189
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
 ##########
 @@ -20,41 +20,23 @@
 
 import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException;
-
-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.cache.CacheBuilder;
-import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
-import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
-import org.apache.hbase.thirdparty.com.google.protobuf.Message;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-
+import org.apache.commons.lang3.NotImplementedException;
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362694697
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
 ##########
 @@ -0,0 +1,36 @@
+/*
+ * 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.exceptions;
+
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Exception thrown when an master registry RPC fails in client. The exception includes the list of
+ * masters to which RPC was attempted.
 
 Review comment:
   They will be logged anyway, didn't want to include them here because it will be too verbose. Thoughts?

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364350614
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -64,7 +64,9 @@
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TestTableName;
 
 Review comment:
   Cool trick!
   
   But wait, this class is both a `@Rule` and a unit test?!? Nooo!

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

[GitHub] [hbase] bharathv commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
bharathv commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-571830020
 
 
   > [ERROR] org.apache.hadoop.hbase.client.TestFromClientSideWithCoprocessor.testCheckAndDeleteWithCompareOp[0]  Time elapsed: 0.142 s  <<< ERROR!
   
   @ndimiduk I think I figured out the flaky test problem. I pushed another commit that deflakes the test and all of these TestClient* now pass in my IDE too. To summarize, JUnit parameterization (with test class hiearchy) is a PITA. Without my last commit, *every* unit test of *every parameterized test* run was reseting the cluster state. JUnit does not yet provide of a way to execute a hook at the end of each parameterized test run. So, I hacked my way around it. I'm hoping it should pass 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:
users@infra.apache.org


With regards,
Apache Git Services

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360535380
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,364 @@
+package org.apache.hadoop.hbase.client;
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360506231
 
 

 ##########
 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:
   How does this timeout relate to general client rpc timeout? If I set rpc timeout for client of 10 seconds, is this timeout subsumed by the general client timeout or does this run indepentent of whatever the overall client timeout setting is?

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362730495
 
 

 ##########
 File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
 ##########
 @@ -172,10 +172,17 @@
   public static final String MASTER_INFO_PORT = "hbase.master.info.port";
 
   /** Configuration key for the list of master host:ports **/
-  public static final String MASTER_ADDRS_KEY = "hbase.master.addrs";
+  public static final String MASTER_ADDRS_KEY = "hbase.masters";
 
   public static final String MASTER_ADDRS_DEFAULT =  "localhost:" + DEFAULT_MASTER_PORT;
 
+  /** Configuration to enable hedged reads on master registry **/
+  public static final String MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY =
+      "hbase.client.master_registry.enable_hedged_reads";
 
 Review comment:
   Rebased and pushed it as a commit.

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360511377
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
 
 Review comment:
   We have a whole world of Callable+Call and it seems like this is introducing a new one of RpcCall... We need this new one? (I don't know.... just asking... because it gets hard tracking Callable/Call/Caller... and adding variants makes for more to decipher... FYI).

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362930568
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -2642,7 +2676,7 @@ public void testJiraTest1182() throws Exception {
    */
   @Test
   public void testJiraTest52() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
 Review comment:
   Missed this too. Not an expert on junit, but I think its possible with chaining. That is probably a bit more cleaner way.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362672281
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
 ##########
 @@ -136,6 +137,10 @@ public HMaster getMaster() {
     } catch (Exception e) {
       throw new IOException(e);
     }
+    // Needed if a master based registry is configured for internal cluster connections. Here, we
+    // just add the current master host port since we do not know other master addresses up front
+    // in mini cluster tests.
+    c.set(HConstants.MASTER_ADDRS_KEY, server.getServerName().getAddress().toString());
 
 Review comment:
   No null-checking required?

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362666055
 
 

 ##########
 File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
 ##########
 @@ -188,4 +191,18 @@ private static long humanReadableIntervalToSec(final String humanReadableInterva
     return ttl;
   }
 
+  /**
+   * Pretty prints a collection of any type to a string. Relies on toString() implementation of the
+   * object type.
+   * @param collection collection to pretty print.
+   * @return Pretty printed string for the collection.
+   */
+  public static String toString(Collection<?> collection) {
 
 Review comment:
   But we have `java.util.StringJoiner`... oh branch-1 :(

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

[GitHub] [hbase] bharathv commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
bharathv commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-569974349
 
 
   @saintstack / @apurtell Any thoughts on the new design? Happy to address any concerns. Thanks.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r365949284
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -64,7 +64,9 @@
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TestTableName;
 
 Review comment:
   https://issues.apache.org/jira/browse/HBASE-23665

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362673672
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -106,23 +108,27 @@
 import org.apache.hadoop.hbase.util.NonRepeatedEnvironmentEdge;
 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
 import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
  * Run tests that use the HBase clients; {@link Table}.
  * Sets up the HBase mini cluster once at start and runs through all client tests.
  * Each creates a table named for the method and does its stuff against that.
+ *
+ * Parameterized to run with different registry implementations.
  */
 @Category({LargeTests.class, ClientTests.class})
 @SuppressWarnings ("deprecation")
+@RunWith(Parameterized.class)
 
 Review comment:
   This class is known for taking a long time to execute (see the sibling classes `TestFromClientSide2,3`. Do you happen to know how out timeout rule interacts with the parameterized test? We'd want the timeout rule to apply to each parameter instance, not all of them together.

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-568135746
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 8 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 35s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 48s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   1m 45s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 32s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   5m  5s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 16s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   4m 46s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m 39s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 31s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  The patch passed checkstyle in hbase-common  |
   | +1 :green_heart: |  checkstyle  |   0m 34s |  The patch passed checkstyle in hbase-client  |
   | +1 :green_heart: |  checkstyle  |   1m 32s |  hbase-server: The patch generated 0 new + 223 unchanged - 46 fixed = 223 total (was 269)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  4s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 29s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 15s |  the patch passed  |
   | -1 :x: |  findbugs  |   1m 18s |  hbase-client generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  2s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 49s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 267m 40s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m 22s |  The patch does not generate ASF License warnings.  |
   |  |   | 346m 53s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-client |
   |  |  Should org.apache.hadoop.hbase.client.MasterRegistry$BatchRpcCtx be a _static_ inner class?  At MasterRegistry.java:inner class?  At MasterRegistry.java:[lines 117-162] |
   | Failed junit tests | hadoop.hbase.client.TestSplitOrMergeStatus |
   |   | hadoop.hbase.client.TestFromClientSideWithCoprocessor |
   |   | hadoop.hbase.master.TestMasterShutdown |
   |   | hadoop.hbase.client.TestScannersFromClientSide |
   |   | hadoop.hbase.client.TestMobSnapshotCloneIndependence |
   |   | hadoop.hbase.client.TestCIPutOperationTimeout |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 2c688ac1523d 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / e41b46cc28 |
   | Default Java | 1.8.0_181 |
   | findbugs | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/2/artifact/out/new-findbugs-hbase-client.html |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/2/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/2/testReport/ |
   | Max. process+thread count | 9917 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/2/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364359974
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -131,61 +138,115 @@
 
   // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide.
   private static final Logger LOG = LoggerFactory.getLogger(TestFromClientSide.class);
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static HBaseTestingUtility TEST_UTIL;
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static final byte[] INVALID_FAMILY = Bytes.toBytes("invalidTestFamily");
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static byte [] VALUE = Bytes.toBytes("testValue");
   protected static int SLAVES = 3;
 
-  @Rule
-  public TestName name = new TestName();
+  @Rule public TestTableName name = new TestTableName();
+
+  // To keep the child classes happy.
+  TestFromClientSide() {}
 
-  protected static final void initialize(Class<?>... cps) throws Exception {
+  public TestFromClientSide(Class registry, int numHedgedReqs) throws Exception {
+    initialize(registry, numHedgedReqs, MultiRowMutationEndpoint.class);
+  }
+
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { MasterRegistry.class, 1},
+        { MasterRegistry.class, 2},
+        { ZKConnectionRegistry.class, 1}
+    });
+  }
+
+  /**
+   * JUnit does not provide an easy way to run a method after each parameterized run. Without that
 
 Review comment:
   Doesn't work. We are on 4.12. 

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r363885862
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
 ##########
 @@ -72,6 +74,14 @@ protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws I
     return new BlockingRpcConnection(this, remoteId);
   }
 
+  @Override
+  public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362718068
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    ((HBaseRpcController) controller).setFailed(lastBatchCtx.getLastFailedRpcReason());
+    done.run(null);
+  }
+
+  @Override
+  public void callMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    // To make the channel non-blocking, we run the actual doCalMethod() async. The call back is
+    // called once the hedging finishes.
+    CompletableFuture.runAsync(
 
 Review comment:
   Not totally clear how the usage of common pool has a potential for deadlocks. I thought its actually the other way. Common pool threads have lower overhead for creation and are good short running tasks, no?

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362695859
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    ((HBaseRpcController) controller).setFailed(lastBatchCtx.getLastFailedRpcReason());
 
 Review comment:
   No, there is a Preconditions check in the c'tor after address parsing. Still added the null check.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362939851
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362694229
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public List<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableList(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Function<T, Boolean> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc) {
 
 Review comment:
   Oops, I meant to use java.util but intellij imported from Guava.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362675053
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
 ##########
 @@ -363,6 +365,101 @@ public void testAsyncEcho() throws IOException {
     }
   }
 
+  /**
+   * Tests the happy path for various request fan out values using a simple RPC hedged across
+   * a mix of running and failing servers.
+   */
+  @Test
+  public void testHedgedAsyncEcho() throws Exception {
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    // Create a mix of running and failing servers.
+    final int numRunningServers = 5;
+    final int numFailingServers = 3;
 
 Review comment:
   If this is happy-path testing, why are there any failing servers at all? When I think "happy path", I would expect a test with all running servers, and assertions that (1) the correct value is received every time and (2) no more than `fanOut` RPC calls are ever issued.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362660570
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    ((HBaseRpcController) controller).setFailed(lastBatchCtx.getLastFailedRpcReason());
 
 Review comment:
   Is it possible that the while loop never executed, thus leaving `lastBatchCtx` as null? Hmm, I suppose not, but refactors happen a null check doesn't hurt.

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360505185
 
 

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

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362696454
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    ((HBaseRpcController) controller).setFailed(lastBatchCtx.getLastFailedRpcReason());
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362656584
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
 
 Review comment:
   Odd that the null check is vs `result` but it's the value of `rpcController.getFailed()` that is actually used. I would expect the null check against the value of getFailed. There would need to be a subsequent null check against `result` to check for a should-not-happen kind of scenario.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235825
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
 
 Review comment:
   The callers don't care right? Why return success/failure?

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-568891936
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  1s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 10 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 35s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 13s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   1m 44s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 24s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   4m 38s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   4m 26s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m 18s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m  7s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | +1 :green_heart: |  checkstyle  |   0m 31s |  hbase-client: The patch generated 0 new + 3 unchanged - 6 fixed = 3 total (was 9)  |
   | +1 :green_heart: |  checkstyle  |   1m 24s |  hbase-server: The patch generated 0 new + 227 unchanged - 46 fixed = 227 total (was 273)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 39s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 44s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 22s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   6m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m 11s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 53s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 189m 42s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m 45s |  The patch does not generate ASF License warnings.  |
   |  |   | 264m  2s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestFromClientSide |
   |   | hadoop.hbase.client.TestScannersFromClientSide |
   |   | hadoop.hbase.client.TestFromClientSideWithCoprocessor |
   |   | hadoop.hbase.quotas.TestQuotaAdmin |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 0e3f574cae6c 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / e41b46cc28 |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/4/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/4/testReport/ |
   | Max. process+thread count | 9909 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/4/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r363458389
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
 ##########
 @@ -527,6 +528,21 @@ public RpcChannel createRpcChannel(ServerName sn, User user, int rpcTimeout)
     return new RpcChannelImplementation(this, createAddr(sn), user, rpcTimeout);
   }
 
+  @Override
+  public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
+      throws UnknownHostException {
+    final int hedgedRpcFanOut = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY,
+        HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT);
+    Set<InetSocketAddress> addresses = new HashSet<>();
+    for (ServerName sn: sns) {
+      addresses.add(createAddr(sn));
+    }
+    Preconditions.checkState(this instanceof NettyRpcClient,
+        "Hedging only supported for non-blocking connection implementations.");
 
 Review comment:
   Done.

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

[GitHub] [hbase] bharathv commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
bharathv commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-569152093
 
 
   Fixed all the test failures from the last run in the latest push. One of the fixes comes in as a separate PR to master https://github.com/apache/hbase/pull/963 . To pull that in, the current feature branch needs a rebase on master. Also, from the last test run, the thread usage has substantially dropped due to the test cleanup. I think this is in a good shape for 2nd round of review. Appreciate any feedback.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362928069
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -106,23 +108,27 @@
 import org.apache.hadoop.hbase.util.NonRepeatedEnvironmentEdge;
 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
 import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
  * Run tests that use the HBase clients; {@link Table}.
  * Sets up the HBase mini cluster once at start and runs through all client tests.
  * Each creates a table named for the method and does its stuff against that.
+ *
+ * Parameterized to run with different registry implementations.
  */
 @Category({LargeTests.class, ClientTests.class})
 @SuppressWarnings ("deprecation")
+@RunWith(Parameterized.class)
 
 Review comment:
   Missed this in the first iteration. I think it applies to each parameter instance. Thats the behavior I saw in my local tests. Even if one parameterized run times out, others start and finish.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235913
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,381 @@
+/*
+ * 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.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 org.apache.hadoop.hbase.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 org.apache.hadoop.hbase.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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
+      // This is the most general (and default) case. We want to avoid the thread creation and
+      // synchronization overhead and hence a special optimization for this case that just loops
+      // through the available masters in that order.
+      return doSequentialRPCs(rpcCall, isvalidResp, debug);
+    }
+    return doHedgedRPCs(rpcCall, isvalidResp, debug);
+  }
+
+  /**
+   * Makes RPCs in batches of {HConstants#MASTER_REGISTRY_NUM_HEDGED_REQS_KEY}. Currently should
+   * only be used if the request fan out is > 1. For the default case, refer to doSequentialRPCs().
+   */
+  private <RESP> Optional<RESP> doHedgedRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    Preconditions.checkState(requestFanOut > 1);
+    Preconditions.checkNotNull(masterRpcPool);
+    int i = 0;
+    while (i < masterServers.size()){
 
 Review comment:
   Agree.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362694181
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public List<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableList(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Function<T, Boolean> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc) {
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r363446712
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public List<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableList(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Function<T, Boolean> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc) {
+    return rpcResult -> {
+      if (rpcResult == null) {
+        future.completeExceptionally(
+            new MasterRegistryFetchException(masterServers, hrc.getFailed()));
+      }
+      if (!isValidResp.apply(rpcResult)) {
+        // Rpc returned ok, but result was malformed.
 
 Review comment:
   This satisfies my concern. thanks!

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-568968651
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  8s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 10 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 36s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 53s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   1m 43s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 38s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   5m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 17s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   4m 43s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m 37s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 47s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | +1 :green_heart: |  checkstyle  |   0m 35s |  hbase-client: The patch generated 0 new + 3 unchanged - 6 fixed = 3 total (was 9)  |
   | +1 :green_heart: |  checkstyle  |   1m 33s |  hbase-server: The patch generated 0 new + 227 unchanged - 46 fixed = 227 total (was 273)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  3s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 21s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 20s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   7m  3s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  1s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 50s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 271m 40s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m 10s |  The patch does not generate ASF License warnings.  |
   |  |   | 350m 55s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestFromClientSideWithCoprocessor |
   |   | hadoop.hbase.quotas.TestClusterScopeQuotaThrottle |
   |   | hadoop.hbase.client.TestScannersFromClientSide |
   |   | hadoop.hbase.client.TestFromClientSide |
   |   | hadoop.hbase.regionserver.TestSplitTransactionOnCluster |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux f3a8a8df9404 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / e41b46cc28 |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/5/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/5/testReport/ |
   | Max. process+thread count | 5491 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/5/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
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.
+ */
+@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.
 
 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:
users@infra.apache.org


With regards,
Apache Git Services

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360510548
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java
 ##########
 @@ -27,8 +28,6 @@
 @InterfaceAudience.Private
 final class AsyncRegistryFactory {
 
-  static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl";
 
 Review comment:
   First reaction is ... what sort of registry is this? Need qualifier? Or Registry is a grand notion that we'll just 'know' what it does going forward (Funny.. the registry factory doesn't say either...). nit.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362675909
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
 ##########
 @@ -348,4 +352,32 @@ public void testToCell() throws Exception {
         ProtobufUtil.toCell(ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY), cell);
     assertTrue(CellComparatorImpl.COMPARATOR.compare(offheapKV, newOffheapKV) == 0);
   }
+
+  @Test
+  public void testMetaRegionState() throws Exception {
 
 Review comment:
   💯 

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362653446
 
 

 ##########
 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.
 
 Review comment:
   > 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?
   
   This is the state of things before you arrived, right? I'd say file this as a separate ticket, as another cleanup project for rebase, or for after this code lands. I think I'd prefer to see the former, but it's not a strong preference ATM. I do consider it a blocker for back port to a branch-2, however.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360800323
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistryFactory.java
 ##########
 @@ -27,8 +28,6 @@
 @InterfaceAudience.Private
 final class AsyncRegistryFactory {
 
-  static final String REGISTRY_IMPL_CONF_KEY = "hbase.client.registry.impl";
 
 Review comment:
   More discussion in HBASE-23604. PR for it is ready at https://github.com/apache/hbase/pull/957. Once that is merged, I can rebase this patch. Didn't want to pollute this PR with those changes.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235717
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
 
 Review comment:
   Removed this and implemented a generic hedging framework for any RPC at the rpc layer.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362910060
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
 
 Review comment:
   In that case, the sleep would be handled there as well, so there's nothing `TODO`.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362716840
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
 
 Review comment:
   Clarified.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362659110
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
 
 Review comment:
   nit: since there's explicitly no order to these addresses as far as the caller is concerned, and all must be unique for the hedging functionality to make sense, can this be a `Set<InetSocketAddress>` instead? Internally, we care about the randomized order, so it makes sense to retain the member variable as a list.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362643252
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public List<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableList(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Function<T, Boolean> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc) {
 
 Review comment:
   For `transformResult` please use `java.util.function.Function` instead of the Guava type. The better to avoid new dependencies on Guava if we can, and that class's javadoc says it's a legacy interface anyway.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362674237
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
 ##########
 @@ -0,0 +1,186 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MultithreadedTestUtil;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MetaRegionLocationCache;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({SmallTests.class, MasterTests.class })
+public class TestMetaRegionLocationCache {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMetaRegionLocationCache.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static AsyncRegistry REGISTRY;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
+    TEST_UTIL.startMiniCluster(3);
+    REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    RegionReplicaTestHelper.waitUntilAllMetaReplicasHavingRegionLocation(
+        TEST_UTIL.getConfiguration(), REGISTRY, 3);
+    TEST_UTIL.getAdmin().balancerSwitch(false, true);
+  }
+
+  @AfterClass
+  public static void cleanUp() throws Exception {
+    IOUtils.closeQuietly(REGISTRY);
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private List<HRegionLocation> getCurrentMetaLocations(ZKWatcher zk) throws Exception {
+    List<HRegionLocation> result = new ArrayList<>();
+    for (String znode: zk.getMetaReplicaNodes()) {
+      String path = ZNodePaths.joinZNode(zk.getZNodePaths().baseZNode, znode);
+      int replicaId = zk.getZNodePaths().getMetaReplicaIdFromPath(path);
+      RegionState state = MetaTableLocator.getMetaRegionState(zk, replicaId);
+      result.add(new HRegionLocation(state.getRegion(), state.getServerName()));
+    }
+    return result;
+  }
+
+  // Verifies that the cached meta locations in the given master are in sync with what is in ZK.
+  private void verifyCachedMetaLocations(HMaster master) throws Exception {
+    // Wait until initial meta locations are loaded.
+    int retries = 0;
+    while (!master.getMetaRegionLocationCache().getMetaRegionLocations().isPresent()) {
+      Thread.sleep(1000);
+      if (++retries == 10) {
+        break;
+      }
+    }
+    List<HRegionLocation> metaHRLs =
+        master.getMetaRegionLocationCache().getMetaRegionLocations().get();
+    assertFalse(metaHRLs.isEmpty());
+    ZKWatcher zk = master.getZooKeeper();
+    List<String> metaZnodes = zk.getMetaReplicaNodes();
+    assertEquals(metaZnodes.size(), metaHRLs.size());
+    List<HRegionLocation> actualHRLs = getCurrentMetaLocations(zk);
+    Collections.sort(metaHRLs);
+    Collections.sort(actualHRLs);
+    assertEquals(actualHRLs, metaHRLs);
+  }
+
+  @Test public void testInitialMetaLocations() throws Exception {
+    verifyCachedMetaLocations(TEST_UTIL.getMiniHBaseCluster().getMaster());
+  }
+
+  @Test public void testStandByMetaLocations() throws Exception {
+    HMaster standBy = TEST_UTIL.getMiniHBaseCluster().startMaster().getMaster();
+    verifyCachedMetaLocations(standBy);
+  }
+
+  /*
 
 Review comment:
   not: might as well be javadoc.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364359361
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -131,61 +138,115 @@
 
   // NOTE: Increment tests were moved to their own class, TestIncrementsFromClientSide.
   private static final Logger LOG = LoggerFactory.getLogger(TestFromClientSide.class);
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static HBaseTestingUtility TEST_UTIL;
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static final byte[] INVALID_FAMILY = Bytes.toBytes("invalidTestFamily");
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static byte [] VALUE = Bytes.toBytes("testValue");
   protected static int SLAVES = 3;
 
-  @Rule
-  public TestName name = new TestName();
+  @Rule public TestTableName name = new TestTableName();
+
+  // To keep the child classes happy.
+  TestFromClientSide() {}
 
-  protected static final void initialize(Class<?>... cps) throws Exception {
+  public TestFromClientSide(Class registry, int numHedgedReqs) throws Exception {
+    initialize(registry, numHedgedReqs, MultiRowMutationEndpoint.class);
+  }
+
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { MasterRegistry.class, 1},
+        { MasterRegistry.class, 2},
+        { ZKConnectionRegistry.class, 1}
+    });
+  }
+
+  /**
+   * JUnit does not provide an easy way to run a method after each parameterized run. Without that
 
 Review comment:
   https://issues.apache.org/jira/browse/HBASE-23664

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360508649
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
 
 Review comment:
   You don't want to declare and assign in the one go? Can the private members be final?

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235901
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,381 @@
+/*
+ * 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.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 org.apache.hadoop.hbase.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 org.apache.hadoop.hbase.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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
 
 Review comment:
   Done. Re-did the whole thing. Now the hedging happens in the RPC layer and is not specific to MasterRegistry anymore.

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360512897
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
 
 Review comment:
   good

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362657347
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
 
 Review comment:
   Perhaps something more descriptive like "Canceled because sibling hedged call succeeded". It's odd to see the message explaining an exception describe a successful result.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362912035
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    ((HBaseRpcController) controller).setFailed(lastBatchCtx.getLastFailedRpcReason());
+    done.run(null);
+  }
+
+  @Override
+  public void callMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    // To make the channel non-blocking, we run the actual doCalMethod() async. The call back is
+    // called once the hedging finishes.
+    CompletableFuture.runAsync(
 
 Review comment:
   One cannot assume anything doing a blocking network operation will be short-lived, and the common pool is usually quite small.
   
   I admit I don't have a firm understanding re: deadlocks that I suspect arise from the use common pool from and with our async client. I suppose if tests are consistently passing, it's fine.

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-571394871
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 46s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 10 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 37s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   7m 29s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   2m  9s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 55s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   5m 11s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 19s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   5m  2s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m 56s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | +1 :green_heart: |  checkstyle  |   0m 33s |  hbase-client: The patch generated 0 new + 0 unchanged - 12 fixed = 0 total (was 12)  |
   | +1 :green_heart: |  checkstyle  |   1m 30s |  hbase-server: The patch generated 0 new + 223 unchanged - 50 fixed = 223 total (was 273)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  3s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 25s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 21s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   7m 14s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  9s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 51s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 168m  1s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m 17s |  The patch does not generate ASF License warnings.  |
   |  |   | 251m  4s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestScannersFromClientSide |
   |   | hadoop.hbase.client.TestFromClientSideWithCoprocessor |
   |   | hadoop.hbase.client.TestFromClientSide |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 209b7b181d42 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / d016cd9962 |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/10/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/10/testReport/ |
   | Max. process+thread count | 4757 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/10/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235728
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
 
 Review comment:
   Done.

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-570737099
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 20s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 31 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 36s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 59s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   2m  3s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 39s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   5m 15s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   4m 43s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m 35s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | -1 :x: |  checkstyle  |   0m 33s |  hbase-client: The patch generated 2 new + 14 unchanged - 10 fixed = 16 total (was 24)  |
   | +1 :green_heart: |  checkstyle  |   1m 34s |  hbase-server: The patch generated 0 new + 224 unchanged - 50 fixed = 224 total (was 274)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  5s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 34s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 16s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   7m 33s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  7s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 55s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 166m  9s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m 13s |  The patch does not generate ASF License warnings.  |
   |  |   | 247m 11s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestFromClientSideWithCoprocessor |
   |   | hadoop.hbase.client.TestFromClientSide |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 1bcc84663300 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / dffa9be899 |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/8/artifact/out/diff-checkstyle-hbase-client.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/8/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/8/testReport/ |
   | Max. process+thread count | 6395 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/8/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235879
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362650185
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
 ##########
 @@ -0,0 +1,36 @@
+/*
+ * 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.exceptions;
+
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Exception thrown when an master registry RPC fails in client. The exception includes the list of
+ * masters to which RPC was attempted.
 
 Review comment:
   Should this include a throwable associated with each master? They can be failing for different reasons.

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-568842494
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 10 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 36s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 17s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   1m 47s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 24s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   4m 33s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 22s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   4m 22s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m 13s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m  0s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 45s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | +1 :green_heart: |  checkstyle  |   0m 32s |  hbase-client: The patch generated 0 new + 3 unchanged - 6 fixed = 3 total (was 9)  |
   | -1 :x: |  checkstyle  |   1m 23s |  hbase-server: The patch generated 2 new + 227 unchanged - 46 fixed = 229 total (was 273)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | -1 :x: |  shadedjars  |   2m 28s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 42s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 21s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   6m 30s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m 10s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 54s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 175m 37s |  hbase-server in the patch failed.  |
   | -1 :x: |  asflicense  |   1m 46s |  The patch generated 1 ASF License warnings.  |
   |  |   | 247m 22s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestFromClientSide |
   |   | hadoop.hbase.client.TestFromClientSideWithCoprocessor |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux dfed66e1cbf1 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / e41b46cc28 |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/3/artifact/out/diff-checkstyle-hbase-server.txt |
   | shadedjars | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/3/artifact/out/patch-shadedjars.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/3/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/3/testReport/ |
   | asflicense | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/3/artifact/out/patch-asflicense-problems.txt |
   | Max. process+thread count | 9903 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/3/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362665624
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
 
 Review comment:
   So `fanOutSize` is the number of concurrent RPCs that "race", the size of a "batch"? That's not explicitly stated in your (otherwise excellent) class java doc. Maybe a constructor-level javadoc is in order?

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362694875
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
 
 Review comment:
   Done.

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

[GitHub] [hbase] bharathv edited a comment on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
bharathv edited a comment on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-572324821
 
 
   Works for me. Try re-running? I noticed the mini cluster in tests is very flaky, especially when you run it in a dev setup.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362654931
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
 
 Review comment:
   This HedgedChannel is an excellent encapsulation. Would love to see this used for the region replica read requests as well. Nice work!

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360503952
 
 

 ##########
 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.
 
 Review comment:
   More detail on how hedged read works? Do we read from all always? Or a delay before we read from others? Thanks.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r363839731
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java
 ##########
 @@ -72,6 +74,14 @@ protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws I
     return new BlockingRpcConnection(this, remoteId);
   }
 
+  @Override
+  public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
 
 Review comment:
   There's no need for an implementation of this method on this class -- it doesn't support this part of the interface and the parent abstract class's implementation is sufficient.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362675545
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
 ##########
 @@ -363,6 +365,101 @@ public void testAsyncEcho() throws IOException {
     }
   }
 
+  /**
+   * Tests the happy path for various request fan out values using a simple RPC hedged across
+   * a mix of running and failing servers.
+   */
+  @Test
+  public void testHedgedAsyncEcho() throws Exception {
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    // Create a mix of running and failing servers.
+    final int numRunningServers = 5;
+    final int numFailingServers = 3;
+    final int numServers = numRunningServers + numFailingServers;
+    for (int i = 0; i < numRunningServers; i++) {
+      RpcServer rpcServer = createRpcServer(null, "testRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    for (int i = 0; i < numFailingServers; i++) {
+      RpcServer rpcServer = createTestFailingRpcServer(null, "testFailingRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    Configuration conf = HBaseConfiguration.create();
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
+      // Try out various fan out values starting from 1 -> numServers.
+      for (int reqFanOut = 1; reqFanOut <= numServers; reqFanOut++) {
+        // Update the client's underlying conf, should be ok for the test.
+        LOG.debug("Testing with request fan out: " + reqFanOut);
+        conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, reqFanOut);
+        Interface stub = newStub(client, addresses);
+        BlockingRpcCallback<EchoResponseProto> done = new BlockingRpcCallback<>();
+        stub.echo(new HBaseRpcControllerImpl(),
+            EchoRequestProto.newBuilder().setMessage("hello").build(), done);
+        TestProtos.EchoResponseProto responseProto = done.get();
+        assertNotNull(responseProto);
+        assertEquals("hello", responseProto.getMessage());
+        LOG.debug("Ended test with request fan out: " + reqFanOut);
+      }
+    } finally {
+      for (RpcServer rpcServer: rpcServers) {
+        rpcServer.stop();
+      }
+    }
+  }
+
+  @Test
+  public void testHedgedAsyncTimeouts() throws Exception {
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    // Create a mix of running and failing servers.
 
 Review comment:
   stale comment? I only see running servers in this test.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362718926
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public List<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableList(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Function<T, Boolean> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc) {
+    return rpcResult -> {
+      if (rpcResult == null) {
+        future.completeExceptionally(
+            new MasterRegistryFetchException(masterServers, hrc.getFailed()));
+      }
+      if (!isValidResp.apply(rpcResult)) {
+        // Rpc returned ok, but result was malformed.
 
 Review comment:
   > I take it failures that warrant a DoNotRetryIOException are handled by the RPC layer. Is there a reason why an RPC would succeed but the isValidResp would need to fail with a DNRIOE ?
   
   It happens when the server doing the RPC does not have enough information to service the request. For example, if there is no active master in the cluster but someone does a "get active master" request. The rPC resturns ok but no server name set. The predicate catches such scenarios.
   
   >It would be nice if this was an interface was enriched to support adding meaningful logging into the loop. For instance, it would be nice to have DEBUG logging that said something like "GetClusterIdResponse does not contain a clusterId value. retrying." Right now it's just RPC into the void over and over without leaving a trail as to the error scenario.
   
   My original implementation had something like that. It made the code ugly. Fwiw, the stack trace will have the actual failing RPC in the protobuf stack. However it is probably not as readable to a user, unless they know how to interpret it.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362673004
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -106,23 +108,27 @@
 import org.apache.hadoop.hbase.util.NonRepeatedEnvironmentEdge;
 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
 import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
  * Run tests that use the HBase clients; {@link Table}.
  * Sets up the HBase mini cluster once at start and runs through all client tests.
  * Each creates a table named for the method and does its stuff against that.
+ *
+ * Parameterized to run with different registry implementations.
 
 Review comment:
   Excellent.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362674414
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
 ##########
 @@ -81,46 +85,60 @@
 
   private static final Logger LOG = LoggerFactory.getLogger(TestScannersFromClientSide.class);
 
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static HBaseTestingUtility TEST_UTIL;
   private static byte [] ROW = Bytes.toBytes("testRow");
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static byte [] VALUE = Bytes.toBytes("testValue");
 
   @Rule
-  public TestName name = new TestName();
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 10 * 1024 * 1024);
-    TEST_UTIL.startMiniCluster(3);
-  }
+  public TestTableName name = new TestTableName();
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
+    if (TEST_UTIL != null) {
+      TEST_UTIL.shutdownMiniCluster();
+    }
   }
 
-  @Before
-  public void setUp() throws Exception {
-    // Nothing to do.
+  @Parameterized.Parameters
+  public static Collection parameters() {
+    return Arrays.asList(new Object[][] {
+        { MasterRegistry.class, 1},
+        { MasterRegistry.class, 2},
+        { ZKAsyncRegistry.class, 1}
+    });
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
-  @After
-  public void tearDown() throws Exception {
-    // Nothing to do.
+  public TestScannersFromClientSide(Class registryImpl, int numHedgedReqs) throws Exception {
+    if (TEST_UTIL != null) {
+      // Might be a remnant of another parameterized run. JUnit does not provide a hook to run after
+      // each parameterized test run.
+      TEST_UTIL.shutdownMiniCluster();
+    }
+    TEST_UTIL = new HBaseTestingUtility();
+    Configuration conf = TEST_UTIL.getConfiguration();
+    conf.setLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 10 * 1024 * 1024);
+    conf.setClass(HConstants.REGISTRY_IMPL_CONF_KEY, registryImpl, AsyncRegistry.class);
+    if (numHedgedReqs == 1) {
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, false);
+    } else {
+      Preconditions.checkArgument(numHedgedReqs > 1);
+      conf.setBoolean(HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY, true);
+      conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, numHedgedReqs);
+    }
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    // Multiple masters needed only when hedged reads for master registry are enabled.
+    builder.numMasters(numHedgedReqs > 1 ? 3 : 1).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
   }
 
   /**
    * Test from client side for batch of scan
    */
   @Test
   public void testScanBatch() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
 Review comment:
   same question about test method name in the `@Before` method.

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360515595
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
+      // This is the most general (and default) case. We want to avoid the thread creation and
+      // synchronization overhead and hence a special optimization for this case that just loops
+      // through the available masters in that order.
+      return doSequentialRPCs(rpcCall, isvalidResp, debug);
+    }
+    return doHedgedRPCs(rpcCall, isvalidResp, debug);
+  }
+
+  /**
+   * Makes RPCs in batches of {HConstants#MASTER_REGISTRY_NUM_HEDGED_REQS_KEY}. Currently should
+   * only be used if the request fan out is > 1. For the default case, refer to doSequentialRPCs().
+   */
+  private <RESP> Optional<RESP> doHedgedRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    Preconditions.checkState(requestFanOut > 1);
+    Preconditions.checkNotNull(masterRpcPool);
+    int i = 0;
+    while (i < masterServers.size()){
+      // Each iteration of loop picks requestFanOut masters
+      int subListSize = Math.min(masterServers.size(), i + requestFanOut);
+      List<ServerName> masterSubList = masterServers.subList(i, subListSize);
+      // Create a new RPC context for this batch of RPCs and submit the RPCs to the pool.
+      BatchRpcCtx<RESP> batchRpcCtx = new BatchRpcCtx<>();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Attempting to make rpc {} to batch {}.", debug,
+            masterSubList.stream().map(Objects::toString).collect(Collectors.toList()));
+      }
+      for (ServerName master: masterSubList) {
+        masterRpcPool.submit(new MasterRpc<>(batchRpcCtx, master, rpcCall, isvalidResp, debug));
+      }
+      if (batchRpcCtx.waitForResults(rpcTimeoutNs)) {
+        // Results set by some RPC, no point in doing rest of the calls.
+        return Optional.of(batchRpcCtx.getResult());
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Failed to make rpc {} to batch {}. Trying others.", debug,
+            masterSubList.stream().map(Objects::toString).collect(Collectors.toList()));
+      }
+      i = subListSize;
+    }
+    return Optional.empty();
+  }
+
+
+  /**
+   * Sequentially calls the masters to make an RPC in random order. This removes the unnecessary
+   * thread overhead and synchronization when hedged RPCs are disabled (maxRequestFanOut == 1).
+   */
+  private <RESP> Optional<RESP> doSequentialRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    Preconditions.checkState(requestFanOut == 1, "Invalid request fan out.");
+    for (ServerName master: masterServers) {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (!isvalidResp.apply(resp)) {
+          continue;
+        }
+        return Optional.of(resp);
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debug, master, e);
+      }
+    }
+    // Failed on all the masters.
+    return Optional.empty();
+  }
+
+  /**
+   * Util that generates a master stub for a given ServerName.
+   */
+  private ClientMetaService.BlockingInterface getMasterStub(ServerName server) throws IOException {
+    return ClientMetaService.newBlockingStub(
+        rpcClient.createBlockingRpcChannel(server, User.getCurrent(), rpcTimeoutNs));
+  }
+
+  @Override
+  public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+    CompletableFuture<RegionLocations> result = new CompletableFuture<>();
+    CompletableFuture.runAsync(() -> {
+      Optional<GetMetaRegionLocationsResponse> resp = doRPCs(stub -> stub.getMetaRegionLocations(
+          rpcControllerFactory.newController(), GetMetaRegionLocationsRequest.getDefaultInstance()),
+        (rpcResp) -> rpcResp.getMetaLocationsCount() != 0,"GetMetaRegionLocations()");
+      if (!resp.isPresent()) {
+        result.completeExceptionally(new MasterRegistryFetchException(masterServers,
+            "GetMetaRegionLocations()"));
+      }
+      List<HRegionLocation> regionLocations = new ArrayList<>();
+      resp.get().getMetaLocationsList().forEach(
+        location -> regionLocations.add(ProtobufUtil.toRegionLocation(location)));
+      result.complete(new RegionLocations(regionLocations));
+    });
+    return result;
+  }
+
+  @Override
+  public CompletableFuture<String> getClusterId() {
+    CompletableFuture<String> result = new CompletableFuture<>();
+    CompletableFuture.runAsync(() -> {
+      Optional<GetClusterIdResponse> resp = doRPCs(stub -> stub.getClusterId(
+          rpcControllerFactory.newController(), GetClusterIdRequest.getDefaultInstance()),
+          GetClusterIdResponse::hasClusterId, "GetClusterId()");
+      if (!resp.isPresent()) {
+        result.completeExceptionally(new MasterRegistryFetchException(masterServers,
+            "GetClusterId()"));
+      }
+      result.complete(resp.get().getClusterId());
+    });
+    return result;
+  }
+
+  @Override
+  public CompletableFuture<ServerName> getMasterAddress() {
 
 Review comment:
   s/getMasterAddress/getActiveMaster/ or getMaster ?

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360507791
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
 
 Review comment:
   This is like AsyncAdminRequestRetryingCaller#Callable only it throws a ServiceException. Should it take a Controller? Should it be in a class of its own?
   
   On ClientMetaService, 'meta' is overloaded in hbase. Usually it is about the hbase:meta table but here it is about something else? Should it be called something other than ClientMetaService? Somethign to do w/ Registry?

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362661992
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
 
 Review comment:
   Another use-case for a more generic `RetryingCallable` implementation.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362664837
 
 

 ##########
 File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
 ##########
 @@ -172,10 +172,17 @@
   public static final String MASTER_INFO_PORT = "hbase.master.info.port";
 
   /** Configuration key for the list of master host:ports **/
-  public static final String MASTER_ADDRS_KEY = "hbase.master.addrs";
+  public static final String MASTER_ADDRS_KEY = "hbase.masters";
 
   public static final String MASTER_ADDRS_DEFAULT =  "localhost:" + DEFAULT_MASTER_PORT;
 
+  /** Configuration to enable hedged reads on master registry **/
+  public static final String MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY =
+      "hbase.client.master_registry.enable_hedged_reads";
 
 Review comment:
   TODO: update this config key when something is settled for the `AsyncRegistry` PR.

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

[GitHub] [hbase] bharathv commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
bharathv commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-573888707
 
 
   > @apurtell, @saintstack no further comments? I'd like to merge this one today.
   
   Just had an offline chat with @apurtell about this. He mentioned he has no objections for the patch to go in today.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360499925
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,364 @@
+package org.apache.hadoop.hbase.client;
 
 Review comment:
   Missing license.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362694767
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
 ##########
 @@ -217,7 +215,9 @@ private void cleanupIdleConnections() {
         // have some pending calls on connection so we should not shutdown the connection outside.
         // The connection itself will disconnect if there is no pending call for maxIdleTime.
         if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) {
-          if (LOG.isTraceEnabled()) LOG.trace("Cleanup idle connection to " + conn.remoteId().address);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Cleanup idle connection to " + conn.remoteId().address);
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362662466
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    ((HBaseRpcController) controller).setFailed(lastBatchCtx.getLastFailedRpcReason());
 
 Review comment:
   Since you already have a precondition on the controller being an instance of `HBaseRpcController`, why not use that class as the internal interface throughout? Cast once up-front and be done with it.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362716456
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360541305
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,381 @@
+/*
+ * 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.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 org.apache.hadoop.hbase.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 org.apache.hadoop.hbase.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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
+      masterServers.add(ServerName.valueOf(masterAddr, ServerName.NON_STARTCODE));
+    }
+    // (Pseudo) Randomized so that not all clients hot spot the same set of masters.
+    Collections.shuffle(masterServers);
+    Preconditions.checkArgument(!masterServers.isEmpty());
+  }
+
+  /**
+   * Makes a given RPC to master servers.
+   * @param rpcCall Call to make.
+   * @param debug String used for debug logging the RPC details.
+   * @param <RESP> Response type for the RPC.
+   * @param isvalidResp Used to verify if the response returned from RPC is valid.
+   * @return Optional response from the RPCs to parsed masters.
+   */
+  @VisibleForTesting
+  <RESP> Optional<RESP> doRPCs(RpcCall<RESP> rpcCall,
+      Function<RESP, Boolean> isvalidResp, String debug) {
+    if (requestFanOut == 1) {
 
 Review comment:
   I was expecting for the first cut we'd do just one request to a random host on the list at a time, and retry with another random choice. (So above randomization of list is good and important.) This is what the zookeeper client does now so is no different from current state of play. 
   
   Hedged reading is ahead of the game.
   
   Good that it is off by default, though.
   
   Also, it's nice that fan out factor is configurable, but I would want an adaptive policy by default. Only if single requests are failing at some threshold of unacceptable probability (i suppose controlled by a config param) would you want to start loading up more than one per request in trade for faster response, hopefully, on average.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362695281
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
 
 Review comment:
   Actually didn't do that because the test coverage also includes BlockingRpcClient. Is there a way to know if we are inside a running test? That way I can add an assert that runs only in the release bits.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362718349
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
 ##########
 @@ -363,6 +365,101 @@ public void testAsyncEcho() throws IOException {
     }
   }
 
+  /**
+   * Tests the happy path for various request fan out values using a simple RPC hedged across
+   * a mix of running and failing servers.
+   */
+  @Test
+  public void testHedgedAsyncEcho() throws Exception {
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    // Create a mix of running and failing servers.
+    final int numRunningServers = 5;
+    final int numFailingServers = 3;
 
 Review comment:
   reworded.

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

[GitHub] [hbase] bharathv commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
bharathv commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-572742767
 
 
   Rebased on top of my other patch that was merged few minutes ago.

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364933868
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
 ##########
 @@ -20,41 +20,23 @@
 
 import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException;
-
-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.cache.CacheBuilder;
-import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
-import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
-import org.apache.hbase.thirdparty.com.google.protobuf.Message;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
-
-import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.UnknownHostException;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-
+import org.apache.commons.lang3.NotImplementedException;
 
 Review comment:
   Please don't use this. Use java.lang.UnsupportedOperationException

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362730792
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
 ##########
 @@ -136,6 +137,10 @@ public HMaster getMaster() {
     } catch (Exception e) {
       throw new IOException(e);
     }
+    // Needed if a master based registry is configured for internal cluster connections. Here, we
+    // just add the current master host port since we do not know other master addresses up front
+    // in mini cluster tests.
+    c.set(HConstants.MASTER_ADDRS_KEY, server.getServerName().getAddress().toString());
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362646157
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public List<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableList(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Function<T, Boolean> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc) {
+    return rpcResult -> {
+      if (rpcResult == null) {
+        future.completeExceptionally(
+            new MasterRegistryFetchException(masterServers, hrc.getFailed()));
+      }
+      if (!isValidResp.apply(rpcResult)) {
+        // Rpc returned ok, but result was malformed.
 
 Review comment:
   I take it failures that warrant a `DoNotRetryIOException` are handled by the RPC layer. Is there a reason why an RPC would succeed but the `isValidResp` would need to fail with a `DNRIOE` ?
   
   It would be nice if this was an interface was enriched to support adding meaningful logging into the loop. For instance, it would be nice to have `DEBUG` logging that said something like "GetClusterIdResponse does not contain a clusterId value. retrying." Right now it's just RPC into the void over and over without leaving a trail as to the error scenario.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362930798
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
 ##########
 @@ -44,35 +45,38 @@
 import org.apache.hadoop.hbase.HTestConst;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.TestTableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
 import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
+import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 
 /**
- * A client-side test, mostly testing scanners with various parameters.
+ * A client-side test, mostly testing scanners with various parameters. Parameterized on different
+ * registry implementations.
  */
 @Category({MediumTests.class, ClientTests.class})
+@RunWith(Parameterized.class)
 
 Review comment:
   ya, I think its per parameterized run, not across all of them. 

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362695728
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
 
 Review comment:
   I think the retries are an upper layer, above channel.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362715271
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
 
 Review comment:
   Done.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362666718
 
 

 ##########
 File path: hbase-protocol-shaded/src/main/protobuf/Master.proto
 ##########
 @@ -1200,3 +1200,47 @@ service HbckService {
   rpc FixMeta(FixMetaRequest)
     returns(FixMetaResponse);
 }
+
+/** Request and response to get the clusterID for this cluster */
+message GetClusterIdRequest {
+}
+message GetClusterIdResponse {
+  /** Not set if cluster ID could not be determined. */
+  optional string cluster_id = 1;
+}
+
+/** Request and response to get the currently active master name for this cluster */
+message GetActiveMasterRequest {
+}
+message GetActiveMasterResponse {
+  /** Not set if an active master could not be determined. */
+  optional ServerName server_name = 1;
+}
+
+/** Request and response to get the current list of meta region locations */
+message GetMetaRegionLocationsRequest {
+}
+message GetMetaRegionLocationsResponse {
+  /** Not set if meta region locations could not be determined. */
+  repeated RegionLocation meta_locations = 1;
 
 Review comment:
   So what if this included an `optional ServerName active_master = 2;` ? That way if the client happens to be in communication with the active master, it can skip the `GetActiveMaster` call entirely.

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

[GitHub] [hbase] ndimiduk merged pull request #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
ndimiduk merged pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954
 
 
   

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362670382
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
 ##########
 @@ -106,6 +111,30 @@ void handle(final String path) {
     }
   }
 
+  /**
+   * Fetches the active master's ServerName from zookeeper.
+   */
+  private void fetchAndSetActiveMasterServerName() {
+    LOG.debug("Attempting to fetch active master sn from zk");
+    try {
+      activeMasterServerName = MasterAddressTracker.getMasterAddress(watcher);
+    } catch (IOException | KeeperException e) {
+      // Log and ignore for now and re-fetch later if needed.
+      LOG.error("Error fetching active master information", e);
+    }
+  }
+
+  public Optional<ServerName> getActiveMasterServerName() {
+    if (!clusterHasActiveMaster.get()) {
+      return Optional.empty();
+    }
+    if (activeMasterServerName == null) {
+      fetchAndSetActiveMasterServerName();
+    }
+    // It could still be null, but return whatever we have.
+    return Optional.ofNullable(activeMasterServerName);
 
 Review comment:
   `ofNullable` protects against returning a null value. good. The comment is a little misleading because an empty `Optional` is discarded at the RPC layer.

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360510722
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
 
 Review comment:
   Return boolean which is true if succeeded in setting result and false otherwise? nit.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362696661
 
 

 ##########
 File path: hbase-common/src/main/java/org/apache/hadoop/hbase/util/PrettyPrinter.java
 ##########
 @@ -188,4 +191,18 @@ private static long humanReadableIntervalToSec(final String humanReadableInterva
     return ttl;
   }
 
+  /**
+   * Pretty prints a collection of any type to a string. Relies on toString() implementation of the
+   * object type.
+   * @param collection collection to pretty print.
+   * @return Pretty printed string for the collection.
+   */
+  public static String toString(Collection<?> collection) {
 
 Review comment:
   Ya thats another pain. Hoping the backports go smoothly. There will be a ton of rewrite.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362674098
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
 ##########
 @@ -0,0 +1,120 @@
+/*
+ * 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.HConstants.META_REPLICAS_NUM;
+import static org.junit.Assert.assertEquals;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestMasterRegistry {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterRegistry.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3);
+    StartMiniClusterOption.Builder builder = StartMiniClusterOption.builder();
+    builder.numMasters(3).numRegionServers(3);
+    TEST_UTIL.startMiniCluster(builder.build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * Generates a string of dummy master addresses in host:port format. Every other hostname won't
+   * have a port number.
+   */
+  private static String generateDummyMastersList(int size) {
+    List<String> masters = new ArrayList<>();
+    for (int i = 0; i < size; i++) {
+      masters.add(" localhost" + (i % 2 == 0 ? ":" + (1000 + i) : ""));
+    }
+    return String.join(",", masters);
+  }
+
+  /**
+   * Makes sure the master registry parses the master end points in the configuration correctly.
+   */
+  @Test public void TestMasterAddressParsing() {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    int numMasters = 10;
+    conf.set(HConstants.MASTER_ADDRS_KEY, generateDummyMastersList(numMasters));
+    try (MasterRegistry registry = new MasterRegistry(conf)) {
+      List<ServerName> parsedMasters = registry.getParsedMasterServers();
+      assertEquals(numMasters, parsedMasters.size());
+      for (int i = 0; i < numMasters; i++) {
+        ServerName sn = parsedMasters.get(i);
+        assertEquals("localhost", sn.getHostname());
+        if (i % 2 == 0) {
+          assertEquals(1000 + i, sn.getPort());
+        } else {
+          assertEquals(HConstants.DEFAULT_MASTER_PORT, sn.getPort());
+        }
+      }
+    }
+  }
+
+  @Test public void TestRegistryRPCs() throws Exception {
 
 Review comment:
   nit: lowercase method name.

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360515941
 
 

 ##########
 File path: hbase-common/src/test/java/org/apache/hadoop/hbase/TestTableName.java
 ##########
 @@ -51,7 +51,21 @@
    */
   @Override
   protected void starting(Description description) {
-    tableName = TableName.valueOf(description.getMethodName());
+    tableName = TableName.valueOf(cleanUpTestName(description.getMethodName()));
+  }
+
+  /**
+   * Helper to handle parameterized method names. Unlike regular test methods, parameterized method
+   * names look like 'foo[x]'. This is problematic for tests that use this name for HBase tables.
+   * This helper strips out the parameter suffixes.
+   * @return current test method name with out parameterized suffixes.
+   */
+  private static String cleanUpTestName(String methodName) {
 
 Review comment:
   smile... yeah, this is a pain.

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360505681
 
 

 ##########
 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.
 
 Review comment:
   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 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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360539166
 
 

 ##########
 File path: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
 ##########
 @@ -172,10 +172,17 @@
   public static final String MASTER_INFO_PORT = "hbase.master.info.port";
 
   /** Configuration key for the list of master host:ports **/
-  public static final String MASTER_ADDRS_KEY = "hbase.master.addrs";
+  public static final String MASTER_ADDRS_KEY = "hbase.masters";
 
 Review comment:
   I like this

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362663990
 
 

 ##########
 File path: hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegistryLeak.java
 ##########
 @@ -70,7 +71,7 @@ public void close() {
 
   @BeforeClass
   public static void setUp() {
-    CONF.setClass(AsyncRegistryFactory.REGISTRY_IMPL_CONF_KEY, AsyncRegistryForTest.class,
+    CONF.setClass(HConstants.REGISTRY_IMPL_CONF_KEY, AsyncRegistryForTest.class,
 
 Review comment:
   👍 

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r363839020
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
 ##########
 @@ -527,6 +528,12 @@ public RpcChannel createRpcChannel(ServerName sn, User user, int rpcTimeout)
     return new RpcChannelImplementation(this, createAddr(sn), user, rpcTimeout);
   }
 
+  @Override
+  public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
+      throws UnknownHostException {
+    throw new NotImplementedException("Should be called on an AbstractRpcClient implementation.");
 
 Review comment:
   I think you mean NettyRpcClient. Simply "method not supported for this implementation." is fine.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361235888
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
 
 Review comment:
   Yikes. Added some test coverage too.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r361236611
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/MasterRegistryFetchException.java
 ##########
 @@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Exception thrown when an master registry RPC fails in client. The exception includes the list of
+ * masters to which RPC was attempted.
+ */
+@InterfaceAudience.Private
+public class MasterRegistryFetchException extends HBaseIOException {
+  public MasterRegistryFetchException(List<ServerName> masters, String failedRPC) {
+    super(String.format("Exception making rpc %s to masters %s", failedRPC,
+        masters.stream().map(Objects::toString).collect(Collectors.toList())));
 
 Review comment:
   Moved this to PrettyPrinter as a general util for any collection. 
   
   > but this one is less readable than a simple string join using Collections.
   
   Fair point. Switched to good old better performing loops.

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-572856681
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m 34s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 10 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 37s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 45s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   1m 47s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 34s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   5m  5s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   5m  5s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   7m  0s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | +1 :green_heart: |  checkstyle  |   0m 32s |  hbase-client: The patch generated 0 new + 0 unchanged - 12 fixed = 0 total (was 12)  |
   | +1 :green_heart: |  checkstyle  |   1m 30s |  hbase-server: The patch generated 0 new + 223 unchanged - 50 fixed = 223 total (was 273)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  4s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 29s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   7m  7s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  8s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 55s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 167m 11s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m 14s |  The patch does not generate ASF License warnings.  |
   |  |   | 250m 44s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestFromClientSide |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/13/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 2294597fcad8 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / d2d7b2bb24 |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/13/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/13/testReport/ |
   | Max. process+thread count | 5489 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/13/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

[GitHub] [hbase] Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-571884012
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 11s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 10 new or modified test files.  |
   ||| _ HBASE-18095/client-locate-meta-no-zookeeper Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 50s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  compile  |   1m 45s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  checkstyle  |   2m 33s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +1 :green_heart: |  shadedjars  |   5m  2s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 17s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   | +0 :ok: |  spotbugs  |   4m 55s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   6m 50s |  HBASE-18095/client-locate-meta-no-zookeeper passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 26s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 46s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  hbase-common: The patch generated 0 new + 4 unchanged - 1 fixed = 4 total (was 5)  |
   | +1 :green_heart: |  checkstyle  |   0m 33s |  hbase-client: The patch generated 0 new + 0 unchanged - 12 fixed = 0 total (was 12)  |
   | +1 :green_heart: |  checkstyle  |   1m 30s |  hbase-server: The patch generated 0 new + 223 unchanged - 50 fixed = 223 total (was 273)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  6s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 25s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   1m 16s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   7m 22s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   3m  8s |  hbase-common in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 55s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 174m 22s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   1m  9s |  The patch does not generate ASF License warnings.  |
   |  |   | 253m 59s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.client.TestFromClientSide |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/11/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/954 |
   | JIRA Issue | HBASE-23305 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 629e2ad8c337 4.15.0-66-generic #75-Ubuntu SMP Tue Oct 1 05:24:09 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/HBase-PreCommit-GitHub-PR_PR-954/out/precommit/personality/provided.sh |
   | git revision | HBASE-18095/client-locate-meta-no-zookeeper / d016cd9962 |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/11/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/11/testReport/ |
   | Max. process+thread count | 4468 (vs. ulimit of 10000) |
   | modules | C: hbase-common hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-954/11/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360543287
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
 
 Review comment:
   We don't need a new one, right? Use existing RPC/Call facilities and set a retry policy using the factory where there are no retries. 

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

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

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360512870
 
 

 ##########
 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;
+
+  // A simple interface that callers can implement to make an RPC to master. This is used to
+  // abstract out the logic needed to hedge the requests to multiple masters. For more details, look
+  // at doRPCs().
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface RpcCall<RESP> {
+    RESP doRpc(ClientMetaService.BlockingInterface stub) throws ServiceException;
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different masters fetching the result. All the methods are
+   * thread-safe.
+   * @param <RESP> Return response type for the RPCs.
+   */
+  private class BatchRpcCtx<RESP> {
+    // Result set by the thread finishing first. Set only once.
+    private AtomicReference<RESP> result;
+    // Caller waits on this latch being set.
+    private CountDownLatch resultsReady;
+    // Book-keeping for number of failed RPCs.
+    private AtomicInteger failedRPCs;
+
+    BatchRpcCtx() {
+      result = new AtomicReference<>();
+      // We set this to 1, so that the first successful RPC result is returned to the client.
+      resultsReady = new CountDownLatch(1);
+      failedRPCs = new AtomicInteger(0);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(RESP result) {
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+      }
+    }
+
+    /**
+     * Caller can use this method to wait for results to be fetched.
+     * @param timeoutNs Waits until this timeout hits or the results are set. Whatever happens
+     *                  first.
+     * @return True if the results are ready. False otherwise.
+     */
+    public boolean waitForResults(int timeoutNs) {
+      try {
+        return resultsReady.await(timeoutNs, TimeUnit.NANOSECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+        return false;
+      }
+    }
+
+    /**
+     * Helper to increment the number of failed RPCs.
+     */
+    public void incrementFailedRPCs() {
+      failedRPCs.incrementAndGet();
+    }
+
+    /**
+     * Onus is on the caller to wait for the results and call this.
+     * @return the current result.
+     */
+    public RESP getResult() {
+      return result.get();
+    }
+  }
+
+  /**
+   * A runnable implementation of an RPC call to a given master. Updates the results in a shared
+   * rpc context.
+   * @param <RESP> Response type of the RPC.
+   */
+  private class MasterRpc<RESP> implements Runnable {
+    private final BatchRpcCtx<RESP> rpcCtx;
+    private final ServerName master;
+    private final RpcCall<RESP> rpcCall;
+    private final Function<RESP, Boolean> isValidResp;
+    private final String debugStr;
+
+    MasterRpc(BatchRpcCtx<RESP> rpcCtx, ServerName master, RpcCall<RESP> rpcCall,
+       Function<RESP, Boolean> isValidResp, String debugStr) {
+      this.rpcCtx = rpcCtx;
+      this.master = master;
+      this.rpcCall = rpcCall;
+      this.isValidResp = isValidResp;
+      this.debugStr = debugStr;
+    }
+
+    @Override
+    public void run() {
+      try {
+        RESP resp = rpcCall.doRpc(getMasterStub(master));
+        if (isValidResp.apply(resp)) {
+          // Valid result, set if not set by other threads.
+          rpcCtx.setResultIfNotSet(resp);
+          return;
+        }
+      } catch (Exception e) {
+        LOG.warn("Error calling {} on master {}. Trying other masters.", debugStr, master, e);
+      }
+      rpcCtx.incrementFailedRPCs();
+    }
+  }
+
+  MasterRegistry(Configuration conf) {
+    masterServers = new ArrayList<>();
+    requestFanOut =
+        conf.getInt(MASTER_REGISTRY_NUM_HEDGED_REQS_KEY, MASTER_REGISTRY_NUM_HEDGED_REQS_DEFAULT);
+    Preconditions.checkArgument(requestFanOut >= 1);
+    if (requestFanOut > 1) {
+      masterRpcPool = Executors.newFixedThreadPool(requestFanOut,
+          new ThreadFactoryBuilder().setDaemon(true).setNameFormat("MasterRegistryRPC-%d").build());
+    }
+    parseMasterAddrs(conf);
+    rpcTimeoutNs = (int) Math.min(Integer.MAX_VALUE,
+        TimeUnit.MILLISECONDS.toNanos(conf.getLong(HBASE_RPC_TIMEOUT_KEY,
+        DEFAULT_HBASE_RPC_TIMEOUT)));
+    // TODO(HBASE-23330): Fix clients using cluster ID based token auth.
+    rpcClient = RpcClientFactory.createClient(conf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(conf);
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(",")) {
 
 Review comment:
   nit: Fragile? If space after comma, this breaks?

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362913375
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
 
 Review comment:
   If this class only support an async client, why is it being called from tests using the blocking client? Can the tests' not be reduced?

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362913523
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
 
 Review comment:
   (I very much prefer to avoid test-aware code in production)

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

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

Posted by GitBox <gi...@apache.org>.
apurtell commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r360547820
 
 

 ##########
 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.
 
 Review comment:
   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. 

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362655233
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
 
 Review comment:
   This instance variable needs to be templatized. Probably `<?>` is sufficient.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362650510
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
 ##########
 @@ -217,7 +215,9 @@ private void cleanupIdleConnections() {
         // have some pending calls on connection so we should not shutdown the connection outside.
         // The connection itself will disconnect if there is no pending call for maxIdleTime.
         if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) {
-          if (LOG.isTraceEnabled()) LOG.trace("Cleanup idle connection to " + conn.remoteId().address);
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Cleanup idle connection to " + conn.remoteId().address);
 
 Review comment:
   nit: use format string.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362642647
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
 ##########
 @@ -0,0 +1,218 @@
+/*
+ * 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.HConstants.MASTER_ADDRS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_ADDRS_KEY;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT;
+import static org.apache.hadoop.hbase.HConstants.MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+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.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.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Function;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+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 org.apache.hadoop.hbase.HConstants#MASTER_ADDRS_KEY}.
+ *
+ * It supports hedged reads, which can be enabled by setting
+ * {@value org.apache.hadoop.hbase.HConstants#MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY} to True. Fan
+ * out the requests batch is controlled by
+ * {@value org.apache.hadoop.hbase.HConstants#HBASE_RPCS_HEDGED_REQS_FANOUT_KEY}.
+ *
+ * TODO: Handle changes to the configuration dynamically without having to restart the client.
+ */
+@InterfaceAudience.Private
+public class MasterRegistry implements AsyncRegistry {
+  private static final String MASTER_ADDRS_CONF_SEPARATOR = ",";
+
+  // Configured list of masters to probe the meta information from.
+  private final List<ServerName> masterServers;
+
+  // RPC client used to talk to the masters.
+  private final RpcClient rpcClient;
+  private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+
+  MasterRegistry(Configuration conf) {
+    boolean hedgedReadsEnabled = conf.getBoolean(MASTER_REGISTRY_ENABLE_HEDGED_READS_KEY,
+        MASTER_REGISTRY_ENABLE_HEDGED_READS_DEFAULT);
+    Configuration finalConf;
+    if (!hedgedReadsEnabled) {
+      // If hedged reads are disabled, it is equivalent to setting a fan out of 1. We make a copy of
+      // the configuration so that other places reusing this reference is not affected.
+      finalConf = new Configuration(conf);
+      finalConf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, 1);
+    } else {
+      finalConf = conf;
+    }
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE, conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY,
+        HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
+    masterServers = new ArrayList<>();
+    parseMasterAddrs(finalConf);
+    rpcClient = RpcClientFactory.createClient(finalConf, HConstants.CLUSTER_ID_DEFAULT);
+    rpcControllerFactory = RpcControllerFactory.instantiate(finalConf);
+  }
+
+  /**
+   * @return Stub needed to make RPC using a hedged channel to the master end points.
+   */
+  private ClientMetaService.Interface getMasterStub() throws IOException {
+    return ClientMetaService.newStub(
+        rpcClient.createHedgedRpcChannel(masterServers, User.getCurrent(), rpcTimeoutMs));
+  }
+
+  /**
+   * Parses the list of master addresses from the provided configuration. Supported format is
+   * comma separated host[:port] values. If no port number if specified, default master port is
+   * assumed.
+   * @param conf Configuration to parse from.
+   */
+  private void parseMasterAddrs(Configuration conf) {
+    String configuredMasters = conf.get(MASTER_ADDRS_KEY, MASTER_ADDRS_DEFAULT);
+    for (String masterAddr: configuredMasters.split(MASTER_ADDRS_CONF_SEPARATOR)) {
+      HostAndPort masterHostPort =
+          HostAndPort.fromString(masterAddr.trim()).withDefaultPort(HConstants.DEFAULT_MASTER_PORT);
+      masterServers.add(ServerName.valueOf(masterHostPort.toString(), ServerName.NON_STARTCODE));
+    }
+    Preconditions.checkArgument(!masterServers.isEmpty(), "At least one master address is needed");
+  }
+
+  @VisibleForTesting
+  public List<ServerName> getParsedMasterServers() {
+    return Collections.unmodifiableList(masterServers);
+  }
+
+  /**
+   * Returns a call back that can be passed along to the non-blocking rpc call. It is invoked once
+   * the rpc finishes and the response is propagated to the passed future.
+   * @param future Result future to which the rpc response is propagated.
+   * @param isValidResp Checks if the rpc response has a valid result.
+   * @param transformResult Transforms the result to a different form as expected by callers.
+   * @param hrc RpcController instance for this rpc.
+   * @param <T> RPC result type.
+   * @param <R> Transformed type of the result.
+   * @return A call back that can be embedded in the non-blocking rpc call.
+   */
+  private <T, R> RpcCallback<T> getRpcCallBack(CompletableFuture<R> future,
+      Function<T, Boolean> isValidResp, Function<T, R> transformResult, HBaseRpcController hrc) {
 
 Review comment:
   nit: use `java.util.function.Predicate` instead of `Function<T, Boolean>` for `isValidResp`.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362675855
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestRegisterPeerWorkerWhenRestarting.java
 ##########
 @@ -53,7 +52,7 @@
 
   public static final class HMasterForTest extends HMaster {
 
-    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+    public HMasterForTest(Configuration conf) throws IOException {
 
 Review comment:
   Your second miracle.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362942554
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HedgedRpcChannel.java
 ##########
 @@ -0,0 +1,266 @@
+/*
+ * 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.ipc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.PrettyPrinter;
+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.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+
+/**
+ * A non-blocking implementation of RpcChannel that hedges requests to multiple service end points.
+ * First received response is returned to the caller. This abstracts out the logic needed to batch
+ * requests to multiple end points underneath and presents itself as a single logical RpcChannel to
+ * the client.
+ *
+ * Hedging Details:
+ * ---------------
+ * - Hedging of RPCs happens in multiple batches. In each iteration, we select a 'batch' of address
+ * end points to make the call to. We do multiple iterations until we get a proper response to the
+ * rpc call or all the service addresses are exhausted, which ever happens first.
+ *
+ * - We randomize the addresses up front so that the batch order per client is non deterministic.
+ * This avoids hot spots on the service side. The size of each batch is controlled via 'fanOutSize'.
+ * Higher fanOutSize implies we make more rpc calls in a single batch. One needs to mindful of the
+ * load on the client and server side when configuring the fan out.
+ *
+ * - In a happy case, once we receive a response from one end point, we cancel all the
+ * other inflight rpcs in the same batch and return the response to the caller. If we do not get a
+ * valid response from any address end point, we propagate the error back to the caller.
+ *
+ * - Rpc timeouts are applied to every hedged rpc.
+ *
+ * - Callers need to be careful about what rpcs they are trying to hedge. Not every kind of call can
+ * be hedged (for example: cluster state changing rpcs).
+ *
+ * (TODO) Retries and Adaptive hedging policy:
+ * ------------------------------------------
+ *
+ * - No retries are handled at the channel level. Retries can be built in upper layers. However the
+ * question is, do we even need retries? Hedging in fact is a substitute for retries.
+ *
+ * - Clearly hedging puts more load on the service side. To mitigate this, we can make the hedging
+ * policy more adaptive. In most happy cases, the rpcs from the first few end points should return
+ * right away (especially short lived rpcs, that do not take up much time). In such cases, hedging
+ * is not needed. So, the idea is to make this request pattern pluggable so that the requests are
+ * hedged only when needed.
+ */
+class HedgedRpcChannel implements RpcChannel {
+  private static final Logger LOG = LoggerFactory.getLogger(HedgedRpcChannel.class);
+
+  private final AbstractRpcClient rpcClient;
+  // List of service addresses to hedge the requests to.
+  private final List<InetSocketAddress> addrs;
+  private final User ticket;
+  private final int rpcTimeout;
+  // Controls the size of request fan out (number of rpcs per a single batch).
+  private final int fanOutSize;
+
+  /**
+   * A simple rpc call back implementation to notify the batch context if any rpc is successful.
+   */
+  private static class BatchRpcCtxCallBack implements RpcCallback<Message> {
+    private  final BatchRpcCtx batchRpcCtx;
+    private final HBaseRpcController rpcController;
+    BatchRpcCtxCallBack(BatchRpcCtx batchRpcCtx, HBaseRpcController rpcController) {
+      this.batchRpcCtx = batchRpcCtx;
+      this.rpcController = rpcController;
+    }
+    @Override
+    public void run(Message result) {
+      batchRpcCtx.setResultIfNotSet(result, rpcController);
+    }
+  }
+
+  /**
+   * A shared RPC context between a batch of hedged RPCs. Tracks the state and helpers needed to
+   * synchronize on multiple RPCs to different end points fetching the result. All the methods are
+   * thread-safe.
+   */
+  private static class BatchRpcCtx {
+    // Result set by the thread finishing first. Set only once.
+    private final AtomicReference<Message> result = new AtomicReference<>();
+    // Caller waits on this latch being set.
+    // We set this to 1, so that the first successful RPC result is returned to the client.
+    private CountDownLatch resultsReady = new CountDownLatch(1);
+    // Failed rpc book-keeping.
+    private AtomicInteger failedRpcCount = new AtomicInteger();
+    // All the call handles for this batch.
+    private final List<Call> callsInFlight = Collections.synchronizedList(new ArrayList<>());
+
+    // Target addresses.
+    private final List<InetSocketAddress> addresses;
+    // Called when the result is ready.
+    private final RpcCallback<Message> callBack;
+    // Last failed rpc's exception. Used to propagate the reason to the controller.
+    private IOException lastFailedRpcReason;
+
+
+    BatchRpcCtx(List<InetSocketAddress> addresses, RpcCallback<Message> callBack) {
+      this.addresses = addresses;
+      this.callBack = Preconditions.checkNotNull(callBack);
+    }
+
+    /**
+     * Sets the result only if it is not already set by another thread. Thread that successfully
+     * sets the result also count downs the latch.
+     * @param result Result to be set.
+     */
+    public void setResultIfNotSet(Message result, HBaseRpcController rpcController) {
+      if (result == null) {
+        incrementFailedRpcs(rpcController.getFailed());
+        return;
+      }
+      if (this.result.compareAndSet(null, result)) {
+        resultsReady.countDown();
+        // Cancel all pending in flight calls.
+        for (Call call: callsInFlight) {
+          // It is ok to do it for all calls as it is a no-op if the call is already done.
+          call.setException(new CallCancelledException("Hedged call succeeded."));
+        }
+      }
+    }
+
+    /**
+     * Waits until the results are populated and calls the callback if the call is successful.
+     * @return true for successful rpc and false otherwise.
+     */
+    public boolean waitForResults() {
+      try {
+        // We do not set a timeout on await() because we rely on the underlying RPCs to timeout if
+        // something on the remote is broken. Worst case we should wait for rpc time out to kick in.
+        resultsReady.await();
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for batched master RPC results. Aborting wait.", e);
+      }
+      Message message = result.get();
+      if (message != null) {
+        callBack.run(message);
+        return true;
+      }
+      return false;
+    }
+
+    public void addCallInFlight(Call c) {
+      callsInFlight.add(c);
+    }
+
+    public void incrementFailedRpcs(IOException reason) {
+      if (failedRpcCount.incrementAndGet() == addresses.size()) {
+        lastFailedRpcReason = reason;
+        // All the rpcs in this batch have failed. Invoke the waiting threads.
+        resultsReady.countDown();
+      }
+    }
+
+    public IOException getLastFailedRpcReason() {
+      return lastFailedRpcReason;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Batched rpc for target(s) %s", PrettyPrinter.toString(addresses));
+    }
+  }
+
+  public HedgedRpcChannel(AbstractRpcClient<?> rpcClient, List<InetSocketAddress> addrs,
+      User ticket, int rpcTimeout, int fanOutSize) {
+    this.rpcClient = rpcClient;
+    this.addrs = Preconditions.checkNotNull(addrs);
+    Preconditions.checkArgument(this.addrs.size() >= 1);
+    // For non-deterministic client query pattern. Not all clients want to hedge RPCs in the same
+    // order, creating hot spots on the service end points.
+    Collections.shuffle(this.addrs);
+    this.ticket = ticket;
+    this.rpcTimeout = rpcTimeout;
+    this.fanOutSize = fanOutSize;
+  }
+
+  private HBaseRpcController applyRpcTimeout(RpcController controller) {
+    // There is no reason to use any other implementation of RpcController.
+    Preconditions.checkState(controller instanceof HBaseRpcController);
+    HBaseRpcController hBaseRpcController = (HBaseRpcController) controller;
+    int rpcTimeoutToSet =
+        hBaseRpcController.hasCallTimeout() ? hBaseRpcController.getCallTimeout() : rpcTimeout;
+    HBaseRpcController response = new HBaseRpcControllerImpl();
+    response.setCallTimeout(rpcTimeoutToSet);
+    return response;
+  }
+
+  public void doCallMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    int i = 0;
+    BatchRpcCtx lastBatchCtx = null;
+    while (i < addrs.size()) {
+      // Each iteration picks fanOutSize addresses to run as batch.
+      int batchEnd = Math.min(addrs.size(), i + fanOutSize);
+      List<InetSocketAddress> addrSubList = addrs.subList(i, batchEnd);
+      BatchRpcCtx batchRpcCtx = new BatchRpcCtx(addrSubList, done);
+      lastBatchCtx = batchRpcCtx;
+      LOG.debug("Attempting request {}, {}", method.getName(), batchRpcCtx);
+      for (InetSocketAddress address : addrSubList) {
+        HBaseRpcController rpcController = applyRpcTimeout(controller);
+        // ** WARN ** This is a blocking call if the underlying connection for the rpc client is
+        // a blocking implementation (ex: BlockingRpcConnection). That essentially serializes all
+        // the write calls. Handling blocking connection means that this should be run in a separate
+        // thread and hence more code complexity. Is it ok to handle only non-blocking connections?
+        // Should we have a check in the constructor if the underlying connection is a blocking
+        // impl and then log some warning?
+        batchRpcCtx.addCallInFlight(rpcClient.callMethod(method, rpcController, request,
+            responsePrototype, ticket, address,
+            new BatchRpcCtxCallBack(batchRpcCtx, rpcController)));
+      }
+      if (batchRpcCtx.waitForResults()) {
+        return;
+      }
+      // TODO: Sleep between batches?
+      // Entire batch has failed, lets try the next batch.
+      LOG.debug("Failed request {}, {}.", method.getName(), batchRpcCtx);
+      i = batchEnd;
+    }
+    // All the batches failed, mark it a failed rpc.
+    // Propagate the failure reason. We propagate the last batch's last failing rpc reason.
+    // Can we do something better?
+    ((HBaseRpcController) controller).setFailed(lastBatchCtx.getLastFailedRpcReason());
+    done.run(null);
+  }
+
+  @Override
+  public void callMethod(Descriptors.MethodDescriptor method, RpcController controller,
+      Message request, Message responsePrototype, RpcCallback<Message> done) {
+    // To make the channel non-blocking, we run the actual doCalMethod() async. The call back is
+    // called once the hedging finishes.
+    CompletableFuture.runAsync(
 
 Review comment:
   Ya, the tests seem to pass ok so far. Will keep an eye on this once we make this registry as the default implementation. Then we probably will have more data points around performance.

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

[GitHub] [hbase] bharathv commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
bharathv commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-570749954
 
 
   Fixed the check style issues and rebased on the latest tip of the feature branch (and squashed all the commits). Test failures seem to be flakes and run fine for me locally. I think this is good to go for another round of review.

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

[GitHub] [hbase] bharathv commented on issue #954: HBASE-23305: Master based registry implementation

Posted by GitBox <gi...@apache.org>.
bharathv commented on issue #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#issuecomment-572324821
 
 
   Works for me. Try re-running? I noticed the mini cluster in cluster is very flaky, especially when you run it in a dev setup.

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364360377
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -4260,7 +4279,13 @@ public void testUnmanagedHConnection() throws IOException {
    */
   @Test
   public void testUnmanagedHConnectionReconnect() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class registryImpl = conf.getClass(
+        HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
+    // This test does not make sense for MasterRegistry since it stops the only master in the
+    // cluster and starts a new master without populating the underlying config for the connection.
+    Assume.assumeFalse(registryImpl.equals(MasterRegistry.class));
 
 Review comment:
   Yep

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362673301
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -2642,7 +2676,7 @@ public void testJiraTest1182() throws Exception {
    */
   @Test
   public void testJiraTest52() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    final TableName tableName = name.getTableName();
 
 Review comment:
   You are a saint.
   
   I hate to ask, but is the value of `name.getMethodName()` available to the `@Before` method? Maybe this could also be resolved by a chain of test rules...

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

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

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r362718390
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
 ##########
 @@ -363,6 +365,101 @@ public void testAsyncEcho() throws IOException {
     }
   }
 
+  /**
+   * Tests the happy path for various request fan out values using a simple RPC hedged across
+   * a mix of running and failing servers.
+   */
+  @Test
+  public void testHedgedAsyncEcho() throws Exception {
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    // Create a mix of running and failing servers.
+    final int numRunningServers = 5;
+    final int numFailingServers = 3;
+    final int numServers = numRunningServers + numFailingServers;
+    for (int i = 0; i < numRunningServers; i++) {
+      RpcServer rpcServer = createRpcServer(null, "testRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    for (int i = 0; i < numFailingServers; i++) {
+      RpcServer rpcServer = createTestFailingRpcServer(null, "testFailingRpcServer" + i,
+          Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
+          SERVICE, null)), new InetSocketAddress("localhost", 0), CONF,
+          new FifoRpcScheduler(CONF, 1));
+      rpcServer.start();
+      addresses.add(rpcServer.getListenerAddress());
+      rpcServers.add(rpcServer);
+    }
+    Configuration conf = HBaseConfiguration.create();
+    try (AbstractRpcClient<?> client = createRpcClient(conf)) {
+      // Try out various fan out values starting from 1 -> numServers.
+      for (int reqFanOut = 1; reqFanOut <= numServers; reqFanOut++) {
+        // Update the client's underlying conf, should be ok for the test.
+        LOG.debug("Testing with request fan out: " + reqFanOut);
+        conf.setInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY, reqFanOut);
+        Interface stub = newStub(client, addresses);
+        BlockingRpcCallback<EchoResponseProto> done = new BlockingRpcCallback<>();
+        stub.echo(new HBaseRpcControllerImpl(),
+            EchoRequestProto.newBuilder().setMessage("hello").build(), done);
+        TestProtos.EchoResponseProto responseProto = done.get();
+        assertNotNull(responseProto);
+        assertEquals("hello", responseProto.getMessage());
+        LOG.debug("Ended test with request fan out: " + reqFanOut);
+      }
+    } finally {
+      for (RpcServer rpcServer: rpcServers) {
+        rpcServer.stop();
+      }
+    }
+  }
+
+  @Test
+  public void testHedgedAsyncTimeouts() throws Exception {
+    List<RpcServer> rpcServers = new ArrayList<>();
+    List<InetSocketAddress> addresses = new ArrayList<>();
+    // Create a mix of running and failing servers.
 
 Review comment:
   done.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r363448043
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
 ##########
 @@ -527,6 +528,21 @@ public RpcChannel createRpcChannel(ServerName sn, User user, int rpcTimeout)
     return new RpcChannelImplementation(this, createAddr(sn), user, rpcTimeout);
   }
 
+  @Override
+  public RpcChannel createHedgedRpcChannel(Set<ServerName> sns, User user, int rpcTimeout)
+      throws UnknownHostException {
+    final int hedgedRpcFanOut = conf.getInt(HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_KEY,
+        HConstants.HBASE_RPCS_HEDGED_REQS_FANOUT_DEFAULT);
+    Set<InetSocketAddress> addresses = new HashSet<>();
+    for (ServerName sn: sns) {
+      addresses.add(createAddr(sn));
+    }
+    Preconditions.checkState(this instanceof NettyRpcClient,
+        "Hedging only supported for non-blocking connection implementations.");
 
 Review comment:
   Might as well use the inheritance language feature as intended. The abstract parent can provide no implementation at all, or just throw the exception. The implementation of `NettyRpcClient` can override that implementation with its own.

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

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

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #954: HBASE-23305: Master based registry implementation
URL: https://github.com/apache/hbase/pull/954#discussion_r364342781
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
 ##########
 @@ -4260,7 +4279,13 @@ public void testUnmanagedHConnection() throws IOException {
    */
   @Test
   public void testUnmanagedHConnectionReconnect() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
+    Configuration conf = TEST_UTIL.getConfiguration();
+    Class registryImpl = conf.getClass(
+        HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY, ZKConnectionRegistry.class);
+    // This test does not make sense for MasterRegistry since it stops the only master in the
+    // cluster and starts a new master without populating the underlying config for the connection.
+    Assume.assumeFalse(registryImpl.equals(MasterRegistry.class));
 
 Review comment:
   My only gripe about this is that surefire will warn about skipped methods.

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