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

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

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.