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:56 UTC

[geode] branch develop updated (b2f7743 -> 7be2a7e)

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

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


    from b2f7743  GEODE-4928 DistributedLockService doesn't work as expected while the dlock grantor is initialized
     new a966ad0  GEODE-4876: Move VM APIs from Host to VM
     new 034a279  GEODE-4876: Remove use of Host from GetPidAndIdDistributedTest
     new ae4b068  GEODE-4876: Add version of getServerHostName without Host
     new e55082a  GEODE-4876: Update RegionClearStatsDistributedTest use of VM and NetworkUtils
     new 841dde9  GEODE-4876: Update RegionExpirationDistributedTest use of VM and NetworkUtil
     new cb6cd2d  GEODE-4876: Update PRCustomPartitioningDistributedTest use of VM and NetworkUtil
     new b069c1f  GEODE-4876: Update RegionListenerDistributedTest use of VM and NetworkUtil
     new 46e98fb  GEODE-4876: Update FunctionExecutionWithExceptionDistributedTest use of VM and NetworkUtil
     new 99bb4f0  GEODE-4876: Update AcceptorImplClientQueueDistributedTest use of VM and NetworkUtil
     new 4296f21  GEODE-4876: Update SharedErrorCollectorDistributedTest use of VM and NetworkUtil
     new 40795fe  GEODE-4876: Update SharedCountersRuleDistributedTest use of VM and NetworkUtil
     new b6b4ac2  GEODE-4876: Update DistributedTestRuleDistributedTest use of VM and NetworkUtil
     new 5271b6b  GEODE-4876: Update DistributedDisconnectRuleDistributedTest use of VM and NetworkUtil
     new da4044e  GEODE-4876: Update DistributedDisconnectRuleAsClassRuleDistributedTest use of VM
     new 7be2a7e  GEODE-4876: Cleanup CacheRule tests use of VM and javadocs

The 15 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:
 .../cli/JdbcClusterConfigDistributedTest.java      |   8 +-
 .../cache/CacheRegionClearStatsDUnitTest.java      | 241 ---------------------
 .../CreateAndLocalDestroyInTXRegressionTest.java   |   8 +-
 .../PRCreationTotalNumBucketsDistributedTest.java  |   6 +-
 .../cache/RegionClearStatsDistributedTest.java     | 179 +++++++++++++++
 ...t.java => RegionExpirationDistributedTest.java} |  10 +-
 .../cache/RegionExpirationIntegrationTest.java     |   2 +-
 .../ReplicateCacheListenerDistributedTest.java     |  29 +--
 ...okenSerializationConsistencyRegressionTest.java |   5 +-
 .../EntriesDoNotExpireDuringGiiRegressionTest.java |   4 +-
 .../cache/MonthBasedPartitionResolver.java         |   2 +-
 ...va => PRCustomPartitioningDistributedTest.java} |  12 +-
 .../PartitionedRegionCloseDistributedTest.java     |  10 +-
 ...est.java => RegionListenerDistributedTest.java} |  21 +-
 ...eplicateEntryIdleExpirationDistributedTest.java |  12 +-
 .../geode/internal/cache/SerializableMonth.java    |   2 +-
 .../ClientFunctionTimeoutRegressionTest.java       |   6 +-
 ...tionExecutionWithExceptionDistributedTest.java} |  10 +-
 .../BucketRebalanceStatRegressionTest.java         |   6 +-
 ...=> AcceptorImplClientQueueDistributedTest.java} |  73 ++++---
 .../ClientProxyWithDeltaDistributedTest.java       |   9 +-
 .../ClientWithInterestFailoverDistributedTest.java |  12 +-
 ...isterInterestServerMetaDataDistributedTest.java |   8 +-
 .../SecurityManagerLifecycleDistributedTest.java   |  60 +++--
 .../java/org/apache/geode/test/dunit/Host.java     |  74 +++----
 .../org/apache/geode/test/dunit/NetworkUtils.java  |  51 ++++-
 .../test/java/org/apache/geode/test/dunit/VM.java  |  85 ++++++--
 .../examples/AsyncInvokeCallableExampleTest.java   |  13 +-
 .../examples/AsyncInvokeRunnableExampleTest.java   |   7 +-
 .../CatchingUnexpectedExceptionExampleTest.java    |   8 +-
 .../examples/DistributedTestRuleExampleTest.java   |   4 +-
 .../dunit/examples/InvokeRunnableExampleTest.java  |  11 +-
 .../examples/LocatorPortClusterExampleTest.java    |  12 +-
 .../apache/geode/test/dunit/rules/CacheRule.java   |   6 +-
 .../test/dunit/rules/DistributedTestRule.java      |  18 +-
 .../geode/test/dunit/rules/SharedCountersRule.java |   4 +
 ...t.java => CacheRuleBuilderDistributedTest.java} |  53 ++---
 ...RuleTest.java => CacheRuleDistributedTest.java} | 171 ++++++++-------
 ...dDisconnectRuleAsClassRuleDistributedTest.java} |  11 +-
 ... DistributedDisconnectRuleDistributedTest.java} |  16 +-
 ...ava => DistributedTestRuleDistributedTest.java} |   6 +-
 ...java => SharedCountersRuleDistributedTest.java} |  21 +-
 ...va => SharedErrorCollectorDistributedTest.java} |  31 +--
 ...st.java => GetCurrentVmNumDistributedTest.java} |  26 +--
 .../GetPidAndIdAfterBounceDistributedTest.java     |  23 +-
 .../dunit/tests/GetPidAndIdDistributedTest.java    |  11 +-
 .../tier/sockets/DurableClientSimpleDUnitTest.java |   2 -
 .../GatewayLegacyAuthenticationRegressionTest.java |  12 +-
 48 files changed, 709 insertions(+), 702 deletions(-)
 delete mode 100755 geode-core/src/test/java/org/apache/geode/cache/CacheRegionClearStatsDUnitTest.java
 create mode 100755 geode-core/src/test/java/org/apache/geode/cache/RegionClearStatsDistributedTest.java
 rename geode-core/src/test/java/org/apache/geode/cache/{RegionExpirationDUnitTest.java => RegionExpirationDistributedTest.java} (96%)
 rename geode-core/src/test/java/org/apache/geode/internal/cache/{PRCustomPartitioningDUnitTest.java => PRCustomPartitioningDistributedTest.java} (96%)
 rename geode-core/src/test/java/org/apache/geode/internal/cache/{RegionListenerDUnitTest.java => RegionListenerDistributedTest.java} (85%)
 rename geode-core/src/test/java/org/apache/geode/internal/cache/execute/{FunctionExecution_ExceptionDUnitTest.java => FunctionExecutionWithExceptionDistributedTest.java} (98%)
 rename geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/{AcceptorImplClientQueueDUnitTest.java => AcceptorImplClientQueueDistributedTest.java} (84%)
 copy geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/{CacheRuleTest.java => CacheRuleBuilderDistributedTest.java} (86%)
 rename geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/{CacheRuleTest.java => CacheRuleDistributedTest.java} (58%)
 rename geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/{DistributedDisconnectRuleAsClassRuleTest.java => DistributedDisconnectRuleAsClassRuleDistributedTest.java} (95%)
 rename geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/{DistributedDisconnectRuleTest.java => DistributedDisconnectRuleDistributedTest.java} (94%)
 rename geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/{DistributedTestRuleTest.java => DistributedTestRuleDistributedTest.java} (89%)
 rename geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/{SharedCountersRuleTest.java => SharedCountersRuleDistributedTest.java} (90%)
 rename geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/{SharedErrorCollectorTest.java => SharedErrorCollectorDistributedTest.java} (92%)
 copy geode-core/src/test/java/org/apache/geode/test/dunit/tests/{GetPidAndIdDistributedTest.java => GetCurrentVmNumDistributedTest.java} (64%)

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

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

Posted by kl...@apache.org.
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.

[geode] 09/15: GEODE-4876: Update AcceptorImplClientQueueDistributedTest use of VM and NetworkUtil

Posted by kl...@apache.org.
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 99bb4f0fc411f529cd219d8f2f2787f541e0f884
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:34:48 2018 -0700

    GEODE-4876: Update AcceptorImplClientQueueDistributedTest use of VM and NetworkUtil
    
    * rename AcceptorImplClientQueueDUnitTest as AcceptorImplClientQueueDistributedTest
---
 ...=> AcceptorImplClientQueueDistributedTest.java} | 73 +++++++++++-----------
 1 file changed, 37 insertions(+), 36 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplClientQueueDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplClientQueueDistributedTest.java
similarity index 84%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplClientQueueDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplClientQueueDistributedTest.java
index 2752971..7aa7f8c 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplClientQueueDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/AcceptorImplClientQueueDistributedTest.java
@@ -14,7 +14,10 @@
  */
 package org.apache.geode.internal.cache.tier.sockets;
 
-import static org.junit.Assert.assertTrue;
+import static org.apache.geode.test.dunit.VM.getAllVMs;
+import static org.apache.geode.test.dunit.VM.getHostName;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.io.IOException;
@@ -44,14 +47,10 @@ import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.cache.server.ClientSubscriptionConfig;
 import org.apache.geode.cache.util.CacheListenerAdapter;
-import org.apache.geode.distributed.DistributedLockBlackboard;
-import org.apache.geode.distributed.DistributedLockBlackboardImpl;
-import org.apache.geode.internal.cache.CacheServerImpl;
 import org.apache.geode.internal.cache.DiskStoreAttributes;
 import org.apache.geode.internal.cache.InitialImageOperation;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.test.dunit.AsyncInvocation;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.CacheRule;
 import org.apache.geode.test.dunit.rules.DistributedRestoreSystemProperties;
@@ -61,45 +60,48 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolde
 import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
 
 @Category(DistributedTest.class)
