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 2020/07/23 19:53:44 UTC

[GitHub] [hbase] bharathv opened a new pull request #2130: HBASE-24765: Dynamic master discovery

bharathv opened a new pull request #2130:
URL: https://github.com/apache/hbase/pull/2130


   This patch adds the ability to discover newly added masters
   dynamically on the master registry side. The trigger for the
   re-fetch is either 5mins or any registry RPC failure.
   
   I didn't add the method to ZK registry interface since there
   is a design discussion going on in splittable meta doc. We can
   add it later if needed.


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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-671504587


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 36s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 59s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 23s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  1s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 29s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 47s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 20s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 24s |  the patch passed  |
   | -1 :x: |  shadedjars  |   3m 45s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 28s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 48s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 14s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 42s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  |  13m 30s |  hbase-server in the patch failed.  |
   |  |   |  53m 19s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b09ff92a658d 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c1c2e160ec |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/testReport/ |
   | Max. process+thread count | 718 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] bharathv commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r475713630



##########
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);

Review comment:
       That also happens during a regular pool shutdown. Added another log later in the method after loop exist for observability.




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-671592789


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  master passed  |
   | +1 :green_heart: |  compile  |   2m  7s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 37s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 16s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 28s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m  8s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m  8s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 35s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 15s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 44s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  2s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 139m 54s |  hbase-server in the patch passed.  |
   |  |   | 170m 30s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux cd8ab79df759 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c1c2e160ec |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/testReport/ |
   | Max. process+thread count | 4757 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-663229929


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 21s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 26s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 40s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m  3s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   8m 19s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 36s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 31s |  hbase-client: The patch generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m 24s |  hbase-server: The patch generated 2 new + 106 unchanged - 0 fixed = 108 total (was 106)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  14m 33s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   8m 50s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 35s |  The patch does not generate ASF License warnings.  |
   |  |   |  60m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.9 Server=19.03.9 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux 2e51900bfdca 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8191fbdd7d |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-663269079


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 59s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 35s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 15s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m 11s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 28s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 42s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 12s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 32s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 32s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 47s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 27s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  0s |  hbase-protocol-shaded in the patch passed.  |
   | -1 :x: |  unit  |   0m 53s |  hbase-client in the patch failed.  |
   | +1 :green_heart: |  unit  | 129m 52s |  hbase-server in the patch passed.  |
   |  |   | 168m 26s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 1919cfbddf4f 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8191fbdd7d |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-client.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/testReport/ |
   | Max. process+thread count | 4474 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] bharathv commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r464801901



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

##########
File path: hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistryHedgedReads.java
##########
@@ -121,6 +121,11 @@ public boolean hasCellBlockSupport() {
     @Override
     public void callMethod(MethodDescriptor method, RpcController controller, Message request,
       Message responsePrototype, RpcCallback<Message> done) {
+      if (!method.getName().equals("GetClusterId")) {
+        // Master registry internally runs other RPCs to keep the master list up to date. This check

Review comment:
       Will add more detail. That is needed because of the way the test is written. This RpcChannel implementation intercepts all the mock RPCs from unit tests and the just counts the getClusterId calls (depending on the index).. With the patch a single GetClusterID() RPC failure can trigger an extra getMasters() call and that is accounted too.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          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);
+        }
+      }
+    };
+    masterAddrRefresherThread = Threads.newDaemonThreadFactory(
+        "MasterRegistry refresh end-points").newThread(masterEndPointRefresher);
+    masterAddrRefresherThread.start();

Review comment:
       Ok switched. I didn't want to have extra layers on top of a simple thread, but I guess a pool is more readable.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          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);
+        }
+      }
+    };
+    masterAddrRefresherThread = Threads.newDaemonThreadFactory(
+        "MasterRegistry refresh end-points").newThread(masterEndPointRefresher);
+    masterAddrRefresherThread.start();

Review comment:
       There is a start() in the c'tor? (Its hiding between the long comment chains). Anyway I switched to a single threaded pool per the suggestions. Hopefully its more clear.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          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);
+        }
+      }
+    };
+    masterAddrRefresherThread = Threads.newDaemonThreadFactory(
+        "MasterRegistry refresh end-points").newThread(masterEndPointRefresher);

Review comment:
       Ok switched. I think we should use our internal one, but fine.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          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);
+        }
+      }
+    };
+    masterAddrRefresherThread = Threads.newDaemonThreadFactory(
+        "MasterRegistry refresh end-points").newThread(masterEndPointRefresher);

Review comment:
       You mean between words?

##########
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:
       My thinking was that since the RPC is relatively inexpensive and is not in any hot path, we could refresh it for any exception. One small optimization we could do is to do it only "connect" exceptions (timeouts/closed channel etc).
   
   > RPC failure results in more RPCs has me suspicious...
   
   Hmm. It's only on a best effort basis we try to keep the list up to date. If no master is reachable, I think we will be in a deeper trouble. I see your point though. You think we should have some kind of back-off on this thread if we are in that state where all the RPCs are failing and we have repeated getMasters() lookups?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -89,11 +97,17 @@
   private final int hedgedReadFanOut;
 
   // Configured list of masters to probe the meta information from.
-  private final ImmutableMap<ServerName, ClientMetaService.Interface> masterAddr2Stub;
+  private volatile ImmutableMap<ServerName, ClientMetaService.Interface> masterAddr2Stub;
 
   // RPC client used to talk to the masters.
   private final RpcClient rpcClient;
   private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+  // For synchronizing on refreshing the master end-points
+  private final Object refreshMasters = new Object();

Review comment:
       No, it's loosely tied to a single connection (there are some places that create on the fly registries which is something that can be fixed). 
   
   A single application can connect to multiple clusters which means we cannot make it static.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
##########
@@ -126,4 +131,46 @@ public void testRegistryRPCs() throws Exception {
       }
     }
   }
