You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2018/03/29 20:21:59 UTC

[geode] 03/15: GEODE-4876: Add version of getServerHostName without Host

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

klund pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git

commit ae4b068fa767b7f9f5863c988f939b41b8b11108
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:24:29 2018 -0700

    GEODE-4876: Add version of getServerHostName without Host
    
    Add getServerHostName() to NetworkUtils so that tests can avoid
    importing Host.
---
 .../ClientWithInterestFailoverDistributedTest.java | 12 ++---
 ...isterInterestServerMetaDataDistributedTest.java |  8 +--
 .../SecurityManagerLifecycleDistributedTest.java   | 60 +++++++++++-----------
 .../org/apache/geode/test/dunit/NetworkUtils.java  | 51 ++++++++++++++----
 .../tier/sockets/DurableClientSimpleDUnitTest.java |  2 -
 5 files changed, 80 insertions(+), 53 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientWithInterestFailoverDistributedTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientWithInterestFailoverDistributedTest.java
index 1656f1e..9d365f2 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientWithInterestFailoverDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientWithInterestFailoverDistributedTest.java
@@ -18,9 +18,9 @@ import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.geode.internal.AvailablePort.SOCKET;
 import static org.apache.geode.internal.AvailablePort.getRandomAvailablePort;
 import static org.apache.geode.test.dunit.Disconnect.disconnectAllFromDS;
-import static org.apache.geode.test.dunit.Host.getHost;
 import static org.apache.geode.test.dunit.Invoke.invokeInEveryVM;
 import static org.apache.geode.test.dunit.NetworkUtils.getServerHostName;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.awaitility.Awaitility.await;
 