-public class AcceptorImplClientQueueDUnitTest implements Serializable {
-  private final Host host = Host.getHost(0);
+@SuppressWarnings("serial")
+public class AcceptorImplClientQueueDistributedTest implements Serializable {
+
   private static final int NUMBER_OF_ENTRIES = 200;
 
+  private String hostName;
+
   @ClassRule
   public static DistributedTestRule distributedTestRule = new DistributedTestRule();
 
   @Rule
-  public CacheRule cacheRule =
-      CacheRule.builder().createCacheIn(host.getVM(0)).createCacheIn(host.getVM(1))
-          .addSystemProperty("BridgeServer.HANDSHAKE_POOL_SIZE", "1").build();
+  public CacheRule cacheRule = CacheRule.builder().createCacheIn(getVM(0)).createCacheIn(getVM(1))
+      .addSystemProperty("BridgeServer.HANDSHAKE_POOL_SIZE", "1").build();
 
   @Rule
-  public SerializableTestName name = new SerializableTestName();
+  public DistributedRestoreSystemProperties restoreSystemProperties =
+      new DistributedRestoreSystemProperties();
 
   @Rule
-  public SerializableTemporaryFolder tempDir = new SerializableTemporaryFolder();
+  public SerializableTestName name = new SerializableTestName();
 
   @Rule
-  public DistributedRestoreSystemProperties restoreSystemProperties =
-      new DistributedRestoreSystemProperties();
+  public SerializableTemporaryFolder tempDir = new SerializableTemporaryFolder();
 
   @Before
-  public void setup() throws Exception {}
+  public void setUp() throws Exception {
+    hostName = getHostName();
+  }
 
   @After
   public void tearDown() throws RemoteException {
-    host.getAllVMs().forEach((vm) -> vm.invoke(() -> {
+    getAllVMs().forEach((vm) -> vm.invoke(() -> {
       InitialImageOperation.slowImageProcessing = 0;
-      System.getProperties().remove("BridgeServer.HANDSHAKE_POOL_SIZE");
     }));
   }
 
   @Test
   public void clientSubscriptionQueueInitializationShouldNotBlockNewConnections() throws Exception {
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-    VM vm2 = host.getVM(2);
-    VM vm3 = host.getVM(3);
+    VM vm0 = getVM(0);
+    VM vm1 = getVM(1);
+    VM vm2 = getVM(2);
+    VM vm3 = getVM(3);
 
     // Start one server
     int vm0_port = vm0.invoke("Start server with subscription turned on",
@@ -112,7 +114,7 @@ public class AcceptorImplClientQueueDUnitTest implements Serializable {
       clientCacheFactory.setPoolSubscriptionEnabled(true);
       clientCacheFactory.setPoolSubscriptionRedundancy(1);
       clientCacheFactory.setPoolReadTimeout(200);
-      clientCacheFactory.addPoolServer(host.getHostName(), vm0_port);
+      clientCacheFactory.addPoolServer(hostName, vm0_port);
       ClientCache cache = clientCacheFactory.set("durable-client-id", "1")
           .set("durable-client-timeout", "300").set("mcast-port", "0").create();
       ClientRegionFactory<Object, Object> clientRegionFactory =
@@ -127,7 +129,7 @@ public class AcceptorImplClientQueueDUnitTest implements Serializable {
     // Add some entries Which will end up the in the queue
     vm3.invoke("Start Client2 to add entries to region", () -> {
       ClientCacheFactory clientCacheFactory = new ClientCacheFactory();
-      clientCacheFactory.addPoolServer(host.getHostName(), vm0_port);
+      clientCacheFactory.addPoolServer(hostName, vm0_port);
       ClientCache cache = clientCacheFactory.set("mcast-port", "0").create();
       ClientRegionFactory<Object, Object> clientRegionFactory =
           cache.createClientRegionFactory(ClientRegionShortcut.PROXY);
@@ -151,7 +153,7 @@ public class AcceptorImplClientQueueDUnitTest implements Serializable {
     });
 
     // Make copying the queue slow
-    vm0.invoke("Turn on slow image processsing", () -> {
+    vm0.invoke("Turn on slow image processing", () -> {
       InitialImageOperation.slowImageProcessing = 500;
     });
 
@@ -165,7 +167,7 @@ public class AcceptorImplClientQueueDUnitTest implements Serializable {
           clientCacheFactory.setPoolSubscriptionRedundancy(1);
           clientCacheFactory.setPoolMinConnections(1);
           clientCacheFactory.setPoolMaxConnections(1);
-          clientCacheFactory.addPoolServer(host.getHostName(), vm1_port);
+          clientCacheFactory.addPoolServer(hostName, vm1_port);
           ClientCacheFactory cacheFactory = clientCacheFactory.set("durable-client-id", "1")
               .set("durable-client-timeout", "300").set("mcast-port", "0");
           ClientCache cache = cacheFactory.create();
@@ -194,6 +196,7 @@ public class AcceptorImplClientQueueDUnitTest implements Serializable {
           return eventCount.get() == NUMBER_OF_ENTRIES;
         });
 
+    // TODO: replace sleep with Awaitility
     Thread.sleep(500);
 
     // Start a second client, which should not be blocked by the queue copying
@@ -203,15 +206,14 @@ public class AcceptorImplClientQueueDUnitTest implements Serializable {
       clientCacheFactory.setPoolMinConnections(1);
       clientCacheFactory.setPoolMaxConnections(1);
       clientCacheFactory.setPoolSocketConnectTimeout(5000);
-      clientCacheFactory.addPoolServer(host.getHostName(), vm1_port);
+      clientCacheFactory.addPoolServer(hostName, vm1_port);
       ClientCache cache = clientCacheFactory.set("mcast-port", "0").create();
-      ClientRegionFactory<Object, Object> clientRegionFactory =
+      ClientRegionFactory<Integer, Integer> clientRegionFactory =
           cache.createClientRegionFactory(ClientRegionShortcut.PROXY);
-      Region region = clientRegionFactory.create("subscriptionRegion");
+      Region<Integer, Integer> region = clientRegionFactory.create("subscriptionRegion");
 
-      int returnValue = 0;
       for (int i = 0; i < 100; i++) {
-        returnValue = (int) region.get(i);
+        assertThat(region.get(i)).isGreaterThanOrEqualTo(0);
       }
       cache.close();
     });
@@ -220,11 +222,11 @@ public class AcceptorImplClientQueueDUnitTest implements Serializable {
     turnOffSlowImageProcessing(vm0);
     turnOffSlowImageProcessing(vm1);
 
-    assertTrue(completedClient1.get());
+    assertThat(completedClient1.get()).isTrue();
   }
 
   private void turnOffSlowImageProcessing(VM vm0) {
-    vm0.invoke("Turn off slow image processsing", () -> {
+    vm0.invoke("Turn off slow image processing", () -> {
       InitialImageOperation.slowImageProcessing = 0;
     });
   }
@@ -232,7 +234,7 @@ public class AcceptorImplClientQueueDUnitTest implements Serializable {
   private int createSubscriptionServer(InternalCache cache) throws IOException {
     initializeDiskStore(cache);
     initializeReplicateRegion(cache);
-    return initializeCacheServerWithSubscription(host, cache);
+    return initializeCacheServerWithSubscription(cache);
   }
 
   private void initializeDiskStore(InternalCache cache) throws IOException {
@@ -248,15 +250,14 @@ public class AcceptorImplClientQueueDUnitTest implements Serializable {
         .create("subscriptionRegion");
   }
 
-  private int initializeCacheServerWithSubscription(Host host, InternalCache cache)
-      throws IOException {
+  private int initializeCacheServerWithSubscription(InternalCache cache) throws IOException {
     CacheServer cacheServer1 = cache.addCacheServer(false);
     ClientSubscriptionConfig clientSubscriptionConfig = cacheServer1.getClientSubscriptionConfig();
     clientSubscriptionConfig.setEvictionPolicy("entry");
     clientSubscriptionConfig.setCapacity(5);
     clientSubscriptionConfig.setDiskStoreName("clientQueueDS");
     cacheServer1.setPort(0);
-    cacheServer1.setHostnameForClients(host.getHostName());
+    cacheServer1.setHostnameForClients(hostName);
     cacheServer1.start();
     return cacheServer1.getPort();
   }

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

[geode] 06/15: GEODE-4876: Update PRCustomPartitioningDistributedTest use of VM and NetworkUtil

Posted by kl...@apache.org.
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 cb6cd2d295cb007615d479df9ab02211f72c3304
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:31:15 2018 -0700

    GEODE-4876: Update PRCustomPartitioningDistributedTest use of VM and NetworkUtil
    
    * rename PRCustomPartitioningDUnitTest as PRCustomPartitioningDistributedTest
---
 .../geode/internal/cache/MonthBasedPartitionResolver.java    |  2 +-
 ...nitTest.java => PRCustomPartitioningDistributedTest.java} | 12 ++++++------
 .../org/apache/geode/internal/cache/SerializableMonth.java   |  2 +-
 3 files changed, 8 insertions(+), 8 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/MonthBasedPartitionResolver.java b/geode-core/src/test/java/org/apache/geode/internal/cache/MonthBasedPartitionResolver.java
index f083ef0..3949938 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/MonthBasedPartitionResolver.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/MonthBasedPartitionResolver.java
@@ -28,7 +28,7 @@ import org.apache.geode.internal.cache.xmlcache.Declarable2;
  * partitioning.
  *
  * <p>
- * Extracted from {@link PRCustomPartitioningDUnitTest}.
+ * Extracted from {@link PRCustomPartitioningDistributedTest}.
  */
 class MonthBasedPartitionResolver implements PartitionResolver, Declarable2 {
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/PRCustomPartitioningDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/PRCustomPartitioningDistributedTest.java
similarity index 96%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/PRCustomPartitioningDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/PRCustomPartitioningDistributedTest.java
index 4cd5595..70249a1 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/PRCustomPartitioningDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/PRCustomPartitioningDistributedTest.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
-import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.Serializable;
@@ -44,7 +44,7 @@ import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
 @SuppressWarnings("serial")
-public class PRCustomPartitioningDUnitTest implements Serializable {
+public class PRCustomPartitioningDistributedTest implements Serializable {
 
   private static final int TOTAL_NUM_BUCKETS = 7;
 
@@ -68,10 +68,10 @@ public class PRCustomPartitioningDUnitTest implements Serializable {
 
   @Before
   public void setUp() {
-    datastoreVM0 = getHost(0).getVM(0);
-    datastoreVM1 = getHost(0).getVM(1);
-    datastoreVM2 = getHost(0).getVM(2);
-    accessorVM3 = getHost(0).getVM(3);
+    datastoreVM0 = getVM(0);
+    datastoreVM1 = getVM(1);
+    datastoreVM2 = getVM(2);
+    accessorVM3 = getVM(3);
 
     regionName = "PR1";
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/SerializableMonth.java b/geode-core/src/test/java/org/apache/geode/internal/cache/SerializableMonth.java
index 638c521..2670593 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/SerializableMonth.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/SerializableMonth.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import org.apache.geode.DataSerializable;
 
 /**
- * Extracted from {@link PRCustomPartitioningDUnitTest}.
+ * Extracted from {@link PRCustomPartitioningDistributedTest}.
  */
 @SuppressWarnings("serial")
 class SerializableMonth implements DataSerializable {

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

[geode] 01/15: GEODE-4876: Move VM APIs from Host to VM

Posted by kl...@apache.org.
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 a966ad00574844ae1c106051efcb5fd2b5f05495
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 16:29:59 2018 -0700

    GEODE-4876: Move VM APIs from Host to VM
    
    Deprecate Host.
---
 .../cli/JdbcClusterConfigDistributedTest.java      |  8 +-
 .../CreateAndLocalDestroyInTXRegressionTest.java   |  8 +-
 .../PRCreationTotalNumBucketsDistributedTest.java  |  6 +-
 .../ReplicateCacheListenerDistributedTest.java     | 29 ++++----
 ...okenSerializationConsistencyRegressionTest.java |  5 +-
 .../EntriesDoNotExpireDuringGiiRegressionTest.java |  4 +-
 .../PartitionedRegionCloseDistributedTest.java     | 10 +--
 ...eplicateEntryIdleExpirationDistributedTest.java | 12 +--
 .../ClientFunctionTimeoutRegressionTest.java       |  6 +-
 .../BucketRebalanceStatRegressionTest.java         |  6 +-
 .../ClientProxyWithDeltaDistributedTest.java       |  9 ++-
 .../java/org/apache/geode/test/dunit/Host.java     | 74 +++++++------------
 .../test/java/org/apache/geode/test/dunit/VM.java  | 85 +++++++++++++++++-----
 .../examples/AsyncInvokeCallableExampleTest.java   | 13 ++--
 .../examples/AsyncInvokeRunnableExampleTest.java   |  7 +-
 .../CatchingUnexpectedExceptionExampleTest.java    |  8 +-
 .../examples/DistributedTestRuleExampleTest.java   |  4 +-
 .../dunit/examples/InvokeRunnableExampleTest.java  | 11 ++-
 .../examples/LocatorPortClusterExampleTest.java    | 12 +--
 .../test/dunit/rules/DistributedTestRule.java      | 18 +++--
 .../geode/test/dunit/rules/SharedCountersRule.java |  4 +
 .../GetCurrentVmNumDistributedTest.java}           | 22 ++++--
 .../GetPidAndIdAfterBounceDistributedTest.java     | 23 +++---
 .../GatewayLegacyAuthenticationRegressionTest.java | 12 +--
 24 files changed, 222 insertions(+), 174 deletions(-)

diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/JdbcClusterConfigDistributedTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/JdbcClusterConfigDistributedTest.java
index 112ce49..d65e00e 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/JdbcClusterConfigDistributedTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/cli/JdbcClusterConfigDistributedTest.java
@@ -17,6 +17,8 @@ package org.apache.geode.connectors.jdbc.internal.cli;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.test.dunit.Disconnect.disconnectAllFromDS;
+import static org.apache.geode.test.dunit.VM.getHostName;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.awaitility.Awaitility.await;
 import static org.junit.Assert.assertTrue;
@@ -40,7 +42,6 @@ import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.cli.Result;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedRestoreSystemProperties;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
@@ -49,6 +50,7 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolde
 import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
 
 @Category(DistributedTest.class)
+@SuppressWarnings("serial")
 public class JdbcClusterConfigDistributedTest implements Serializable {
 
   private transient InternalCache cache;
@@ -86,7 +88,7 @@ public class JdbcClusterConfigDistributedTest implements Serializable {
     keyInValue = true;
     fieldMappings = new String[] {"field1:column1", "field2:column2"};
 
-    locator = Host.getHost(0).getVM(0);
+    locator = getVM(0);
     String locatorFolder = "vm-" + locator.getId() + "-" + testName.getMethodName();
 
     int port = locator.invoke(() -> {
@@ -97,7 +99,7 @@ public class JdbcClusterConfigDistributedTest implements Serializable {
       await().atMost(2, MINUTES).until(() -> assertTrue(locator.isSharedConfigurationRunning()));
       return Locator.getLocator().getPort();
     });
-    locators = Host.getHost(0).getHostName() + "[" + port + "]";
+    locators = getHostName() + "[" + port + "]";
 
     cache = (InternalCache) new CacheFactory().set(LOCATORS, locators).create();
 
diff --git a/geode-core/src/test/java/org/apache/geode/cache/CreateAndLocalDestroyInTXRegressionTest.java b/geode-core/src/test/java/org/apache/geode/cache/CreateAndLocalDestroyInTXRegressionTest.java
index e0cd355..78f81c5 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/CreateAndLocalDestroyInTXRegressionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/CreateAndLocalDestroyInTXRegressionTest.java
@@ -18,7 +18,7 @@ import static org.apache.geode.cache.RegionShortcut.REPLICATE;
 import static org.apache.geode.cache.RegionShortcut.REPLICATE_PROXY;
 import static org.apache.geode.internal.i18n.LocalizedStrings.TXStateStub_LOCAL_DESTROY_NOT_ALLOWED_IN_TRANSACTION;
 import static org.apache.geode.internal.i18n.LocalizedStrings.TXStateStub_LOCAL_INVALIDATE_NOT_ALLOWED_IN_TRANSACTION;
-import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.mockito.Mockito.mock;
@@ -45,7 +45,6 @@ import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.CacheRule;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.rules.serializable.SerializableErrorCollector;
 
 /**
  * Create and LocalDestroy/LocalInvalidate should create event with NewValue
@@ -75,12 +74,9 @@ public class CreateAndLocalDestroyInTXRegressionTest implements Serializable {
   @Rule
   public CacheRule cacheRule = new CacheRule();
 
-  @Rule
-  public SerializableErrorCollector errorCollector = new SerializableErrorCollector();
-
   @Before
   public void setUp() throws Exception {
-    otherVM = getHost(0).getVM(0);
+    otherVM = getVM(0);
     spyCacheListener = mock(CacheListener.class);
 
     otherVM.invoke(() -> {
diff --git a/geode-core/src/test/java/org/apache/geode/cache/PRCreationTotalNumBucketsDistributedTest.java b/geode-core/src/test/java/org/apache/geode/cache/PRCreationTotalNumBucketsDistributedTest.java
index f73b8f2..22efcb6 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/PRCreationTotalNumBucketsDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/PRCreationTotalNumBucketsDistributedTest.java
@@ -17,8 +17,8 @@ package org.apache.geode.cache;
 import static org.apache.geode.cache.PartitionAttributesFactory.GLOBAL_MAX_BUCKETS_DEFAULT;
 import static org.apache.geode.cache.PartitionAttributesFactory.GLOBAL_MAX_BUCKETS_PROPERTY;
 import static org.apache.geode.cache.RegionShortcut.PARTITION;
-import static org.apache.geode.test.dunit.Host.getHost;
 import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
@@ -54,8 +54,8 @@ public class PRCreationTotalNumBucketsDistributedTest implements Serializable {
 
   @Before
   public void setUp() throws Exception {
-    vm0 = getHost(0).getVM(0);
-    vm1 = getHost(0).getVM(1);
+    vm0 = getVM(0);
+    vm1 = getVM(1);
 
     totalNumBuckets = 7;
   }
diff --git a/geode-core/src/test/java/org/apache/geode/cache/ReplicateCacheListenerDistributedTest.java b/geode-core/src/test/java/org/apache/geode/cache/ReplicateCacheListenerDistributedTest.java
index 6f5268f..c9cb01d 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/ReplicateCacheListenerDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/ReplicateCacheListenerDistributedTest.java
@@ -14,7 +14,8 @@
  */
 package org.apache.geode.cache;
 
-import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.fail;
 import static org.hamcrest.Matchers.anyOf;
@@ -72,7 +73,7 @@ public class ReplicateCacheListenerDistributedTest implements Serializable {
   public SerializableTestName testName = new SerializableTestName();
 
   @Rule
-  public SharedCountersRule sharedCountersRule = SharedCountersRule.builder().build();
+  public SharedCountersRule sharedCountersRule = new SharedCountersRule();
 
   @Rule
   public SharedErrorCollector errorCollector = new SharedErrorCollector();
@@ -91,8 +92,8 @@ public class ReplicateCacheListenerDistributedTest implements Serializable {
   public void afterCreateIsInvokedInEveryMember() throws Exception {
     CacheListener<String, Integer> listener = new CreateCountingCacheListener();
     Region<String, Integer> region = createRegion(regionName, listener);
-    for (int i = 0; i < getHost(0).getVMCount(); i++) {
-      getHost(0).getVM(i).invoke(() -> {
+    for (int i = 0; i < getVMCount(); i++) {
+      getVM(i).invoke(() -> {
         createRegion(regionName, listener);
       });
     }
@@ -106,8 +107,8 @@ public class ReplicateCacheListenerDistributedTest implements Serializable {
   public void afterUpdateIsInvokedInEveryMember() throws Exception {
     CacheListener<String, Integer> listener = new UpdateCountingCacheListener();
     Region<String, Integer> region = createRegion(regionName, listener);
-    for (int i = 0; i < getHost(0).getVMCount(); i++) {
-      getHost(0).getVM(i).invoke(() -> {
+    for (int i = 0; i < getVMCount(); i++) {
+      getVM(i).invoke(() -> {
         createRegion(regionName, listener);
       });
     }
@@ -122,8 +123,8 @@ public class ReplicateCacheListenerDistributedTest implements Serializable {
   public void afterInvalidateIsInvokedInEveryMember() throws Exception {
     CacheListener<String, Integer> listener = new InvalidateCountingCacheListener();
     Region<String, Integer> region = createRegion(regionName, listener);
-    for (int i = 0; i < getHost(0).getVMCount(); i++) {
-      getHost(0).getVM(i).invoke(() -> {
+    for (int i = 0; i < getVMCount(); i++) {
+      getVM(i).invoke(() -> {
         createRegion(regionName, listener);
       });
     }
@@ -139,8 +140,8 @@ public class ReplicateCacheListenerDistributedTest implements Serializable {
   public void afterDestroyIsInvokedInEveryMember() throws Exception {
     CacheListener<String, Integer> listener = new DestroyCountingCacheListener();
     Region<String, Integer> region = createRegion(regionName, listener);
-    for (int i = 0; i < getHost(0).getVMCount(); i++) {
-      getHost(0).getVM(i).invoke(() -> {
+    for (int i = 0; i < getVMCount(); i++) {
+      getVM(i).invoke(() -> {
         createRegion(regionName, listener);
       });
     }
@@ -162,19 +163,19 @@ public class ReplicateCacheListenerDistributedTest implements Serializable {
   }
 
   protected int expectedCreates() {
-    return getHost(0).getVMCount() + 1;
+    return getVMCount() + 1;
   }
 
   protected int expectedUpdates() {
-    return getHost(0).getVMCount() + 1;
+    return getVMCount() + 1;
   }
 
   protected int expectedInvalidates() {
-    return getHost(0).getVMCount() + 1;
+    return getVMCount() + 1;
   }
 
   protected int expectedDestroys() {
-    return getHost(0).getVMCount() + 1;
+    return getVMCount() + 1;
   }
 
   /**
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/BrokenSerializationConsistencyRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/BrokenSerializationConsistencyRegressionTest.java
index c4b6c61..27ec286 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/BrokenSerializationConsistencyRegressionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/BrokenSerializationConsistencyRegressionTest.java
@@ -19,7 +19,7 @@ import static com.googlecode.catchexception.CatchException.caughtException;
 import static org.apache.geode.cache.RegionShortcut.REPLICATE;
 import static org.apache.geode.internal.lang.SystemPropertyHelper.EARLY_ENTRY_EVENT_SERIALIZATION;
 import static org.apache.geode.internal.lang.SystemPropertyHelper.GEODE_PREFIX;
-import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.DataInput;
@@ -47,6 +47,7 @@ import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
+@SuppressWarnings("serial")
 public class BrokenSerializationConsistencyRegressionTest implements Serializable {
 
   private static final String REGION_NAME = "replicateRegion";
@@ -69,7 +70,7 @@ public class BrokenSerializationConsistencyRegressionTest implements Serializabl
 
   @Before
   public void setUpAll() {
-    vm0 = getHost(0).getVM(0);
+    vm0 = getVM(0);
 
     System.setProperty(GEODE_PREFIX + EARLY_ENTRY_EVENT_SERIALIZATION, "true");
     createReplicateRegions();
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/EntriesDoNotExpireDuringGiiRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/EntriesDoNotExpireDuringGiiRegressionTest.java
index 7784b03..6853125 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/EntriesDoNotExpireDuringGiiRegressionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/EntriesDoNotExpireDuringGiiRegressionTest.java
@@ -17,7 +17,7 @@ package org.apache.geode.internal.cache;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.geode.cache.ExpirationAction.INVALIDATE;
 import static org.apache.geode.cache.RegionShortcut.REPLICATE;
-import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.awaitility.Awaitility.await;
 import static org.hamcrest.core.Is.is;
@@ -82,7 +82,7 @@ public class EntriesDoNotExpireDuringGiiRegressionTest implements Serializable {
 
   @Before
   public void setUp() throws Exception {
-    otherVM = getHost(0).getVM(0);
+    otherVM = getVM(0);
 
     otherVM.invoke(() -> {
       RegionFactory<String, String> regionFactory =
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionCloseDistributedTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionCloseDistributedTest.java
index fe5ebf9..db75f05 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionCloseDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionCloseDistributedTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.Serializable;
@@ -34,7 +35,6 @@ import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.internal.cache.partitioned.RegionAdvisor;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.CacheRule;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
@@ -64,11 +64,11 @@ public class PartitionedRegionCloseDistributedTest implements Serializable {
   public void setUp() throws Exception {
     regionName = getClass().getSimpleName();
 
-    accessor = Host.getHost(0).getVM(0);
+    accessor = getVM(0);
     datastores = new VM[3];
-    datastores[0] = Host.getHost(0).getVM(1);
-    datastores[1] = Host.getHost(0).getVM(2);
-    datastores[2] = Host.getHost(0).getVM(3);
+    datastores[0] = getVM(1);
+    datastores[1] = getVM(2);
+    datastores[2] = getVM(3);
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/ReplicateEntryIdleExpirationDistributedTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/ReplicateEntryIdleExpirationDistributedTest.java
index afac1a3..6d2ab11 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/ReplicateEntryIdleExpirationDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/ReplicateEntryIdleExpirationDistributedTest.java
@@ -17,7 +17,7 @@ package org.apache.geode.internal.cache;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static org.apache.geode.cache.ExpirationAction.DESTROY;
 import static org.apache.geode.cache.RegionShortcut.REPLICATE;
-import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.awaitility.Awaitility.await;
 
@@ -55,14 +55,14 @@ public class ReplicateEntryIdleExpirationDistributedTest implements Serializable
   private static final String KEY = "KEY";
   private static final String VALUE = "VALUE";
 
-  private final VM member1 = getHost(0).getVM(0);
-  private final VM member2 = getHost(0).getVM(1);
-  private final VM member3 = getHost(0).getVM(2);
+  private final VM member1 = getVM(0);
+  private final VM member2 = getVM(1);
+  private final VM member3 = getVM(2);
   private final String regionName = getClass().getSimpleName();
 
   @Rule
   public CacheRule cacheRule = CacheRule.builder().createCacheIn(member1).createCacheIn(member2)
-      .createCacheIn(member3).createCacheIn(getHost(0).getVM(3)).build();
+      .createCacheIn(member3).createCacheIn(getVM(3)).build();
 
   @Before
   public void setUp() throws Exception {
@@ -124,7 +124,7 @@ public class ReplicateEntryIdleExpirationDistributedTest implements Serializable
 
   @Test
   public void readsInNormalMemberShouldPreventExpiration() throws Exception {
-    VM member4 = getHost(0).getVM(3);
+    VM member4 = getVM(3);
     member4.invoke(() -> {
       KEEP_READING.set(true);
       ExpiryTask.suspendExpiration();
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/ClientFunctionTimeoutRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/ClientFunctionTimeoutRegressionTest.java
index d709ebf..a3135b2 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/ClientFunctionTimeoutRegressionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/ClientFunctionTimeoutRegressionTest.java
@@ -19,8 +19,8 @@ import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
 import static org.apache.geode.distributed.internal.DistributionConfig.GEMFIRE_PREFIX;
 import static org.apache.geode.test.dunit.DistributedTestUtils.getLocatorPort;
-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.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.IOException;
@@ -95,8 +95,8 @@ public class ClientFunctionTimeoutRegressionTest implements Serializable {
 
   @Before
   public void before() throws Exception {
-    server = getHost(0).getVM(0);
-    client = getHost(0).getVM(1);
+    server = getVM(0);
+    client = getVM(1);
   }
 
   @After
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/BucketRebalanceStatRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/BucketRebalanceStatRegressionTest.java
index 7075d98..0e6e44a 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/BucketRebalanceStatRegressionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/BucketRebalanceStatRegressionTest.java
@@ -17,7 +17,7 @@ package org.apache.geode.internal.cache.partitioned;
 import static org.apache.geode.cache.EvictionAction.OVERFLOW_TO_DISK;
 import static org.apache.geode.cache.EvictionAttributes.createLRUEntryAttributes;
 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.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
@@ -88,8 +88,8 @@ public class BucketRebalanceStatRegressionTest implements Serializable {
 
   @Before
   public void setUp() throws Exception {
-    vm0 = getHost(0).getVM(0);
-    vm1 = getHost(0).getVM(1);
+    vm0 = getVM(0);
+    vm1 = getVM(1);
   }
 
   @After
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java
index 7837b85..1136b2f 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyWithDeltaDistributedTest.java
@@ -18,9 +18,10 @@ 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.getHostName;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.awaitility.Awaitility.await;
 
@@ -83,9 +84,9 @@ public class ClientProxyWithDeltaDistributedTest implements Serializable {
 
   @Before
   public void setUp() throws Exception {
-    server = getHost(0).getVM(0);
-    client1 = getHost(0).getVM(1);
-    client2 = getHost(0).getVM(3);
+    server = getVM(0);
+    client1 = getVM(1);
+    client2 = getVM(3);
 
     hostName = getServerHostName(server.getHost());
 
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/Host.java b/geode-core/src/test/java/org/apache/geode/test/dunit/Host.java
index 74793e7..e3e93c9 100755
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/Host.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/Host.java
@@ -16,7 +16,6 @@ package org.apache.geode.test.dunit;
 
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 
 import org.apache.geode.test.dunit.standalone.RemoteDUnitVMIF;
@@ -30,29 +29,23 @@ import org.apache.geode.test.dunit.standalone.VersionManager;
  * Additionally, it provides access to the Java RMI registry that runs on the host. By default, an
  * RMI registry is only started on the host on which Hydra's Master VM runs. RMI registries may be
  * started on other hosts via additional Hydra configuration.
+ *
+ * @deprecated Please use similar static APIs on {@link VM} instead.
  */
+@Deprecated
 @SuppressWarnings("serial")
 public abstract class Host implements Serializable {
 
   /** The available hosts */
-  protected static List hosts = new ArrayList();
+  private static final List<Host> hosts = new ArrayList<>();
 
   private static VM locator;
 
-  /** Indicates an unstarted RMI registry */
-  protected static int NO_REGISTRY = -1;
-
   /** The name of this host machine */
-  private String hostName;
+  private final String hostName;
 
   /** The VMs that run on this host */
-  private List<VM> vms;
-
-  /** The GemFire systems that are available on this host */
-  private List systems;
-
-  /** Key is system name, value is GemFireSystem instance */
-  private HashMap systemNames;
+  private final List<VM> vms;
 
   /**
    * Returns the number of known hosts
@@ -71,18 +64,18 @@ public abstract class Host implements Serializable {
   /**
    * Returns a given host
    *
-   * @param n A zero-based identifier of the host
+   * @param whichHost A zero-based identifier of the host
    *
    * @throws IllegalArgumentException {@code n} is more than the number of hosts
    */
-  public static Host getHost(int n) {
+  public static Host getHost(int whichHost) {
     int size = hosts.size();
-    if (n >= size) {
-      String s = "Cannot request host " + n + ".  There are only " + size + " hosts.";
-      throw new IllegalArgumentException(s);
+    if (whichHost >= size) {
+      String message = "Cannot request host " + whichHost + ".  There are only " + size + " hosts.";
+      throw new IllegalArgumentException(message);
 
     } else {
-      return (Host) hosts.get(n);
+      return hosts.get(whichHost);
     }
   }
 
@@ -103,40 +96,33 @@ public abstract class Host implements Serializable {
         }
       }
     }
-
   }
 
-  ///////////////////// Constructors //////////////////////
-
   /**
    * Creates a new {@code Host} with the given name
    */
   protected Host(String hostName) {
     if (hostName == null) {
-      String s = "Cannot create a Host with a null name";
-      throw new NullPointerException(s);
+      String message = "Cannot create a Host with a null name";
+      throw new NullPointerException(message);
     }
 
     this.hostName = hostName;
-    this.vms = new ArrayList();
-    this.systems = new ArrayList();
-    this.systemNames = new HashMap();
+    vms = new ArrayList<>();
   }
 
-  //////////////////// Instance Methods ////////////////////
-
   /**
    * Returns the machine name of this host
    */
   public String getHostName() {
-    return this.hostName;
+    return hostName;
   }
 
   /**
    * Returns the number of VMs that run on this host
    */
   public int getVMCount() {
-    return this.vms.size();
+    return vms.size();
   }
 
   /**
@@ -153,7 +139,7 @@ public abstract class Host implements Serializable {
       throw new IllegalArgumentException(s);
 
     } else {
-      VM vm = (VM) vms.get(n);
+      VM vm = vms.get(n);
       vm.makeAvailable();
       return vm;
     }
@@ -179,7 +165,7 @@ public abstract class Host implements Serializable {
    */
   protected void addVM(int vmid, RemoteDUnitVMIF client) {
     VM vm = new VM(this, vmid, client);
-    this.vms.add(vm);
+    vms.add(vm);
   }
 
   public static VM getLocator() {
@@ -194,38 +180,28 @@ public abstract class Host implements Serializable {
     setLocator(new VM(this, vmid, client));
   }
 
-  /**
-   * Returns the number of GemFire systems that run on this host
-   */
-  public int getSystemCount() {
-    return this.systems.size();
-  }
-
+  @Override
   public String toString() {
     StringBuilder sb = new StringBuilder("Host ");
-    sb.append(this.getHostName());
+    sb.append(getHostName());
     sb.append(" with ");
     sb.append(getVMCount());
     sb.append(" VMs");
     return sb.toString();
   }
 
-  /**
-   * Two {@code Host}s are considered equal if they have the same name.
-   */
+  @Override
   public boolean equals(Object o) {
     if (o instanceof Host) {
-      return ((Host) o).getHostName().equals(this.getHostName());
+      return ((Host) o).getHostName().equals(getHostName());
 
     } else {
       return false;
     }
   }
 
-  /**
-   * A {@code Host}'s hash code is based on the hash code of its name.
-   */
+  @Override
   public int hashCode() {
-    return this.getHostName().hashCode();
+    return getHostName().hashCode();
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/VM.java b/geode-core/src/test/java/org/apache/geode/test/dunit/VM.java
index 22a99c0..d25c2f3 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/VM.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/VM.java
@@ -14,11 +14,14 @@
  */
 package org.apache.geode.test.dunit;
 
+import static org.apache.geode.test.dunit.standalone.DUnitLauncher.NUM_VMS;
+
 import java.io.File;
 import java.io.PrintWriter;
 import java.io.Serializable;
 import java.io.StringWriter;
 import java.rmi.RemoteException;
+import java.util.List;
 import java.util.concurrent.Callable;
 
 import hydra.MethExecutorResult;
@@ -31,11 +34,15 @@ import org.apache.geode.test.dunit.standalone.StandAloneDUnitEnv;
 import org.apache.geode.test.dunit.standalone.VersionManager;
 
 /**
- * This class represents a Java Virtual Machine that runs on a host.
+ * This class represents a Java Virtual Machine that runs in a DistributedTest.
  */
-@SuppressWarnings("serial")
+@SuppressWarnings("serial,unused")
 public class VM implements Serializable {
 
+  public static final int CONTROLLER_VM = -1;
+
+  public static final int DEFAULT_VM_COUNT = NUM_VMS;
+
   /** The host on which this VM runs */
   private final Host host;
 
@@ -60,26 +67,54 @@ public class VM implements Serializable {
   }
 
   /**
-   * restart an unavailable VM
+   * Returns true if executed from the main JUnit VM.
    */
-  public synchronized void makeAvailable() {
-    if (!available) {
-      available = true;
-      bounce();
-    }
+  public static boolean isControllerVM() {
+    return getCurrentVMNum() == CONTROLLER_VM;
+  }
+
+  /**
+   * Returns true if executed from a DUnit VM. Returns false if executed from the main JUnit VM.
+   */
+  public static boolean isVM() {
+    return getCurrentVMNum() != CONTROLLER_VM;
   }
 
   /**
-   * Returns the total number of {@code VM}s on all {@code Host}s (note that DUnit currently only
-   * supports one {@code Host}).
+   * Returns a VM that runs in this DistributedTest.
+   *
+   * @param whichVM A zero-based identifier of the VM
+   */
+  public static VM getVM(int whichVM) {
+    return Host.getHost(0).getVM(whichVM);
+  }
+
+  /**
+   * Returns a collection of all DistributedTest VMs.
+   */
+  public static List<VM> getAllVMs() {
+    return Host.getHost(0).getAllVMs();
+  }
+
+  /**
+   * Returns the number of VMs that run in this DistributedTest.
    */
   public static int getVMCount() {
-    int count = 0;
-    for (int h = 0; h < Host.getHostCount(); h++) {
-      Host host = Host.getHost(h);
-      count += host.getVMCount();
-    }
-    return count;
+    return Host.getHost(0).getVMCount();
+  }
+
+  /**
+   * Returns the DistributedTest Locator VM.
+   */
+  public static VM getLocator() {
+    return Host.getLocator();
+  }
+
+  /**
+   * Returns the machine name hosting this DistributedTest.
+   */
+  public static String getHostName() {
+    return Host.getHost(0).getHostName();
   }
 
   /**
@@ -403,6 +438,16 @@ public class VM implements Serializable {
   }
 
   /**
+   * Restart an unavailable VM
+   */
+  public synchronized void makeAvailable() {
+    if (!available) {
+      available = true;
+      bounce();
+    }
+  }
+
+  /**
    * Synchronously bounces (mean kills and restarts) this {@code VM}. Concurrent bounce attempts are
    * synchronized but attempts to invoke methods on a bouncing {@code VM} will cause test failure.
    * Tests using bounce should be placed at the end of the DUnit test suite, since an exception here
@@ -442,16 +487,16 @@ public class VM implements Serializable {
     }
   }
 
+  public File getWorkingDirectory() {
+    return DUnitEnv.get().getWorkingDirectory(getVersion(), getId());
+  }
+
   @Override
   public String toString() {
     return "VM " + getId() + " running on " + getHost()
         + (VersionManager.isCurrentVersion(version) ? "" : (" with version " + version));
   }
 
-  public File getWorkingDirectory() {
-    return DUnitEnv.get().getWorkingDirectory(getVersion(), getId());
-  }
-
   private MethExecutorResult execute(final Class<?> targetClass, final String methodName,
       final Object[] args) {
     try {
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/AsyncInvokeCallableExampleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/AsyncInvokeCallableExampleTest.java
index 30cee5d..8879126 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/AsyncInvokeCallableExampleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/AsyncInvokeCallableExampleTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.test.dunit.examples;
 
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.util.concurrent.TimeUnit;
@@ -23,7 +24,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.test.dunit.AsyncInvocation;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
@@ -37,14 +37,14 @@ public class AsyncInvokeCallableExampleTest {
 
   @Test
   public void invokeAsyncAsFuture() throws Exception {
-    VM workerVM = Host.getHost(0).getVM(0);
+    VM workerVM = getVM(0);
     boolean success = workerVM.invokeAsync(() -> longRunningWorkWithResult()).get();
     assertThat(success).isTrue();
   }
 
   @Test
   public void invokeAsyncAsFutureWithTimeout() throws Exception {
-    VM workerVM = Host.getHost(0).getVM(0);
+    VM workerVM = getVM(0);
     boolean success =
         workerVM.invokeAsync(() -> longRunningWorkWithResult()).get(1, TimeUnit.MINUTES);
     assertThat(success).isTrue();
@@ -52,7 +52,7 @@ public class AsyncInvokeCallableExampleTest {
 
   @Test
   public void invokeAsyncWithExceptionOccurred() throws Exception {
-    VM workerVM = Host.getHost(0).getVM(0);
+    VM workerVM = getVM(0);
 
     AsyncInvocation<Boolean> asyncInvocation =
         workerVM.invokeAsync(() -> longRunningWorkThatThrowsException());
@@ -68,7 +68,7 @@ public class AsyncInvokeCallableExampleTest {
    */
   @Test(expected = AssertionError.class)
   public void invokeAsyncWithAwait() throws Exception {
-    Host.getHost(0).getVM(0).invokeAsync(() -> longRunningWorkThatThrowsException()).await();
+    getVM(0).invokeAsync(() -> longRunningWorkThatThrowsException()).await();
   }
 
   /**
@@ -77,8 +77,7 @@ public class AsyncInvokeCallableExampleTest {
    */
   @Test(expected = AssertionError.class)
   public void invokeAsyncWithAwaitWithTimeout() throws Exception {
-    Host.getHost(0).getVM(0).invokeAsync(() -> longRunningWorkThatThrowsException()).await(1,
-        TimeUnit.MINUTES);
+    getVM(0).invokeAsync(() -> longRunningWorkThatThrowsException()).await(1, TimeUnit.MINUTES);
   }
 
   private static boolean longRunningWorkWithResult() {
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/AsyncInvokeRunnableExampleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/AsyncInvokeRunnableExampleTest.java
index 7a2be5e..ebcc69a 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/AsyncInvokeRunnableExampleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/AsyncInvokeRunnableExampleTest.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.test.dunit.examples;
 
+import static org.apache.geode.test.dunit.VM.getAllVMs;
+
 import java.util.ArrayList;
 import java.util.List;
 
@@ -22,7 +24,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.test.dunit.AsyncInvocation;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
@@ -36,7 +37,7 @@ public class AsyncInvokeRunnableExampleTest {
 
   @Test
   public void invokeAsyncHelloWorldInEachVM() throws Exception {
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invokeAsync(() -> System.out.println(vm + " says Hello World!"));
     }
   }
@@ -44,7 +45,7 @@ public class AsyncInvokeRunnableExampleTest {
   @Test
   public void invokeAsyncHelloWorldInEachVMWithAwait() throws Exception {
     List<AsyncInvocation> invocations = new ArrayList<>();
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       AsyncInvocation invocation =
           vm.invokeAsync(() -> System.out.println(vm + " says Hello World!"));
       invocations.add(invocation);
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/CatchingUnexpectedExceptionExampleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/CatchingUnexpectedExceptionExampleTest.java
index da2b71b..e170dab 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/CatchingUnexpectedExceptionExampleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/CatchingUnexpectedExceptionExampleTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode.test.dunit.examples;
 
 import static org.apache.geode.test.dunit.Assert.fail;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.Serializable;
@@ -30,7 +31,6 @@ import org.apache.geode.cache.CacheException;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.rules.DistributedDisconnectRule;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
@@ -52,7 +52,7 @@ public class CatchingUnexpectedExceptionExampleTest implements Serializable {
    */
   @Test
   public void createRegion_withTryCatch_dontDoThis() throws Exception {
-    Host.getHost(0).getVM(0).invoke(new SerializableRunnable("Create Region") {
+    getVM(0).invoke(new SerializableRunnable("Create Region") {
       @Override
       public void run() {
         try {
@@ -72,7 +72,7 @@ public class CatchingUnexpectedExceptionExampleTest implements Serializable {
    */
   @Test
   public void createRegion_withThrowsException_thisIsBetter() throws Exception {
-    Host.getHost(0).getVM(0).invoke(new SerializableRunnable("Create Region") {
+    getVM(0).invoke(new SerializableRunnable("Create Region") {
       @Override
       public void run() throws Exception {
         Cache cache = new CacheFactory().create();
@@ -88,7 +88,7 @@ public class CatchingUnexpectedExceptionExampleTest implements Serializable {
    */
   @Test
   public void createRegion_withLambda_thisIsBest() throws Exception {
-    Host.getHost(0).getVM(0).invoke("Create Region", () -> {
+    getVM(0).invoke("Create Region", () -> {
       Cache cache = new CacheFactory().create();
       RegionFactory regionFactory = cache.createRegionFactory(new AttributesFactory().create());
       LocalRegion region = (LocalRegion) regionFactory.create("region1");
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/DistributedTestRuleExampleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/DistributedTestRuleExampleTest.java
index a73f4b4..1a2e7bc 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/DistributedTestRuleExampleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/DistributedTestRuleExampleTest.java
@@ -14,13 +14,13 @@
  */
 package org.apache.geode.test.dunit.examples;
 
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
@@ -32,6 +32,6 @@ public class DistributedTestRuleExampleTest {
 
   @Test
   public void shouldHaveFourDUnitVMsByDefault() throws Exception {
-    assertThat(Host.getHost(0).getVMCount()).isEqualTo(4);
+    assertThat(getVMCount()).isEqualTo(4);
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/InvokeRunnableExampleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/InvokeRunnableExampleTest.java
index 3477d5a..31d4195 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/InvokeRunnableExampleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/InvokeRunnableExampleTest.java
@@ -14,11 +14,14 @@
  */
 package org.apache.geode.test.dunit.examples;
 
+import static org.apache.geode.test.dunit.VM.getAllVMs;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMCount;
+
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
@@ -31,15 +34,15 @@ public class InvokeRunnableExampleTest {
 
   @Test
   public void invokeHelloWorldForEachVMInGetAllVMs() throws Exception {
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> System.out.println(vm + " says Hello World!"));
     }
   }
 
   @Test
   public void invokeHelloWorldInEachVMInOrder() throws Exception {
-    for (int whichVM = 0; whichVM < Host.getHost(0).getVMCount(); whichVM++) {
-      VM vm = Host.getHost(0).getVM(whichVM);
+    for (int whichVM = 0; whichVM < getVMCount(); whichVM++) {
+      VM vm = getVM(whichVM);
       vm.invoke(() -> System.out.println(vm + " says Hello World!"));
     }
   }
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/LocatorPortClusterExampleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/LocatorPortClusterExampleTest.java
index f78f5f5..e391357 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/LocatorPortClusterExampleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/examples/LocatorPortClusterExampleTest.java
@@ -16,6 +16,9 @@ package org.apache.geode.test.dunit.examples;
 
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.test.dunit.DistributedTestUtils.getLocatorPort;
+import static org.apache.geode.test.dunit.VM.getAllVMs;
+import static org.apache.geode.test.dunit.VM.getHostName;
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.Serializable;
@@ -30,7 +33,6 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedDisconnectRule;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
@@ -54,11 +56,11 @@ public class LocatorPortClusterExampleTest implements Serializable {
   @Before
   public void setUp() throws Exception {
     config = new Properties();
-    config.put(LOCATORS, Host.getHost(0).getHostName() + "[" + getLocatorPort() + "]");
+    config.put(LOCATORS, getHostName() + "[" + getLocatorPort() + "]");
 
     cache = (InternalCache) new CacheFactory(config).create();
 
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> {
         cache = (InternalCache) new CacheFactory(config).create();
       });
@@ -68,14 +70,14 @@ public class LocatorPortClusterExampleTest implements Serializable {
   @After
   public void tearDown() throws Exception {
     cache = null;
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> cache = null);
     }
   }
 
   @Test
   public void clusterHasDUnitVMCountPlusTwoByDefault() throws Exception {
-    int dunitVMCount = Host.getHost(0).getVMCount();
+    int dunitVMCount = getVMCount();
     assertThat(cache.getDistributionManager().getViewMembers()).hasSize(dunitVMCount + 2);
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedTestRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedTestRule.java
index 563c7d1..c14968b 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedTestRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedTestRule.java
@@ -14,9 +14,10 @@
  */
 package org.apache.geode.test.dunit.rules;
 
+import static org.apache.geode.test.dunit.VM.DEFAULT_VM_COUNT;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.standalone.DUnitLauncher;
 
 /**
@@ -40,6 +41,7 @@ import org.apache.geode.test.dunit.standalone.DUnitLauncher;
  * }
  * </pre>
  */
+@SuppressWarnings("unused")
 public class DistributedTestRule extends DistributedExternalResource {
 
   private final int vmCount;
@@ -49,26 +51,28 @@ public class DistributedTestRule extends DistributedExternalResource {
   }
 
   public DistributedTestRule() {
-    // use 4 DUnit VMs by default
-    this.vmCount = 4;
+    this(new Builder());
+  }
+
+  public DistributedTestRule(final int vmCount) {
+    this(new Builder().withVMCount(vmCount));
   }
 
   DistributedTestRule(final Builder builder) {
-    // use 4 DUnit VMs by default
-    this.vmCount = builder.vmCount;
+    vmCount = builder.vmCount;
   }
 
   @Override
   protected void before() throws Throwable {
     DUnitLauncher.launchIfNeeded();
     for (int i = 0; i < vmCount; i++) {
-      assertThat(Host.getHost(0).getVM(i)).isNotNull();
+      assertThat(getVM(i)).isNotNull();
     }
   }
 
   public static class Builder {
 
-    private int vmCount = 4;
+    private int vmCount = DEFAULT_VM_COUNT;
 
     public Builder withVMCount(final int vmCount) {
       if (vmCount < 0) {
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/SharedCountersRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/SharedCountersRule.java
index 16a0ae0..f1dbfae 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/SharedCountersRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/SharedCountersRule.java
@@ -66,6 +66,10 @@ public class SharedCountersRule extends DistributedExternalResource {
     return new Builder();
   }
 
+  public SharedCountersRule() {
+    this(new Builder(), new RemoteInvoker());
+  }
+
   SharedCountersRule(final Builder builder) {
     this(builder, new RemoteInvoker());
   }
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/DistributedTestRuleExampleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetCurrentVmNumDistributedTest.java
similarity index 61%
copy from geode-core/src/test/java/org/apache/geode/test/dunit/examples/DistributedTestRuleExampleTest.java
copy to geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetCurrentVmNumDistributedTest.java
index a73f4b4..689ed37 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/examples/DistributedTestRuleExampleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetCurrentVmNumDistributedTest.java
@@ -12,26 +12,38 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.test.dunit.examples;
+package org.apache.geode.test.dunit.tests;
 
+import static org.apache.geode.test.dunit.VM.CONTROLLER_VM;
+import static org.apache.geode.test.dunit.VM.DEFAULT_VM_COUNT;
+import static org.apache.geode.test.dunit.VM.getCurrentVMNum;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
-public class DistributedTestRuleExampleTest {
+public class GetCurrentVmNumDistributedTest {
 
   @ClassRule
   public static DistributedTestRule distributedTestRule = new DistributedTestRule();
 
   @Test
-  public void shouldHaveFourDUnitVMsByDefault() throws Exception {
-    assertThat(Host.getHost(0).getVMCount()).isEqualTo(4);
+  public void returnsNegativeOneInController() {
+    assertThat(getCurrentVMNum()).isEqualTo(CONTROLLER_VM);
+  }
+
+  @Test
+  public void returnsWhichVmInVm() {
+    assertThat(getVMCount()).isGreaterThanOrEqualTo(DEFAULT_VM_COUNT);
+    for (int i = 0; i < getVMCount(); i++) {
+      assertThat(getVM(i).invoke(() -> getCurrentVMNum())).isEqualTo(i).isNotEqualTo(CONTROLLER_VM);
+    }
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetPidAndIdAfterBounceDistributedTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetPidAndIdAfterBounceDistributedTest.java
index 8bcb40e..ef5305d 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetPidAndIdAfterBounceDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetPidAndIdAfterBounceDistributedTest.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.test.dunit.tests;
 
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import org.junit.Before;
@@ -21,7 +23,6 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
@@ -36,27 +37,27 @@ public class GetPidAndIdAfterBounceDistributedTest {
 
   @Before
   public void setUp() throws Exception {
-    idsBefore = new int[Host.getHost(0).getVMCount()];
-    pidsBefore = new int[Host.getHost(0).getVMCount()];
+    idsBefore = new int[getVMCount()];
+    pidsBefore = new int[getVMCount()];
 
-    for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-      idsBefore[i] = Host.getHost(0).getVM(i).getId();
-      pidsBefore[i] = Host.getHost(0).getVM(i).getPid();
-      Host.getHost(0).getVM(i).bounce();
+    for (int i = 0; i < getVMCount(); i++) {
+      idsBefore[i] = getVM(i).getId();
+      pidsBefore[i] = getVM(i).getPid();
+      getVM(i).bounce();
     }
   }
 
   @Test
   public void getIdShouldReturnSameValueAfterBounce() throws Exception {
-    for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-      assertThat(Host.getHost(0).getVM(i).getId()).isEqualTo(idsBefore[i]);
+    for (int i = 0; i < getVMCount(); i++) {
+      assertThat(getVM(i).getId()).isEqualTo(idsBefore[i]);
     }
   }
 
   @Test
   public void getPidShouldReturnDifferentValueAfterBounce() throws Exception {
-    for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-      assertThat(Host.getHost(0).getVM(i).getPid()).isNotEqualTo(pidsBefore[i]);
+    for (int i = 0; i < getVMCount(); i++) {
+      assertThat(getVM(i).getPid()).isNotEqualTo(pidsBefore[i]);
     }
   }
 }
diff --git a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/GatewayLegacyAuthenticationRegressionTest.java b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/GatewayLegacyAuthenticationRegressionTest.java
index 5103cd4..c6967b6 100644
--- a/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/GatewayLegacyAuthenticationRegressionTest.java
+++ b/geode-wan/src/test/java/org/apache/geode/internal/cache/wan/GatewayLegacyAuthenticationRegressionTest.java
@@ -24,7 +24,7 @@ import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_PEER
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_PEER_AUTH_INIT;
 import static org.apache.geode.distributed.ConfigurationProperties.START_LOCATOR;
 import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
-import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.awaitility.Awaitility.await;
 import static org.awaitility.Awaitility.waitAtMost;
@@ -71,7 +71,7 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolde
  * <p>
  * GEODE-3117: "Gateway authentication throws NullPointerException"
  */
-@Category({DistributedTest.class, SecurityTest.class, WanTest.class, WanTest.class})
+@Category({DistributedTest.class, SecurityTest.class, WanTest.class})
 public class GatewayLegacyAuthenticationRegressionTest implements Serializable {
 
   private static final String REGION_NAME = "TheRegion";
@@ -109,10 +109,10 @@ public class GatewayLegacyAuthenticationRegressionTest implements Serializable {
   public void before() {
     AUTHENTICATE_COUNT.set(0);
 
-    londonLocatorVM = getHost(0).getVM(0);
-    newYorkLocatorVM = getHost(0).getVM(1);
-    londonServerVM = getHost(0).getVM(2);
-    newYorkServerVM = getHost(0).getVM(3);
+    londonLocatorVM = getVM(0);
+    newYorkLocatorVM = getVM(1);
+    londonServerVM = getVM(2);
+    newYorkServerVM = getVM(3);
 
     londonName = "ln";
     newYorkName = "ny";

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

[geode] 05/15: GEODE-4876: Update RegionExpirationDistributedTest use of VM and NetworkUtil

Posted by kl...@apache.org.
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 841dde9bada8467a5885777da4623dad25a75822
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:30:02 2018 -0700

    GEODE-4876: Update RegionExpirationDistributedTest use of VM and NetworkUtil
    
    * rename RegionExpirationDUnitTest as RegionExpirationDistributedTest
---
 ...tionDUnitTest.java => RegionExpirationDistributedTest.java} | 10 +++++-----
 .../apache/geode/cache/RegionExpirationIntegrationTest.java    |  2 +-
 2 files changed, 6 insertions(+), 6 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/cache/RegionExpirationDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/RegionExpirationDistributedTest.java
similarity index 96%
rename from geode-core/src/test/java/org/apache/geode/cache/RegionExpirationDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/cache/RegionExpirationDistributedTest.java
index 63dc650..be7f6eb 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/RegionExpirationDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/RegionExpirationDistributedTest.java
@@ -16,7 +16,7 @@ package org.apache.geode.cache;
 
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.geode.cache.RegionShortcut.REPLICATE;
-import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.spy;
@@ -56,7 +56,7 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
 @Category(DistributedTest.class)
 @RunWith(JUnitParamsRunner.class)
 @SuppressWarnings("serial")
-public class RegionExpirationDUnitTest implements Serializable {
+public class RegionExpirationDistributedTest implements Serializable {
 
   private static final int TTL_SECONDS = 10;
   private static final String KEY = "key";
@@ -81,8 +81,8 @@ public class RegionExpirationDUnitTest implements Serializable {
   public void setUp() {
     regionName = getClass().getSimpleName() + "_" + testName.getMethodName();
 
-    withExpirationVM0 = getHost(0).getVM(0);
-    withoutExpirationVM1 = getHost(0).getVM(1);
+    withExpirationVM0 = getVM(0);
+    withoutExpirationVM1 = getVM(1);
   }
 
   @Test
@@ -151,7 +151,7 @@ public class RegionExpirationDUnitTest implements Serializable {
     Verification(final Consumer<CacheListener<String, String>> strategy,
         final ExpirationAction evictionAction) {
       this.strategy = strategy;
-      this.expirationAction = evictionAction;
+      expirationAction = evictionAction;
     }
 
     void verify(final CacheListener<String, String> spyCacheListener) {
diff --git a/geode-core/src/test/java/org/apache/geode/cache/RegionExpirationIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/cache/RegionExpirationIntegrationTest.java
index 89c2114..a718b51 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/RegionExpirationIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/RegionExpirationIntegrationTest.java
@@ -41,7 +41,7 @@ import org.mockito.InOrder;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
 /**
- * Extracted from {@link RegionExpirationDUnitTest}.
+ * Extracted from {@link RegionExpirationDistributedTest}.
  */
 @Category(IntegrationTest.class)
 public class RegionExpirationIntegrationTest {

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

[geode] 15/15: GEODE-4876: Cleanup CacheRule tests use of VM and javadocs

Posted by kl...@apache.org.
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 7be2a7e215a85c9d48ec293c024b54d719946008
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:42:44 2018 -0700

    GEODE-4876: Cleanup CacheRule tests use of VM and javadocs
    
    * rename CacheRuleTest as CacheRuleBuilderDistributedTest
    * write new CacheRuleDistributedTest
    * fix javadocs of CacheRule
---
 .../apache/geode/test/dunit/rules/CacheRule.java   |   6 +-
 ...t.java => CacheRuleBuilderDistributedTest.java} |  53 +++----
 ...RuleTest.java => CacheRuleDistributedTest.java} | 171 ++++++++++-----------
 3 files changed, 117 insertions(+), 113 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/CacheRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/CacheRule.java
index 42a3923..990f45c 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/CacheRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/CacheRule.java
@@ -119,6 +119,10 @@ public class CacheRule extends DistributedExternalResource {
     cache = (InternalCache) new CacheFactory(config).create();
   }
 
+  public void createCache(final CacheFactory cacheFactory) {
+    cache = (InternalCache) cacheFactory.create();
+  }
+
   public void createCache(final Properties config) {
     cache = (InternalCache) new CacheFactory(config).create();
   }
@@ -156,7 +160,7 @@ public class CacheRule extends DistributedExternalResource {
   }
 
   /**
-   * Builds an instance of SharedCountersRule
+   * Builds an instance of CacheRule.
    */
   public static class Builder {
 
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleBuilderDistributedTest.java
similarity index 86%
copy from geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleTest.java
copy to geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleBuilderDistributedTest.java
index d10d4be..72c3f11 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleBuilderDistributedTest.java
@@ -15,6 +15,9 @@
 package org.apache.geode.test.dunit.rules.tests;
 
 import static org.apache.geode.distributed.ConfigurationProperties.DISTRIBUTED_SYSTEM_ID;
+import static org.apache.geode.test.dunit.VM.getAllVMs;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.apache.geode.test.junit.runners.TestRunner.runTestWithValidation;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -27,7 +30,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.CacheRule;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
@@ -35,7 +37,7 @@ import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
 @SuppressWarnings("serial")
-public class CacheRuleTest {
+public class CacheRuleBuilderDistributedTest {
 
   @ClassRule
   public static DistributedTestRule distributedTestRule = new DistributedTestRule();
@@ -96,7 +98,7 @@ public class CacheRuleTest {
     @Test
     public void getCache_returnsNullInAllVMs() throws Exception {
       assertThat(cacheRule.getCache()).isNull();
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> assertThat(cacheRule.getCache()).isNull());
       }
     }
@@ -104,7 +106,7 @@ public class CacheRuleTest {
     @Test
     public void getCacheSingleton_returnsNullInAllVMs() throws Exception {
       assertThat(GemFireCacheImpl.getInstance()).isNull();
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> assertThat(GemFireCacheImpl.getInstance()).isNull());
       }
     }
@@ -121,7 +123,7 @@ public class CacheRuleTest {
     @Test
     public void getCache_returnsCacheInLocalOnly() throws Exception {
       assertThat(cacheRule.getCache()).isNotNull();
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> assertThat(cacheRule.getCache()).isNull());
       }
     }
@@ -133,19 +135,18 @@ public class CacheRuleTest {
   public static class CreateCacheInOneVM implements Serializable {
 
     @Rule
-    public CacheRule cacheRule =
-        CacheRule.builder().createCacheIn(Host.getHost(0).getVM(0)).build();
+    public CacheRule cacheRule = CacheRule.builder().createCacheIn(getVM(0)).build();
 
     @Test
     public void getCache_returnsCacheInOneVM() throws Exception {
       assertThat(cacheRule.getCache()).isNull();
 
-      Host.getHost(0).getVM(0).invoke(() -> {
+      getVM(0).invoke(() -> {
         assertThat(cacheRule.getCache()).isNotNull();
       });
 
-      for (int i = 1; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+      for (int i = 1; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache()).isNull();
         });
       }
@@ -158,20 +159,20 @@ public class CacheRuleTest {
   public static class CreateCacheInTwoVMs implements Serializable {
 
     @Rule
-    public CacheRule cacheRule = CacheRule.builder().createCacheIn(Host.getHost(0).getVM(1))
-        .createCacheIn(Host.getHost(0).getVM(3)).build();
+    public CacheRule cacheRule =
+        CacheRule.builder().createCacheIn(getVM(1)).createCacheIn(getVM(3)).build();
 
     @Test
     public void getCache_returnsCacheInTwoVMs() throws Exception {
       assertThat(cacheRule.getCache()).isNull();
 
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
+      for (int i = 0; i < getVMCount(); i++) {
         if (i == 1 || i == 3) {
-          Host.getHost(0).getVM(i).invoke(() -> {
+          getVM(i).invoke(() -> {
             assertThat(cacheRule.getCache()).isNotNull();
           });
         } else {
-          Host.getHost(0).getVM(i).invoke(() -> {
+          getVM(i).invoke(() -> {
             assertThat(cacheRule.getCache()).isNull();
           });
         }
@@ -190,8 +191,8 @@ public class CacheRuleTest {
     @Test
     public void createCacheInAll_returnsCacheInAll() throws Exception {
       assertThat(cacheRule.getCache()).isNotNull();
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+      for (int i = 0; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache()).isNotNull();
         });
       }
@@ -209,8 +210,8 @@ public class CacheRuleTest {
     @Test
     public void createCacheInAllExplicitly_returnsCacheInAll() throws Exception {
       assertThat(cacheRule.getCache()).isNotNull();
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+      for (int i = 0; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache()).isNotNull();
         });
       }
@@ -227,12 +228,12 @@ public class CacheRuleTest {
 
     @Test
     public void createCacheInAll_createsCluster() throws Exception {
-      int vmCount = Host.getHost(0).getVMCount();
+      int vmCount = getVMCount();
 
       assertThat(cacheRule.getCache().getDistributionManager().getViewMembers())
           .hasSize(vmCount + 2);
       for (int i = 0; i < vmCount; i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache().getDistributionManager().getViewMembers())
               .hasSize(vmCount + 2);
         });
@@ -256,12 +257,12 @@ public class CacheRuleTest {
       assertThat(
           cacheRule.getCache().getInternalDistributedSystem().getConfig().getDistributedSystemId())
               .isEqualTo(1);
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+      for (int i = 0; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache().getDistributedSystem().getProperties()
               .getProperty(DISTRIBUTED_SYSTEM_ID)).isEqualTo("1");
         });
-        Host.getHost(0).getVM(i).invoke(() -> {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache().getInternalDistributedSystem().getConfig()
               .getDistributedSystemId()).isEqualTo(1);
         });
@@ -281,8 +282,8 @@ public class CacheRuleTest {
     @Test
     public void replaceConfig_createsLonersInAll() throws Exception {
       assertThat(cacheRule.getCache().getDistributionManager().getViewMembers()).hasSize(1);
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+      for (int i = 0; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache().getDistributionManager().getViewMembers()).hasSize(1);
         });
       }
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleDistributedTest.java
similarity index 58%
rename from geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleTest.java
rename to geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleDistributedTest.java
index d10d4be..a4748e5 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/CacheRuleDistributedTest.java
@@ -14,20 +14,22 @@
  */
 package org.apache.geode.test.dunit.rules.tests;
 
-import static org.apache.geode.distributed.ConfigurationProperties.DISTRIBUTED_SYSTEM_ID;
+import static org.apache.geode.test.dunit.VM.getAllVMs;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.apache.geode.test.junit.runners.TestRunner.runTestWithValidation;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.Serializable;
 import java.util.Properties;
 
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.CacheRule;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
@@ -35,7 +37,7 @@ import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
 @SuppressWarnings("serial")
-public class CacheRuleTest {
+public class CacheRuleDistributedTest {
 
   @ClassRule
   public static DistributedTestRule distributedTestRule = new DistributedTestRule();
@@ -66,23 +68,13 @@ public class CacheRuleTest {
   }
 
   @Test
-  public void createCacheInAllExplicitly() throws Exception {
-    runTestWithValidation(CreateCacheInAllExplicitly.class);
-  }
-
-  @Test
   public void createCacheInAllCreatesCluster() throws Exception {
     runTestWithValidation(CreateCacheInAllCreatesCluster.class);
   }
 
   @Test
-  public void addConfigAffectsAllVMs() throws Exception {
-    runTestWithValidation(AddConfigAffectsAllVMs.class);
-  }
-
-  @Test
-  public void replaceConfigCreatesLonersInAll() throws Exception {
-    runTestWithValidation(ReplaceConfigCreatesLonersInAll.class);
+  public void emptyConfigCreatesLonersInAll() throws Exception {
+    runTestWithValidation(EmptyConfigCreatesLonersInAll.class);
   }
 
   /**
@@ -91,12 +83,12 @@ public class CacheRuleTest {
   public static class DefaultDoesNothing implements Serializable {
 
     @Rule
-    public CacheRule cacheRule = CacheRule.builder().build();
+    public CacheRule cacheRule = new CacheRule();
 
     @Test
     public void getCache_returnsNullInAllVMs() throws Exception {
       assertThat(cacheRule.getCache()).isNull();
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> assertThat(cacheRule.getCache()).isNull());
       }
     }
@@ -104,7 +96,7 @@ public class CacheRuleTest {
     @Test
     public void getCacheSingleton_returnsNullInAllVMs() throws Exception {
       assertThat(GemFireCacheImpl.getInstance()).isNull();
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> assertThat(GemFireCacheImpl.getInstance()).isNull());
       }
     }
@@ -116,12 +108,17 @@ public class CacheRuleTest {
   public static class CreateCacheInLocal implements Serializable {
 
     @Rule
-    public CacheRule cacheRule = CacheRule.builder().createCacheInLocal().build();
+    public CacheRule cacheRule = new CacheRule();
+
+    @Before
+    public void setUp() throws Exception {
+      cacheRule.createCache();
+    }
 
     @Test
     public void getCache_returnsCacheInLocalOnly() throws Exception {
       assertThat(cacheRule.getCache()).isNotNull();
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> assertThat(cacheRule.getCache()).isNull());
       }
     }
@@ -133,19 +130,25 @@ public class CacheRuleTest {
   public static class CreateCacheInOneVM implements Serializable {
 
     @Rule
-    public CacheRule cacheRule =
-        CacheRule.builder().createCacheIn(Host.getHost(0).getVM(0)).build();
+    public CacheRule cacheRule = new CacheRule();
+
+    @Before
+    public void setUp() throws Exception {
+      getVM(0).invoke(() -> {
+        cacheRule.createCache();
+      });
+    }
 
     @Test
     public void getCache_returnsCacheInOneVM() throws Exception {
       assertThat(cacheRule.getCache()).isNull();
 
-      Host.getHost(0).getVM(0).invoke(() -> {
+      getVM(0).invoke(() -> {
         assertThat(cacheRule.getCache()).isNotNull();
       });
 
-      for (int i = 1; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+      for (int i = 1; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache()).isNull();
         });
       }
@@ -158,20 +161,29 @@ public class CacheRuleTest {
   public static class CreateCacheInTwoVMs implements Serializable {
 
     @Rule
-    public CacheRule cacheRule = CacheRule.builder().createCacheIn(Host.getHost(0).getVM(1))
-        .createCacheIn(Host.getHost(0).getVM(3)).build();
+    public CacheRule cacheRule = new CacheRule();
+
+    @Before
+    public void setUp() throws Exception {
+      getVM(1).invoke(() -> {
+        cacheRule.createCache();
+      });
+      getVM(3).invoke(() -> {
+        cacheRule.createCache();
+      });
+    }
 
     @Test
     public void getCache_returnsCacheInTwoVMs() throws Exception {
       assertThat(cacheRule.getCache()).isNull();
 
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
+      for (int i = 0; i < getVMCount(); i++) {
         if (i == 1 || i == 3) {
-          Host.getHost(0).getVM(i).invoke(() -> {
+          getVM(i).invoke(() -> {
             assertThat(cacheRule.getCache()).isNotNull();
           });
         } else {
-          Host.getHost(0).getVM(i).invoke(() -> {
+          getVM(i).invoke(() -> {
             assertThat(cacheRule.getCache()).isNull();
           });
         }
@@ -185,32 +197,23 @@ public class CacheRuleTest {
   public static class CreateCacheInAll implements Serializable {
 
     @Rule
-    public CacheRule cacheRule = CacheRule.builder().createCacheInAll().build();
-
-    @Test
-    public void createCacheInAll_returnsCacheInAll() throws Exception {
-      assertThat(cacheRule.getCache()).isNotNull();
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
-          assertThat(cacheRule.getCache()).isNotNull();
+    public CacheRule cacheRule = new CacheRule();
+
+    @Before
+    public void setUp() throws Exception {
+      cacheRule.createCache();
+      for (int i = 0; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
+          cacheRule.createCache();
         });
       }
     }
-  }
-
-  /**
-   * Used by test {@link #createCacheInAllExplicitly()}.
-   */
-  public static class CreateCacheInAllExplicitly implements Serializable {
-
-    @Rule
-    public CacheRule cacheRule = CacheRule.builder().createCacheInAll().build();
 
     @Test
-    public void createCacheInAllExplicitly_returnsCacheInAll() throws Exception {
+    public void createCacheInAll_returnsCacheInAll() throws Exception {
       assertThat(cacheRule.getCache()).isNotNull();
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+      for (int i = 0; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache()).isNotNull();
         });
       }
@@ -222,17 +225,29 @@ public class CacheRuleTest {
    */
   public static class CreateCacheInAllCreatesCluster implements Serializable {
 
+    private int vmCount;
+
     @Rule
-    public CacheRule cacheRule = CacheRule.builder().createCacheInAll().build();
+    public CacheRule cacheRule = new CacheRule();
+
+    @Before
+    public void setUp() throws Exception {
+      vmCount = getVMCount();
+
+      cacheRule.createCache();
+      for (int i = 0; i < vmCount; i++) {
+        getVM(i).invoke(() -> {
+          cacheRule.createCache();
+        });
+      }
+    }
 
     @Test
     public void createCacheInAll_createsCluster() throws Exception {
-      int vmCount = Host.getHost(0).getVMCount();
-
       assertThat(cacheRule.getCache().getDistributionManager().getViewMembers())
           .hasSize(vmCount + 2);
       for (int i = 0; i < vmCount; i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache().getDistributionManager().getViewMembers())
               .hasSize(vmCount + 2);
         });
@@ -241,48 +256,32 @@ public class CacheRuleTest {
   }
 
   /**
-   * Used by test {@link #addConfigAffectsAllVMs()}.
+   * Used by test {@link #emptyConfigCreatesLonersInAll()}.
    */
-  public static class AddConfigAffectsAllVMs implements Serializable {
+  public static class EmptyConfigCreatesLonersInAll implements Serializable {
+
+    private Properties config;
 
     @Rule
-    public CacheRule cacheRule =
-        CacheRule.builder().addConfig(DISTRIBUTED_SYSTEM_ID, "1").createCacheInAll().build();
+    public CacheRule cacheRule = new CacheRule();
 
-    @Test
-    public void addConfig_affectsAllVMs() throws Exception {
-      assertThat(cacheRule.getCache().getDistributedSystem().getProperties()
-          .getProperty(DISTRIBUTED_SYSTEM_ID)).isEqualTo("1");
-      assertThat(
-          cacheRule.getCache().getInternalDistributedSystem().getConfig().getDistributedSystemId())
-              .isEqualTo(1);
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
-          assertThat(cacheRule.getCache().getDistributedSystem().getProperties()
-              .getProperty(DISTRIBUTED_SYSTEM_ID)).isEqualTo("1");
-        });
-        Host.getHost(0).getVM(i).invoke(() -> {
-          assertThat(cacheRule.getCache().getInternalDistributedSystem().getConfig()
-              .getDistributedSystemId()).isEqualTo(1);
+    @Before
+    public void setUp() throws Exception {
+      config = new Properties();
+
+      cacheRule.createCache(config);
+      for (int i = 0; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
+          cacheRule.createCache(config);
         });
       }
     }
-  }
-
-  /**
-   * Used by test {@link #replaceConfigCreatesLonersInAll()}.
-   */
-  public static class ReplaceConfigCreatesLonersInAll implements Serializable {
-
-    @Rule
-    public CacheRule cacheRule =
-        CacheRule.builder().replaceConfig(new Properties()).createCacheInAll().build();
 
     @Test
-    public void replaceConfig_createsLonersInAll() throws Exception {
+    public void emptyConfig_createsLonersInAll() throws Exception {
       assertThat(cacheRule.getCache().getDistributionManager().getViewMembers()).hasSize(1);
-      for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-        Host.getHost(0).getVM(i).invoke(() -> {
+      for (int i = 0; i < getVMCount(); i++) {
+        getVM(i).invoke(() -> {
           assertThat(cacheRule.getCache().getDistributionManager().getViewMembers()).hasSize(1);
         });
       }

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

[geode] 11/15: GEODE-4876: Update SharedCountersRuleDistributedTest use of VM and NetworkUtil

Posted by kl...@apache.org.
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 40795fedf572fb7658edee94f15d20b170ad0408
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:36:53 2018 -0700

    GEODE-4876: Update SharedCountersRuleDistributedTest use of VM and NetworkUtil
    
    * rename SharedCountersRuleTest as SharedCountersRuleDistributedTest
---
 ....java => SharedCountersRuleDistributedTest.java} | 21 +++++++++++----------
 1 file changed, 11 insertions(+), 10 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedCountersRuleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedCountersRuleDistributedTest.java
similarity index 90%
rename from geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedCountersRuleTest.java
rename to geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedCountersRuleDistributedTest.java
index 9508a66..f8778e1 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedCountersRuleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedCountersRuleDistributedTest.java
@@ -16,6 +16,8 @@ package org.apache.geode.test.dunit.rules.tests;
 
 import static java.util.concurrent.CompletableFuture.supplyAsync;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.geode.test.dunit.VM.getAllVMs;
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
@@ -32,7 +34,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.dunit.rules.SharedCountersRule;
@@ -40,7 +41,7 @@ import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
 @SuppressWarnings("serial")
-public class SharedCountersRuleTest implements Serializable {
+public class SharedCountersRuleDistributedTest implements Serializable {
 
   private static final int TWO_MINUTES_MILLIS = 2 * 60 * 1000;
   private static final String ID1 = "ID1";
@@ -104,23 +105,23 @@ public class SharedCountersRuleTest implements Serializable {
   @Test
   public void inc_fromDUnitVMs_getTotal_returnsFour() throws Exception {
     sharedCountersRule.initialize(ID1);
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> {
         sharedCountersRule.increment(ID1);
       });
     }
-    assertThat(sharedCountersRule.getTotal(ID1)).isEqualTo(Host.getHost(0).getVMCount());
+    assertThat(sharedCountersRule.getTotal(ID1)).isEqualTo(getVMCount());
   }
 
   @Test
   public void inc_fromEveryVM_getTotal_returnsFive() throws Exception {
     sharedCountersRule.initialize(ID1).increment(ID1);
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> {
         sharedCountersRule.increment(ID1);
       });
     }
-    assertThat(sharedCountersRule.getTotal(ID1)).isEqualTo(Host.getHost(0).getVMCount() + 1);
+    assertThat(sharedCountersRule.getTotal(ID1)).isEqualTo(getVMCount() + 1);
   }
 
   @Test
@@ -131,19 +132,19 @@ public class SharedCountersRuleTest implements Serializable {
 
     // inc ID1 in numThreads in every VM (4 DUnit VMs + Controller VM)
     submitIncrementTasks(numThreads, ID1);
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> submitIncrementTasks(numThreads, ID1));
     }
 
     // await CompletableFuture in every VM
     Stopwatch stopwatch = Stopwatch.createStarted();
     combined.get(calculateTimeoutMillis(stopwatch), MILLISECONDS);
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       long timeoutMillis = calculateTimeoutMillis(stopwatch);
       vm.invoke(() -> combined.get(timeoutMillis, MILLISECONDS));
     }
 
-    int dunitVMCount = Host.getHost(0).getVMCount();
+    int dunitVMCount = getVMCount();
     int controllerPlusDUnitVMCount = dunitVMCount + 1;
     int expectedIncrements = controllerPlusDUnitVMCount * numThreads;
 
@@ -158,7 +159,7 @@ public class SharedCountersRuleTest implements Serializable {
     executor = Executors.newFixedThreadPool(numThreads);
     futures = new ArrayList<>();
 
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> {
         executor = Executors.newFixedThreadPool(numThreads);
         futures = new ArrayList<>();

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

[geode] 13/15: GEODE-4876: Update DistributedDisconnectRuleDistributedTest use of VM and NetworkUtil

Posted by kl...@apache.org.
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 5271b6b970e2239a4e46e3bc43fb41250c7b39e4
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:38:45 2018 -0700

    GEODE-4876: Update DistributedDisconnectRuleDistributedTest use of VM and NetworkUtil
    
    * rename DistributedDisconnectRuleTest as DistributedDisconnectRuleDistributedTest
---
 ...ava => DistributedDisconnectRuleDistributedTest.java} | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleDistributedTest.java
similarity index 94%
rename from geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleTest.java
rename to geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleDistributedTest.java
index 9363be2..01bbcee 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleDistributedTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.test.dunit.rules.tests;
 
+import static org.apache.geode.test.dunit.VM.getAllVMs;
 import static org.apache.geode.test.dunit.rules.DistributedDisconnectRule.disconnect;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -28,7 +29,6 @@ import org.junit.runners.MethodSorters;
 
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedDisconnectRule;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
@@ -36,19 +36,19 @@ import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.runners.TestRunner;
 
 @Category(DistributedTest.class)
-public class DistributedDisconnectRuleTest {
+public class DistributedDisconnectRuleDistributedTest {
+
+  private static InternalCache cache;
 
   @ClassRule
   public static DistributedTestRule distributedTestRule = new DistributedTestRule();
 
-  static InternalCache cache;
-
   @After
   public void tearDown() throws Exception {
     disconnect();
     cache = null;
 
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> {
         disconnect();
         cache = null;
@@ -93,7 +93,7 @@ public class DistributedDisconnectRuleTest {
   static void createCacheInEveryVM() throws Exception {
     cache = (InternalCache) new CacheFactory().create();
 
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> {
         cache = (InternalCache) new CacheFactory().create();
       });
@@ -104,7 +104,7 @@ public class DistributedDisconnectRuleTest {
     assertThat(cache.isClosed()).isFalse();
     assertThat(cache.getInternalDistributedSystem().isConnected()).isTrue();
 
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> {
         assertThat(cache.isClosed()).isFalse();
         assertThat(cache.getInternalDistributedSystem().isConnected()).isTrue();
@@ -116,7 +116,7 @@ public class DistributedDisconnectRuleTest {
     assertThat(cache.getInternalDistributedSystem().isConnected()).isFalse();
     assertThat(cache.isClosed()).isTrue();
 
-    for (VM vm : Host.getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       vm.invoke(() -> {
         assertThat(cache.getInternalDistributedSystem().isConnected()).isFalse();
         assertThat(cache.isClosed()).isTrue();

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

[geode] 08/15: GEODE-4876: Update FunctionExecutionWithExceptionDistributedTest use of VM and NetworkUtil

Posted by kl...@apache.org.
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 46e98fb5e1be80c0ac6568848565c15519631d98
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:33:47 2018 -0700

    GEODE-4876: Update FunctionExecutionWithExceptionDistributedTest use of VM and NetworkUtil
    
    * rename FunctionExecution_ExceptionDUnitTest as FunctionExecutionWithExceptionDistributedTest
---
 ...java => FunctionExecutionWithExceptionDistributedTest.java} | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/FunctionExecution_ExceptionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/FunctionExecutionWithExceptionDistributedTest.java
similarity index 98%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/execute/FunctionExecution_ExceptionDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/execute/FunctionExecutionWithExceptionDistributedTest.java
index c45fe39..5429c35 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/execute/FunctionExecution_ExceptionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/execute/FunctionExecutionWithExceptionDistributedTest.java
@@ -20,7 +20,7 @@ import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_
 import static org.apache.geode.internal.cache.functions.TestFunction.TEST_FUNCTION_SEND_EXCEPTION;
 import static org.apache.geode.internal.cache.functions.TestFunction.TEST_FUNCTION_THROW_EXCEPTION;
 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.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
@@ -54,7 +54,7 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
 
 @Category({DistributedTest.class, FunctionServiceTest.class})
 @SuppressWarnings("serial")
-public class FunctionExecution_ExceptionDUnitTest implements Serializable {
+public class FunctionExecutionWithExceptionDistributedTest implements Serializable {
 
   private String regionName;
   private VM datastoreVM0;
@@ -80,9 +80,9 @@ public class FunctionExecution_ExceptionDUnitTest implements Serializable {
   public void setUp() throws Exception {
     regionName = getClass().getSimpleName() + "_" + testName.getMethodName();
 
-    datastoreVM0 = getHost(0).getVM(0);
-    datastoreVM1 = getHost(0).getVM(1);
-    datastoreVM2 = getHost(0).getVM(2);
+    datastoreVM0 = getVM(0);
+    datastoreVM1 = getVM(1);
+    datastoreVM2 = getVM(2);
 
     stringKey = "execKey";
     stringKeys = new HashSet<>();

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

[geode] 12/15: GEODE-4876: Update DistributedTestRuleDistributedTest use of VM and NetworkUtil

Posted by kl...@apache.org.
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 b6b4ac2de7f48d3b45298c872bda6f9453c59409
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:37:51 2018 -0700

    GEODE-4876: Update DistributedTestRuleDistributedTest use of VM and NetworkUtil
    
    * rename DistributedTestRuleTest as DistributedTestRuleDistributedTest
---
 ...tedTestRuleTest.java => DistributedTestRuleDistributedTest.java} | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedTestRuleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedTestRuleDistributedTest.java
similarity index 89%
rename from geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedTestRuleTest.java
rename to geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedTestRuleDistributedTest.java
index be0b518..18ad269 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedTestRuleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedTestRuleDistributedTest.java
@@ -14,24 +14,24 @@
  */
 package org.apache.geode.test.dunit.rules.tests;
 
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
-public class DistributedTestRuleTest {
+public class DistributedTestRuleDistributedTest {
 
   @ClassRule
   public static DistributedTestRule distributedTestRule = new DistributedTestRule();
 
   @Test
   public void shouldHaveFourOrMoreDUnitVMsByDefault() throws Exception {
-    assertThat(Host.getHost(0).getVMCount()).isGreaterThanOrEqualTo(4);
+    assertThat(getVMCount()).isGreaterThanOrEqualTo(4);
   }
 }

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

[geode] 10/15: GEODE-4876: Update SharedErrorCollectorDistributedTest use of VM and NetworkUtil

Posted by kl...@apache.org.
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 4296f218aa9122e910ce6db45a1b7fd0664b9366
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:35:49 2018 -0700

    GEODE-4876: Update SharedErrorCollectorDistributedTest use of VM and NetworkUtil
    
    * rename SharedErrorCollectorTest as SharedErrorCollectorDistributedTest
---
 ...va => SharedErrorCollectorDistributedTest.java} | 31 +++++++++++-----------
 1 file changed, 16 insertions(+), 15 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedErrorCollectorTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedErrorCollectorDistributedTest.java
similarity index 92%
rename from geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedErrorCollectorTest.java
rename to geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedErrorCollectorDistributedTest.java
index 82283c8..4f40724 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedErrorCollectorTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/SharedErrorCollectorDistributedTest.java
@@ -14,6 +14,9 @@
  */
 package org.apache.geode.test.dunit.rules.tests;
 
+import static org.apache.geode.test.dunit.VM.getAllVMs;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.hamcrest.Matchers.is;
 
@@ -29,7 +32,6 @@ import org.junit.rules.ErrorCollector;
 import org.junit.runner.Result;
 import org.junit.runner.notification.Failure;
 
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.dunit.rules.SharedErrorCollector;
@@ -38,7 +40,7 @@ import org.apache.geode.test.junit.runners.TestRunner;
 
 @Category(DistributedTest.class)
 @SuppressWarnings("serial")
-public class SharedErrorCollectorTest {
+public class SharedErrorCollectorDistributedTest {
 
   static final String MESSAGE = "Failure message";
 
@@ -90,7 +92,7 @@ public class SharedErrorCollectorTest {
 
     assertThat(result.wasSuccessful()).isFalse();
     List<Failure> failures = result.getFailures();
-    assertThat(failures).hasSize(Host.getHost(0).getVMCount());
+    assertThat(failures).hasSize(getVMCount());
     int i = 0;
     for (Failure failure : failures) {
       assertThat(failure.getException()).isInstanceOf(AssertionError.class)
@@ -104,7 +106,7 @@ public class SharedErrorCollectorTest {
 
     assertThat(result.wasSuccessful()).isFalse();
     List<Failure> failures = result.getFailures();
-    assertThat(failures).hasSize(Host.getHost(0).getVMCount() + 1);
+    assertThat(failures).hasSize(getVMCount() + 1);
     boolean first = true;
     int i = 0;
     for (Failure failure : failures) {
@@ -147,7 +149,7 @@ public class SharedErrorCollectorTest {
 
     assertThat(result.wasSuccessful()).isFalse();
     List<Failure> failures = result.getFailures();
-    assertThat(failures).hasSize(Host.getHost(0).getVMCount());
+    assertThat(failures).hasSize(getVMCount());
     int i = 0;
     for (Failure failure : failures) {
       assertThat(failure.getException()).isInstanceOf(NullPointerException.class)
@@ -161,7 +163,7 @@ public class SharedErrorCollectorTest {
 
     assertThat(result.wasSuccessful()).isFalse();
     List<Failure> failures = result.getFailures();
-    assertThat(failures).hasSize(Host.getHost(0).getVMCount() + 1);
+    assertThat(failures).hasSize(getVMCount() + 1);
     boolean first = true;
     int i = 0;
     for (Failure failure : failures) {
@@ -225,7 +227,7 @@ public class SharedErrorCollectorTest {
 
     @Test
     public void assertionFailsInDUnitVM() throws Exception {
-      Host.getHost(0).getVM(0).invoke(() -> errorCollector.checkThat(MESSAGE, false, is(true)));
+      getVM(0).invoke(() -> errorCollector.checkThat(MESSAGE, false, is(true)));
     }
   }
 
@@ -239,7 +241,7 @@ public class SharedErrorCollectorTest {
 
     @Test
     public void assertionFailsInEveryDUnitVM() throws Exception {
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(
             () -> errorCollector.checkThat(MESSAGE + " in VM-" + vm.getId(), false, is(true)));
       }
@@ -257,7 +259,7 @@ public class SharedErrorCollectorTest {
     @Test
     public void assertionFailsInEveryDUnitVM() throws Exception {
       errorCollector.checkThat(MESSAGE + " in VM-CONTROLLER", false, is(true));
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(
             () -> errorCollector.checkThat(MESSAGE + " in VM-" + vm.getId(), false, is(true)));
       }
@@ -274,7 +276,7 @@ public class SharedErrorCollectorTest {
 
     @Test
     public void assertionFailsInDUnitVM() throws Exception {
-      Host.getHost(0).getVM(0).invoke(() -> checkThat());
+      getVM(0).invoke(() -> checkThat());
     }
 
     private void checkThat() {
@@ -292,8 +294,7 @@ public class SharedErrorCollectorTest {
 
     @Test
     public void exceptionInDUnitVM() throws Exception {
-      Host.getHost(0).getVM(0)
-          .invoke(() -> errorCollector.addError(new NullPointerException(MESSAGE)));
+      getVM(0).invoke(() -> errorCollector.addError(new NullPointerException(MESSAGE)));
     }
   }
 
@@ -307,7 +308,7 @@ public class SharedErrorCollectorTest {
 
     @Test
     public void exceptionInEveryDUnitVM() throws Exception {
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> errorCollector
             .addError(new NullPointerException(MESSAGE + " in VM-" + vm.getId())));
       }
@@ -325,7 +326,7 @@ public class SharedErrorCollectorTest {
     @Test
     public void exceptionInEveryDUnitVM() throws Exception {
       errorCollector.addError(new NullPointerException(MESSAGE + " in VM-CONTROLLER"));
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> errorCollector
             .addError(new NullPointerException(MESSAGE + " in VM-" + vm.getId())));
       }
@@ -342,7 +343,7 @@ public class SharedErrorCollectorTest {
 
     @Test
     public void exceptionInDUnitVM() throws Exception {
-      Host.getHost(0).getVM(0).invoke(() -> addError());
+      getVM(0).invoke(() -> addError());
     }
 
     private void addError() {

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

[geode] 14/15: GEODE-4876: Update DistributedDisconnectRuleAsClassRuleDistributedTest use of VM

Posted by kl...@apache.org.
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 da4044ec9af30cca84f8eb51354efe381cfc307b
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:39:53 2018 -0700

    GEODE-4876: Update DistributedDisconnectRuleAsClassRuleDistributedTest use of VM
    
    * renamed from DistributedDisconnectRuleAsClassRuleTest
---
 ... DistributedDisconnectRuleAsClassRuleDistributedTest.java} | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleAsClassRuleTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleAsClassRuleDistributedTest.java
similarity index 95%
rename from geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleAsClassRuleTest.java
rename to geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleAsClassRuleDistributedTest.java
index 3425d32..5a52574 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleAsClassRuleTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/tests/DistributedDisconnectRuleAsClassRuleDistributedTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.test.dunit.rules.tests;
 
+import static org.apache.geode.test.dunit.VM.getAllVMs;
 import static org.apache.geode.test.dunit.rules.DistributedDisconnectRule.disconnect;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -37,7 +38,7 @@ import org.apache.geode.test.junit.rules.RuleList;
 import org.apache.geode.test.junit.runners.TestRunner;
 
 @Category(DistributedTest.class)
-public class DistributedDisconnectRuleAsClassRuleTest {
+public class DistributedDisconnectRuleAsClassRuleDistributedTest {
 
   @After
   public void tearDown() throws Exception {
@@ -105,7 +106,7 @@ public class DistributedDisconnectRuleAsClassRuleTest {
       assertThat(cache.isClosed()).isFalse();
       assertThat(cache.getInternalDistributedSystem().isConnected()).isTrue();
 
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> {
           cache = (InternalCache) new CacheFactory().create();
           assertThat(cache.isClosed()).isFalse();
@@ -119,7 +120,7 @@ public class DistributedDisconnectRuleAsClassRuleTest {
       assertThat(cache.isClosed()).isFalse();
       assertThat(cache.getInternalDistributedSystem().isConnected()).isTrue();
 
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> {
           assertThat(cache.isClosed()).isFalse();
           assertThat(cache.getInternalDistributedSystem().isConnected()).isTrue();
@@ -146,7 +147,7 @@ public class DistributedDisconnectRuleAsClassRuleTest {
       assertThat(cache.isClosed()).isFalse();
       assertThat(cache.getInternalDistributedSystem().isConnected()).isTrue();
 
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> {
           cache = (InternalCache) new CacheFactory().create();
           assertThat(cache.isClosed()).isFalse();
@@ -160,7 +161,7 @@ public class DistributedDisconnectRuleAsClassRuleTest {
       assertThat(cache.isClosed()).isFalse();
       assertThat(cache.getInternalDistributedSystem().isConnected()).isTrue();
 
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         vm.invoke(() -> {
           assertThat(cache.isClosed()).isFalse();
           assertThat(cache.getInternalDistributedSystem().isConnected()).isTrue();

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

[geode] 07/15: GEODE-4876: Update RegionListenerDistributedTest use of VM and NetworkUtil

Posted by kl...@apache.org.
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 b069c1fa0121b4ce6a1ffbe206be699d89ade41f
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:32:50 2018 -0700

    GEODE-4876: Update RegionListenerDistributedTest use of VM and NetworkUtil
    
    * rename RegionListenerDUnitTest as RegionListenerDistributedTest
---
 ...Test.java => RegionListenerDistributedTest.java} | 21 ++++++++++-----------
 1 file changed, 10 insertions(+), 11 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/RegionListenerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/RegionListenerDistributedTest.java
similarity index 85%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/RegionListenerDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/RegionListenerDistributedTest.java
index 1c1e821..46d2889 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/RegionListenerDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/RegionListenerDistributedTest.java
@@ -14,9 +14,9 @@
  */
 package org.apache.geode.internal.cache;
 
+import static org.apache.geode.test.dunit.VM.getVM;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.junit.Assert.fail;
 
 import java.io.Serializable;
 import java.util.Set;
@@ -30,14 +30,14 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.CacheRule;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
-public class RegionListenerDUnitTest implements Serializable {
+@SuppressWarnings("serial")
+public class RegionListenerDistributedTest implements Serializable {
 
   @ClassRule
   public static DistributedTestRule distributedTestRule = new DistributedTestRule();
@@ -47,9 +47,8 @@ public class RegionListenerDUnitTest implements Serializable {
 
   @Test
   public void testCleanupFailedInitializationInvoked() {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
+    VM vm0 = getVM(0);
+    VM vm1 = getVM(1);
 
     // Add RegionListener in both members
     vm0.invoke(() -> installRegionListener());
@@ -68,11 +67,11 @@ public class RegionListenerDUnitTest implements Serializable {
   }
 
   private void installRegionListener() {
-    this.cacheRule.getCache().addRegionListener(new TestRegionListener());
+    cacheRule.getCache().addRegionListener(new TestRegionListener());
   }
 
   private void createRegion(String regionName, boolean addAsyncEventQueueId) {
-    RegionFactory rf = this.cacheRule.getCache().createRegionFactory(RegionShortcut.REPLICATE);
+    RegionFactory rf = cacheRule.getCache().createRegionFactory(RegionShortcut.REPLICATE);
     if (addAsyncEventQueueId) {
       rf.addAsyncEventQueueId("aeqId");
     }
@@ -80,13 +79,13 @@ public class RegionListenerDUnitTest implements Serializable {
   }
 
   private void createRegion(String regionName, Class exception) {
-    RegionFactory rf = this.cacheRule.getCache().createRegionFactory(RegionShortcut.REPLICATE)
+    RegionFactory rf = cacheRule.getCache().createRegionFactory(RegionShortcut.REPLICATE)
         .addAsyncEventQueueId("aeqId");
     assertThatThrownBy(() -> rf.create(regionName)).isInstanceOf(exception);
   }
 
   private void verifyRegionListenerCleanupFailedInitializationInvoked() {
-    Set<RegionListener> listeners = this.cacheRule.getCache().getRegionListeners();
+    Set<RegionListener> listeners = cacheRule.getCache().getRegionListeners();
     assertThat(listeners.size()).isEqualTo(1);
     RegionListener listener = listeners.iterator().next();
     assertThat(listener).isInstanceOf(TestRegionListener.class);
@@ -104,7 +103,7 @@ public class RegionListenerDUnitTest implements Serializable {
     }
 
     public boolean getCleanupFailedInitializationInvoked() {
-      return this.cleanupFailedInitializationInvoked.get();
+      return cleanupFailedInitializationInvoked.get();
     }
   }
 }

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

[geode] 04/15: GEODE-4876: Update RegionClearStatsDistributedTest use of VM and NetworkUtils

Posted by kl...@apache.org.
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 e55082a280ad65d47933400dcc4a575834fe7cae
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:27:37 2018 -0700

    GEODE-4876: Update RegionClearStatsDistributedTest use of VM and NetworkUtils
    
    * rename CacheRegionClearStatsDUnitTest as RegionClearStatsDistributedTest
    * cleanup test
---
 .../cache/CacheRegionClearStatsDUnitTest.java      | 241 ---------------------
 .../cache/RegionClearStatsDistributedTest.java     | 179 +++++++++++++++
 2 files changed, 179 insertions(+), 241 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/cache/CacheRegionClearStatsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/CacheRegionClearStatsDUnitTest.java
deleted file mode 100755
index 41704d8..0000000
--- a/geode-core/src/test/java/org/apache/geode/cache/CacheRegionClearStatsDUnitTest.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.cache;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-import java.util.Properties;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.cache.client.PoolManager;
-import org.apache.geode.cache.client.internal.PoolImpl;
-import org.apache.geode.cache.server.CacheServer;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.internal.AvailablePort;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.test.dunit.Assert;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.NetworkUtils;
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
-
-/**
- * verifies the count of clear operation
- */
-@Category(DistributedTest.class)
-public class CacheRegionClearStatsDUnitTest extends JUnit4DistributedTestCase {
-
-  /** the cache */
-  private static GemFireCacheImpl cache = null;
-
-  private VM server1 = null;
-
-  private static VM client1 = null;
-
-  /** name of the test region */
-  private static final String REGION_NAME = "CacheRegionClearStatsDUnitTest_Region";
-
-  private static final String k1 = "k1";
-
-  private static final String k2 = "k2";
-
-  private static final String client_k1 = "client-k1";
-
-  private static final String client_k2 = "client-k2";
-
-  private static final int clearOp = 2;
-
-  @Override
-  public final void postSetUp() throws Exception {
-    final Host host = Host.getHost(0);
-    server1 = host.getVM(0);
-    client1 = host.getVM(1);
-  }
-
-  private void createCache(Properties props) throws Exception {
-    DistributedSystem ds = getSystem(props);
-    ds.disconnect();
-    ds = getSystem(props);
-    assertNotNull(ds);
-    cache = (GemFireCacheImpl) CacheFactory.create(ds);
-    assertNotNull(cache);
-  }
-
-  public static void createClientCache(String host, Integer port1) throws Exception {
-    new CacheRegionClearStatsDUnitTest();
-    Properties props = new Properties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(LOCATORS, "");
-    new CacheRegionClearStatsDUnitTest().createCache(props);
-    PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(host, port1.intValue())
-        .setSubscriptionEnabled(false).setThreadLocalConnections(true).setMinConnections(1)
-        .setReadTimeout(20000).setPingInterval(10000).setRetryAttempts(1)
-        .create("CacheRegionClearStatsDUnitTest");
-
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setPoolName(p.getName());
-
-    RegionAttributes attrs = factory.create();
-    Region region = cache.createRegion(REGION_NAME, attrs);
-    // region.registerInterest("ALL_KEYS");
-  }
-
-  public static Integer createServerCacheDisk() throws Exception {
-    return createCache(DataPolicy.PERSISTENT_REPLICATE);
-  }
-
-  private static Integer createCache(DataPolicy dataPolicy) throws Exception {
-    new CacheRegionClearStatsDUnitTest().createCache(new Properties());
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setDataPolicy(dataPolicy);
-    RegionAttributes attrs = factory.create();
-    cache.createRegion(REGION_NAME, attrs);
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    CacheServer server1 = cache.addCacheServer();
-    server1.setPort(port);
-    server1.setNotifyBySubscription(true);
-    server1.start();
-    return new Integer(server1.getPort());
-  }
-
-  public static Integer createServerCache() throws Exception {
-    return createCache(DataPolicy.REPLICATE);
-  }
-
-  public static void createClientCacheDisk(String host, Integer port1) throws Exception {
-    new CacheRegionClearStatsDUnitTest();
-    Properties props = new Properties();
-    props.setProperty(MCAST_PORT, "0");
-    props.setProperty(LOCATORS, "");
-    new CacheRegionClearStatsDUnitTest().createCache(props);
-    PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(host, port1.intValue())
-        .setSubscriptionEnabled(false).setThreadLocalConnections(true).setMinConnections(1)
-        .setReadTimeout(20000).setPingInterval(10000).setRetryAttempts(1)
-        .create("CacheRegionClearStatsDUnitTest");
-
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setPoolName(p.getName());
-    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
-    RegionAttributes attrs = factory.create();
-    Region region = cache.createRegion(REGION_NAME, attrs);
-    // region.registerInterest("ALL_KEYS");
-  }
-
-  /**
-   * This test does the following (<b> clear stats counter </b>):<br>
-   * 1)Verifies that clear operation count matches with stats count<br>
-   */
-  @Test
-  public void testClearStatsWithNormalRegion() {
-    Integer port1 =
-        ((Integer) server1.invoke(() -> CacheRegionClearStatsDUnitTest.createServerCache()));
-
-    client1.invoke(() -> CacheRegionClearStatsDUnitTest
-        .createClientCache(NetworkUtils.getServerHostName(server1.getHost()), port1));
-    client1.invoke(() -> CacheRegionClearStatsDUnitTest.put());
-
-    try {
-      Thread.sleep(10000);
-    } catch (Exception e) {
-      // sleep
-    }
-
-    client1.invoke(() -> CacheRegionClearStatsDUnitTest.validationClearStat());
-
-    server1.invoke(() -> CacheRegionClearStatsDUnitTest.validationClearStat());
-  }
-
-  /**
-   * This test does the following (<b> clear stats counter when disk involved </b>):<br>
-   * 1)Verifies that clear operation count matches with stats count <br>
-   */
-  @Test
-  public void testClearStatsWithDiskRegion() {
-    Integer port1 =
-        ((Integer) server1.invoke(() -> CacheRegionClearStatsDUnitTest.createServerCacheDisk()));
-
-    client1.invoke(() -> CacheRegionClearStatsDUnitTest
-        .createClientCacheDisk(NetworkUtils.getServerHostName(server1.getHost()), port1));
-    client1.invoke(() -> CacheRegionClearStatsDUnitTest.put());
-
-    try {
-      Thread.sleep(10000);
-    } catch (Exception e) {
-      // sleep
-    }
-
-    client1.invoke(() -> CacheRegionClearStatsDUnitTest.validationClearStat());
-
-    server1.invoke(() -> CacheRegionClearStatsDUnitTest.validationClearStat());
-  }
-
-  @Override
-  public final void preTearDown() throws Exception {
-    client1.invoke(() -> CacheRegionClearStatsDUnitTest.closeCache());
-    // then close the servers
-    server1.invoke(() -> CacheRegionClearStatsDUnitTest.closeCache());
-  }
-
-  public static void closeCache() {
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache.getDistributedSystem().disconnect();
-    }
-  }
-
-  public static void put() {
-    try {
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-      assertNotNull(r1);
-
-      r1.put(k1, client_k1);
-      assertEquals(r1.getEntry(k1).getValue(), client_k1);
-      r1.put(k2, client_k2);
-      assertEquals(r1.getEntry(k2).getValue(), client_k2);
-      try {
-        Thread.sleep(10000);
-      } catch (Exception e) {
-        // sleep
-      }
-      r1.clear();
-
-      r1.put(k1, client_k1);
-      assertEquals(r1.getEntry(k1).getValue(), client_k1);
-      r1.put(k2, client_k2);
-      assertEquals(r1.getEntry(k2).getValue(), client_k2);
-      try {
-        Thread.sleep(10000);
-      } catch (Exception e) {
-        // sleep
-      }
-      r1.clear();
-    } catch (Exception ex) {
-      Assert.fail("failed while put", ex);
-    }
-  }
-
-  public static void validationClearStat() {
-    assertEquals(cache.getCachePerfStats().getClearCount(), clearOp);
-  }
-
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/cache/RegionClearStatsDistributedTest.java b/geode-core/src/test/java/org/apache/geode/cache/RegionClearStatsDistributedTest.java
new file mode 100755
index 0000000..1b955e2
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/cache/RegionClearStatsDistributedTest.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.cache;
+
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+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;
+import java.io.Serializable;
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.client.PoolManager;
+import org.apache.geode.cache.client.internal.PoolImpl;
+import org.apache.geode.cache.server.CacheServer;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.CacheRule;
+import org.apache.geode.test.dunit.rules.DistributedTestRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+
+/**
+ * verifies the count of clear operation
+ */
+@Category(DistributedTest.class)
+@SuppressWarnings("serial")
+public class RegionClearStatsDistributedTest implements Serializable {
+
+  private static final String REGION_NAME = RegionClearStatsDistributedTest.class.getSimpleName();
+  private static final String KEY1 = "k1";
+  private static final String KEY2 = "k2";
+  private static final String VALUE1 = "client-k1";
+  private static final String VALUE2 = "client-k2";
+  private static final int EXPECTED_CLEAR_COUNT_STAT_VALUE = 2;
+
+  private VM server1;
+  private VM client1;
+
+  @ClassRule
+  public static DistributedTestRule distributedTestRule = new DistributedTestRule();
+
+  @Rule
+  public CacheRule cacheRule = new CacheRule();
+
+  @Before
+  public void setUp() throws Exception {
+    server1 = getVM(0);
+    client1 = getVM(1);
+  }
+
+  /**
+   * Verifies that clear operation count matches with stats count
+   */
+  @Test
+  public void testClearStatsWithNormalRegion() {
+    int port = server1.invoke(() -> createServerCache());
+    client1.invoke(() -> createClientCache(getServerHostName(), port));
+
+    client1.invoke(() -> doPutsAndClear());
+    client1.invoke(() -> doPutsAndClear());
+
+    client1.invoke(() -> validateClearCountStat());
+    server1.invoke(() -> validateClearCountStat());
+  }
+
+  /**
+   * Verifies that clear operation count matches with stats count with persistence
+   */
+  @Test
+  public void testClearStatsWithDiskRegion() {
+    int port = server1.invoke(() -> createServerCacheWithPersistence());
+    client1.invoke(() -> createClientCacheWithPersistence(getServerHostName(), port));
+
+    client1.invoke(() -> doPutsAndClear());
+    client1.invoke(() -> doPutsAndClear());
+
+    client1.invoke(() -> validateClearCountStat());
+    server1.invoke(() -> validateClearCountStat());
+  }
+
+  private int createCache(DataPolicy dataPolicy) throws IOException {
+    cacheRule.createCache();
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setDataPolicy(dataPolicy);
+
+    cacheRule.getCache().createRegion(REGION_NAME, factory.create());
+
+    CacheServer server1 = cacheRule.getCache().addCacheServer();
+    server1.setPort(0);
+    server1.setNotifyBySubscription(true);
+    server1.start();
+    return server1.getPort();
+  }
+
+  private int createServerCacheWithPersistence() throws IOException {
+    return createCache(DataPolicy.PERSISTENT_REPLICATE);
+  }
+
+  private int createServerCache() throws IOException {
+    return createCache(DataPolicy.REPLICATE);
+  }
+
+  private void createClientCache(String host, int port) {
+    Properties config = new Properties();
+    config.setProperty(MCAST_PORT, "0");
+    config.setProperty(LOCATORS, "");
+
+    cacheRule.createCache(config);
+
+    PoolImpl pool =
+        (PoolImpl) PoolManager.createFactory().addServer(host, port).setSubscriptionEnabled(false)
+            .setThreadLocalConnections(true).setMinConnections(1).setReadTimeout(20000)
+            .setPingInterval(10000).setRetryAttempts(1).create(getClass().getSimpleName());
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setPoolName(pool.getName());
+
+    cacheRule.getCache().createRegion(REGION_NAME, factory.create());
+  }
+
+  private void createClientCacheWithPersistence(String host, int port) {
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
+
+    cacheRule.createCache(props);
+
+    PoolImpl pool =
+        (PoolImpl) PoolManager.createFactory().addServer(host, port).setSubscriptionEnabled(false)
+            .setThreadLocalConnections(true).setMinConnections(1).setReadTimeout(20000)
+            .setPingInterval(10000).setRetryAttempts(1).create(getClass().getSimpleName());
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setPoolName(pool.getName());
+    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+
+    cacheRule.getCache().createRegion(REGION_NAME, factory.create());
+  }
+
+  private void doPutsAndClear() {
+    Region region = cacheRule.getCache().getRegion(REGION_NAME);
+
+    region.put(KEY1, VALUE1);
+    region.put(KEY2, VALUE2);
+
+    assertThat(region.getEntry(KEY1).getValue()).isEqualTo(VALUE1);
+    assertThat(region.getEntry(KEY2).getValue()).isEqualTo(VALUE2);
+
+    region.clear();
+  }
+
+  private void validateClearCountStat() {
+    assertThat(cacheRule.getCache().getCachePerfStats().getClearCount())
+        .isEqualTo(EXPECTED_CLEAR_COUNT_STAT_VALUE);
+  }
+}

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

[geode] 02/15: GEODE-4876: Remove use of Host from GetPidAndIdDistributedTest

Posted by kl...@apache.org.
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 034a279f67cd949333141ef8a729d567b6fe295b
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Mon Mar 26 17:14:57 2018 -0700

    GEODE-4876: Remove use of Host from GetPidAndIdDistributedTest
---
 .../geode/test/dunit/tests/GetPidAndIdDistributedTest.java    | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetPidAndIdDistributedTest.java b/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetPidAndIdDistributedTest.java
index 1d1df64..b389538 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetPidAndIdDistributedTest.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/tests/GetPidAndIdDistributedTest.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.test.dunit.tests;
 
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMCount;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import org.junit.ClassRule;
@@ -21,7 +23,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.internal.process.ProcessUtils;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedTestRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
@@ -34,16 +35,16 @@ public class GetPidAndIdDistributedTest {
 
   @Test
   public void getId_returnsVMSequentialId() throws Exception {
-    for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-      VM vm = Host.getHost(0).getVM(i);
+    for (int i = 0; i < getVMCount(); i++) {
+      VM vm = getVM(i);
       assertThat(vm.getId()).isEqualTo(i);
     }
   }
 
   @Test
   public void getPid_returnsVMProcessId() throws Exception {
-    for (int i = 0; i < Host.getHost(0).getVMCount(); i++) {
-      VM vm = Host.getHost(0).getVM(i);
+    for (int i = 0; i < getVMCount(); i++) {
+      VM vm = getVM(i);
       int remotePid = vm.invoke(() -> ProcessUtils.identifyPid());
       assertThat(vm.getPid()).isEqualTo(remotePid);
     }

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