+
+  /**
+   * Tests that the list of masters configured in the MasterRegistry is dynamically refreshed in the
+   * event of errors.
+   */
+  @Test
+  public void testDynamicMasterConfigurationRefresh() throws Exception {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    String currentMasterAddrs = Preconditions.checkNotNull(conf.get(HConstants.MASTER_ADDRS_KEY));
+    HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster();
+    String clusterId = activeMaster.getClusterId();
+    // Add a non-working master
+    ServerName badServer = ServerName.valueOf("localhost", 1234, -1);
+    conf.set(HConstants.MASTER_ADDRS_KEY, badServer.toShortString() + "," + currentMasterAddrs);
+    // Set the hedging fan out so that all masters are queried.
+    conf.setInt(MasterRegistry.MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY, 4);
+    try (MasterRegistry registry = new MasterRegistry(conf)) {
+      final Set<ServerName> masters = registry.getParsedMasterServers();
+      assertTrue(masters.contains(badServer));
+      // Make a registry RPC, this should trigger a refresh since one of the hedged RPC fails.
+      assertEquals(registry.getClusterId().get(), clusterId);
+      // Wait for new set of masters to be populated.
+      TEST_UTIL.waitFor(5000,
+          (Waiter.Predicate<Exception>) () -> !registry.getParsedMasterServers().equals(masters));

Review comment:
       Skipped it because it bloats up the code, we need to expand the lambda. I think the intent there is pretty clear if the  test fails. Let me know if you feel strongly, I can change it.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          populateMasterStubs(newMasters);
+          LOG.debug("Finished refreshing master end points. {}", newMasters);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted during wait, aborting refresh-masters-thread.", e);

Review comment:
       Done.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          populateMasterStubs(newMasters);
+          LOG.debug("Finished refreshing master end points. {}", newMasters);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted during wait, aborting refresh-masters-thread.", e);

Review comment:
       > Should this interrupt be accompanied by a shutdown of the managing thread pool?
   
   This is InterruptedException thrown by wait() call. try-catch seems like the right way to do it? (unless I misunderstood what you are suggesting).

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());

Review comment:
       I haven't thought about this. Are you talking about this RPC specifically? If so, may I ask why? 
   
   Adding it on the server makes sense to me, like we want some metrics around where most of the time is spent (grouped by RPC) but curious what purpose it serves on client. 




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



[GitHub] [hbase] virajjasani commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-673624360


   Ah, somehow missed this today. Will take a look tomorrow 👍 


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



[GitHub] [hbase] bharathv commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r460132859



##########
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
+        // refreshes, but that is okay since the RPC is not expensive and not in a hot path.
+        synchronized (refreshMasters) {
+          refreshMasters.notify();

Review comment:
       > Maybe we can expedite populating masters with the help of AtomicBoolean check
   
   Not sure I follow this, mind rephrasing?
   
   > assuming no other RPC call happens and masters list is stale
   
   If not other RPC call happens, it doesn't matter if the list is stale or not? 
   
   > Even if we have network issue, we don't want to delay populate masters by 5 min right?
   
   Not sure I follow, if we have a network issue, how can we populate?




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-671599849


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 57s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 27s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 41s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 46s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 58s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 11s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 134m 52s |  hbase-server in the patch passed.  |
   |  |   | 167m 29s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 364f0e75aa63 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c1c2e160ec |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/testReport/ |
   | Max. process+thread count | 3739 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-663301795


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 25s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 56s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 52s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 33s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 26s |  hbase-client: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m  0s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  0s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m  5s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 31s |  The patch does not generate ASF License warnings.  |
   |  |   |  48m 29s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux c0dacdaedae8 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e7963f6486 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r470252040



##########
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:
       Sorry I didn't notice this earlier. Maybe a [`ScheduledThreadPoolExecutor`](https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/ScheduledThreadPoolExecutor.html) would simplify your scheduling need?




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-668778886


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   2m  5s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 39s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 31s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m  5s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 32s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 11s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 45s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  3s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 159m 38s |  hbase-server in the patch failed.  |
   |  |   | 190m 16s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a850f338ec67 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / d2f5a5f27b |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/testReport/ |
   | Max. process+thread count | 3679 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-664025076


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 13s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m  5s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 30s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 11s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 46s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  8s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 142m 16s |  hbase-server in the patch passed.  |
   |  |   | 172m 54s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c866b401b8ad 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 975cdf7b88 |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/testReport/ |
   | Max. process+thread count | 4256 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-668711613


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  9s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 43s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m  7s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 24s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 27s |  hbase-client: The patch generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 10s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 59s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   6m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 39s |  The patch does not generate ASF License warnings.  |
   |  |   |  46m 35s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux e9182e365d47 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / d2f5a5f27b |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-674279122


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 40s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 36s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 20s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 27s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 24s |  hbase-client: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | -0 :warning: |  checkstyle  |   1m  4s |  hbase-server: The patch generated 1 new + 106 unchanged - 0 fixed = 107 total (was 106)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 11s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 52s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   6m 43s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 30s |  The patch does not generate ASF License warnings.  |
   |  |   |  45m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux 28353b3c9f2e 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] virajjasani commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
virajjasani commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-679006296


   Keeping backupMasters volatile list up-to date in ActiveMasterManager is nice move with this 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



[GitHub] [hbase] virajjasani commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r465695297



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +132,49 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    masterAddrRefresher = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("MasterRegistry refresh end-points").setDaemon(true).build());

Review comment:
       nit: `MasterRegistry-refresh-end-points-%d` or `MasterRegistry-refresh-pool-%d` ?




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-671497201


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 35s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 39s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 43s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 12s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 20s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 27s |  hbase-client: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | -0 :warning: |  checkstyle  |   1m  7s |  hbase-server: The patch generated 1 new + 106 unchanged - 0 fixed = 107 total (was 106)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 17s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   1m 56s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   6m 39s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  asflicense  |   0m 38s |  The patch generated 2 ASF License warnings.  |
   |  |   |  46m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux a818e95de5e1 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c1c2e160ec |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | asflicense | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-general-check/output/patch-asflicense-problems.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-674741453


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 53s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 30s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   3m  1s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   8m  7s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 27s |  hbase-client: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | -0 :warning: |  checkstyle  |   0m 11s |  hbase-zookeeper: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m 14s |  hbase-server: The patch generated 4 new + 108 unchanged - 1 fixed = 112 total (was 109)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 15s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 18s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 46s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 43s |  The patch does not generate ASF License warnings.  |
   |  |   |  55m 17s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux f684ebe537b8 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-general-check/output/diff-checkstyle-hbase-zookeeper.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] Apache9 commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r459835061