@@ -79,10 +79,10 @@ public class ClientWithInterestFailoverDistributedTest implements Serializable {
 
   @Before
   public void setUp() throws Exception {
-    client = getHost(0).getVM(0);
+    client = getVM(0);
 
-    server = getHost(0).getVM(1);
-    server2 = getHost(0).getVM(2);
+    server = getVM(1);
+    server2 = getVM(2);
 
     primaryServerPort = givenTwoCacheServers();
   }
@@ -194,8 +194,8 @@ public class ClientWithInterestFailoverDistributedTest implements Serializable {
     serverPort1 = server.invoke(() -> createServerCache());
     serverPort2 = server2.invoke(() -> createServerCache());
 
-    return client.invoke(() -> createClientCacheWithTwoRegions(getServerHostName(server.getHost()),
-        serverPort1, getServerHostName(server2.getHost()), serverPort2));
+    return client.invoke(() -> createClientCacheWithTwoRegions(getServerHostName(), serverPort1,
+        getServerHostName(), serverPort2));
   }
 
   private VM getPrimaryServerVM() {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/RegisterInterestServerMetaDataDistributedTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/RegisterInterestServerMetaDataDistributedTest.java
index b364c03..fac8b5e 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/RegisterInterestServerMetaDataDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/RegisterInterestServerMetaDataDistributedTest.java
@@ -18,9 +18,9 @@ import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.geode.internal.AvailablePort.SOCKET;
 import static org.apache.geode.internal.AvailablePort.getRandomAvailablePort;
 import static org.apache.geode.test.dunit.Disconnect.disconnectAllFromDS;
-import static org.apache.geode.test.dunit.Host.getHost;
 import static org.apache.geode.test.dunit.Invoke.invokeInEveryVM;
 import static org.apache.geode.test.dunit.NetworkUtils.getServerHostName;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.awaitility.Awaitility.await;
 
@@ -76,10 +76,10 @@ public class RegisterInterestServerMetaDataDistributedTest implements Serializab
 
   @Before
   public void setUp() throws Exception {
-    server = getHost(0).getVM(0);
-    client = getHost(0).getVM(1);
+    server = getVM(0);
+    client = getVM(1);
 
-    hostName = getServerHostName(server.getHost());
+    hostName = getServerHostName();
 
     serverPort1 = server.invoke(() -> createServerCache());
     client.invoke(() -> createClientCacheWithTwoRegions(hostName, serverPort1));
diff --git a/geode-core/src/test/java/org/apache/geode/security/SecurityManagerLifecycleDistributedTest.java b/geode-core/src/test/java/org/apache/geode/security/SecurityManagerLifecycleDistributedTest.java
index b0ab1ad..055e9bc 100644
--- a/geode-core/src/test/java/org/apache/geode/security/SecurityManagerLifecycleDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/SecurityManagerLifecycleDistributedTest.java
@@ -25,8 +25,8 @@ import static org.apache.geode.distributed.ConfigurationProperties.USE_CLUSTER_C
 import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.management.ManagementService.getExistingManagementService;
 import static org.apache.geode.test.dunit.DistributedTestUtils.deleteLocatorStateFile;
-import static org.apache.geode.test.dunit.Host.getHost;
 import static org.apache.geode.test.dunit.NetworkUtils.getServerHostName;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.IOException;
@@ -38,12 +38,11 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.server.CacheServer;
-import org.apache.geode.cache30.CacheTestCase;
 import org.apache.geode.internal.security.IntegratedSecurityService;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.management.ManagementService;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.cache.CacheTestCase;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
 
@@ -54,32 +53,31 @@ public class SecurityManagerLifecycleDistributedTest extends CacheTestCase {
   private VM locatorVM;
 
   @Before
-  public void before() throws Exception {
-    Host host = getHost(0);
-    this.locatorVM = host.getVM(0);
+  public void setUp() throws Exception {
+    locatorVM = getVM(0);
 
     int[] ports = getRandomAvailableTCPPorts(2);
     int locatorPort = ports[0];
     int managerPort = ports[1];
 
-    this.locators = getServerHostName(host) + "[" + locatorPort + "]";
+    locators = getServerHostName() + "[" + locatorPort + "]";
 
-    this.locatorVM.invoke(() -> {
+    locatorVM.invoke(() -> {
       deleteLocatorStateFile(locatorPort);
 
-      Properties properties = new Properties();
-      properties.setProperty(LOCATORS, locators);
-      properties.setProperty(MCAST_PORT, "0");
-      properties.setProperty(START_LOCATOR, locators);
-      properties.setProperty(JMX_MANAGER, "true");
-      properties.setProperty(JMX_MANAGER_PORT, String.valueOf(managerPort));
-      properties.setProperty(JMX_MANAGER_START, "true");
-      properties.setProperty(USE_CLUSTER_CONFIGURATION, "false");
-      properties.setProperty(SECURITY_MANAGER, SpySecurityManager.class.getName());
-      properties.setProperty("security-username", "secure");
-      properties.setProperty("security-password", "secure");
-
-      getSystem(properties);
+      Properties config = new Properties();
+      config.setProperty(LOCATORS, locators);
+      config.setProperty(MCAST_PORT, "0");
+      config.setProperty(START_LOCATOR, locators);
+      config.setProperty(JMX_MANAGER, "true");
+      config.setProperty(JMX_MANAGER_PORT, String.valueOf(managerPort));
+      config.setProperty(JMX_MANAGER_START, "true");
+      config.setProperty(USE_CLUSTER_CONFIGURATION, "false");
+      config.setProperty(SECURITY_MANAGER, SpySecurityManager.class.getName());
+      config.setProperty("security-username", "secure");
+      config.setProperty("security-password", "secure");
+
+      getSystem(config);
       getCache();
     });
   }
@@ -95,21 +93,21 @@ public class SecurityManagerLifecycleDistributedTest extends CacheTestCase {
 
     verifyCallbacksRegardlessOfManager(false);
 
-    this.locatorVM.invoke(() -> {
+    locatorVM.invoke(() -> {
       verifyCallbacksRegardlessOfManager(true);
     });
   }
 
   private void connectServer() throws IOException {
-    Properties properties = new Properties();
-    properties.setProperty(LOCATORS, locators);
-    properties.setProperty(MCAST_PORT, "0");
-    properties.setProperty(USE_CLUSTER_CONFIGURATION, "false");
-    properties.setProperty(SECURITY_MANAGER, SpySecurityManager.class.getName());
-    properties.setProperty("security-username", "secure");
-    properties.setProperty("security-password", "secure");
-
-    getSystem(properties);
+    Properties config = new Properties();
+    config.setProperty(LOCATORS, locators);
+    config.setProperty(MCAST_PORT, "0");
+    config.setProperty(USE_CLUSTER_CONFIGURATION, "false");
+    config.setProperty(SECURITY_MANAGER, SpySecurityManager.class.getName());
+    config.setProperty("security-username", "secure");
+    config.setProperty("security-password", "secure");
+
+    getSystem(config);
 
     CacheServer server1 = getCache().addCacheServer();
     server1.setPort(0);
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/NetworkUtils.java b/geode-core/src/test/java/org/apache/geode/test/dunit/NetworkUtils.java
index 244354c..a29c8c5 100755
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/NetworkUtils.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/NetworkUtils.java
@@ -14,20 +14,24 @@
  */
 package org.apache.geode.test.dunit;
 
+import static org.apache.geode.distributed.ConfigurationProperties.SERVER_BIND_ADDRESS;
+import static org.apache.geode.distributed.internal.DistributionConfig.GEMFIRE_PREFIX;
+
+import java.io.UncheckedIOException;
+import java.net.InetAddress;
 import java.net.UnknownHostException;
 
-import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.net.SocketCreator;
 
 /**
- * <code>NetworkUtils</code> provides static utility methods to perform network DNS lookups or
+ * {@code NetworkUtils} provides static utility methods to perform network DNS lookups or
  * similar actions.
  *
- * These methods can be used directly: <code>NetworkUtils.getIPLiteral()</code>, however, they are
+ * These methods can be used directly: {@code NetworkUtils.getIPLiteral()}, however, they are
  * intended to be referenced through static import:
  *
  * <pre>
- * import static org.apache.geode.test.dunit.NetworkUtils.*;
+ * import static org.apache.geode.test.dunit.NetworkUtils.getIPLiteral;
  *    ...
  *    String hostName = getIPLiteral();
  * </pre>
@@ -36,10 +40,12 @@ import org.apache.geode.internal.net.SocketCreator;
  */
 public class NetworkUtils {
 
-  protected NetworkUtils() {}
+  protected NetworkUtils() {
+    // nothing
+  }
 
   /**
-   * Get the IP literal name for the current host. Use this instead of "localhost" to avoid IPv6
+   * Returns the IP literal name for the current host. Use this instead of "localhost" to avoid IPv6
    * name resolution bugs in the JDK/machine config. This method honors java.net.preferIPvAddresses
    *
    * @return an IP literal which honors java.net.preferIPvAddresses
@@ -53,14 +59,39 @@ public class NetworkUtils {
   }
 
   /**
-   * Get the host name to use for a server cache in client/server dunit testing.
+   * Returns the host name to use for a server cache in client/server DUnit testing.
+   *
+   * @return the host name
+   */
+  public static String getServerHostName() {
+    String serverBindAddress = System.getProperty(GEMFIRE_PREFIX + SERVER_BIND_ADDRESS);
+    return serverBindAddress != null ? serverBindAddress : getCanonicalHostName();
+  }
+
+  /**
+   * Returns the host name to use for a server cache in client/server DUnit testing.
    *
-   * @param host the dunit Host to get a machine host name for
+   * @param host the DUnit Host to get a machine host name for
    * @return the host name
+   * @deprecated Please use {@link #getServerHostName()} instead.
    */
+  @Deprecated
   public static String getServerHostName(final Host host) {
-    String serverBindAddress =
-        System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "server-bind-address");
+    String serverBindAddress = System.getProperty(GEMFIRE_PREFIX + SERVER_BIND_ADDRESS);
     return serverBindAddress != null ? serverBindAddress : host.getHostName();
   }
+
+  /**
+   * Returns {@code InetAddress.getLocalHost().getCanonicalHostName()}.
+   *
+   * @return the canonical host name
+   * @throws UncheckedIOException if underlying call threw {@code UnknownHostException}.
+   */
+  private static String getCanonicalHostName() {
+    try {
+      return InetAddress.getLocalHost().getCanonicalHostName();
+    } catch (UnknownHostException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
 }
diff --git a/geode-cq/src/test/java/org/apache/geode/internal/cache/tier/sockets/DurableClientSimpleDUnitTest.java b/geode-cq/src/test/java/org/apache/geode/internal/cache/tier/sockets/DurableClientSimpleDUnitTest.java
index 85c4f6b..ea0394e 100644
--- a/geode-cq/src/test/java/org/apache/geode/internal/cache/tier/sockets/DurableClientSimpleDUnitTest.java
+++ b/geode-cq/src/test/java/org/apache/geode/internal/cache/tier/sockets/DurableClientSimpleDUnitTest.java
@@ -20,7 +20,6 @@ import static org.apache.geode.test.dunit.NetworkUtils.getServerHostName;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.util.Iterator;
 import java.util.List;
@@ -63,7 +62,6 @@ import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.LogWriterUtils;
-import org.apache.geode.test.dunit.NetworkUtils;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.Wait;
 import org.apache.geode.test.dunit.WaitCriterion;

-- 
To stop receiving notification emails like this one, please contact
klund@apache.org.