You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2020/08/11 03:59:26 UTC

[hbase] branch branch-2.2 updated (813b324 -> 0e36228)

This is an automated email from the ASF dual-hosted git repository.

zghao pushed a change to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/hbase.git.


    from 813b324  HBASE-24841 Change the jenkins job urls in our jenkinsfile (#2227)
     new d1774e6  HBASE-23851 Log networks and bind addresses when multicast publisher/listener enabled (#1173)
     new 0e36228  HBASE-23866 More test classifications

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../hadoop/hbase/client/AsyncConnectionImpl.java   |  3 +-
 .../hadoop/hbase/client/ClusterStatusListener.java | 20 +++----
 .../org/apache/hadoop/hbase/ScheduledChore.java    |  4 +-
 .../hbase/master/ClusterStatusPublisher.java       | 37 ++++++++----
 .../org/apache/hadoop/hbase/master/HMaster.java    |  1 +
 .../hbase/client/TestAsyncMetaRegionLocator.java   |  5 +-
 .../hbase/client/TestAsyncResultScannerCursor.java |  6 +-
 .../hbase/client/TestAsyncTableRSCrashPublish.java | 68 ++++++++++++++--------
 .../hadoop/hbase/client/TestAsyncTableScanAll.java |  6 +-
 .../hbase/client/TestScannersFromClientSide2.java  |  6 +-
 .../hbase/client/TestSeparateClientZKCluster.java  |  5 +-
 .../hbase/filter/TestMultiRowRangeFilter.java      |  6 +-
 .../hadoop/hbase/http/TestInfoServersACL.java      | 20 +++++++
 .../TestStochasticLoadBalancerRegionReplica.java   |  6 +-
 .../hbase/security/token/SecureTestCluster.java    |  2 +-
 15 files changed, 124 insertions(+), 71 deletions(-)


[hbase] 02/02: HBASE-23866 More test classifications

Posted by zg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zghao pushed a commit to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 0e3622852599e8d6816d402d6e1052b6945b226b
Author: stack <st...@apache.org>
AuthorDate: Tue Feb 18 21:03:15 2020 -0800

    HBASE-23866 More test classifications
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
     Add wait on no region in transition in setup before proceeding.
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java
     Make test large. Takes longer than 50seconds.
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
     Make large because it fails when networking is weird... when VPN or
     something is up. See HBASE-23850. When large, doesn't get run
     when you do mvn test.
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java
    hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide2.java
    hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
    hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java
     Make it large. Takes too long.
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
     Use data dir instead of /tmp.
    
    M hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
     Wait till mbean is up before proceeding w/ test.
    
    hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
     Test for null CLUSTER in tearDown.
---
 .../hbase/client/TestAsyncMetaRegionLocator.java   |  5 +-
 .../hbase/client/TestAsyncResultScannerCursor.java |  6 +-
 .../hbase/client/TestAsyncTableRSCrashPublish.java | 68 ++++++++++++++--------
 .../hadoop/hbase/client/TestAsyncTableScanAll.java |  6 +-
 .../hbase/client/TestScannersFromClientSide2.java  |  6 +-
 .../hbase/client/TestSeparateClientZKCluster.java  |  5 +-
 .../hbase/filter/TestMultiRowRangeFilter.java      |  6 +-
 .../hadoop/hbase/http/TestInfoServersACL.java      | 20 +++++++
 .../TestStochasticLoadBalancerRegionReplica.java   |  6 +-
 .../hbase/security/token/SecureTestCluster.java    |  2 +-
 10 files changed, 84 insertions(+), 46 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
index 1f0d40b..a228043 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator;
-import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.AfterClass;
@@ -51,9 +50,9 @@ public class TestAsyncMetaRegionLocator {
 
   @BeforeClass
   public static void setUp() throws Exception {
-    TEST_UTIL.getConfiguration().set(BaseLoadBalancer.TABLES_ON_MASTER, "none");
     TEST_UTIL.getConfiguration().setInt(HConstants.META_REPLICAS_NUM, 3);
     TEST_UTIL.startMiniCluster(3);
+    TEST_UTIL.waitUntilNoRegionsInTransition();
     REGISTRY = AsyncRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     RegionReplicaTestHelper
       .waitUntilAllMetaReplicasHavingRegionLocation(TEST_UTIL.getConfiguration(), REGISTRY, 3);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java
index 638ae8c..d8e293c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncResultScannerCursor.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -20,12 +20,12 @@ package org.apache.hadoop.hbase.client;
 import java.util.concurrent.ForkJoinPool;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 
-@Category({ MediumTests.class, ClientTests.class })
+@Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncResultScannerCursor extends AbstractTestResultScannerCursor {
 
   @ClassRule
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
index 849feb8..461bf1b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,15 +18,16 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertNotEquals;
-
 import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -34,9 +35,10 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
-
-@Category({ MediumTests.class, ClientTests.class })
+// Categorized as a large test so not run as part of general 'test' suite (which is small
+// and mediums). This test fails if networking is odd -- say if you are connected to a
+// VPN... See HBASE-23850
+@Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncTableRSCrashPublish {
 
   @ClassRule
@@ -45,40 +47,56 @@ public class TestAsyncTableRSCrashPublish {
 
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
-  private static AsyncConnectionImpl CONN;
-
   private static TableName TABLE_NAME = TableName.valueOf("Publish");
 
   private static byte[] FAMILY = Bytes.toBytes("family");
 
   @BeforeClass
-  public static void setUp() throws Exception {
+  public static void beforeClass() throws Exception {
     UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true);
+    /* Below is code for choosing a NetworkInterface and then setting it into
+      configs so can be picked up by the client and server.
+    String niName = UTIL.getConfiguration().get(HConstants.STATUS_MULTICAST_NI_NAME);
+    NetworkInterface ni;
+    if (niName != null) {
+      ni = NetworkInterface.getByName(niName);
+    } else {
+      String mcAddress = UTIL.getConfiguration().get(HConstants.STATUS_MULTICAST_ADDRESS,
+        HConstants.DEFAULT_STATUS_MULTICAST_ADDRESS);
+      InetAddress ina = InetAddress.getByName(mcAddress);
+      boolean inet6Address = ina instanceof Inet6Address;
+      ni = NetworkInterface.getByInetAddress(inet6Address?
+        Addressing.getIp6Address(): Addressing.getIp4Address());
+    }
+    UTIL.getConfiguration().set(HConstants.STATUS_MULTICAST_NI_NAME, ni.getName());
+    */
     UTIL.startMiniCluster(2);
     UTIL.createTable(TABLE_NAME, FAMILY);
     UTIL.waitTableAvailable(TABLE_NAME);
-    CONN =
-      (AsyncConnectionImpl) ConnectionFactory.createAsyncConnection(UTIL.getConfiguration()).get();
   }
 
   @AfterClass
-  public static void tearDown() throws Exception {
-    Closeables.close(CONN, true);
+  public static void afterClass() throws Exception {
     UTIL.shutdownMiniCluster();
   }
 
   @Test
-  public void test() throws IOException {
-    AsyncNonMetaRegionLocator locator = CONN.getLocator().getNonMetaRegionLocator();
-    CONN.getTable(TABLE_NAME).get(new Get(Bytes.toBytes(0))).join();
-    ServerName serverName = locator.getRegionLocationInCache(TABLE_NAME, HConstants.EMPTY_START_ROW)
-      .getDefaultRegionLocation().getServerName();
-    UTIL.getMiniHBaseCluster().stopRegionServer(serverName);
-    UTIL.waitFor(60000,
-      () -> locator.getRegionLocationInCache(TABLE_NAME, HConstants.EMPTY_START_ROW) == null);
-    CONN.getTable(TABLE_NAME).get(new Get(Bytes.toBytes(0))).join();
-    assertNotEquals(serverName,
-      locator.getRegionLocationInCache(TABLE_NAME, HConstants.EMPTY_START_ROW)
-        .getDefaultRegionLocation().getServerName());
+  public void test() throws IOException, ExecutionException, InterruptedException {
+    Configuration conf = UTIL.getHBaseCluster().getMaster().getConfiguration();
+    try (AsyncConnection connection = ConnectionFactory.createAsyncConnection(conf).get()) {
+      AsyncNonMetaRegionLocator locator =
+        ((AsyncConnectionImpl) connection).getLocator().getNonMetaRegionLocator();
+      connection.getTable(TABLE_NAME).get(new Get(Bytes.toBytes(0))).join();
+      ServerName serverName =
+        locator.getRegionLocationInCache(TABLE_NAME, HConstants.EMPTY_START_ROW)
+        .getDefaultRegionLocation().getServerName();
+      UTIL.getMiniHBaseCluster().stopRegionServer(serverName);
+      UTIL.waitFor(60000,
+        () -> locator.getRegionLocationInCache(TABLE_NAME, HConstants.EMPTY_START_ROW) == null);
+      connection.getTable(TABLE_NAME).get(new Get(Bytes.toBytes(0))).join();
+      assertNotEquals(serverName,
+        locator.getRegionLocationInCache(TABLE_NAME, HConstants.EMPTY_START_ROW)
+          .getDefaultRegionLocation().getServerName());
+    }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java
index b8adbce..d9a5395 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -21,7 +21,7 @@ import java.util.List;
 import java.util.function.Supplier;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -30,7 +30,7 @@ import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(Parameterized.class)
-@Category({ MediumTests.class, ClientTests.class })
+@Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncTableScanAll extends AbstractTestAsyncTableScan {
 
   @ClassRule
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide2.java
index 0a10818..d973051 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide2.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -45,7 +45,7 @@ import org.junit.runners.Parameterized.Parameters;
  * inclusive/exclusive, limit for rows, etc.
  */
 @RunWith(Parameterized.class)
-@Category({ MediumTests.class, ClientTests.class })
+@Category({ LargeTests.class, ClientTests.class })
 public class TestScannersFromClientSide2 {
 
   @ClassRule
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
index f44ef28..816801c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -48,7 +48,8 @@ import org.slf4j.LoggerFactory;
 public class TestSeparateClientZKCluster {
   private static final Logger LOG = LoggerFactory.getLogger(TestSeparateClientZKCluster.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final File clientZkDir = new File("/tmp/TestSeparateClientZKCluster");
+  private static final File clientZkDir =
+    new File(TEST_UTIL.getDataTestDir("TestSeparateClientZKCluster").toString());
   private static final int ZK_SESSION_TIMEOUT = 5000;
   private static MiniZooKeeperCluster clientZkCluster;
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
index 00c557c..9f8cca2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -48,7 +48,7 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Category(MediumTests.class)
+@Category(LargeTests.class)
 public class TestMultiRowRangeFilter {
 
   @ClassRule
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
index fa5dea6..69b0a2d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
+import java.lang.management.ManagementFactory;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
 import org.apache.hadoop.hbase.security.token.TokenProvider;
@@ -41,6 +43,7 @@ import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.http.auth.AuthSchemeProvider;
@@ -70,6 +73,10 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.management.InstanceNotFoundException;
+import javax.management.MBeanInfo;
+import javax.management.ObjectName;
+
 /**
  * Testing info servers for admin acl.
  */
@@ -300,6 +307,19 @@ public class TestInfoServersACL {
   @Test
   public void testJmxAvailableForAdmins() throws Exception {
     final String expectedAuthorizedContent = "Hadoop:service=HBase";
+    UTIL.waitFor(30000, new Waiter.Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        for (ObjectName name: ManagementFactory.getPlatformMBeanServer().
+          queryNames(new ObjectName("*:*"), null)) {
+          if (name.toString().contains(expectedAuthorizedContent)) {
+            LOG.info("{}", name);
+            return true;
+          }
+        }
+        return false;
+      }
+    });
     UserGroupInformation admin = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
         USER_ADMIN_STR, KEYTAB_FILE.getAbsolutePath());
     admin.doAs(new PrivilegedExceptionAction<Void>() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java
index f5a8cba..9634f9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancerRegionReplica.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -36,13 +36,13 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.master.RackManager;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-@Category({ MasterTests.class, MediumTests.class })
+@Category({ MasterTests.class, LargeTests.class })
 public class TestStochasticLoadBalancerRegionReplica extends BalancerTestBase {
 
   @ClassRule
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
index 1323946..5b6dadc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/SecureTestCluster.java
@@ -98,8 +98,8 @@ public class SecureTestCluster {
     try {
       if (CLUSTER != null) {
         CLUSTER.shutdown();
+        CLUSTER.join();
       }
-      CLUSTER.join();
       if (KDC != null) {
         KDC.stop();
       }


[hbase] 01/02: HBASE-23851 Log networks and bind addresses when multicast publisher/listener enabled (#1173)

Posted by zg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zghao pushed a commit to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit d1774e67b6ebba5d73e695d176a422ef8e34f06e
Author: Michael Stack <sa...@users.noreply.github.com>
AuthorDate: Tue Mar 10 10:55:44 2020 -0700

    HBASE-23851 Log networks and bind addresses when multicast publisher/listener enabled (#1173)
    
    Signed-off-by: Sean Busbey <bu...@apache.org>
    Signed-off-by: Nick Dimiduk <nd...@apache.org>
    Signed-off-by: Viraj Jasani <vj...@apache.org>
---
 .../hadoop/hbase/client/AsyncConnectionImpl.java   |  3 +-
 .../hadoop/hbase/client/ClusterStatusListener.java | 20 ++++++------
 .../org/apache/hadoop/hbase/ScheduledChore.java    |  4 +--
 .../hbase/master/ClusterStatusPublisher.java       | 37 ++++++++++++++++------
 .../org/apache/hadoop/hbase/master/HMaster.java    |  1 +
 5 files changed, 40 insertions(+), 25 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index f58dfba..e0b7407 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -165,8 +165,7 @@ class AsyncConnectionImpl implements AsyncConnection {
               }
             }, conf, listenerClass);
         } catch (IOException e) {
-          LOG.warn("Failed to create ClusterStatusListener, not a critical problem, ignoring...",
-            e);
+          LOG.warn("Failed create of ClusterStatusListener, not a critical, ignoring...", e);
         }
       }
     }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
index 7361238..ccdfec7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
@@ -37,10 +37,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.io.netty.bootstrap.Bootstrap;
 import org.apache.hbase.thirdparty.io.netty.buffer.ByteBufInputStream;
 import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
@@ -51,8 +47,10 @@ import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramChannel;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramPacket;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioDatagramChannel;
-
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A class that receives the cluster status, and provide it as a set of service to the client.
@@ -208,10 +206,9 @@ class ClusterStatusListener implements Closeable {
       try {
         Bootstrap b = new Bootstrap();
         b.group(group)
-            .channel(NioDatagramChannel.class)
-            .option(ChannelOption.SO_REUSEADDR, true)
-            .handler(new ClusterStatusHandler());
-
+          .channel(NioDatagramChannel.class)
+          .option(ChannelOption.SO_REUSEADDR, true)
+          .handler(new ClusterStatusHandler());
         channel = (DatagramChannel)b.bind(bindAddress, port).sync().channel();
       } catch (InterruptedException e) {
         close();
@@ -225,9 +222,11 @@ class ClusterStatusListener implements Closeable {
         ni = NetworkInterface.getByInetAddress(Addressing.getIpAddress());
       }
 
+      LOG.debug("Channel bindAddress={}, networkInterface={}, INA={}", bindAddress, ni, ina);
       channel.joinGroup(ina, ni, null, channel.newPromise());
     }
 
+
     @Override
     public void close() {
       if (channel != null) {
@@ -252,8 +251,7 @@ class ClusterStatusListener implements Closeable {
       }
 
       @Override
-      public boolean acceptInboundMessage(Object msg)
-          throws Exception {
+      public boolean acceptInboundMessage(Object msg) throws Exception {
         return super.acceptInboundMessage(msg);
       }
 
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
index 358ab23..28af186 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ScheduledChore.java
@@ -369,7 +369,7 @@ public abstract class ScheduledChore implements Runnable {
   @InterfaceAudience.Private
   @Override
   public String toString() {
-    return "[ScheduledChore: Name: " + getName() + " Period: " + getPeriod() + " Unit: "
-        + getTimeUnit() + "]";
+    return "ScheduledChore name=" + getName() + ", period=" + getPeriod() +
+      ", unit=" + getTimeUnit();
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
index af35ce4..8257466 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
  * 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
@@ -49,12 +48,11 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
-
 import org.apache.hbase.thirdparty.io.netty.bootstrap.Bootstrap;
-import org.apache.hbase.thirdparty.io.netty.bootstrap.ChannelFactory;
 import org.apache.hbase.thirdparty.io.netty.buffer.Unpooled;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.ChannelException;
+import org.apache.hbase.thirdparty.io.netty.channel.ChannelFactory;
 import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;
 import org.apache.hbase.thirdparty.io.netty.channel.ChannelOption;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
@@ -65,6 +63,8 @@ import org.apache.hbase.thirdparty.io.netty.channel.socket.InternetProtocolFamil
 import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioDatagramChannel;
 import org.apache.hbase.thirdparty.io.netty.handler.codec.MessageToMessageEncoder;
 import org.apache.hbase.thirdparty.io.netty.util.internal.StringUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
@@ -75,6 +75,7 @@ import org.apache.hbase.thirdparty.io.netty.util.internal.StringUtil;
  */
 @InterfaceAudience.Private
 public class ClusterStatusPublisher extends ScheduledChore {
+  private static Logger LOG = LoggerFactory.getLogger(ClusterStatusPublisher.class);
   /**
    * The implementation class used to publish the status. Default is null (no publish).
    * Use org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher to multicast the
@@ -113,7 +114,7 @@ public class ClusterStatusPublisher extends ScheduledChore {
   public ClusterStatusPublisher(HMaster master, Configuration conf,
                                 Class<? extends Publisher> publisherClass)
       throws IOException {
-    super("HBase clusterStatusPublisher for " + master.getName(), master, conf.getInt(
+    super("ClusterStatusPublisher for=" + master.getName(), master, conf.getInt(
       STATUS_PUBLISH_PERIOD, DEFAULT_STATUS_PUBLISH_PERIOD));
     this.master = master;
     this.messagePeriod = conf.getInt(STATUS_PUBLISH_PERIOD, DEFAULT_STATUS_PUBLISH_PERIOD);
@@ -126,6 +127,11 @@ public class ClusterStatusPublisher extends ScheduledChore {
     connected = true;
   }
 
+  @Override
+  public String toString() {
+    return super.toString() + ", publisher=" + this.publisher + ", connected=" + this.connected;
+  }
+
   // For tests only
   protected ClusterStatusPublisher() {
     master = null;
@@ -246,6 +252,11 @@ public class ClusterStatusPublisher extends ScheduledChore {
     }
 
     @Override
+    public String toString() {
+      return "channel=" + this.channel;
+    }
+
+    @Override
     public void connect(Configuration conf) throws IOException {
       String mcAddress = conf.get(HConstants.STATUS_MULTICAST_ADDRESS,
           HConstants.DEFAULT_STATUS_MULTICAST_ADDRESS);
@@ -262,7 +273,6 @@ public class ClusterStatusPublisher extends ScheduledChore {
         close();
         throw new IOException("Can't connect to " + mcAddress, e);
       }
-
       final InetSocketAddress isa = new InetSocketAddress(mcAddress, port);
 
       InternetProtocolFamily family;
@@ -285,17 +295,23 @@ public class ClusterStatusPublisher extends ScheduledChore {
         }
         ni = NetworkInterface.getByInetAddress(localAddress);
       }
-
       Bootstrap b = new Bootstrap();
       b.group(group)
         .channelFactory(new HBaseDatagramChannelFactory<Channel>(NioDatagramChannel.class, family))
         .option(ChannelOption.SO_REUSEADDR, true)
         .handler(new ClusterMetricsEncoder(isa));
-
       try {
+        LOG.debug("Channel bindAddress={}, networkInterface={}, INA={}", bindAddress, ni, ina);
         channel = (DatagramChannel) b.bind(bindAddress, 0).sync().channel();
         channel.joinGroup(ina, ni, null, channel.newPromise()).sync();
         channel.connect(isa).sync();
+        // Set into configuration in case many networks available. Do this for tests so that
+        // server and client use same Interface (presuming share same Configuration).
+        // TestAsyncTableRSCrashPublish was failing when connected to VPN because extra networks
+        // available with Master binding on one Interface and client on another so test failed.
+        if (ni != null) {
+          conf.set(HConstants.STATUS_MULTICAST_NI_NAME, ni.getName());
+        }
       } catch (InterruptedException e) {
         close();
         throw ExceptionUtil.asInterrupt(e);
@@ -303,9 +319,9 @@ public class ClusterStatusPublisher extends ScheduledChore {
     }
 
     private static final class HBaseDatagramChannelFactory<T extends Channel>
-      implements ChannelFactory<T> {
+        implements ChannelFactory<T> {
       private final Class<? extends T> clazz;
-      private InternetProtocolFamily family;
+      private final InternetProtocolFamily family;
 
       HBaseDatagramChannelFactory(Class<? extends T> clazz, InternetProtocolFamily family) {
         this.clazz = clazz;
@@ -347,6 +363,7 @@ public class ClusterStatusPublisher extends ScheduledChore {
 
     @Override
     public void publish(ClusterMetrics cs) {
+      LOG.info("PUBLISH {}", cs);
       channel.writeAndFlush(cs).syncUninterruptibly();
     }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 0dc2997..f244e19 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -544,6 +544,7 @@ public class HMaster extends HRegionServer implements MasterServices {
               " is not set - not publishing status");
         } else {
           clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass);
+          LOG.debug("Created {}", this.clusterStatusPublisherChore);
           getChoreService().scheduleChore(clusterStatusPublisherChore);
         }
       }