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/04/23 05:46:42 UTC

[geode] 04/13: GEODE-5033: Improve TearDown of DistributedTestRule

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 3849147c309bfad9e81ea0cf21dfe9905aee0618
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Thu Apr 12 12:39:10 2018 -0700

    GEODE-5033: Improve TearDown of DistributedTestRule
    
    Change Distributed rules to extend AbstractDistributedTestRule to ensure
    that all DUnit VMs were launched prior to setUp of these rules.
    
    Add DistributedTestRule.TearDown which includes everything that the
    old DistributedTestCase handled during tearDown.
    
    Using DistributedTestRule as a non-static Rule is probably the best way
    to use it going forward. If you use it as a static ClassRule then you
    should also register DistributedTestRule.TearDown as a non-static Rule
    so that removal of IgnoredExceptions and grep for suspect strings are
    both properly invoked after every test method.
---
 .../dunit/cache/internal/JUnit4CacheTestCase.java  |   5 +-
 .../dunit/internal/JUnit4DistributedTestCase.java  |   2 +-
 .../dunit/rules/AbstractDistributedTestRule.java   |  94 +++++++++++
 .../apache/geode/test/dunit/rules/CacheRule.java   | 105 +++++++++++--
 .../test/dunit/rules/CleanupDUnitVMsRule.java      |   5 -
 .../geode/test/dunit/rules/ClientCacheRule.java    |   6 +-
 .../test/dunit/rules/DistributedDiskDirRule.java   |   2 +
 .../rules/DistributedRestoreSystemProperties.java  |  37 +----
 .../test/dunit/rules/DistributedTestRule.java      | 172 +++++++++++++++++++--
 .../DistributedUseJacksonForJsonPathRule.java      |  20 ++-
 .../geode/test/dunit/rules/SharedCountersRule.java |  13 +-
 .../test/dunit/rules/SharedErrorCollector.java     |  46 ++----
 12 files changed, 392 insertions(+), 115 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/cache/internal/JUnit4CacheTestCase.java b/geode-core/src/test/java/org/apache/geode/test/dunit/cache/internal/JUnit4CacheTestCase.java
index c93f921..fa9a6f2 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/cache/internal/JUnit4CacheTestCase.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/cache/internal/JUnit4CacheTestCase.java
@@ -41,6 +41,7 @@ import org.apache.geode.cache.TimeoutException;
 import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientCacheFactory;
 import org.apache.geode.cache.client.PoolManager;
+import org.apache.geode.cache.client.internal.InternalClientCache;
 import org.apache.geode.cache30.CacheSerializableRunnable;
 import org.apache.geode.distributed.internal.DistributionMessageObserver;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
@@ -262,8 +263,8 @@ public abstract class JUnit4CacheTestCase extends JUnit4DistributedTestCase
     }
   }
 