##########
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
+        // refreshes, but that is okay since the RPC is not expensive and not in a hot path.
+        synchronized (refreshMasters) {

Review comment:
       Usually a notify without any flag will have strange race problem...
   At least let's have a 'triggerRefresh' flag to guard redundant notification?

##########
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:
       I think here we should assign masterAddr2Stub to a local variable before using it?

##########
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 think this should be done in this patch, as now this method will replace the old getActiveMaster method, which makes it not only be used in our internal refresh but also be used by end users, we should not let users still have the ability to harmmer zookeeper...

##########
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:
       This is a behavior change? I do not think we will throw IllegalStateException when there is no active master before this patch?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -89,11 +97,17 @@
   private final int hedgedReadFanOut;
 
   // Configured list of masters to probe the meta information from.
-  private final ImmutableMap<ServerName, ClientMetaService.Interface> masterAddr2Stub;
+  private volatile ImmutableMap<ServerName, ClientMetaService.Interface> masterAddr2Stub;
 
   // RPC client used to talk to the masters.
   private final RpcClient rpcClient;
   private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+  // For synchronizing on refreshing the master end-points
+  private final Object refreshMasters = new Object();
+  // Refreshed every WAIT_TIME_OUT_MS or unless explicitly invoked.
+  private static final int WAIT_TIME_OUT_MS = 5 * 60 * 1000; // 5 mins

Review comment:
       Do we want to make it configurable?

##########
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:
       Do not need to be public?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      while (!Thread.interrupted()) {

Review comment:
       I prefer we add a stop flag for the Registry and test it to determine whether we should quit.

##########
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:
       I think we can test the exception type? Only if it is a connect exception, we will refresh the address list?

##########
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:
       I think this could be a follow on issue. Let's get this feature in first as I think it is very useful for a long running service which depends on HBase? I have an idea on making use of the HashedWheelTimeoutTimer to do the refreshing work, will file an improvement issue if I have time.




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-663324629


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 30s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 42s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 56s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 31s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 46s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 57s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 11s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 133m 26s |  hbase-server in the patch failed.  |
   |  |   | 166m 21s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a15a5146bc82 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e7963f6486 |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/testReport/ |
   | Max. process+thread count | 4758 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] bharathv commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r466035689



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());

Review comment:
       Okay. Looks like we have all the plumbing for client metrics already in place via HBASE-12911. I can add metrics for this RPC.

##########
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, I think thats a valid concern. I was thinking about a similar edge case, hence the exponential back-off proposal. I think we can either implement a back-off or wrap it in a simple rate limiter (ex: no more than 1 getMasters in a 2 min window). WDYT?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          populateMasterStubs(newMasters);
+          LOG.debug("Finished refreshing master end points. {}", newMasters);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted during wait, aborting refresh-masters-thread.", e);

Review comment:
       >  is there a reason the thread could be interrupted (and thus terminated) while the connection remains active
   
   I don't think so. The only interruptor for this thread should be from the shutdown from the registry#close(), which is called during connection clean up.
   
   > So can a connection enter a state where it's still alive with no masterRefreshThread running?
   
   AFAICT that is not possible. The only way it can break the loop is via interruption.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +132,49 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    masterAddrRefresher = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("MasterRegistry refresh end-points").setDaemon(true).build());

Review comment:
       will do




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-671533878


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 49s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 35s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 55s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 49s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   7m 20s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 58s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 29s |  hbase-client: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | -0 :warning: |  checkstyle  |   1m 16s |  hbase-server: The patch generated 1 new + 106 unchanged - 0 fixed = 107 total (was 106)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 39s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 11s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 55s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 35s |  The patch does not generate ASF License warnings.  |
   |  |   |  53m 49s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux bc4e7165f0de 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c1c2e160ec |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] bharathv merged pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv merged pull request #2130:
URL: https://github.com/apache/hbase/pull/2130


   


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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-678919771


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 16s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 57s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m 34s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 24s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 26s |  hbase-client: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | -0 :warning: |  checkstyle  |   0m 13s |  hbase-zookeeper: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m  6s |  hbase-server: The patch generated 4 new + 106 unchanged - 1 fixed = 110 total (was 107)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m  9s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 19s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   7m 26s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 50s |  The patch does not generate ASF License warnings.  |
   |  |   |  48m 55s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux 7699366d90a2 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2874f00a2f |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-general-check/output/diff-checkstyle-hbase-zookeeper.txt |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] Apache9 commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache9 commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-678948670


   The only concern is about the IllegalStateException. We used to throw IOException for this case.
   
   And please fix the checkstyle issues?


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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r466070307



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());

Review comment:
       Either client-side or server-side is fine with me. Just having something that an operator can expose would be useful.




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-674825062


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 18s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  6s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 24s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in master failed.  |
   | -0 :warning: |  javadoc  |   0m 16s |  hbase-zookeeper in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 35s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 11s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 11s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 28s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 16s |  hbase-zookeeper in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  3s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 17s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 45s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  | 191m  6s |  hbase-server in the patch passed.  |
   |  |   | 229m 50s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux f91228a0ebdf 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/testReport/ |
   | Max. process+thread count | 3417 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-679344131


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 46s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 28s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 26s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m  7s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   8m 14s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 11s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  The patch passed checkstyle in hbase-client  |
   | +1 :green_heart: |  checkstyle  |   0m 13s |  The patch passed checkstyle in hbase-zookeeper  |
   | +1 :green_heart: |  checkstyle  |   1m 16s |  hbase-server: The patch generated 0 new + 107 unchanged - 1 fixed = 107 total (was 108)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  13m 43s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   8m 25s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 51s |  The patch does not generate ASF License warnings.  |
   |  |   |  58m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux 38fe84e87b80 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 6ad73b9668 |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-668775483


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 22s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 39s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 54s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 28s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 12s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 34s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 34s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 50s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 19s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 20s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 147m 34s |  hbase-server in the patch failed.  |
   |  |   | 182m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2a198205c2e8 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / d2f5a5f27b |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/testReport/ |
   | Max. process+thread count | 3797 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] virajjasani commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r471664544



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

Review comment:
       nit: keep this `int` similar to `PERIODIC_REFRESH_INTERVAL_SECS_DEFAULT`?

