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/02/07 01:44:21 UTC

[GitHub] [hbase] ndimiduk opened a new pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

ndimiduk opened a new pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141
 
 
   …StartingAnyRegionServer
   
   Be a bit more dogmatic about terminating the minicluster between test
   methods. I doubt this resolves the root issue, but we'll see.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r378018435
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
 ##########
 @@ -2804,9 +2804,15 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
    * Master runs a coordinated stop of all RegionServers and then itself.
    */
   public void shutdown() throws IOException {
+    if (!isInitialized()) {
+      LOG.info("Shutdown requested but we're not the active master. Proceeding as a stop.");
 
 Review comment:
   Well, I'm not entirely sure if this is the intended way to stop stand-by masters. Reading the code.. it looks like the active master coordinates a proper cluster shutdown. In that process, it removes the /hbase/running znode that the standb-bys keep a watch on. See the following code in ActiveMasterManager..
   
   ```
   @Override
     public void nodeDeleted(String path) {
   
       // We need to keep track of the cluster's shutdown status while
       // we wait on the current master. We consider that, if the cluster
       // was already in a "shutdown" state when we started, that this master
       // is part of a new cluster that was started shortly after the old cluster
       // shut down, so that state is now irrelevant. This means that the shutdown
       // state must be set while we wait on the active master in order
       // to shutdown this master. See HBASE-8519.
       if(path.equals(watcher.getZNodePaths().clusterStateZNode) && !master.isStopped()) {
         clusterShutDown.set(true);
       }
   
   ```
   Ideally they should shut themselves down if the ZK event notifications happen as expected. Is that not the case?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
bharathv commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-585425228
 
 
   > Thanks for the discussion guys. I'm going to back out the change to HMaster, address the remaining TODO's and restore the waitFor on the instance of ServerManager. We can file a follow-on ticket to fix this race condition in cluster shutdown. Sound good?
   
   Sgtm.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r377778169
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -128,41 +142,50 @@ public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
 
     // Start the cluster
-    final HBaseTestingUtility util = new HBaseTestingUtility(conf);
-    util.startMiniDFSCluster(3);
-    util.startMiniZKCluster();
-    util.createRootDir();
-    final LocalHBaseCluster cluster =
-        new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
-            MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
-    final int MASTER_INDEX = 0;
-    final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
-    master.start();
-    LOG.info("Called master start on " + master.getName());
-    Thread shutdownThread = new Thread("Shutdown-Thread") {
-      @Override
-      public void run() {
-        LOG.info("Before call to shutdown master");
-        try (Connection connection = createConnection(util); Admin admin = connection.getAdmin()) {
-          admin.shutdown();
-        } catch (Exception e) {
-          LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
+    LocalHBaseCluster cluster = null;
+    try {
+      htu = new HBaseTestingUtility(conf);
+      htu.startMiniDFSCluster(3);
+      htu.startMiniZKCluster();
+      htu.createRootDir();
+      cluster = new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
+        MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
+      final int MASTER_INDEX = 0;
+      final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
+      master.start();
+      LOG.info("Called master start on " + master.getName());
+      final LocalHBaseCluster finalCluster = cluster;
+      Thread shutdownThread = new Thread("Shutdown-Thread") {
+        @Override
+        public void run() {
+          LOG.info("Before call to shutdown master");
+          try (Connection connection = createConnection(htu); Admin admin = connection.getAdmin()) {
+            admin.shutdown();
+          } catch (Exception e) {
+            LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
+          }
+          LOG.info("After call to shutdown master");
+          finalCluster.waitOnMaster(MASTER_INDEX);
         }
-        LOG.info("After call to shutdown master");
-        cluster.waitOnMaster(MASTER_INDEX);
+      };
+      shutdownThread.start();
+      LOG.info("Called master join on " + master.getName());
+      master.join();
+      shutdownThread.join();
+
+      List<MasterThread> masterThreads = cluster.getMasters();
+      // make sure all the masters properly shutdown
+      assertEquals(0, masterThreads.size());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      if (htu != null) {
+        htu.shutdownMiniZKCluster();
 
 Review comment:
   _nod_

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r376199298
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -128,41 +142,50 @@ public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
 
     // Start the cluster
-    final HBaseTestingUtility util = new HBaseTestingUtility(conf);
-    util.startMiniDFSCluster(3);
-    util.startMiniZKCluster();
-    util.createRootDir();
-    final LocalHBaseCluster cluster =
-        new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
-            MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
-    final int MASTER_INDEX = 0;
-    final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
-    master.start();
-    LOG.info("Called master start on " + master.getName());
-    Thread shutdownThread = new Thread("Shutdown-Thread") {
-      @Override
-      public void run() {
-        LOG.info("Before call to shutdown master");
-        try (Connection connection = createConnection(util); Admin admin = connection.getAdmin()) {
-          admin.shutdown();
-        } catch (Exception e) {
-          LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
+    LocalHBaseCluster cluster = null;
+    try {
+      htu = new HBaseTestingUtility(conf);
+      htu.startMiniDFSCluster(3);
+      htu.startMiniZKCluster();
+      htu.createRootDir();
+      cluster = new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
+        MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
+      final int MASTER_INDEX = 0;
+      final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
+      master.start();
+      LOG.info("Called master start on " + master.getName());
+      final LocalHBaseCluster finalCluster = cluster;
+      Thread shutdownThread = new Thread("Shutdown-Thread") {
+        @Override
+        public void run() {
+          LOG.info("Before call to shutdown master");
+          try (Connection connection = createConnection(htu); Admin admin = connection.getAdmin()) {
+            admin.shutdown();
+          } catch (Exception e) {
+            LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
+          }
+          LOG.info("After call to shutdown master");
+          finalCluster.waitOnMaster(MASTER_INDEX);
         }
-        LOG.info("After call to shutdown master");
-        cluster.waitOnMaster(MASTER_INDEX);
+      };
+      shutdownThread.start();
+      LOG.info("Called master join on " + master.getName());
+      master.join();
+      shutdownThread.join();
+
+      List<MasterThread> masterThreads = cluster.getMasters();
+      // make sure all the masters properly shutdown
+      assertEquals(0, masterThreads.size());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      if (htu != null) {
+        htu.shutdownMiniZKCluster();
 
 Review comment:
   nit: You could just use shutdownMiniCluster(). It appears null safe on underlying minicluster

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-585976400
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 39s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 52s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 56s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 11s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 20s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   4m 18s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m  0s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  0s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 13s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 39s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 48s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   0m 38s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   4m 26s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  64m  4s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 40s |  The patch does not generate ASF License warnings.  |
   |  |   | 122m 18s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.6 Server=19.03.6 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1141 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 7d8a8db7b4e7 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1141/out/precommit/personality/provided.sh |
   | git revision | master / 488c2b2483 |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/6/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/6/testReport/ |
   | Max. process+thread count | 8559 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/6/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r376536908
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -128,41 +142,50 @@ public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
 
     // Start the cluster
-    final HBaseTestingUtility util = new HBaseTestingUtility(conf);
-    util.startMiniDFSCluster(3);
-    util.startMiniZKCluster();
-    util.createRootDir();
-    final LocalHBaseCluster cluster =
-        new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
-            MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
-    final int MASTER_INDEX = 0;
-    final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
-    master.start();
-    LOG.info("Called master start on " + master.getName());
-    Thread shutdownThread = new Thread("Shutdown-Thread") {
-      @Override
-      public void run() {
-        LOG.info("Before call to shutdown master");
-        try (Connection connection = createConnection(util); Admin admin = connection.getAdmin()) {
-          admin.shutdown();
-        } catch (Exception e) {
-          LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
+    LocalHBaseCluster cluster = null;
+    try {
+      htu = new HBaseTestingUtility(conf);
+      htu.startMiniDFSCluster(3);
+      htu.startMiniZKCluster();
+      htu.createRootDir();
+      cluster = new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
+        MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
+      final int MASTER_INDEX = 0;
+      final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
+      master.start();
+      LOG.info("Called master start on " + master.getName());
+      final LocalHBaseCluster finalCluster = cluster;
+      Thread shutdownThread = new Thread("Shutdown-Thread") {
+        @Override
+        public void run() {
+          LOG.info("Before call to shutdown master");
+          try (Connection connection = createConnection(htu); Admin admin = connection.getAdmin()) {
+            admin.shutdown();
 
 Review comment:
   No, not after adding this check https://github.com/apache/hbase/blob/d110c08dce482ef3161294ab931d2559d8e57fca/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java#L141
   
   This check basically works around the actual problem without fixing it. Ideally you could do the same because that check is anyway applied in the master after the branch merge.
   
   Fwiw, I think this race is exposed after committing HBASE-23764, because that speeds up the connections.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-585920811
 
 
   Follow-on ticket is https://issues.apache.org/jira/browse/HBASE-23836

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r377391768
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -33,23 +34,34 @@
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
-@Category({MasterTests.class, LargeTests.class})
+@Category({MasterTests.class, MediumTests.class})
 
 Review comment:
   Seems to be snappy for me... What's the idea behind these annotations? They're supposed to indicate the local dev experience, or the overworked Jenkins experience?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r376625684
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -33,23 +34,34 @@
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
-@Category({MasterTests.class, LargeTests.class})
+@Category({MasterTests.class, MediumTests.class})
 
 Review comment:
   For sure? Maybe medium when you run it standalone but under load, maybe it goes over the 50s upper-bound (only saying because did a pass on these recently).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-584820517
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 13s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 42s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   4m 40s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 25s |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 59s |  the patch passed  |
   | +1 :green_heart: |  javac  |   0m 59s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 12s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  4s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 11s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   0m 42s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   5m 13s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  91m  9s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 25s |  The patch does not generate ASF License warnings.  |
   |  |   | 153m 13s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.master.TestMasterShutdown |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.4 Server=19.03.4 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1141 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux c49df62887e3 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1141/out/precommit/personality/provided.sh |
   | git revision | master / 12f98bc50e |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/2/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/2/testReport/ |
   | Max. process+thread count | 5642 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/2/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r378035422
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -30,26 +34,38 @@
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 @Category({MasterTests.class, LargeTests.class})
 public class TestMasterShutdown {
+  private static final Logger LOG = LoggerFactory.getLogger(TestMasterShutdown.class);
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestMasterShutdown.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestMasterShutdown.class);
+  private HBaseTestingUtility htu;
+
+  @Before
+  public void shutdownCluster() throws IOException {
+    if (htu != null) {
 
 Review comment:
   We can skip an @After ? i.e. should the shutdown of cluster be in @After if not there already.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r378036496
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -66,103 +82,172 @@ public void testMasterShutdown() throws Exception {
     Configuration conf = HBaseConfiguration.create();
 
     // Start the cluster
-    HBaseTestingUtility htu = new HBaseTestingUtility(conf);
-    StartMiniClusterOption option = StartMiniClusterOption.builder()
-        .numMasters(NUM_MASTERS).numRegionServers(NUM_RS).numDataNodes(NUM_RS).build();
-    htu.startMiniCluster(option);
-    MiniHBaseCluster cluster = htu.getHBaseCluster();
-
-    // get all the master threads
-    List<MasterThread> masterThreads = cluster.getMasterThreads();
-
-    // wait for each to come online
-    for (MasterThread mt : masterThreads) {
-      assertTrue(mt.isAlive());
-    }
-
-    // find the active master
-    HMaster active = null;
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        active = masterThreads.get(i).getMaster();
-        break;
+    try {
+      htu = new HBaseTestingUtility(conf);
+      StartMiniClusterOption option = StartMiniClusterOption.builder()
+        .numMasters(NUM_MASTERS)
+        .numRegionServers(NUM_RS)
+        .numDataNodes(NUM_RS)
+        .build();
+      final MiniHBaseCluster cluster = htu.startMiniCluster(option);
+
+      // wait for all master thread to spawn and start their run loop.
+      final long thirtySeconds = TimeUnit.SECONDS.toMillis(30);
+      final long oneSecond = TimeUnit.SECONDS.toMillis(1);
+      assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond, () -> {
+        final List<MasterThread> masterThreads = cluster.getMasterThreads();
+        return masterThreads != null
+          && masterThreads.size() >= 3
+          && masterThreads.stream().allMatch(Thread::isAlive);
+      }));
+
+      // find the active master
+      final HMaster active = cluster.getMaster();
+      assertNotNull(active);
+
+      // make sure the other two are backup masters
+      ClusterMetrics status = active.getClusterMetrics();
+      assertEquals(2, status.getBackupMasterNames().size());
+
+      // tell the active master to shutdown the cluster
+      active.shutdown();
+      assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond,
+        () -> CollectionUtils.isEmpty(cluster.getLiveMasterThreads())));
+      assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond,
+        () -> CollectionUtils.isEmpty(cluster.getLiveRegionServerThreads())));
+    } finally {
+      if (htu != null) {
 
 Review comment:
   Somehow we are skipping this? Odd. Move to an @After?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r378036278
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -66,43 +78,45 @@ public void testMasterShutdown() throws Exception {
     Configuration conf = HBaseConfiguration.create();
 
     // Start the cluster
-    HBaseTestingUtility htu = new HBaseTestingUtility(conf);
-    StartMiniClusterOption option = StartMiniClusterOption.builder()
-        .numMasters(NUM_MASTERS).numRegionServers(NUM_RS).numDataNodes(NUM_RS).build();
-    htu.startMiniCluster(option);
-    MiniHBaseCluster cluster = htu.getHBaseCluster();
-
-    // get all the master threads
-    List<MasterThread> masterThreads = cluster.getMasterThreads();
-
-    // wait for each to come online
-    for (MasterThread mt : masterThreads) {
-      assertTrue(mt.isAlive());
-    }
-
-    // find the active master
-    HMaster active = null;
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        active = masterThreads.get(i).getMaster();
-        break;
+    try {
+      htu = new HBaseTestingUtility(conf);
 
 Review comment:
   On hbase.ipc.client.failed.servers.expiry being aggressive... remove the config?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
saintstack commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-603944444
 
 
   @ndimiduk yeah, me too.. studying....

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r377988314
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -66,103 +82,172 @@ public void testMasterShutdown() throws Exception {
     Configuration conf = HBaseConfiguration.create();
 
     // Start the cluster
-    HBaseTestingUtility htu = new HBaseTestingUtility(conf);
-    StartMiniClusterOption option = StartMiniClusterOption.builder()
-        .numMasters(NUM_MASTERS).numRegionServers(NUM_RS).numDataNodes(NUM_RS).build();
-    htu.startMiniCluster(option);
-    MiniHBaseCluster cluster = htu.getHBaseCluster();
-
-    // get all the master threads
-    List<MasterThread> masterThreads = cluster.getMasterThreads();
-
-    // wait for each to come online
-    for (MasterThread mt : masterThreads) {
-      assertTrue(mt.isAlive());
-    }
-
-    // find the active master
-    HMaster active = null;
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        active = masterThreads.get(i).getMaster();
-        break;
+    try {
+      htu = new HBaseTestingUtility(conf);
+      StartMiniClusterOption option = StartMiniClusterOption.builder()
+        .numMasters(NUM_MASTERS)
+        .numRegionServers(NUM_RS)
+        .numDataNodes(NUM_RS)
+        .build();
+      final MiniHBaseCluster cluster = htu.startMiniCluster(option);
+
+      // wait for all master thread to spawn and start their run loop.
+      final long thirtySeconds = TimeUnit.SECONDS.toMillis(30);
+      final long oneSecond = TimeUnit.SECONDS.toMillis(1);
+      assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond, () -> {
+        final List<MasterThread> masterThreads = cluster.getMasterThreads();
+        return masterThreads != null
+          && masterThreads.size() >= 3
+          && masterThreads.stream().allMatch(Thread::isAlive);
+      }));
+
+      // find the active master
+      final HMaster active = cluster.getMaster();
+      assertNotNull(active);
+
+      // make sure the other two are backup masters
+      ClusterMetrics status = active.getClusterMetrics();
+      assertEquals(2, status.getBackupMasterNames().size());
+
+      // tell the active master to shutdown the cluster
+      active.shutdown();
+      assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond,
+        () -> CollectionUtils.isEmpty(cluster.getLiveMasterThreads())));
+      assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond,
+        () -> CollectionUtils.isEmpty(cluster.getLiveRegionServerThreads())));
+    } finally {
+      if (htu != null) {
+        htu.shutdownMiniCluster();
+        htu = null;
       }
     }
-    assertNotNull(active);
-    // make sure the other two are backup masters
-    ClusterMetrics status = active.getClusterMetrics();
-    assertEquals(2, status.getBackupMasterNames().size());
-
-    // tell the active master to shutdown the cluster
-    active.shutdown();
-
-    for (int i = NUM_MASTERS - 1; i >= 0 ;--i) {
-      cluster.waitOnMaster(i);
-    }
-    // make sure all the masters properly shutdown
-    assertEquals(0, masterThreads.size());
-
-    htu.shutdownMiniCluster();
   }
 
-  private Connection createConnection(HBaseTestingUtility util) throws InterruptedException {
-    // the cluster may have not been initialized yet which means we can not get the cluster id thus
-    // an exception will be thrown. So here we need to retry.
-    for (;;) {
-      try {
-        return ConnectionFactory.createConnection(util.getConfiguration());
-      } catch (Exception e) {
-        Thread.sleep(10);
+  /**
+   * This test appears to be an intentional race between a thread that issues a shutdown RPC to the
+   * master, while the master is concurrently realizing it cannot initialize because there are no
+   * region servers available to it. The expected behavior is that master initialization is
+   * interruptable via said shutdown RPC.
+   */
+  @Test
+  public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
+    LocalHBaseCluster hbaseCluster = null;
+    try {
+      htu =  new HBaseTestingUtility(
+        createMasterShutdownBeforeStartingAnyRegionServerConfiguration());
+
+      // configure a cluster with
+      final StartMiniClusterOption options = StartMiniClusterOption.builder()
+        .numDataNodes(1)
+        .numMasters(1)
+        .numRegionServers(0)
+        .masterClass(HMaster.class)
+        .rsClass(MiniHBaseCluster.MiniHBaseClusterRegionServer.class)
+        .createRootDir(true)
+        .build();
+
+      // Can't simply `htu.startMiniCluster(options)` because that method waits for the master to
+      // start completely. However, this test's premise is that a partially started master should
+      // still respond to a shutdown RPC. So instead, we manage each component lifecycle
+      // independently.
+      // I think it's not worth refactoring HTU's helper methods just for this class.
+      htu.startMiniDFSCluster(options.getNumDataNodes());
+      htu.startMiniZKCluster(options.getNumZkServers());
+      htu.createRootDir();
+      hbaseCluster = new LocalHBaseCluster(htu.getConfiguration(), options.getNumMasters(),
+        options.getNumRegionServers(), options.getMasterClass(), options.getRsClass());
+      final MasterThread masterThread = hbaseCluster.getMasters().get(0);
+
+      final CompletableFuture<Void> shutdownFuture = CompletableFuture.runAsync(() -> {
+        // Switching to master registry exacerbated a race in the master bootstrap that can result
+        // in a lost shutdown command (HBASE-8422). The race is essentially because the server
+        // manager in HMaster is not initialized by the time shutdown() RPC (below) is made to the
+        // master. The suspected reason as to why it was uncommon before HBASE-18095 is because the
+        // connection creation with ZK registry is so slow that by then the server manager is
+        // usually init'ed in time for the RPC to be made. For now, adding an explicit wait() in
+        // the test, waiting for the server manager to become available.
+        final long timeout = TimeUnit.MINUTES.toMillis(10);
+//        assertNotEquals("timeout waiting for server manager to become available.",
 
 Review comment:
   Bah. need to delete this commented code.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-583349397
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   5m 19s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m 12s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  5s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 14s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 48s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 39s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m  9s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   4m  7s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 39s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  1s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m  8s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  6s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m  7s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   4m 31s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |   4m 46s |  hbase-server in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 16s |  The patch does not generate ASF License warnings.  |
   |  |   |  70m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1141 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 3007616e834e 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1141/out/precommit/personality/provided.sh |
   | git revision | master / 93c5e7691f |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/1/artifact/out/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/1/testReport/ |
   | Max. process+thread count | 896 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/1/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-584956495
 
 
   > > More digging, will be back.
   > 
   > Is there more to it than fixing the ServerManager initialization? Meaning, does the test still fail after my waitFor() fix?
   
   That fixes the test, but I think that's exactly the kind of gap that HBASE-8422 was trying to close.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r376199438
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -128,41 +142,50 @@ public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
 
     // Start the cluster
-    final HBaseTestingUtility util = new HBaseTestingUtility(conf);
-    util.startMiniDFSCluster(3);
-    util.startMiniZKCluster();
-    util.createRootDir();
-    final LocalHBaseCluster cluster =
-        new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
-            MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
-    final int MASTER_INDEX = 0;
-    final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
-    master.start();
-    LOG.info("Called master start on " + master.getName());
-    Thread shutdownThread = new Thread("Shutdown-Thread") {
-      @Override
-      public void run() {
-        LOG.info("Before call to shutdown master");
-        try (Connection connection = createConnection(util); Admin admin = connection.getAdmin()) {
-          admin.shutdown();
-        } catch (Exception e) {
-          LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
+    LocalHBaseCluster cluster = null;
+    try {
+      htu = new HBaseTestingUtility(conf);
+      htu.startMiniDFSCluster(3);
+      htu.startMiniZKCluster();
+      htu.createRootDir();
+      cluster = new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
+        MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
+      final int MASTER_INDEX = 0;
+      final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
+      master.start();
+      LOG.info("Called master start on " + master.getName());
+      final LocalHBaseCluster finalCluster = cluster;
+      Thread shutdownThread = new Thread("Shutdown-Thread") {
+        @Override
+        public void run() {
+          LOG.info("Before call to shutdown master");
+          try (Connection connection = createConnection(htu); Admin admin = connection.getAdmin()) {
+            admin.shutdown();
 
 Review comment:
   This is the place that the master registry exposed a race ..(shutdown goes missing..). Rebase will not be clean now :'(

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk merged pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk merged pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r376481912
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -128,41 +142,50 @@ public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
 
     // Start the cluster
-    final HBaseTestingUtility util = new HBaseTestingUtility(conf);
-    util.startMiniDFSCluster(3);
-    util.startMiniZKCluster();
-    util.createRootDir();
-    final LocalHBaseCluster cluster =
-        new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
-            MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
-    final int MASTER_INDEX = 0;
-    final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
-    master.start();
-    LOG.info("Called master start on " + master.getName());
-    Thread shutdownThread = new Thread("Shutdown-Thread") {
-      @Override
-      public void run() {
-        LOG.info("Before call to shutdown master");
-        try (Connection connection = createConnection(util); Admin admin = connection.getAdmin()) {
-          admin.shutdown();
-        } catch (Exception e) {
-          LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
+    LocalHBaseCluster cluster = null;
+    try {
+      htu = new HBaseTestingUtility(conf);
+      htu.startMiniDFSCluster(3);
+      htu.startMiniZKCluster();
+      htu.createRootDir();
+      cluster = new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
+        MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
+      final int MASTER_INDEX = 0;
+      final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
+      master.start();
+      LOG.info("Called master start on " + master.getName());
+      final LocalHBaseCluster finalCluster = cluster;
+      Thread shutdownThread = new Thread("Shutdown-Thread") {
+        @Override
+        public void run() {
+          LOG.info("Before call to shutdown master");
+          try (Connection connection = createConnection(htu); Admin admin = connection.getAdmin()) {
+            admin.shutdown();
 
 Review comment:
   I think this is the source of the instability I've seen in this test, but I cannot reproduce it reliably.
   
   You see this test failing reliably on your branch?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r379049688
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -30,26 +34,38 @@
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 @Category({MasterTests.class, LargeTests.class})
 public class TestMasterShutdown {
+  private static final Logger LOG = LoggerFactory.getLogger(TestMasterShutdown.class);
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestMasterShutdown.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestMasterShutdown.class);
+  private HBaseTestingUtility htu;
+
+  @Before
+  public void shutdownCluster() throws IOException {
+    if (htu != null) {
 
 Review comment:
   I tried writing a test for `HBaseClassTestRule ` and the `Timeout` class it uses to assert the behavior of a miniCluster in the face of the interrupt, but that took me round in circles. Going to pass on that for the time being.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r377391906
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -66,43 +78,45 @@ public void testMasterShutdown() throws Exception {
     Configuration conf = HBaseConfiguration.create();
 
     // Start the cluster
-    HBaseTestingUtility htu = new HBaseTestingUtility(conf);
-    StartMiniClusterOption option = StartMiniClusterOption.builder()
-        .numMasters(NUM_MASTERS).numRegionServers(NUM_RS).numDataNodes(NUM_RS).build();
-    htu.startMiniCluster(option);
-    MiniHBaseCluster cluster = htu.getHBaseCluster();
-
-    // get all the master threads
-    List<MasterThread> masterThreads = cluster.getMasterThreads();
-
-    // wait for each to come online
-    for (MasterThread mt : masterThreads) {
-      assertTrue(mt.isAlive());
-    }
-
-    // find the active master
-    HMaster active = null;
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        active = masterThreads.get(i).getMaster();
-        break;
+    try {
+      htu = new HBaseTestingUtility(conf);
 
 Review comment:
   This is a new conf, created above... Let me move them in here after htu is created but before cluster start. I'm not clear on why `hbase.ipc.client.failed.servers.expiry` is set so aggressively.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r377394544
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -128,41 +142,50 @@ public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
 
     // Start the cluster
-    final HBaseTestingUtility util = new HBaseTestingUtility(conf);
-    util.startMiniDFSCluster(3);
-    util.startMiniZKCluster();
-    util.createRootDir();
-    final LocalHBaseCluster cluster =
-        new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
-            MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
-    final int MASTER_INDEX = 0;
-    final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
-    master.start();
-    LOG.info("Called master start on " + master.getName());
-    Thread shutdownThread = new Thread("Shutdown-Thread") {
-      @Override
-      public void run() {
-        LOG.info("Before call to shutdown master");
-        try (Connection connection = createConnection(util); Admin admin = connection.getAdmin()) {
-          admin.shutdown();
-        } catch (Exception e) {
-          LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
+    LocalHBaseCluster cluster = null;
+    try {
+      htu = new HBaseTestingUtility(conf);
+      htu.startMiniDFSCluster(3);
+      htu.startMiniZKCluster();
+      htu.createRootDir();
+      cluster = new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
+        MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
+      final int MASTER_INDEX = 0;
+      final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
+      master.start();
+      LOG.info("Called master start on " + master.getName());
+      final LocalHBaseCluster finalCluster = cluster;
+      Thread shutdownThread = new Thread("Shutdown-Thread") {
+        @Override
+        public void run() {
+          LOG.info("Before call to shutdown master");
+          try (Connection connection = createConnection(htu); Admin admin = connection.getAdmin()) {
+            admin.shutdown();
 
 Review comment:
   I re-read your changes on the branch. Let me try to merge those with the general cleanup I'm attempting here. Hopefully when it's done, you can just drop the change to this test from the feature branch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r377222438
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -66,43 +78,45 @@ public void testMasterShutdown() throws Exception {
     Configuration conf = HBaseConfiguration.create();
 
     // Start the cluster
-    HBaseTestingUtility htu = new HBaseTestingUtility(conf);
-    StartMiniClusterOption option = StartMiniClusterOption.builder()
-        .numMasters(NUM_MASTERS).numRegionServers(NUM_RS).numDataNodes(NUM_RS).build();
-    htu.startMiniCluster(option);
-    MiniHBaseCluster cluster = htu.getHBaseCluster();
-
-    // get all the master threads
-    List<MasterThread> masterThreads = cluster.getMasterThreads();
-
-    // wait for each to come online
-    for (MasterThread mt : masterThreads) {
-      assertTrue(mt.isAlive());
-    }
-
-    // find the active master
-    HMaster active = null;
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        active = masterThreads.get(i).getMaster();
-        break;
+    try {
+      htu = new HBaseTestingUtility(conf);
 
 Review comment:
   You don't want to get a new conf to go w/ the new htu?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-584993304
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 1 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 48s |  master passed  |
   | +1 :green_heart: |  compile  |   0m 58s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 23s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 21s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 40s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 38s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   4m 36s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 16s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m  2s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedjars  |   4m 41s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  16m 42s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   4m 17s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  58m 39s |  hbase-server in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   | 118m 41s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1141 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 32ae11ce03fa 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1141/out/precommit/personality/provided.sh |
   | git revision | master / 12f98bc50e |
   | Default Java | 1.8.0_181 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/3/testReport/ |
   | Max. process+thread count | 7309 (vs. ulimit of 10000) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1141/3/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r377993610
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
 ##########
 @@ -2804,9 +2804,15 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
    * Master runs a coordinated stop of all RegionServers and then itself.
    */
   public void shutdown() throws IOException {
+    if (!isInitialized()) {
+      LOG.info("Shutdown requested but we're not the active master. Proceeding as a stop.");
 
 Review comment:
   I think another reasonable alternative here is to reject the RPC back to the client with some form of `DoNotRetryIOException` and a message about cannot shutdown from an inactive master.
   
   What do you think?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r376196608
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -66,43 +78,45 @@ public void testMasterShutdown() throws Exception {
     Configuration conf = HBaseConfiguration.create();
 
     // Start the cluster
-    HBaseTestingUtility htu = new HBaseTestingUtility(conf);
-    StartMiniClusterOption option = StartMiniClusterOption.builder()
-        .numMasters(NUM_MASTERS).numRegionServers(NUM_RS).numDataNodes(NUM_RS).build();
-    htu.startMiniCluster(option);
-    MiniHBaseCluster cluster = htu.getHBaseCluster();
-
-    // get all the master threads
-    List<MasterThread> masterThreads = cluster.getMasterThreads();
-
-    // wait for each to come online
-    for (MasterThread mt : masterThreads) {
-      assertTrue(mt.isAlive());
-    }
-
-    // find the active master
-    HMaster active = null;
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        active = masterThreads.get(i).getMaster();
-        break;
+    try {
+      htu = new HBaseTestingUtility(conf);
+      StartMiniClusterOption option = StartMiniClusterOption.builder()
+        .numMasters(NUM_MASTERS)
+        .numRegionServers(NUM_RS)
+        .numDataNodes(NUM_RS)
+        .build();
+      final MiniHBaseCluster cluster = htu.startMiniCluster(option);
+
+      // wait for all master thread to spawn and start their run loop.
+      final long thirtySeconds = TimeUnit.SECONDS.toMillis(30);
+      final long oneSecond = TimeUnit.SECONDS.toMillis(1);
+      assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond, () -> {
+        final List<MasterThread> masterThreads = cluster.getMasterThreads();
+        return CollectionUtils.isNotEmpty(masterThreads)
 
 Review comment:
   nit: Can probably be simplified to masterThreads != null && masterThreads.size() >=3...
   because the second check automatically means isNotEmpty().

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r378503162
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -66,43 +78,45 @@ public void testMasterShutdown() throws Exception {
     Configuration conf = HBaseConfiguration.create();
 
     // Start the cluster
-    HBaseTestingUtility htu = new HBaseTestingUtility(conf);
-    StartMiniClusterOption option = StartMiniClusterOption.builder()
-        .numMasters(NUM_MASTERS).numRegionServers(NUM_RS).numDataNodes(NUM_RS).build();
-    htu.startMiniCluster(option);
-    MiniHBaseCluster cluster = htu.getHBaseCluster();
-
-    // get all the master threads
-    List<MasterThread> masterThreads = cluster.getMasterThreads();
-
-    // wait for each to come online
-    for (MasterThread mt : masterThreads) {
-      assertTrue(mt.isAlive());
-    }
-
-    // find the active master
-    HMaster active = null;
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        active = masterThreads.get(i).getMaster();
-        break;
+    try {
+      htu = new HBaseTestingUtility(conf);
 
 Review comment:
   yeah, removed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-584748786
 
 
   Updated per PR feedback. Also some further restructuring of the test, including some learnings from HBASE-23647.
   
   I think there's a real bug here. I can issue a `shutdown` RPC that is acknowledged by the master and commences shutdown. However, at least one subsystem is not getting the message and continues on. This seems to leave the master in a quasi-initialized zombie state. (I believe this is what @bharathv was seeing when he added the `waitFor` on the `ServerManager` instance) Eventually the 200sec master initialization timeout trips and the test fails.
   
   More digging, will be back.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r378500802
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
 ##########
 @@ -30,26 +34,38 @@
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 @Category({MasterTests.class, LargeTests.class})
 public class TestMasterShutdown {
+  private static final Logger LOG = LoggerFactory.getLogger(TestMasterShutdown.class);
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestMasterShutdown.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestMasterShutdown.class);
+  private HBaseTestingUtility htu;
+
+  @Before
+  public void shutdownCluster() throws IOException {
+    if (htu != null) {
 
 Review comment:
   I'm not exactly sure what the junit contract is on executing `finally` clauses and `after` method in the face of timeouts, as implemented with our classifier rule. I was being extra-special redundant with this cleanup. Probably one or the other is fine. Let me see if i can understand what junit does.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r378035129
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
 ##########
 @@ -2804,9 +2804,15 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
    * Master runs a coordinated stop of all RegionServers and then itself.
    */
   public void shutdown() throws IOException {
+    if (!isInitialized()) {
+      LOG.info("Shutdown requested but we're not the active master. Proceeding as a stop.");
 
 Review comment:
   Yeah, this stuff is tricky and there be dragons if you disturb the current order.
   
   The standbys don't go down currently?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#discussion_r378510163
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
 ##########
 @@ -2804,9 +2804,15 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
    * Master runs a coordinated stop of all RegionServers and then itself.
    */
   public void shutdown() throws IOException {
+    if (!isInitialized()) {
+      LOG.info("Shutdown requested but we're not the active master. Proceeding as a stop.");
 
 Review comment:
   @bharathv has the gist of it. At the point of this race condition -- all four of these fields are `null` at the time the rpc is received -- the master will simply do nothing. However, any master (active or backup) can currently receive the rpc and if it's `clusterStatusTracker` is non-null, it will delete this ZK node. From there, in the case of a backup master, the `ActiveMasterManager` will notice and `stop` itself.
   
   Related, looks like there's an early-out in `ServerManager#shutdown` that can result in a master `stop`ping without properly shutting down its procedure store.
   
   ```java
       if (onlineServers.isEmpty()) {
         // we do not synchronize here so this may cause a double stop, but not a big deal
         master.stop("OnlineServer=0 right after cluster shutdown set");
       }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #1141: HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore…
URL: https://github.com/apache/hbase/pull/1141#issuecomment-603931441
 
 
   This test failed again for me last night. attaching a log to the jira.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services