-  public final ClientCache getClientCache() {
-    return (ClientCache) cache;
+  public final InternalClientCache getClientCache() {
+    return (InternalClientCache) cache;
   }
 
   /**
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/JUnit4DistributedTestCase.java b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/JUnit4DistributedTestCase.java
index 1ecbefe..4790e87 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/JUnit4DistributedTestCase.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/JUnit4DistributedTestCase.java
@@ -572,7 +572,7 @@ public abstract class JUnit4DistributedTestCase implements DistributedTestFixtur
     }
   }
 
-  private static final void cleanupAllVms() {
+  public static final void cleanupAllVms() {
     tearDownVM();
     invokeInEveryVM("tearDownVM", () -> tearDownVM());
     invokeInLocator(() -> {
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/AbstractDistributedTestRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/AbstractDistributedTestRule.java
new file mode 100644
index 0000000..fcba9c5
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/AbstractDistributedTestRule.java
@@ -0,0 +1,94 @@
+/*
+ * 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.test.dunit.rules;
+
+import static org.apache.geode.test.dunit.VM.DEFAULT_VM_COUNT;
+import static org.apache.geode.test.dunit.VM.getVMCount;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+import org.apache.geode.test.dunit.standalone.DUnitLauncher;
+import org.apache.geode.test.junit.rules.serializable.SerializableStatement;
+import org.apache.geode.test.junit.rules.serializable.SerializableTestRule;
+
+class AbstractDistributedTestRule implements SerializableTestRule {
+
+  private final int vmCount;
+  private final RemoteInvoker invoker;
+
+  private volatile int beforeVmCount;
+
+  protected AbstractDistributedTestRule() {
+    this(DEFAULT_VM_COUNT);
+  }
+
+  protected AbstractDistributedTestRule(final int vmCount) {
+    this(vmCount, new RemoteInvoker());
+  }
+
+  protected AbstractDistributedTestRule(final int vmCount, final RemoteInvoker invoker) {
+    this.vmCount = vmCount;
+    this.invoker = invoker;
+  }
+
+  @Override
+  public Statement apply(final Statement base, final Description description) {
+    return statement(base);
+  }
+
+  private Statement statement(final Statement base) {
+    return new SerializableStatement() {
+      @Override
+      public void evaluate() throws Throwable {
+        beforeDistributedTest();
+        before();
+        try {
+          base.evaluate();
+        } finally {
+          after();
+          afterDistributedTest();
+        }
+      }
+    };
+  }
+
+  private void beforeDistributedTest() throws Throwable {
+    DUnitLauncher.launchIfNeeded();
+    beforeVmCount = getVMCount();
+  }
+
+  private void afterDistributedTest() throws Throwable {
+    int afterVmCount = getVMCount();
+    assertThat(afterVmCount).isEqualTo(beforeVmCount);
+  }
+
+  protected void before() throws Throwable {
+    // override
+  }
+
+  protected void after() throws Throwable {
+    // override
+  }
+
+  protected RemoteInvoker invoker() {
+    return invoker;
+  }
+
+  protected int vmCount() {
+    return vmCount;
+  }
+}
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 990f45c..9b2bbf5 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
@@ -14,35 +14,61 @@
  */
 package org.apache.geode.test.dunit.rules;
 
-import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.test.dunit.DistributedTestUtils.getLocators;
+import static org.apache.geode.test.dunit.Disconnect.disconnectAllFromDS;
+import static org.apache.geode.test.dunit.standalone.DUnitLauncher.getDistributedSystemProperties;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
+import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.Region;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.cache.HARegion;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.test.dunit.Disconnect;
+import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.test.dunit.VM;
 
 /**
  * JUnit Rule that creates Cache instances in DistributedTest VMs without {@code CacheTestCase}.
  *
  * <p>
- * {@code CacheRule} follows the standard convention of using a {@code Builder} for configuration as
- * introduced in the JUnit {@code Timeout} rule.
+ * {@code CacheRule} can be used in DistributedTests as a {@code Rule}:
+ *
+ * <pre>
+ * {@literal @}Rule
+ * public DistributedTestRule distributedTestRule = new DistributedTestRule();
+ *
+ * {@literal @}Rule
+ * public CacheRule cacheRule = new CacheRule();
+ *
+ * {@literal @}Before
+ * public void setUp() {
+ *   getVM(0).invoke(() -> cacheRule.createCache(new CacheFactory().setPdxDiskStore(myDiskStore))));
+ * }
+ *
+ * {@literal @}Test
+ * public void createRegionWithRegionFactory() {
+ *   getVM(0).invoke(() -> {
+ *     RegionFactory regionFactory = cacheRule.getCache().createRegionFactory();
+ *     ...
+ *   });
+ * }
+ * </pre>
  *
  * <p>
- * {@code CacheRule} can be used in DistributedTests as a {@code Rule}:
+ * {@link CacheRule.Builder} can be used to construct an instance with more options:
  *
  * <pre>
  * {@literal @}ClassRule
  * public static DistributedTestRule distributedTestRule = new DistributedTestRule();
  *
  * {@literal @}Rule
+ * public DistributedTestRule.TearDown tearDown = new DistributedTestRule.TearDown();
+ *
+ * {@literal @}Rule
  * public CacheRule cacheRule = CacheRule.builder().createCacheInAll().build();
  *
  * {@literal @}Test
@@ -55,17 +81,22 @@ import org.apache.geode.test.dunit.VM;
  * </pre>
  */
 @SuppressWarnings({"serial", "unused"})
-public class CacheRule extends DistributedExternalResource {
+public class CacheRule extends AbstractDistributedTestRule {
 
   private static volatile InternalCache cache;
 
   private final boolean createCacheInAll;
   private final boolean createCache;
   private final boolean disconnectAfter;
+  private final boolean destroyRegions;
+  private final boolean replaceConfig;
   private final List<VM> createCacheInVMs;
   private final Properties config;
   private final Properties systemProperties;
 
+  /**
+   * Use {@code Builder} for more options in constructing {@code CacheRule}.
+   */
   public static Builder builder() {
     return new Builder();
   }
@@ -78,6 +109,8 @@ public class CacheRule extends DistributedExternalResource {
     createCacheInAll = builder.createCacheInAll;
     createCache = builder.createCache;
     disconnectAfter = builder.disconnectAfter;
+    destroyRegions = builder.destroyRegions;
+    replaceConfig = builder.replaceConfig;
     createCacheInVMs = builder.createCacheInVMs;
     config = builder.config;
     systemProperties = builder.systemProperties;
@@ -86,13 +119,13 @@ public class CacheRule extends DistributedExternalResource {
   @Override
   protected void before() {
     if (createCacheInAll) {
-      invoker().invokeInEveryVMAndController(() -> createCache(config, systemProperties));
+      invoker().invokeInEveryVMAndController(() -> createCache(config(), systemProperties));
     } else {
       if (createCache) {
-        createCache(config, systemProperties);
+        createCache(config(), systemProperties);
       }
       for (VM vm : createCacheInVMs) {
-        vm.invoke(() -> createCache(config, systemProperties));
+        vm.invoke(() -> createCache(config(), systemProperties));
       }
     }
   }
@@ -103,8 +136,17 @@ public class CacheRule extends DistributedExternalResource {
     invoker().invokeInEveryVMAndController(() -> closeAndNullCache());
 
     if (disconnectAfter) {
-      Disconnect.disconnectAllFromDS();
+      disconnectAllFromDS();
+    }
+  }
+
+  private Properties config() {
+    if (replaceConfig) {
+      return config;
     }
+    Properties allConfig = getDistributedSystemProperties();
+    allConfig.putAll(config);
+    return allConfig;
   }
 
   public InternalCache getCache() {
@@ -116,7 +158,7 @@ public class CacheRule extends DistributedExternalResource {
   }
 
   public void createCache() {
-    cache = (InternalCache) new CacheFactory(config).create();
+    cache = (InternalCache) new CacheFactory(config()).create();
   }
 
   public void createCache(final CacheFactory cacheFactory) {
@@ -140,14 +182,17 @@ public class CacheRule extends DistributedExternalResource {
     return cache;
   }
 
-  private static void closeAndNullCache() {
+  private void closeAndNullCache() {
     closeCache();
     nullCache();
   }
 
-  private static void closeCache() {
+  private void closeCache() {
     try {
       if (cache != null) {
+        if (destroyRegions) {
+          destroyRegions(cache);
+        }
         cache.close();
       }
     } catch (Exception ignored) {
@@ -159,6 +204,23 @@ public class CacheRule extends DistributedExternalResource {
     cache = null;
   }
 
+  private static void destroyRegions(final Cache cache) {
+    if (cache != null && !cache.isClosed()) {
+      // try to destroy the root regions first so that we clean up any persistent files.
+      for (Region<?, ?> root : cache.rootRegions()) {
+        String regionFullPath = root == null ? null : root.getFullPath();
+        // for colocated regions you can't locally destroy a partitioned region.
+        if (root.isDestroyed() || root instanceof HARegion || root instanceof PartitionedRegion) {
+          continue;
+        }
+        try {
+          root.localDestroyRegion("CacheRule_tearDown");
+        } catch (Exception ignore) {
+        }
+      }
+    }
+  }
+
   /**
    * Builds an instance of CacheRule.
    */
@@ -167,12 +229,14 @@ public class CacheRule extends DistributedExternalResource {
     private boolean createCacheInAll;
     private boolean createCache;
     private boolean disconnectAfter;
+    private boolean destroyRegions;
+    private boolean replaceConfig;
     private List<VM> createCacheInVMs = new ArrayList<>();
     private Properties config = new Properties();
     private Properties systemProperties = new Properties();
 
     public Builder() {
-      config.setProperty(LOCATORS, getLocators());
+      // nothing
     }
 
     /**
@@ -210,8 +274,19 @@ public class CacheRule extends DistributedExternalResource {
       return this;
     }
 
+    /**
+     * Destroy all Regions before closing the Cache. This will cleanup the presence of each Region
+     * in DiskStores, but this is not needed if the disk files are on a TemporaryFolder. Default is
+     * false.
+     */
+    public Builder destroyRegions() {
+      destroyRegions = true;
+      return this;
+    }
+
     public Builder replaceConfig(final Properties config) {
       this.config = config;
+      replaceConfig = true;
       return this;
     }
 
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/CleanupDUnitVMsRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/CleanupDUnitVMsRule.java
index ea1cd49..73ccad7 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/CleanupDUnitVMsRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/CleanupDUnitVMsRule.java
@@ -14,13 +14,8 @@
  */
 package org.apache.geode.test.dunit.rules;
 
-import static org.apache.geode.test.dunit.Host.getHost;
 import static org.apache.geode.test.dunit.Host.getHostCount;
 
-import java.io.Serializable;
-
-import org.junit.rules.ExternalResource;
-
 import org.apache.geode.test.dunit.VM;
 
 public class CleanupDUnitVMsRule extends ExternalResource implements Serializable {
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ClientCacheRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ClientCacheRule.java
index a97f302..9391e2b 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ClientCacheRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ClientCacheRule.java
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.test.dunit.rules;
 
-import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.test.dunit.DistributedTestUtils.getLocators;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.util.ArrayList;
@@ -53,7 +51,7 @@ import org.apache.geode.test.dunit.VM;
  * </pre>
  */
 @SuppressWarnings({"serial", "unused"})
-public class ClientCacheRule extends DistributedExternalResource {
+public class ClientCacheRule extends AbstractDistributedTestRule {
 
   private static volatile InternalClientCache clientCache;
 
@@ -163,7 +161,7 @@ public class ClientCacheRule extends DistributedExternalResource {
     private Properties systemProperties = new Properties();
 
     public Builder() {
-      config.setProperty(LOCATORS, getLocators());
+      // nothing
     }
 
     /**
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedDiskDirRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedDiskDirRule.java
index 1ada67b..7a25148 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedDiskDirRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedDiskDirRule.java
@@ -31,6 +31,7 @@ import org.junit.rules.TestName;
 import org.junit.runner.Description;
 
 import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.standalone.DUnitLauncher;
 import org.apache.geode.test.junit.rules.DiskDirRule;
 import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
 import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
@@ -124,6 +125,7 @@ public class DistributedDiskDirRule extends DiskDirRule implements SerializableT
 
   @Override
   protected void before(Description description) throws Exception {
+    DUnitLauncher.launchIfNeeded();
     beforeVmCount = getVMCount();
 
     if (initializeHelperRules) {
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedRestoreSystemProperties.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedRestoreSystemProperties.java
index 2fcf128..d93502a 100755
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedRestoreSystemProperties.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedRestoreSystemProperties.java
@@ -14,48 +14,29 @@
  */
 package org.apache.geode.test.dunit.rules;
 
-import static org.apache.geode.test.dunit.Host.getHost;
-import static org.assertj.core.api.Assertions.assertThat;
-
 import org.apache.geode.test.junit.rules.accessible.AccessibleRestoreSystemProperties;
-import org.apache.geode.test.junit.rules.serializable.SerializableTestRule;
 
 /**
  * Distributed version of RestoreSystemProperties which affects all DUnit JVMs including the Locator
  * JVM.
  */
-public class DistributedRestoreSystemProperties extends AccessibleRestoreSystemProperties
-    implements SerializableTestRule {
+public class DistributedRestoreSystemProperties extends AbstractDistributedTestRule {
 
   private static final AccessibleRestoreSystemProperties restoreSystemProperties =
       new AccessibleRestoreSystemProperties();
 
-  private final RemoteInvoker invoker;
-
-  private volatile int beforeVmCount;
-
   public DistributedRestoreSystemProperties() {
-    this(new RemoteInvoker());
-  }
-
-  public DistributedRestoreSystemProperties(final RemoteInvoker invoker) {
-    super();
-    this.invoker = invoker;
+    // nothing
   }
 
   @Override
-  public void before() throws Throwable {
-    beforeVmCount = getVMCount();
-
-    invoker.invokeInEveryVMAndController(() -> invokeBefore());
+  public void before() throws Exception {
+    invoker().invokeInEveryVMAndController(() -> invokeBefore());
   }
 
   @Override
   public void after() {
-    int afterVmCount = getVMCount();
-    assertThat(afterVmCount).isEqualTo(beforeVmCount);
-
-    invoker.invokeInEveryVMAndController(() -> invokeAfter());
+    invoker().invokeInEveryVMAndController(() -> invokeAfter());
   }
 
   private void invokeBefore() throws Exception {
@@ -72,12 +53,4 @@ public class DistributedRestoreSystemProperties extends AccessibleRestoreSystemP
   private void invokeAfter() {
     restoreSystemProperties.after();
   }
-
-  private int getVMCount() {
-    try {
-      return getHost(0).getVMCount();
-    } catch (IllegalArgumentException e) {
-      throw new IllegalStateException("DUnit VMs have not been launched");
-    }
-  }
 }
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 c14968b..5dcb1e5 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,38 +14,90 @@
  */
 package org.apache.geode.test.dunit.rules;
 
+import static org.apache.geode.test.dunit.Disconnect.disconnectFromDS;
+import static org.apache.geode.test.dunit.DistributedTestUtils.unregisterInstantiatorsInThisVM;
+import static org.apache.geode.test.dunit.Invoke.invokeInEveryVM;
+import static org.apache.geode.test.dunit.Invoke.invokeInLocator;
 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.cache.query.QueryTestUtils;
+import org.apache.geode.cache.query.internal.QueryObserverHolder;
+import org.apache.geode.cache30.ClientServerTestCase;
+import org.apache.geode.cache30.GlobalLockingDUnitTest;
+import org.apache.geode.cache30.MultiVMRegionTestCase;
+import org.apache.geode.cache30.RegionTestCase;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionMessageObserver;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.tcpserver.TcpClient;
+import org.apache.geode.internal.admin.ClientStatsManager;
+import org.apache.geode.internal.cache.CacheServerLauncher;
+import org.apache.geode.internal.cache.DiskStoreObserver;
+import org.apache.geode.internal.cache.InitialImageOperation;
+import org.apache.geode.internal.cache.tier.InternalClientMembership;
+import org.apache.geode.internal.cache.tier.sockets.CacheServerTestUtil;
+import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
+import org.apache.geode.internal.cache.tier.sockets.Message;
+import org.apache.geode.internal.cache.xmlcache.CacheCreation;
+import org.apache.geode.internal.net.SocketCreator;
+import org.apache.geode.internal.net.SocketCreatorFactory;
+import org.apache.geode.management.internal.cli.LogWrapper;
+import org.apache.geode.pdx.internal.TypeRegistry;
+import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.standalone.DUnitLauncher;
+import org.apache.geode.test.junit.rules.serializable.SerializableExternalResource;
 
 /**
- * JUnit Rule that launches DistributedTest VMs without {@code DistributedTestCase}. Class may need
- * to implement {@code Serializable}.
+ * JUnit Rule that launches DistributedTest VMs without {@code DistributedTestCase}. Test class may
+ * need to implement {@code Serializable}.
  *
  * <p>
- * {@code DistributedTestRule} follows the standard convention of using a {@code Builder} for
- * configuration as introduced in the JUnit {@code Timeout} rule.
- *
- * <p>
- * {@code DistributedTestRule} can be used in DistributedTests as a {@code ClassRule}:
+ * {@code DistributedTestRule} can be used in DistributedTests as a {@code ClassRule}. This ensures
+ * that DUnit VMs will be available to non-Class {@code Rule}s. Unfortunately, you will need to
+ * declare {@code DistributedTestRule.TearDown} as a non-Class {@code Rule}. Without
+ * {@code DistributedTestRule.TearDown} grep for suspect strings will not be invoked after each
+ * test.
  *
  * <pre>
  * {@literal @}ClassRule
  * public static DistributedTestRule distributedTestRule = new DistributedTestRule();
  *
+ * {@literal @}Rule
+ * public DistributedTestRule.TearDown tearDownRule = new DistributedTestRule.TearDown();
+ *
+ * {@literal @}Test
+ * public void shouldHaveFourDUnitVMsByDefault() {
+ *   assertThat(getVMCount()).isEqualTo(4);
+ * }
+ * </pre>
+ *
+ * <p>
+ * Or as a non-Class {@code Rule}. This usage does <bold>not</bold> require separate declaration of
+ * {@code DistributedTestRule.TearDown}:
+ *
+ * <pre>
+ * {@literal @}Rule
+ * public DistributedTestRule distributedTestRule = new DistributedTestRule();
+ *
+ * {@literal @}Rule
+ * public DistributedTestRule.TearDown tearDownRule = new DistributedTestRule.TearDown();
+ *
  * {@literal @}Test
  * public void shouldHaveFourDUnitVMsByDefault() {
- *   assertThat(Host.getHost(0).getVMCount()).isEqualTo(4);
+ *   assertThat(getVMCount()).isEqualTo(4);
  * }
  * </pre>
  */
 @SuppressWarnings("unused")
-public class DistributedTestRule extends DistributedExternalResource {
+public class DistributedTestRule extends AbstractDistributedTestRule {
 
   private final int vmCount;
 
+  /**
+   * Use {@code Builder} for more options in constructing {@code DistributedTestRule}.
+   */
   public static Builder builder() {
     return new Builder();
   }
@@ -63,13 +115,21 @@ public class DistributedTestRule extends DistributedExternalResource {
   }
 
   @Override
-  protected void before() throws Throwable {
+  protected void before() throws Exception {
     DUnitLauncher.launchIfNeeded();
     for (int i = 0; i < vmCount; i++) {
       assertThat(getVM(i)).isNotNull();
     }
   }
 
+  @Override
+  protected void after() {
+    TearDown.doTearDown();
+  }
+
+  /**
+   * Builds an instance of CacheRule.
+   */
   public static class Builder {
 
     private int vmCount = DEFAULT_VM_COUNT;
@@ -86,4 +146,96 @@ public class DistributedTestRule extends DistributedExternalResource {
       return new DistributedTestRule(this);
     }
   }
+
+  /**
+   * Cleans up horrendous things like static state and non-default instances in Geode.
+   *
+   * <p>
+   * {@link DistributedTestRule#after()} invokes the same cleanup that this Rule does, but if you
+   * defined {@code DistributedTestRule} as a {@code ClassRule} then you should declare TearDown
+   * as a non-class {@code Rule} in your test:
+   *
+   * <pre>
+   * {@literal @}ClassRule
+   * public static DistributedTestRule distributedTestRule = new DistributedTestRule();
+   *
+   * {@literal @}Rule
+   * public DistributedTestRule.TearDown tearDownRule = new DistributedTestRule.TearDown();
+   *
+   * {@literal @}Test
+   * public void shouldHaveFourDUnitVMsByDefault() {
+   *   assertThat(getVMCount()).isEqualTo(4);
+   * }
+   * </pre>
+   *
+   * <p>
+   * Note: {@link CacheRule} handles its own cleanup of Cache and Regions.
+   */
+  public static class TearDown extends SerializableExternalResource {
+
+    @Override
+    protected void before() throws Exception {
+      // nothing
+    }
+
+    @Override
+    protected void after() {
+      doTearDown();
+    }
+
+    static void doTearDown() {
+      tearDownInVM();
+      invokeInEveryVM(() -> {
+        tearDownInVM();
+      });
+      invokeInLocator(() -> {
+        DistributionMessageObserver.setInstance(null);
+        unregisterInstantiatorsInThisVM();
+      });
+      DUnitLauncher.closeAndCheckForSuspects();
+    }
+
+    private static void tearDownInVM() {
+      // 1. Please do NOT add to this list. I'm trying to DELETE this list.
+      // 2. Instead, please add to the after() of your test or your rule.
+
+      disconnectFromDS();
+
+      // keep alphabetized to detect duplicate lines
+      CacheCreation.clearThreadLocals();
+      CacheServerLauncher.clearStatics();
+      CacheServerTestUtil.clearCacheReference();
+      ClientProxyMembershipID.system = null;
+      ClientServerTestCase.AUTO_LOAD_BALANCE = false;
+      ClientStatsManager.cleanupForTests();
+      DiskStoreObserver.setInstance(null);
+      unregisterInstantiatorsInThisVM();
+      DistributionMessageObserver.setInstance(null);
+      GlobalLockingDUnitTest.region_testBug32356 = null;
+      InitialImageOperation.slowImageProcessing = 0;
+      InternalClientMembership.unregisterAllListeners();
+      LogWrapper.close();
+      MultiVMRegionTestCase.CCRegion = null;
+      QueryObserverHolder.reset();
+      QueryTestUtils.setCache(null);
+      RegionTestCase.preSnapshotRegion = null;
+      SocketCreator.resetHostNameCache();
+      SocketCreator.resolve_dns = true;
+      TcpClient.clearStaticData();
+
+      // clear system properties -- keep alphabetized
+      System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + "log-level");
+      System.clearProperty("jgroups.resolve_dns");
+      System.clearProperty(Message.MAX_MESSAGE_SIZE_PROPERTY);
+
+      if (InternalDistributedSystem.systemAttemptingReconnect != null) {
+        InternalDistributedSystem.systemAttemptingReconnect.stopReconnecting();
+      }
+
+      IgnoredException.removeAllExpectedExceptions();
+      SocketCreatorFactory.close();
+      TypeRegistry.setPdxSerializer(null);
+      TypeRegistry.init();
+    }
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedUseJacksonForJsonPathRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedUseJacksonForJsonPathRule.java
index 8c85775..ca81b2c 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedUseJacksonForJsonPathRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/DistributedUseJacksonForJsonPathRule.java
@@ -14,14 +14,20 @@
  */
 package org.apache.geode.test.dunit.rules;
 
+import static org.apache.geode.test.dunit.VM.getVMCount;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.geode.test.dunit.standalone.DUnitLauncher;
 import org.apache.geode.test.junit.rules.UseJacksonForJsonPathRule;
 
 public class DistributedUseJacksonForJsonPathRule extends UseJacksonForJsonPathRule {
 
-  private static UseJacksonForJsonPathRule instance = new UseJacksonForJsonPathRule();
+  private static volatile UseJacksonForJsonPathRule instance = new UseJacksonForJsonPathRule();
 
   private final RemoteInvoker invoker;
 
+  private volatile int beforeVmCount;
+
   public DistributedUseJacksonForJsonPathRule() {
     this(new RemoteInvoker());
   }
@@ -32,19 +38,27 @@ public class DistributedUseJacksonForJsonPathRule extends UseJacksonForJsonPathR
 
   @Override
   public void before() {
-    this.invoker.invokeInEveryVMAndController(DistributedUseJacksonForJsonPathRule::invokeBefore);
+    DUnitLauncher.launchIfNeeded();
+    beforeVmCount = getVMCount();
+
+    invoker.invokeInEveryVMAndController(() -> invokeBefore());
   }
 
   @Override
   public void after() {
-    this.invoker.invokeInEveryVMAndController(DistributedUseJacksonForJsonPathRule::invokeAfter);
+    int afterVmCount = getVMCount();
+    assertThat(afterVmCount).isEqualTo(beforeVmCount);
+
+    invoker.invokeInEveryVMAndController(() -> invokeAfter());
   }
 
   private static void invokeBefore() {
+    instance = new UseJacksonForJsonPathRule();
     instance.before();
   }
 
   private static void invokeAfter() {
     instance.after();
+    instance = null;
   }
 }
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 f1dbfae..9aa0273 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
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.test.dunit.rules;
 
-import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.VM.getAllVMs;
 
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -56,7 +56,7 @@ import org.apache.geode.test.dunit.VM;
  * </pre>
  */
 @SuppressWarnings({"serial", "unused"})
-public class SharedCountersRule extends DistributedExternalResource {
+public class SharedCountersRule extends AbstractDistributedTestRule {
 
   private static volatile Map<Serializable, AtomicInteger> counters;
 
@@ -67,15 +67,10 @@ public class SharedCountersRule extends DistributedExternalResource {
   }
 
   public SharedCountersRule() {
-    this(new Builder(), new RemoteInvoker());
+    this(new Builder());
   }
 
   SharedCountersRule(final Builder builder) {
-    this(builder, new RemoteInvoker());
-  }
-
-  SharedCountersRule(final Builder builder, final RemoteInvoker invoker) {
-    super(invoker);
     idsToInitInBefore.addAll(builder.ids);
   }
 
@@ -148,7 +143,7 @@ public class SharedCountersRule extends DistributedExternalResource {
    */
   public int getTotal(final Serializable id) {
     int total = counters.get(id).get();
-    for (VM vm : getHost(0).getAllVMs()) {
+    for (VM vm : getAllVMs()) {
       total += vm.invoke(() -> counters.get(id).get());
     }
     return total;
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/SharedErrorCollector.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/SharedErrorCollector.java
index 93b9f94..350ea00 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/SharedErrorCollector.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/SharedErrorCollector.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.test.dunit.rules;
 
+import static org.apache.geode.test.dunit.VM.getAllVMs;
+
 import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.List;
@@ -21,12 +23,8 @@ import java.util.concurrent.Callable;
 
 import org.hamcrest.Matcher;
 import org.junit.rules.ErrorCollector;
-import org.junit.runner.Description;
-import org.junit.runners.model.Statement;
 
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.junit.rules.serializable.SerializableTestRule;
 
 /**
  * JUnit Rule that provides a shared ErrorCollector in all DistributedTest VMs. In particular, this
@@ -52,50 +50,31 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTestRule;
  * </pre>
  */
 @SuppressWarnings({"serial", "unused"})
-public class SharedErrorCollector implements SerializableTestRule {
+public class SharedErrorCollector extends AbstractDistributedTestRule {
 
   private static volatile ProtectedErrorCollector errorCollector;
 
-  private final RemoteInvoker invoker;
-
   public SharedErrorCollector() {
-    this(new RemoteInvoker());
-  }
-
-  SharedErrorCollector(final RemoteInvoker invoker) {
-    this.invoker = invoker;
+    // nothing
   }
 
   @Override
-  public Statement apply(final Statement base, Description description) {
-    return new Statement() {
-      @Override
-      public void evaluate() throws Throwable {
-        before();
-        try {
-          base.evaluate();
-        } finally {
-          after();
-        }
-      }
-    };
-  }
-
   protected void before() throws Throwable {
-    invoker.invokeInEveryVMAndController(() -> errorCollector = new ProtectedErrorCollector());
+    invoker().invokeInEveryVMAndController(() -> errorCollector = new ProtectedErrorCollector());
   }
 
+  @Override
   protected void after() throws Throwable {
     ProtectedErrorCollector allErrors = errorCollector;
     try {
-      for (VM vm : Host.getHost(0).getAllVMs()) {
+      for (VM vm : getAllVMs()) {
         List<Throwable> remoteFailures = new ArrayList<>();
         remoteFailures.addAll(vm.invoke(() -> errorCollector.errors()));
         for (Throwable t : remoteFailures) {
           allErrors.addError(t);
         }
       }
-      invoker.invokeInEveryVMAndController(() -> errorCollector = null);
+      invoker().invokeInEveryVMAndController(() -> errorCollector = null);
     } finally {
       allErrors.verify();
     }
@@ -135,20 +114,19 @@ public class SharedErrorCollector implements SerializableTestRule {
    */
   private static class ProtectedErrorCollector extends ErrorCollector {
 
-    protected final List<Throwable> protectedErrors;
+    private final List<Throwable> protectedErrors;
 
-    public ProtectedErrorCollector() {
-      super();
+    ProtectedErrorCollector() {
       try {
         Field superErrors = ErrorCollector.class.getDeclaredField("errors");
         superErrors.setAccessible(true);
-        this.protectedErrors = (List<Throwable>) superErrors.get(this);
+        protectedErrors = (List<Throwable>) superErrors.get(this);
       } catch (IllegalAccessException | NoSuchFieldException e) {
         throw new RuntimeException(e);
       }
     }
 
-    public List<Throwable> errors() {
+    List<Throwable> errors() {
       return protectedErrors;
     }
 

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