##########
File path: hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MasterAddressTracker.java
##########
@@ -278,4 +282,59 @@ public static boolean deleteIfEquals(ZKWatcher zkw, final String content) {
 
     return false;
   }
+
+  public List<ServerName> getBackupMasters() throws InterruptedIOException {
+    return getBackupMastersAndRenewWatch(watcher);
+  }
+
+  /**
+   * Retrieves the list of registered backup masters and renews a watch on the znode for children
+   * updates.
+   * @param zkw Zookeeper watcher to use
+   * @return List of backup masters.
+   * @throws InterruptedIOException
+   */
+  public static List<ServerName> getBackupMastersAndRenewWatch(
+      ZKWatcher zkw) throws InterruptedIOException {
+    // Build Set of backup masters from ZK nodes
+    List<String> backupMasterStrings;
+    try {
+      backupMasterStrings = ZKUtil.listChildrenAndWatchForNewChildren(zkw,
+          zkw.getZNodePaths().backupMasterAddressesZNode);
+    } catch (KeeperException e) {
+      LOG.warn(zkw.prefix("Unable to list backup servers"), e);
+      backupMasterStrings = null;

Review comment:
       nit: `return Collections.emptyList();`

##########
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()
+        .setNameFormat("master-registry-refresh-end-points").setDaemon(true).build());
+    periodicRefreshMs = 1000 * conf.getLong(PERIODIC_REFRESH_INTERVAL_SECS,
+        PERIODIC_REFRESH_INTERVAL_SECS_DEFAULT);
+    timeBetweenRefreshesMs = 1000 * conf.getLong(MIN_SECS_BETWEEN_REFRESHES,
+        MIN_SECS_BETWEEN_REFRESHES_DEFAULT);

Review comment:
       nit: `TimeUnit.SECONDS.toMillis(conf.getLong(,))`

##########
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);

Review comment:
       We are aborting refresh by breaking out of the loop and basically, we are done refreshing master stubs. Better to log this at `ERROR`?




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-679401754


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 36s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m  8s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 18s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m  2s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 29s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-zookeeper in master failed.  |
   | -0 :warning: |  javadoc  |   0m 46s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 24s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 50s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 28s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-zookeeper in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 43s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  9s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 17s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 46s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  | 142m 12s |  hbase-server in the patch passed.  |
   |  |   | 183m 43s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux dfb6fd97521d 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 6ad73b9668 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/testReport/ |
   | Max. process+thread count | 3973 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] virajjasani commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r460552343



##########
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
+        // refreshes, but that is okay since the RPC is not expensive and not in a hot path.
+        synchronized (refreshMasters) {
+          refreshMasters.notify();

Review comment:
       > > Maybe we can expedite populating masters with the help of AtomicBoolean check
   > 
   > Not sure I follow this, mind rephrasing?
   
   I meant to say if refresh thread misses this `notify` because it is already done `waiting` on `refreshMasters`, for the next loop, it should not again wait 5 min on `refreshMasters` and rather quickly perform RPC call to populate masters.
   
   > > Even if we have network issue, we don't want to delay populate masters by 5 min right?
   >
   > Not sure I follow, if we have a network issue, how can we populate?
   
   I meant same as above that even if network issue causes `notify` to refresh thread when it was already past `waiting` state, maybe next time the thread better quickly make an RPC call rather than waiting 5 min on `refreshMasters`. But yes, for network issues, we will keep making RPC calls without any progress.
   
   > If not other RPC call happens, it doesn't matter if the list is stale or not?
   
   Hmm that's true.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -89,11 +97,17 @@
   private final int hedgedReadFanOut;
 
   // Configured list of masters to probe the meta information from.
-  private final ImmutableMap<ServerName, ClientMetaService.Interface> masterAddr2Stub;
+  private volatile ImmutableMap<ServerName, ClientMetaService.Interface> masterAddr2Stub;
 
   // RPC client used to talk to the masters.
   private final RpcClient rpcClient;
   private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+  // For synchronizing on refreshing the master end-points
+  private final Object refreshMasters = new Object();

Review comment:
       nit: `static final` here? Anyways, `MasterRegistry` is singleton right? (if not by design, but by usage)




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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r466069574



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          populateMasterStubs(newMasters);
+          LOG.debug("Finished refreshing master end points. {}", newMasters);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted during wait, aborting refresh-masters-thread.", e);

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



[GitHub] [hbase] virajjasani commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r465691996



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -89,11 +97,17 @@
   private final int hedgedReadFanOut;
 
   // Configured list of masters to probe the meta information from.
-  private final ImmutableMap<ServerName, ClientMetaService.Interface> masterAddr2Stub;
+  private volatile ImmutableMap<ServerName, ClientMetaService.Interface> masterAddr2Stub;
 
   // RPC client used to talk to the masters.
   private final RpcClient rpcClient;
   private final RpcControllerFactory rpcControllerFactory;
+  private final int rpcTimeoutMs;
+  // For synchronizing on refreshing the master end-points
+  private final Object refreshMasters = new Object();

Review comment:
       Oops, yes. This better be just 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



[GitHub] [hbase] bharathv commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-679252642


   > The only concern is about the IllegalStateException. We used to throw IOException for this case.
   
   I added a checked exception. TBH I don't fully understand the concern. Like I explained it happens only if an invariant fails, which means a code bug, so unchecked exception should be fine. Either way its not a big deal what exception we throw, so made the changed. Also, fixed the check style issues.


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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r466021691



##########
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:
       It seems like something we want to ensure is well-throttled, so that 100's of clients are not overwhelming 3 masters while they're trying to manage cluster recovery. I've recently been investigating an event where we have the rs hosting meta get overwhelmed by client requests, while the master is concurrently trying to process server crash procedures, thus the idea of the stampeding herd is fresh in my mind. Our IPC QoS story isn't great right now, so I want to think carefully about how failure and recovery scenarios play out with ratios of client:server that are on the order of 100:1.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());

Review comment:
       Someone who has the HBase client embedded in their application might be interested in observing this behavior, if they notice sudden spikes in RPC traffic not directly correlated with their application's data path.
   
   Or maybe it's enough to track these calls on the server-side. We'd see the same spikes, though with less granularity.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          populateMasterStubs(newMasters);
