bharathv commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r470724925



##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -226,17 +276,19 @@ private IOException badResponse(String debug) {
 
   private <T extends Message> CompletableFuture<T> call(Callable<T> callable,
     Predicate<T> isValidResp, String debug) {
+    Set<ServerName> masterServers = masterAddr2Stub.keySet();

Review comment:
       Right, done.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -259,17 +311,40 @@ private RegionLocations 
transformMetaRegionLocations(GetMetaRegionLocationsRespo
       .thenApply(GetClusterIdResponse::getClusterId);
   }
 
-  private ServerName transformServerName(GetActiveMasterResponse resp) {
-    return ProtobufUtil.toServerName(resp.getServerName());
+  private static boolean hasActiveMaster(GetMastersResponse resp) {
+    List<GetMastersResponseEntry> activeMasters =
+        
resp.getMasterServersList().stream().filter(GetMastersResponseEntry::getIsActive).collect(
+        Collectors.toList());
+    return activeMasters.size() == 1;
+  }
+
+  private static ServerName filterActiveMaster(GetMastersResponse resp) {
+    List<GetMastersResponseEntry> activeMasters =
+        
resp.getMasterServersList().stream().filter(GetMastersResponseEntry::getIsActive).collect(
+            Collectors.toList());
+    Preconditions.checkState(activeMasters.size() == 1);

Review comment:
       Don't think so. If you see getActiveMaster() call, "hasActiveMaster" is 
applied on the response. If that fails, we throw right away. This is only 
called on "thenApply()" which filters the output. Given the way the code is 
structured, I had to do it twice (since the call() takes a predicate that 
returns a boolean). 

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterAddressRefresher.java
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import 
org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ClientMetaService;
+
+/**
+ * Thread safe utility that keeps master end points used by {@link 
MasterRegistry} up to date. This
+ * uses the RPC {@link ClientMetaService#getMasters} to fetch the latest list 
of registered masters.
+ * By default the refresh happens periodically (configured via
+ * {@link #PERIODIC_REFRESH_INTERVAL_SECS}). The refresh can also be triggered 
on demand via
+ * {@link #refreshNow()}. To prevent a flood of on-demand refreshes we expect 
that any attempts two
+ * should be spaced at least {@link #MIN_SECS_BETWEEN_REFRESHES} seconds apart.
+ */
+@InterfaceAudience.Private
+public class MasterAddressRefresher implements Closeable {
+  private static final Logger LOG = 
LoggerFactory.getLogger(MasterAddressRefresher.class);
+  public static final String PERIODIC_REFRESH_INTERVAL_SECS =
+      "hbase.client.master_registry.refresh_interval_secs";
+  private static final int PERIODIC_REFRESH_INTERVAL_SECS_DEFAULT = 300;
+  public static final String MIN_SECS_BETWEEN_REFRESHES =
+      "hbase.client.master_registry.min_secs_between_refreshes";
+  private static final long MIN_SECS_BETWEEN_REFRESHES_DEFAULT = 60;
+
+  private final ExecutorService pool;
+  private final MasterRegistry registry;
+  private final long periodicRefreshMs;
+  private final long timeBetweenRefreshesMs;
+  private final Object refreshMasters = new Object();
+
+  @Override
+  public void close() {
+    pool.shutdownNow();
+  }
+
+  /**
+   * Thread that refreshes the master end points until it is interrupted via 
{@link #close()}.
+   * Multiple callers attempting to refresh at the same time synchronize on 
{@link #refreshMasters}.
+   */
+  private class RefreshThread implements Runnable {
+    @Override
+    public void run() {
+      long lastRpcTs = 0;
+      while (!Thread.interrupted()) {
+        try {
+          // Spurious wake ups are okay, worst case we make an extra RPC call 
to refresh. We won't
+          // have duplicate refreshes because once the thread is past the 
wait(), notify()s are
+          // ignored until the thread is back to the waiting state.
+          synchronized (refreshMasters) {
+            refreshMasters.wait(periodicRefreshMs);
+          }
+          long currentTs = EnvironmentEdgeManager.currentTime();
+          if (lastRpcTs != 0 && currentTs - lastRpcTs <= 
timeBetweenRefreshesMs) {
+            continue;
+          }
+          lastRpcTs = currentTs;
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new 
HashSet<>(registry.getMasters().get());
+          registry.populateMasterStubs(newMasters);
+          LOG.debug("Finished refreshing master end points. {}", newMasters);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted during wait, aborting 
refresh-masters-thread.", e);
+          break;
+        } catch (ExecutionException | IOException e) {
+          LOG.debug("Error populating latest list of masters.", e);
+        }
+      }
+    }
+  }
+
+  MasterAddressRefresher(Configuration conf, MasterRegistry registry) {
+    pool = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()

Review comment:
       Whats your idea? Just curious.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -170,6 +214,11 @@ public static String getMasterAddr(Configuration conf) 
throws UnknownHostExcepti
     callable.call(controller, stub, resp -> {
       if (controller.failed()) {
         future.completeExceptionally(controller.getFailed());
+        // RPC has failed, trigger a refresh of master end points. We can have 
some spurious

Review comment:
       Ya, that was what I wanted to do originally, but I thought the rpc is 
cheap and it doesn't hurt to do for any exception and left it this way. Let me 
add it, I don't have a strong preference.

##########
File path: 
hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -259,17 +311,40 @@ private RegionLocations 
transformMetaRegionLocations(GetMetaRegionLocationsRespo
       .thenApply(GetClusterIdResponse::getClusterId);
   }
 
-  private ServerName transformServerName(GetActiveMasterResponse resp) {
-    return ProtobufUtil.toServerName(resp.getServerName());
+  private static boolean hasActiveMaster(GetMastersResponse resp) {
+    List<GetMastersResponseEntry> activeMasters =
+        
resp.getMasterServersList().stream().filter(GetMastersResponseEntry::getIsActive).collect(
+        Collectors.toList());
+    return activeMasters.size() == 1;
+  }
+
+  private static ServerName filterActiveMaster(GetMastersResponse resp) {
+    List<GetMastersResponseEntry> activeMasters =
+        
resp.getMasterServersList().stream().filter(GetMastersResponseEntry::getIsActive).collect(
+            Collectors.toList());
+    Preconditions.checkState(activeMasters.size() == 1);
+    return ProtobufUtil.toServerName(activeMasters.get(0).getServerName());
   }
 
   @Override
   public CompletableFuture<ServerName> getActiveMaster() {
     return this
-      .<GetActiveMasterResponse> call(
-        (c, s, d) -> s.getActiveMaster(c, 
GetActiveMasterRequest.getDefaultInstance(), d),
-        GetActiveMasterResponse::hasServerName, "getActiveMaster()")
-      .thenApply(this::transformServerName);
+      .<GetMastersResponse> call(
+        (c, s, d) -> s.getMasters(c, GetMastersRequest.getDefaultInstance(), 
d),
+        MasterRegistry::hasActiveMaster, "getMasters()")
+      .thenApply(MasterRegistry::filterActiveMaster);
+  }
+
+  private static List<ServerName> transformServerNames(GetMastersResponse 
resp) {
+    return resp.getMasterServersList().stream().map(s -> 
ProtobufUtil.toServerName(
+        s.getServerName())).collect(Collectors.toList());
+  }
+
+  public CompletableFuture<List<ServerName>> getMasters() {

Review comment:
       Done.

##########
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -2931,6 +2935,27 @@ public GetActiveMasterResponse 
getActiveMaster(RpcController rpcController,
     return resp.build();
   }
 
+  @Override
+  public GetMastersResponse getMasters(RpcController rpcController, 
GetMastersRequest request)
+      throws ServiceException {
+    GetMastersResponse.Builder resp = GetMastersResponse.newBuilder();
+    // Active master
+    Optional<ServerName> serverName = master.getActiveMaster();
+    serverName.ifPresent(name -> 
resp.addMasterServers(GetMastersResponseEntry.newBuilder()
+        
.setServerName(ProtobufUtil.toServerName(name)).setIsActive(true).build()));
+    // Backup masters
+    try {
+      // TODO: Cache the backup masters to avoid a ZK RPC for each 
getMasters() call.

Review comment:
       I have a separate patch for this, didn't want to scope creep this one. 
Mind if do a separate PR? I will back port 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:
us...@infra.apache.org


Reply via email to