diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java index 23c97dc6748c..98514cb83a4c 100644 --- a/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java +++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java @@ -66,7 +66,7 @@ public static void setUp() throws Exception { .createTable(TableDescriptorBuilder.newBuilder(NAME) .setCoprocessor(ZooKeeperScanPolicyObserver.class.getName()) .setValue(ZooKeeperScanPolicyObserver.ZK_ENSEMBLE_KEY, - "localhost:" + UTIL.getZkCluster().getClientPort()) + UTIL.getZkCluster().getAddress().toString()) .setValue(ZooKeeperScanPolicyObserver.ZK_SESSION_TIMEOUT_KEY, "2000") .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()).build()); TABLE = UTIL.getConnection().getTable(NAME); diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java index 4a2c3cd51707..348271905fcd 100644 --- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java +++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.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 @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.apache.zookeeper.KeeperException; import org.junit.Test; @@ -373,7 +374,8 @@ protected void populateQueues() throws ReplicationException { } // Add peers for the corresponding queues so they are not orphans rp.getPeerStorage().addPeer("qId" + i, - ReplicationPeerConfig.newBuilder().setClusterKey("localhost:2818:/bogus" + i).build(), + ReplicationPeerConfig.newBuilder(). + setClusterKey(MiniZooKeeperCluster.HOST + ":2818:/bogus" + i).build(), true, SyncReplicationState.NONE); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index 0d140ae0ec1c..137cb28573a3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -38,6 +38,7 @@ 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.zookeeper.MiniZooKeeperCluster; import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -101,7 +102,7 @@ public void test() throws InterruptedException, ExecutionException, IOException public void testIndependentZKConnections() throws IOException { try (ReadOnlyZKClient zk1 = REGISTRY.getZKClient()) { Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration()); - otherConf.set(HConstants.ZOOKEEPER_QUORUM, "localhost"); + otherConf.set(HConstants.ZOOKEEPER_QUORUM, MiniZooKeeperCluster.HOST); try (ZKConnectionRegistry otherRegistry = new ZKConnectionRegistry(otherConf)) { ReadOnlyZKClient zk2 = otherRegistry.getZKClient(); assertNotSame("Using a different configuration / quorum should result in different " + diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java index e9fcc6634664..0cbb5cf0f417 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java @@ -25,7 +25,6 @@ import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; - import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -78,7 +77,7 @@ private void doTest() throws IOException { Admin admin = UTIL.getAdmin(); String peerId = "1"; ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder() - .setClusterKey("localhost:" + UTIL.getZkCluster().getClientPort() + ":/hbase2").build(); + .setClusterKey(UTIL.getZkCluster().getAddress().toString() + ":/hbase2").build(); admin.addReplicationPeer(peerId, peerConfig, true); assertEquals(peerConfig.getClusterKey(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index 79ca400ac4eb..61c3cfefd83a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.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 @@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; - import java.io.IOException; import java.lang.reflect.Field; import java.net.URLEncoder; @@ -101,10 +100,8 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; - import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor; @@ -502,7 +499,7 @@ public void testPeerRemovalCleanup() throws Exception{ String replicationSourceImplName = conf.get("replication.replicationsource.implementation"); final String peerId = "FakePeer"; final ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder() - .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase").build(); + .setClusterKey(utility.getZkCluster().getAddress().toString() + ":/hbase").build(); try { DummyServer server = new DummyServer(); ReplicationQueueStorage rq = ReplicationStorageFactory @@ -557,7 +554,7 @@ private static long getSizeOfLatestPath() { public void testRemovePeerMetricsCleanup() throws Exception { final String peerId = "DummyPeer"; final ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder() - .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase").build(); + .setClusterKey(utility.getZkCluster().getAddress().toString() + ":/hbase").build(); try { MetricsReplicationSourceSource globalSource = getGlobalSource(); final int globalLogQueueSizeInitial = globalSource.getSizeOfLogQueue(); @@ -700,8 +697,6 @@ private static void waitPeer(final String peerId, /** * Remove a peer and wait for it to get cleaned up - * @param peerId - * @throws Exception */ private void removePeerAndWait(final String peerId) throws Exception { final ReplicationPeers rp = manager.getReplicationPeers(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java index cdc183e0e28c..f8ef9fa739ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.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 @@ -31,7 +31,6 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; - import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -62,7 +61,6 @@ import org.mockito.ArgumentMatcher; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; - import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; @RunWith(MockitoJUnitRunner.class) @@ -104,7 +102,8 @@ public void testBasicZookeeperCanaryWorks() throws Exception { @Test public void testZookeeperCanaryPermittedFailuresArgumentWorks() throws Exception { - final String[] args = { "-t", "10000", "-zookeeper", "-treatFailureAsError", "-permittedZookeeperFailures", "1" }; + final String[] args = { "-t", "10000", "-zookeeper", "-treatFailureAsError", + "-permittedZookeeperFailures", "1" }; testZookeeperCanaryWithArgs(args); } @@ -211,8 +210,10 @@ public void testReadTableTimeouts() throws Exception { assertEquals(0, ToolRunner.run(testingUtility.getConfiguration(), canary, args)); verify(sink, times(tableNames.length)).initializeAndGetReadLatencyForTable(isA(String.class)); for (int i=0; i<2; i++) { - assertNotEquals("verify non-null read latency", null, sink.getReadLatencyMap().get(tableNames[i].getNameAsString())); - assertNotEquals("verify non-zero read latency", 0L, sink.getReadLatencyMap().get(tableNames[i].getNameAsString())); + assertNotEquals("verify non-null read latency", null, + sink.getReadLatencyMap().get(tableNames[i].getNameAsString())); + assertNotEquals("verify non-zero read latency", 0L, + sink.getReadLatencyMap().get(tableNames[i].getNameAsString())); } // One table's timeout is set for 0 ms and thus, should lead to an error. verify(mockAppender, times(1)).doAppend(argThat(new ArgumentMatcher() { @@ -309,7 +310,8 @@ private void runRegionserverCanary() throws Exception { private void testZookeeperCanaryWithArgs(String[] args) throws Exception { Integer port = Iterables.getOnlyElement(testingUtility.getZkCluster().getClientPortList(), null); - testingUtility.getConfiguration().set(HConstants.ZOOKEEPER_QUORUM, "localhost:" + port); + String hostPort = testingUtility.getZkCluster().getAddress().toString(); + testingUtility.getConfiguration().set(HConstants.ZOOKEEPER_QUORUM, hostPort); ExecutorService executor = new ScheduledThreadPoolExecutor(2); CanaryTool.ZookeeperStdOutSink sink = spy(new CanaryTool.ZookeeperStdOutSink()); CanaryTool canary = new CanaryTool(executor, sink); @@ -317,7 +319,6 @@ private void testZookeeperCanaryWithArgs(String[] args) throws Exception { String baseZnode = testingUtility.getConfiguration() .get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT); - verify(sink, atLeastOnce()) - .publishReadTiming(eq(baseZnode), eq("localhost:" + port), anyLong()); + verify(sink, atLeastOnce()).publishReadTiming(eq(baseZnode), eq(hostPort), anyLong()); } } diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb index 3a3c44745178..c669c2a4212b 100644 --- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb +++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb @@ -598,7 +598,7 @@ def assert_tablecfs_equal(table_cfs, table_cfs_map) end define_test "set_peer_bandwidth: works with peer bandwidth upper limit" do - cluster_key = "localhost:2181:/hbase-test" + cluster_key = org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster::HOST + ":2181:/hbase-test" args = { CLUSTER_KEY => cluster_key } command(:add_peer, @peer_id, args) @@ -644,7 +644,7 @@ def assert_tablecfs_equal(table_cfs, table_cfs_map) end define_test "get_peer_config: works with simple clusterKey peer" do - cluster_key = "localhost:2181:/hbase-test" + cluster_key = org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster::HOST + ":2181:/hbase-test" args = { CLUSTER_KEY => cluster_key } command(:add_peer, @peer_id, args) peer_config = command(:get_peer_config, @peer_id) @@ -654,7 +654,7 @@ def assert_tablecfs_equal(table_cfs, table_cfs_map) end define_test "get_peer_config: works with replicationendpointimpl peer and config params" do - cluster_key = 'localhost:2181:/hbase-test' + cluster_key = org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster::HOST + ":2181:/hbase-test" repl_impl = 'org.apache.hadoop.hbase.replication.DummyReplicationEndpoint' config_params = { "config1" => "value1", "config2" => "value2" } args = { CLUSTER_KEY => cluster_key, ENDPOINT_CLASSNAME => repl_impl, @@ -670,7 +670,7 @@ def assert_tablecfs_equal(table_cfs, table_cfs_map) end define_test "list_peer_configs: returns all peers' ReplicationPeerConfig objects" do - cluster_key = "localhost:2181:/hbase-test" + cluster_key = org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster::HOST + ":2181:/hbase-test" args = { CLUSTER_KEY => cluster_key } peer_id_second = '2' command(:add_peer, @peer_id, args) diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java index 4ec079e15a5e..3a8f9d30a555 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java @@ -25,12 +25,14 @@ import java.io.StringWriter; import java.net.BindException; import java.net.ConnectException; +import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.net.Address; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.yetus.audience.InterfaceAudience; @@ -49,12 +51,13 @@ @InterfaceAudience.Public public class MiniZooKeeperCluster { private static final Logger LOG = LoggerFactory.getLogger(MiniZooKeeperCluster.class); - private static final int TICK_TIME = 2000; private static final int TIMEOUT = 1000; private static final int DEFAULT_CONNECTION_TIMEOUT = 30000; private static final byte[] STATIC_BYTES = Bytes.toBytes("stat"); private final int connectionTimeout; + public static final String LOOPBACK_HOST = InetAddress.getLoopbackAddress().getHostName(); + public static final String HOST = LOOPBACK_HOST; private boolean started; @@ -236,7 +239,7 @@ public int startup(File baseDir, int numZooKeeperServers) while (true) { try { standaloneServerFactory = new NIOServerCnxnFactory(); - standaloneServerFactory.configure(new InetSocketAddress(currentClientPort), + standaloneServerFactory.configure(new InetSocketAddress(LOOPBACK_HOST, currentClientPort), configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, HConstants.DEFAULT_ZOOKEEPER_MAX_CLIENT_CNXNS)); } catch (BindException e) { @@ -418,7 +421,7 @@ private static boolean waitForServerDown(int port, long timeout) throws IOExcept long start = System.currentTimeMillis(); while (true) { try { - send4LetterWord("localhost", port, "stat", (int)timeout); + send4LetterWord(HOST, port, "stat", (int)timeout); } catch (IOException e) { return true; } @@ -441,7 +444,7 @@ private static boolean waitForServerUp(int port, long timeout) throws IOExceptio long start = System.currentTimeMillis(); while (true) { try { - String result = send4LetterWord("localhost", port, "stat", (int)timeout); + String result = send4LetterWord(HOST, port, "stat", (int)timeout); if (result.startsWith("Zookeeper version:") && !result.contains("READ-ONLY")) { return true; } else { @@ -449,10 +452,10 @@ private static boolean waitForServerUp(int port, long timeout) throws IOExceptio } } catch (ConnectException e) { // ignore as this is expected, do not log stacktrace - LOG.info("localhost:{} not up: {}", port, e.toString()); + LOG.info("{}:{} not up: {}", HOST, port, e.toString()); } catch (IOException e) { // ignore as this is expected - LOG.info("localhost:{} not up", port, e); + LOG.info("{}:{} not up", HOST, port, e); } if (System.currentTimeMillis() > start + timeout) { @@ -472,6 +475,13 @@ public int getClientPort() { : clientPortList.get(activeZKServerIndex); } + /** + * @return Address for this cluster instance. + */ + public Address getAddress() { + return Address.fromParts(HOST, getClientPort()); + } + List getZooKeeperServers() { return zooKeeperServers; } diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java index b5b95307b3d5..ca661458fe92 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.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,7 +36,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; - import java.io.IOException; import java.util.Collections; import java.util.List; @@ -83,8 +82,9 @@ public class TestReadOnlyZKClient { @BeforeClass public static void setUp() throws Exception { final int port = UTIL.startMiniZKCluster().getClientPort(); + String hostPort = UTIL.getZkCluster().getAddress().toString(); - ZooKeeper zk = ZooKeeperHelper.getConnectedZooKeeper("localhost:" + port, 10000); + ZooKeeper zk = ZooKeeperHelper.getConnectedZooKeeper(hostPort, 10000); DATA = new byte[10]; ThreadLocalRandom.current().nextBytes(DATA); zk.create(PATH, DATA, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); @@ -93,7 +93,7 @@ public static void setUp() throws Exception { } zk.close(); Configuration conf = UTIL.getConfiguration(); - conf.set(HConstants.ZOOKEEPER_QUORUM, "localhost:" + port); + conf.set(HConstants.ZOOKEEPER_QUORUM, hostPort); conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY, 3); conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY_INTERVAL_MILLIS, 100); conf.setInt(ReadOnlyZKClient.KEEPALIVE_MILLIS, 3000); diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java index 264baad277a4..0fda614e6245 100644 --- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java +++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; - import java.security.Permission; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -84,8 +83,8 @@ public void testCommandLineWorks() throws Exception { ZKUtil.checkExists(zkw, znode); boolean exception = false; try { - ZKMainServer.main(new String [] {"-server", - "localhost:" + htu.getZkCluster().getClientPort(), "delete", znode}); + ZKMainServer.main(new String [] {"-server", htu.getZkCluster().getAddress().toString(), + "delete", znode}); } catch (ExitException ee) { // ZKMS calls System.exit which should trigger this exception. exception = true;