+          LOG.debug("Finished refreshing master end points. {}", newMasters);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted during wait, aborting refresh-masters-thread.", e);

Review comment:
       Let me try again, and ask about the application lifecycle of this thread. It looks like a thread we always want running, unless the Connection is being closed. Now that there's a pool, I'm asking about the lifecycle of the pool, when it's `shutdown` method is called. I'm also asking about the presence of the thread -- is there a reason the thread could be interrupted (and thus terminated) while the connection remains active? There's only one call to submit, and there's nothing that checks for the presence of the thread. So can a connection enter a state where it's still alive with no masterRefreshThread running?




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



[GitHub] [hbase] virajjasani commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r460075547



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          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);
+        }
+      }
+    };
+    masterAddrRefresherThread = Threads.newDaemonThreadFactory(
+        "MasterRegistry refresh end-points").newThread(masterEndPointRefresher);

Review comment:
       Can you please use guava library's ThreadFactoryBuilder? So far the consensus on [HBASE-24750](https://issues.apache.org/jira/browse/HBASE-24750) is to get rid of our internally maintained ThreadFactory :)

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          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);
+        }
+      }
+    };
+    masterAddrRefresherThread = Threads.newDaemonThreadFactory(
+        "MasterRegistry refresh end-points").newThread(masterEndPointRefresher);
+    masterAddrRefresherThread.start();

Review comment:
       Don't want to use SingleThreadExecutor.submit()?

##########
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:
       We are planning to have a cache with ZKWatcher for backupMasters ZNode right? I believe as of now, we don't subscribe for any event.

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
##########
@@ -126,4 +131,46 @@ public void testRegistryRPCs() throws Exception {
       }
     }
   }
+
+  /**
+   * Tests that the list of masters configured in the MasterRegistry is dynamically refreshed in the
+   * event of errors.
+   */
+  @Test
+  public void testDynamicMasterConfigurationRefresh() throws Exception {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    String currentMasterAddrs = Preconditions.checkNotNull(conf.get(HConstants.MASTER_ADDRS_KEY));
+    HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster();
+    String clusterId = activeMaster.getClusterId();
+    // Add a non-working master
+    ServerName badServer = ServerName.valueOf("localhost", 1234, -1);
+    conf.set(HConstants.MASTER_ADDRS_KEY, badServer.toShortString() + "," + currentMasterAddrs);
+    // Set the hedging fan out so that all masters are queried.
+    conf.setInt(MasterRegistry.MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY, 4);
+    try (MasterRegistry registry = new MasterRegistry(conf)) {
+      final Set<ServerName> masters = registry.getParsedMasterServers();
+      assertTrue(masters.contains(badServer));
+      // Make a registry RPC, this should trigger a refresh since one of the hedged RPC fails.
+      assertEquals(registry.getClusterId().get(), clusterId);
+      // Wait for new set of masters to be populated.
+      TEST_UTIL.waitFor(5000,
+          (Waiter.Predicate<Exception>) () -> !registry.getParsedMasterServers().equals(masters));
+      // new set of masters should not include the bad server
+      final Set<ServerName> newMasters = registry.getParsedMasterServers();
+      // Bad one should be out.
+      assertEquals(3, newMasters.size());
+      assertFalse(newMasters.contains(badServer));
+      // Kill the active master
+      activeMaster.stopMaster();
+      TEST_UTIL.waitFor(10000,
+        () -> TEST_UTIL.getMiniHBaseCluster().getLiveMasterThreads().size() == 2);
+      assertEquals(registry.getClusterId().get(), clusterId);
+      // Make another registry RPC call which should trigger another refresh.
+      TEST_UTIL.waitFor(100000, (Waiter.Predicate<Exception>) () ->
+          registry.getParsedMasterServers().size() == 2);
+      final Set<ServerName> newMasters2 = registry.getParsedMasterServers();
+      assertEquals(2, newMasters2.size());
+      assertFalse(newMasters2.contains(activeMaster));

Review comment:
       `newMasters2.contains(activeMaster.getServerName())`

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
##########
@@ -126,4 +131,46 @@ public void testRegistryRPCs() throws Exception {
       }
     }
   }
+
+  /**
+   * Tests that the list of masters configured in the MasterRegistry is dynamically refreshed in the
+   * event of errors.
+   */
+  @Test
+  public void testDynamicMasterConfigurationRefresh() throws Exception {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    String currentMasterAddrs = Preconditions.checkNotNull(conf.get(HConstants.MASTER_ADDRS_KEY));
+    HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster();
+    String clusterId = activeMaster.getClusterId();
+    // Add a non-working master
+    ServerName badServer = ServerName.valueOf("localhost", 1234, -1);
+    conf.set(HConstants.MASTER_ADDRS_KEY, badServer.toShortString() + "," + currentMasterAddrs);
+    // Set the hedging fan out so that all masters are queried.
+    conf.setInt(MasterRegistry.MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY, 4);
+    try (MasterRegistry registry = new MasterRegistry(conf)) {
+      final Set<ServerName> masters = registry.getParsedMasterServers();
+      assertTrue(masters.contains(badServer));
+      // Make a registry RPC, this should trigger a refresh since one of the hedged RPC fails.
+      assertEquals(registry.getClusterId().get(), clusterId);
+      // Wait for new set of masters to be populated.
+      TEST_UTIL.waitFor(5000,
+          (Waiter.Predicate<Exception>) () -> !registry.getParsedMasterServers().equals(masters));

Review comment:
       nit: upto you if you want to use `ExplainingPredicate` to throw Exception with specific message

##########
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
+        // refreshes, but that is okay since the RPC is not expensive and not in a hot path.
+        synchronized (refreshMasters) {
+          refreshMasters.notify();

Review comment:
       For any generic RPC failure, we want to expedite populating masters with another RPC call.
   
   Let's say there are some sequence of events:
   1. getClusterId() RPC call failed
   2. master refresher thread was in `waiting` state, so we notify it and it will trigger getMasters() call
   3. the call fails again and we `notify` refreshMasters but no one is waiting on it, notify is ignored
   4. master refresher thread again waits for 5 min before populating masters.
   
   Do we really want step 4 to wait for 5 min (assuming no other RPC call happens and masters list is stale)? Maybe we can expedite populating masters with the help of AtomicBoolean check (and also avoid `synchronized + wait` calls i.e 5 min wait)?
   
   Even if we have network issue, we don't want to delay populate masters by 5 min right?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          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);
+        }
+      }
+    };
+    masterAddrRefresherThread = Threads.newDaemonThreadFactory(
+        "MasterRegistry refresh end-points").newThread(masterEndPointRefresher);

Review comment:
       nit: avoid space in Thread prefix name?

##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
##########
@@ -126,4 +131,46 @@ public void testRegistryRPCs() throws Exception {
       }
     }
   }
+
+  /**
+   * Tests that the list of masters configured in the MasterRegistry is dynamically refreshed in the
+   * event of errors.
+   */
+  @Test
+  public void testDynamicMasterConfigurationRefresh() throws Exception {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    String currentMasterAddrs = Preconditions.checkNotNull(conf.get(HConstants.MASTER_ADDRS_KEY));
+    HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster();
+    String clusterId = activeMaster.getClusterId();
+    // Add a non-working master
+    ServerName badServer = ServerName.valueOf("localhost", 1234, -1);
+    conf.set(HConstants.MASTER_ADDRS_KEY, badServer.toShortString() + "," + currentMasterAddrs);
+    // Set the hedging fan out so that all masters are queried.
+    conf.setInt(MasterRegistry.MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY, 4);
+    try (MasterRegistry registry = new MasterRegistry(conf)) {
+      final Set<ServerName> masters = registry.getParsedMasterServers();
+      assertTrue(masters.contains(badServer));
+      // Make a registry RPC, this should trigger a refresh since one of the hedged RPC fails.
+      assertEquals(registry.getClusterId().get(), clusterId);
+      // Wait for new set of masters to be populated.
+      TEST_UTIL.waitFor(5000,
+          (Waiter.Predicate<Exception>) () -> !registry.getParsedMasterServers().equals(masters));
+      // new set of masters should not include the bad server
+      final Set<ServerName> newMasters = registry.getParsedMasterServers();
+      // Bad one should be out.
+      assertEquals(3, newMasters.size());
+      assertFalse(newMasters.contains(badServer));
+      // Kill the active master
+      activeMaster.stopMaster();
+      TEST_UTIL.waitFor(10000,
+        () -> TEST_UTIL.getMiniHBaseCluster().getLiveMasterThreads().size() == 2);
+      assertEquals(registry.getClusterId().get(), clusterId);
+      // Make another registry RPC call which should trigger another refresh.
+      TEST_UTIL.waitFor(100000, (Waiter.Predicate<Exception>) () ->
+          registry.getParsedMasterServers().size() == 2);
+      final Set<ServerName> newMasters2 = registry.getParsedMasterServers();
+      assertEquals(2, newMasters2.size());

Review comment:
       After stopping activeMaster, maybe add an extra check to confirm list contains one Active and one Backup?




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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r466069499



##########
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:
       Between a throttle and an exponential backoff, i have no preference. I think we won't know what works until someone suffers the the consequences. Having something in place will address my concern.




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



[GitHub] [hbase] bharathv commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

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



[GitHub] [hbase] bharathv commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-678906608


   @Apache9 / @virajjasani Any more comments or is this good to go?
   
   This has been open for a while, would like to get it in asap unless you have any comments.


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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-663337641


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 14s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 55s |  master passed  |
   | +1 :green_heart: |  compile  |   2m  8s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 20s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 15s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  1s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 21s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 21s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 12s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 11s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 48s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 16s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 207m 50s |  hbase-server in the patch passed.  |
   |  |   | 241m 15s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a40a8f1ed21b 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e7963f6486 |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/testReport/ |
   | Max. process+thread count | 4005 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] bharathv commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r470268400



##########
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:
       This is a long running thread and wakes up if there is an on-demand refresh (via refreshNow()). If we use a scheduled executor, on demand scheduling logic becomes  a bit tricky, right? (unless there is some easy way of doing it that I missed).




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-679020090


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 39s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 57s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 12s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 48s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 21s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 59s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 59s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m  0s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  1s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 29s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 52s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  | 226m 52s |  hbase-server in the patch passed.  |
   |  |   | 267m 19s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux c7e07fa9f7d1 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2874f00a2f |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/testReport/ |
   | Max. process+thread count | 2735 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-679400952


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 42s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 42s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 35s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 10s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 27s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 26s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 52s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  7s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 45s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  | 151m 16s |  hbase-server in the patch passed.  |
   |  |   | 188m  7s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux f7ba282bf066 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 6ad73b9668 |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/testReport/ |
   | Max. process+thread count | 4584 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/11/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-680220425


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 23s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m  3s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   7m 21s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 56s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m  9s |  The patch passed checkstyle in hbase-protocol-shaded  |
   | +1 :green_heart: |  checkstyle  |   0m 28s |  The patch passed checkstyle in hbase-client  |
   | +1 :green_heart: |  checkstyle  |   0m 12s |  The patch passed checkstyle in hbase-zookeeper  |
   | +1 :green_heart: |  checkstyle  |   1m 14s |  hbase-server: The patch generated 0 new + 107 unchanged - 1 fixed = 107 total (was 108)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 53s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m 26s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   8m 28s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 46s |  The patch does not generate ASF License warnings.  |
   |  |   |  54m 57s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/12/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux 7b679973ec57 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ebe321a99b |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/12/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-664010221


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 25s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  0s |  prototool was not available.  |
   | +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.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 36s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 46s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   6m  5s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 19s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 27s |  hbase-client: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m 35s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   2m  3s |  the patch passed  |
   | +1 :green_heart: |  spotbugs  |   6m 58s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   |  46m 35s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux 626d957f7e94 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 975cdf7b88 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-client.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-678970912


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  8s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 57s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 27s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-zookeeper in master failed.  |
   | -0 :warning: |  javadoc  |   0m 46s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 22s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 12s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 12s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m  3s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 27s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-zookeeper in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  0s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 10s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 45s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  | 131m  4s |  hbase-server in the patch passed.  |
   |  |   | 167m 49s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 7ee216d4ef28 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 2874f00a2f |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-zookeeper.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/testReport/ |
   | Max. process+thread count | 3942 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/10/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-674808415


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 50s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 33s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 25s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 28s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 29s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 56s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  1s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 19s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 48s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  | 156m 49s |  hbase-server in the patch passed.  |
   |  |   | 191m  0s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 484bdd2c6b29 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/testReport/ |
   | Max. process+thread count | 3671 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-674311474


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 16s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 23s |  master passed  |
   | +1 :green_heart: |  compile  |   2m  7s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 54s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 44s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 12s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 48s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  6s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 143m 32s |  hbase-server in the patch failed.  |
   |  |   | 175m  2s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e37a0ce08f45 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | Default Java | 1.8.0_232 |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/testReport/ |
   | Max. process+thread count | 4162 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-664030914


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  6s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 41s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 41s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 18s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 25s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 27s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 40s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 40s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 22s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  2s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m 20s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  | 193m 47s |  hbase-server in the patch passed.  |
   |  |   | 229m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux a5068c8af5cb 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 975cdf7b88 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/testReport/ |
   | Max. process+thread count | 3498 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] bharathv commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r461737913



##########
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
+        // refreshes, but that is okay since the RPC is not expensive and not in a hot path.
+        synchronized (refreshMasters) {
+          refreshMasters.notify();

Review comment:
       > I meant to say if refresh thread misses this notify because it is already done waiting on refreshMasters, for the next loop, it should not again wait 5 min on refreshMasters and rather quickly perform RPC call to populate masters.
   
   I don't think thats needed. If the thread has just fetched the masters (in cases where it missed the notification), it is very unlikely that something new has been added/removed. Typically this is a very rare event, probably less rare in K8s environment than DC deployments but even then I don't think things usually change for days if not weeks.




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-671232903


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 12s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 32s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 49s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   1m 18s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 13s |  hbase-client in the patch failed.  |
   | -1 :x: |  compile  |   1m  4s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 13s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javac  |   1m  4s |  hbase-server in the patch failed.  |
   | -1 :x: |  shadedjars  |   2m 38s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m  0s |  hbase-protocol-shaded in the patch passed.  |
   | -1 :x: |  unit  |   0m 14s |  hbase-client in the patch failed.  |
   | -1 :x: |  unit  |   1m  2s |  hbase-server in the patch failed.  |
   |  |   |  26m 21s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d9cdfd0a661e 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7b099eaa75 |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-client.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-client.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-client.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/testReport/ |
   | Max. process+thread count | 104 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-671492440


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  3s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  master passed  |
   | +1 :green_heart: |  compile  |   2m  8s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 15s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 26s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m  5s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m  5s |  the patch passed  |
   | -1 :x: |  shadedjars  |   2m 43s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 14s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 47s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  1s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  |   6m 33s |  hbase-server in the patch failed.  |
   |  |   |  34m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 300c33288488 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c1c2e160ec |
   | Default Java | 1.8.0_232 |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/testReport/ |
   | Max. process+thread count | 998 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] bharathv commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r474150302



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

##########
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:
       Its an unchecked exception because if this happens, it is a code bug. This means that someone called filterActiveMaster() without calling hasActiveMaster(). This is impossible to happen because its a private utility and we control the callers. It doesn't seem worth propagating the exceptions for 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-674309213


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 27s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 31s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 44s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-client in master failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 29s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 29s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 41s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 24s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 59s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  9s |  hbase-client in the patch passed.  |
   | -1 :x: |  unit  | 130m  0s |  hbase-server in the patch failed.  |
   |  |   | 163m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 485055fdf240 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / c81ef7368e |
   | Default Java | 2020-01-14 |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-client.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/testReport/ |
   | Max. process+thread count | 3970 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] virajjasani commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r465693339



##########
File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java
##########
@@ -126,4 +131,46 @@ public void testRegistryRPCs() throws Exception {
       }
     }
   }
+
+  /**
+   * Tests that the list of masters configured in the MasterRegistry is dynamically refreshed in the
+   * event of errors.
+   */
+  @Test
+  public void testDynamicMasterConfigurationRefresh() throws Exception {
+    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
+    String currentMasterAddrs = Preconditions.checkNotNull(conf.get(HConstants.MASTER_ADDRS_KEY));
+    HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster();
+    String clusterId = activeMaster.getClusterId();
+    // Add a non-working master
+    ServerName badServer = ServerName.valueOf("localhost", 1234, -1);
+    conf.set(HConstants.MASTER_ADDRS_KEY, badServer.toShortString() + "," + currentMasterAddrs);
+    // Set the hedging fan out so that all masters are queried.
+    conf.setInt(MasterRegistry.MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY, 4);
+    try (MasterRegistry registry = new MasterRegistry(conf)) {
+      final Set<ServerName> masters = registry.getParsedMasterServers();
+      assertTrue(masters.contains(badServer));
+      // Make a registry RPC, this should trigger a refresh since one of the hedged RPC fails.
+      assertEquals(registry.getClusterId().get(), clusterId);
+      // Wait for new set of masters to be populated.
+      TEST_UTIL.waitFor(5000,
+          (Waiter.Predicate<Exception>) () -> !registry.getParsedMasterServers().equals(masters));

Review comment:
       Sure no worries, not a strong point.




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-671233273


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 45s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  5s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 26s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 31s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 22s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | -1 :x: |  mvninstall  |   1m 10s |  root in the patch failed.  |
   | -1 :x: |  compile  |   0m 14s |  hbase-client in the patch failed.  |
   | -1 :x: |  compile  |   1m  1s |  hbase-server in the patch failed.  |
   | -0 :warning: |  javac  |   0m 14s |  hbase-client in the patch failed.  |
   | -0 :warning: |  javac  |   1m  1s |  hbase-server in the patch failed.  |
   | -1 :x: |  shadedjars  |   2m 55s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 18s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 52s |  hbase-protocol-shaded in the patch passed.  |
   | -1 :x: |  unit  |   0m 13s |  hbase-client in the patch failed.  |
   | -1 :x: |  unit  |   1m  0s |  hbase-server in the patch failed.  |
   |  |   |  27m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux e54180b67926 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 7b099eaa75 |
   | Default Java | 1.8.0_232 |
   | mvninstall | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk8-hadoop3-check/output/patch-mvninstall-root.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-client.txt |
   | compile | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-server.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-client.txt |
   | javac | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk8-hadoop3-check/output/patch-compile-hbase-server.txt |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-client.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/testReport/ |
   | Max. process+thread count | 88 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-663289529


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 21s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 57s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 11s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  5s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 13s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 47s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 10s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m  5s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 10s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 49s |  hbase-protocol-shaded in the patch passed.  |
   | -1 :x: |  unit  |   0m 56s |  hbase-client in the patch failed.  |
   | +1 :green_heart: |  unit  | 208m 58s |  hbase-server in the patch passed.  |
   |  |   | 241m 29s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d4ae8b4b288e 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8191fbdd7d |
   | Default Java | 1.8.0_232 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-client.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/testReport/ |
   | Max. process+thread count | 3936 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-2130/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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



[GitHub] [hbase] Apache9 commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache9 commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-673840254


   Oh shit, I forgot to hit the submit review button...
   
   Sorry @bharathv , I've commented long ago but forgot to submit them...
   
   Hope you still have patience to fix the review comments...


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



[GitHub] [hbase] bharathv commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
bharathv commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-673604488


   any more thoughts on this? i think I addressed the review comments, let me know if I missed something. 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



[GitHub] [hbase] Apache9 commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache9 commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r473656744



##########
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:
       Introduce a global HashedWheelTimeoutTimer to be used at client side. Now there is one in AsyncConnectionImpl, but in MasterRegistry we can not depend on AsyncConnection so we should move it to another place.
   And schedule a timer task to HashedWheelTimeoutTimer to do the refresh. To avoid blocking the thread, we could make use of async rpc call.

##########
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:
       But I do not think we will throw IllegalStateException? Usually we will throw IOException here? IllegalStateException is an unchecked exception...




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



[GitHub] [hbase] ndimiduk commented on a change in pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#discussion_r464597532



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());

Review comment:
       Where would I find metrics regarding calls to `getMasters()`? I suppose either client or server-side would be good.

##########
File path: hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistryHedgedReads.java
##########
@@ -121,6 +121,11 @@ public boolean hasCellBlockSupport() {
     @Override
     public void callMethod(MethodDescriptor method, RpcController controller, Message request,
       Message responsePrototype, RpcCallback<Message> done) {
+      if (!method.getName().equals("GetClusterId")) {
+        // Master registry internally runs other RPCs to keep the master list up to date. This check

Review comment:
       Can you say more here? Why is it that the internal RPCs that keep the master list up to date are sufficient to skip a call to "GetClusterId"? Can you provide a "see also" comment that points the reader off to the counting logic, or at least the counter that this condition protects?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          populateMasterStubs(newMasters);
+          LOG.debug("Finished refreshing master end points. {}", newMasters);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted during wait, aborting refresh-masters-thread.", e);

Review comment:
       Oh, reading below, there is no thread pool. I +1 Viraj's suggestion of a single thread executor service.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          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);
+        }
+      }
+    };
+    masterAddrRefresherThread = Threads.newDaemonThreadFactory(
+        "MasterRegistry refresh end-points").newThread(masterEndPointRefresher);
+    masterAddrRefresherThread.start();

Review comment:
       Should the thread be started in the constructor? Now I see that `ConnectionRegistry` defines a `close()` but not a `start()` :(

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MasterRegistry.java
##########
@@ -115,20 +129,50 @@
   MasterRegistry(Configuration conf) throws IOException {
     this.hedgedReadFanOut = Math.max(1, conf.getInt(MASTER_REGISTRY_HEDGED_REQS_FANOUT_KEY,
       MASTER_REGISTRY_HEDGED_REQS_FANOUT_DEFAULT));
-    int rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
+    rpcTimeoutMs = (int) Math.min(Integer.MAX_VALUE,
       conf.getLong(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
     // XXX: we pass cluster id as null here since we do not have a cluster id yet, we have to fetch
     // this through the master registry...
     // This is a problem as we will use the cluster id to determine the authentication method
     rpcClient = RpcClientFactory.createClient(conf, null);
     rpcControllerFactory = RpcControllerFactory.instantiate(conf);
-    Set<ServerName> masterAddrs = parseMasterAddrs(conf);
+    // Generate the seed list of master stubs. Subsequent RPCs try to keep a live list of masters
+    // by fetching the end points from this list.
+    populateMasterStubs(parseMasterAddrs(conf));
+    Runnable masterEndPointRefresher = () -> {
+      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(WAIT_TIME_OUT_MS);
+          }
+          LOG.debug("Attempting to refresh master address end points.");
+          Set<ServerName> newMasters = new HashSet<>(getMasters().get());
+          populateMasterStubs(newMasters);
+          LOG.debug("Finished refreshing master end points. {}", newMasters);
+        } catch (InterruptedException e) {
+          LOG.debug("Interrupted during wait, aborting refresh-masters-thread.", e);

Review comment:
       Should this interrupt be accompanied by a shutdown of the managing thread pool? Maybe in an attached exception handler. Or it's okay to leave the pool as abandoned, on the assumption that there was only the single thread in the pool and the whole process is being terminated.

##########
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:
       You sure that _any_ failure should require refreshing the list? A mechanism wherein RPC failure results in more RPCs has me suspicious...




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



[GitHub] [hbase] Apache-HBase commented on pull request #2130: HBASE-24765: Dynamic master discovery

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2130:
URL: https://github.com/apache/hbase/pull/2130#issuecomment-680284516


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 41s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 38s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 44s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 36s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 36s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 36s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 47s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 38s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 47s |  hbase-protocol-shaded in the patch passed.  |
   | +1 :green_heart: |  unit  |   1m  8s |  hbase-client in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 44s |  hbase-zookeeper in the patch passed.  |
   | +1 :green_heart: |  unit  | 161m 29s |  hbase-server in the patch passed.  |
   |  |   | 197m 25s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.12 Server=19.03.12 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/12/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2130 |
   | JIRA Issue | HBASE-24765 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 7a42cc8df5d0 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / ebe321a99b |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/12/testReport/ |
   | Max. process+thread count | 4306 (vs. ulimit of 12500) |
   | modules | C: hbase-protocol-shaded hbase-client hbase-zookeeper hbase-server U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2130/12/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | 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