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 2022/06/01 21:19:47 UTC

[geode] branch support/1.15 updated: GEODE-10327: Overhaul GfshRule to kill processes and save artifacts (#7731)

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

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


The following commit(s) were added to refs/heads/support/1.15 by this push:
     new 6a0e744f1c GEODE-10327: Overhaul GfshRule to kill processes and save artifacts (#7731)
6a0e744f1c is described below

commit 6a0e744f1cbcca75c2a5a5b6465f010a3f135a8c
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Wed Jun 1 14:16:35 2022 -0700

    GEODE-10327: Overhaul GfshRule to kill processes and save artifacts (#7731)
    
    PROBLEM
    
    Tests that use GfshRule leave behind orphaned processes and do not save
    artifacts for debugging failures.
    
    SOLUTION
    
    GfshRule needs to cleanup all processes it forks. It also needs to save
    off all runtime artifacts such as logging, stats, pid files, diskstores
    to enable debugging of test failures.
    
    DETAILS
    
    Enhance GfshRule and modify all tests using it for proper debugging and
    to prevent test pollution.
    
    Overhaul of GfshRule:
    
    * kill ALL geode processes during cleanup
    * use FolderRule to ensure all logs and files are properly saved off
      when a test fails
    * extract GfshExecutor from JUnit rule code
    * GfshExecutor allows a test to use any number of Geode versions with
      just one GfshRule
    * add Gfsh log level support for easier debugging
    * add support for new VmConfiguration to allow control over Geode and
      Java versions
    * overhaul API of GfshRule and companion classes for better consistency
      and design
    
    New FolderRule:
    
    * replaces TemporaryFolder and saves off all content when a test fails
    * creates root directory under the gradle worker instead of under temp
    
    Update HTTP session caching module tests:
    
    * use new FolderRule to save all artifacts when a test fails
    * use nio Paths for filesystem variables
    
    Update acceptance and upgrade tests that use GfshRule:
    
    * use new improved GfshRule and GfshExecutor
    * use new FolderRule instead of TemporaryFolder to save all artifacts
      when a test fails
    * use --disable-default-server in tests with no clients
    * fix flakiness of many tests by using random ports instead of default
      or hardcoded port values
    * reformat GfshRule API usage in tests to improve readability and
      consistency
    * add GfshStopper to provide common place to await process stop (stop
      locator/server is async so restarting with same ports is very prone
      to hitting BindExceptions)
    
    Update ProcessUtils:
    
    * extract NativeProcessUtils and make it public for direct use
    * rename InternalProcessUtils as ProcessUtilsProvider and move to its
      own class
    * rethrow IOExceptions as UncheckedIOExceptions
    * fix flakiness in NativeProcessUtilsTest by moving findAvailablePid
      into test method
    
    Minor changes:
    
    * improve code formatting and readability
    * convert from old io File to nio Path APIs as much as possible
    * close output streams to fix filesystem issues on Windows
    
    Fixes flaky test tickets:
    
    * DeployJarAcceptanceTest GEODE-9615
    * possibly other tests that uses GfshRule
    
    Changes for resubmit:
    
    * log error message if unable to delete folder
    
    NOTES
    
    The jdk8, jdk17 and windows labels were used to run tests on more
    environments.
    
    This PR contains mostly test and framework changes. The only product
    code altered is ServerLauncher and several classes in
    org.apache.geode.internal.process, all of which is in geode-core.
    
    (cherry picked from commit 3f8f8db595ca4b99b25fe4d109a8ed118a712701)
---
 .../geode/session/tests/ContainerInstall.java      | 182 ++++-----
 .../geode/session/tests/ServerContainer.java       | 106 +++---
 .../geode/session/tests/TomcatContainer.java       |  45 ++-
 .../apache/geode/session/tests/TomcatInstall.java  |  99 +++--
 .../MissingDiskStoreAcceptanceTest.java            |  20 +-
 ...gDiskStoreAfterServerRestartAcceptanceTest.java |  61 ++--
 .../launchers/ServerStartupNotificationTest.java   |  40 +-
 .../geode/launchers/ServerStartupOnlineTest.java   |  44 +--
 ...rStartupRedundancyRecoveryNotificationTest.java |  27 +-
 ...ServerStartupValueRecoveryNotificationTest.java |  42 ++-
 .../LocatorWithCustomLogConfigAcceptanceTest.java  |  34 +-
 .../ServerWithCustomLogConfigAcceptanceTest.java   |  28 +-
 .../cli/commands/ConfigureEvictionThroughGfsh.java | 196 ++++++----
 .../cli/commands/DeployWithLargeJarTest.java       |  50 ++-
 .../cli/commands/DestroyIndexIfExistsTest.java     |  37 +-
 .../GfshStartLocatorLogAcceptanceTest.java         |  37 +-
 .../cli/commands/ImportClusterConfigTest.java      |  26 +-
 ...scribeConfigAreFullyRedactedAcceptanceTest.java |  73 ++--
 .../cli/commands/PutCommandWithJsonTest.java       |  35 +-
 .../cli/commands/StartLocatorAcceptanceTest.java   |  44 ++-
 .../commands/StartServerCommandAcceptanceTest.java | 138 ++++---
 .../cli/commands/StatusLocatorRealGfshTest.java    |  41 ++-
 .../cli/commands/StopServerAcceptanceTest.java     |  42 ++-
 .../StopServerWithSecurityAcceptanceTest.java      |  88 +++--
 .../cli/shell/GfshDisconnectWithinScript.java      |  20 +-
 .../shell/StatusLocatorExitCodeAcceptanceTest.java | 144 ++++----
 .../shell/StatusServerExitCodeAcceptanceTest.java  |  95 +++--
 .../rest/RegionManagementAcceptanceTest.java       |  34 +-
 ...tandaloneClientManagementAPIAcceptanceTest.java | 117 +++---
 .../apache/geode/metrics/CacheGetsTimerTest.java   |  48 +--
 .../geode/metrics/GatewayReceiverMetricsTest.java  |  76 ++--
 .../geode/metrics/MemberTypeCommonTagsTest.java    |  25 +-
 .../apache/geode/metrics/MicrometerBinderTest.java |  27 +-
 .../geode/metrics/RegionEntriesGaugeTest.java      |  76 ++--
 .../FunctionExecutionsTimerClusterTest.java        |  62 ++--
 .../FunctionExecutionsTimerLonerTest.java          |  45 ++-
 .../FunctionExecutionsTimerNoResultTest.java       |  37 +-
 .../geode/modules/DeployJarAcceptanceTest.java     | 406 +++++++++++++--------
 .../GfshRuleExampleTest.java}                      |  43 ++-
 ...artLocatorGlobalSerialFilterAcceptanceTest.java |  11 +-
 .../StartLocatorJmxSerialFilterAcceptanceTest.java |  11 +-
 ...tartServerGlobalSerialFilterAcceptanceTest.java |  30 +-
 .../StartServerJmxSerialFilterAcceptanceTest.java  |  11 +-
 .../apache/geode/ssl/CertificateRotationTest.java  | 115 +++---
 .../apache/geode/session/tests/CargoTestBase.java  |   2 +-
 .../session/tests/GenericAppServerContainer.java   |  23 +-
 .../session/tests/GenericAppServerInstall.java     |  21 +-
 .../Tomcat8ClientServerCustomCacheXmlTest.java     |   2 +-
 .../session/tests/TomcatClientServerTest.java      |   7 +-
 .../DeploymentManagementUpgradeTest.java           | 105 +++---
 .../management/OperationManagementUpgradeTest.java | 114 +++---
 .../RollingUpgradeWithGfshDUnitTest.java           | 130 ++++---
 .../management/RollingUpgradeWithSslDUnitTest.java | 137 ++++---
 .../cli/commands/ConnectCommandUpgradeTest.java    |  93 +++--
 .../Tomcat8ClientServerRollingUpgradeTest.java     | 213 +++++------
 ...omcatSessionBackwardsCompatibilityTestBase.java |   2 +-
 .../test/junit/rules/GfshExecutorVersionTest.java  |  53 +++
 .../test/junit/rules/GfshRuleUpgradeTest.java      |  85 -----
 .../ClusterConfigDeployJarDUnitTest.java           |  13 +-
 .../ClusterConfigImportDUnitTest.java              |  13 +-
 .../ClusterConfigStartMemberDUnitTest.java         |  16 +-
 .../geode/internal/process/AttachProcessUtils.java |   4 +-
 .../internal/process/FileControllableProcess.java  |   3 +
 .../internal/process/LocalProcessLauncher.java     |   9 +-
 .../geode/internal/process/NativeProcessUtils.java |   7 +-
 .../apache/geode/internal/process/ProcessType.java |   6 +
 .../geode/internal/process/ProcessUtils.java       |  29 +-
 .../internal/process/ProcessUtilsProvider.java     |  27 +-
 .../internal/process/NativeProcessUtilsTest.java   |  17 +-
 .../geode/ClusterCommunicationsDUnitTest.java      |   2 +-
 .../internal/net/SocketCreatorUpgradeTest.java     | 176 ++++-----
 .../configuration/ClusterConfigTestBase.java       |  59 ++-
 .../geode/gfsh/GfshWithSslAcceptanceTest.java      |  78 ++--
 .../apache/geode/codeAnalysis/excludedClasses.txt  |   1 +
 .../org/apache/geode/cache/ssl/CertStores.java     |  25 +-
 .../org/apache/geode/test/junit/rules/Folder.java  |  51 +++
 .../geode/test/junit/rules/FolderFactory.java      |  32 +-
 .../apache/geode/test/junit/rules/FolderRule.java  |  97 +++++
 .../test/junit/rules/ResourceReporterRule.java     | 100 +++++
 .../geode/test/junit/rules/gfsh/GfshContext.java   | 305 ++++++++++++++++
 .../geode/test/junit/rules/gfsh/GfshExecution.java | 220 +++++------
 .../geode/test/junit/rules/gfsh/GfshExecutor.java  |  27 +-
 .../geode/test/junit/rules/gfsh/GfshRule.java      | 290 +++++----------
 .../geode/test/junit/rules/gfsh/GfshScript.java    |  22 +-
 .../geode/test/junit/rules/gfsh/GfshStopper.java   |  93 +++++
 .../serializable/SerializableTemporaryFolder.java  |  16 +-
 .../org/apache/geode/test/util/ResourceUtils.java  |  30 +-
 .../org/apache/geode/test/version/TestVersion.java |  15 +-
 .../apache/geode/test/version/TestVersions.java    |   2 +-
 .../apache/geode/test/version/VersionManager.java  |   4 +-
 .../apache/geode/test/version/VmConfiguration.java |   7 +-
 .../geode/test/version/VmConfigurations.java       |   3 +-
 .../sanctioned-geode-junit-serializables.txt       |   1 +
 ...ccessfulWhenAllServersRollToCurrentVersion.java |   2 +-
 .../TcpServerProductVersionUpgradeTest.java        |   6 +-
 ...ateGatewaySenderMixedSiteOneCurrentSiteTwo.java |   2 +-
 96 files changed, 3443 insertions(+), 2322 deletions(-)

diff --git a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/ContainerInstall.java b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/ContainerInstall.java
index 979e4356ee..9649500a21 100644
--- a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/ContainerInstall.java
+++ b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/ContainerInstall.java
@@ -14,15 +14,21 @@
  */
 package org.apache.geode.session.tests;
 
+import static java.nio.file.Files.isDirectory;
+import static org.apache.commons.io.FileUtils.deleteDirectory;
+import static org.apache.commons.io.FilenameUtils.getBaseName;
+import static org.apache.commons.io.FilenameUtils.getExtension;
+import static org.apache.geode.management.internal.configuration.utils.ZipUtils.unzip;
 import static org.apache.geode.test.util.ResourceUtils.getResource;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.URL;
 import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.HashMap;
 import java.util.Properties;
 import java.util.function.IntSupplier;
@@ -34,8 +40,6 @@ import javax.xml.transform.TransformerFactory;
 import javax.xml.transform.dom.DOMSource;
 import javax.xml.transform.stream.StreamResult;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.FilenameUtils;
 import org.apache.logging.log4j.Logger;
 import org.codehaus.cargo.container.installer.Installer;
 import org.codehaus.cargo.container.installer.ZipURLInstaller;
@@ -46,7 +50,6 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
 import org.apache.geode.logging.internal.log4j.api.LogService;
-import org.apache.geode.management.internal.configuration.utils.ZipUtils;
 
 /**
  * Base class for handling downloading and configuring J2EE containers.
@@ -58,25 +61,25 @@ import org.apache.geode.management.internal.configuration.utils.ZipUtils;
  */
 public abstract class ContainerInstall {
 
-  private final IntSupplier portSupplier;
   static final Logger logger = LogService.getLogger();
-  static final String TMP_DIR = createTempDir();
-  static final String GEODE_BUILD_HOME = System.getenv("GEODE_HOME");
-  static final String GEODE_BUILD_HOME_LIB = GEODE_BUILD_HOME + "/lib/";
-  private static final String DEFAULT_INSTALL_DIR = TMP_DIR + "/cargo_containers/";
-  private static final String DEFAULT_MODULE_EXTRACTION_DIR = TMP_DIR + "/cargo_modules/";
-  static final String DEFAULT_MODULE_LOCATION = GEODE_BUILD_HOME + "/tools/Modules/";
+
+  private final IntSupplier portSupplier;
+
+  static final Path GEODE_HOME_PATH = Paths.get(System.getenv("GEODE_HOME"));
+  static final Path GEODE_LIB_PATH = GEODE_HOME_PATH.resolve("lib");
+  static final Path DEFAULT_MODULE_PATH = GEODE_HOME_PATH.resolve("tools").resolve("Modules");
 
   protected IntSupplier portSupplier() {
     return portSupplier;
   }
 
   private final ConnectionType connType;
-  private final String installPath;
-  private final String modulePath;
-  private final String warFilePath;
-
+  private final Path rootDir;
+  private final Path installPath;
+  private final Path modulePath;
+  private final Path warFilePath;
   private final String defaultLocatorAddress;
+
   private int defaultLocatorPort;
 
   /**
@@ -122,12 +125,15 @@ public abstract class ContainerInstall {
     }
   }
 
-  public ContainerInstall(String name, String downloadURL, ConnectionType connectionType,
+  public ContainerInstall(Path rootDir, String name, String downloadURL,
+      ConnectionType connectionType,
       String moduleName, IntSupplier portSupplier) throws IOException {
-    this(name, downloadURL, connectionType, moduleName, DEFAULT_MODULE_LOCATION, portSupplier);
+    this(rootDir, name, downloadURL, connectionType, moduleName, DEFAULT_MODULE_PATH, portSupplier);
   }
 
   /**
+   * @param rootDir The root folder used by default for cargo logs, container configs and other
+   *        files and directories
    * @param name used to name install directory
    * @param downloadURL the URL from which to download the container
    * @param connType Enum representing the connection type of this installation (either client
@@ -138,28 +144,34 @@ public abstract class ContainerInstall {
    * @param portSupplier the port supplier
    * @throws IOException if an exception is encountered when deleting or accessing files
    */
-  public ContainerInstall(String name, String downloadURL, ConnectionType connType,
-      String moduleName, String geodeModuleLocation, IntSupplier portSupplier) throws IOException {
+  public ContainerInstall(Path rootDir, String name, String downloadURL, ConnectionType connType,
+      String moduleName, Path geodeModuleLocation, IntSupplier portSupplier)
+      throws IOException {
+    this.rootDir = rootDir;
+
     this.connType = connType;
     this.portSupplier = portSupplier;
 
-    String installDir = DEFAULT_INSTALL_DIR + name;
+    Path installDir = rootDir.toAbsolutePath().resolve("cargo_containers").resolve(name);
 
     clearPreviousInstall(installDir);
 
     URL url = getResource(getClass(), "/" + downloadURL);
-    logger.info("Installing container from URL " + url);
+    logger.info("Installing container from URL {}", url);
 
     // Optional step to install the container from a URL pointing to its distribution
     Installer installer =
-        new ZipURLInstaller(url, TMP_DIR + "/downloads", installDir);
+        new ZipURLInstaller(url, installDir + "/downloads", installDir.toString());
     installer.install();
 
     // Set install home
-    installPath = installer.getHome();
+    installPath = Paths.get(installer.getHome());
+
     // Find and extract the module path
-    modulePath = findAndExtractModule(geodeModuleLocation, moduleName);
-    logger.info("Extracted module " + moduleName + " to " + modulePath);
+    Path modulesDir = rootDir.toAbsolutePath().resolve("cargo_modules");
+    modulePath = findAndExtractModule(modulesDir, geodeModuleLocation, moduleName);
+    logger.info("Extracted module {} to {}", moduleName, modulePath);
+
     // Find the session testing war path
     warFilePath = findSessionTestingWar();
 
@@ -171,18 +183,17 @@ public abstract class ContainerInstall {
   }
 
   ServerContainer generateContainer(String containerDescriptors) throws IOException {
-    return generateContainer(null, containerDescriptors);
+    return generateContainer(rootDir, null, containerDescriptors);
   }
 
   /**
    * Cleans up the installation by deleting the extracted module and downloaded installation folders
    */
-  private void clearPreviousInstall(String installDir) throws IOException {
-    File installFolder = new File(installDir);
+  private void clearPreviousInstall(Path installDir) throws IOException {
     // Remove installs from previous runs in the same folder
-    if (installFolder.exists()) {
-      logger.info("Deleting previous install folder " + installFolder.getAbsolutePath());
-      FileUtils.deleteDirectory(installFolder);
+    if (Files.exists(installDir)) {
+      logger.info("Deleting previous install folder {}", installDir);
+      deleteDirectory(installDir.toFile());
     }
   }
 
@@ -200,10 +211,14 @@ public abstract class ContainerInstall {
     return connType.isClientServer();
   }
 
+  Path getRootDir() {
+    return rootDir;
+  }
+
   /*
    * Where the installation is located
    */
-  public String getHome() {
+  public Path getHome() {
     return installPath;
   }
 
@@ -214,13 +229,13 @@ public abstract class ContainerInstall {
    * needed XML files.
    */
   String getModulePath() {
-    return modulePath;
+    return modulePath.toString();
   }
 
   /**
    * The path to the session testing WAR file
    */
-  String getWarFilePath() {
+  Path getWarFilePath() {
     return warFilePath;
   }
 
@@ -255,8 +270,8 @@ public abstract class ContainerInstall {
   /*
    * Gets the cache XML file to use by default for this installation
    */
-  File getCacheXMLFile() {
-    return new File(modulePath + "/conf/" + getConnectionType().getCacheXMLFileName());
+  Path getCacheXMLFile() {
+    return modulePath.resolve("conf").resolve(getConnectionType().getCacheXMLFileName());
   }
 
   /*
@@ -275,15 +290,18 @@ public abstract class ContainerInstall {
   public abstract String getContextSessionManagerClass();
 
   /**
-   * Generates a {@link ServerContainer} from the given {@link ContainerInstall}
+   * Generates a {@link ServerContainer} from the given {@code ContainerInstall}
    *
+   * @param rootDir The root folder used by default for cargo logs, container configs and other
+   *        files and directories
    * @param containerConfigHome The folder that the container configuration folder should be setup
    *        in
    * @param containerDescriptors Additional descriptors used to identify a container
    * @return the newly generated {@link ServerContainer}
    * @throws IOException if an exception is encountered
    */
-  public abstract ServerContainer generateContainer(File containerConfigHome,
+  public abstract ServerContainer generateContainer(Path rootDir,
+      Path containerConfigHome,
       String containerDescriptors) throws IOException;
 
   /**
@@ -292,7 +310,7 @@ public abstract class ContainerInstall {
    * NOTE::This walks into the extensions folder and then uses a hardcoded path from there making it
    * very unreliable if things are moved.
    */
-  private static String findSessionTestingWar() {
+  private static Path findSessionTestingWar() {
     // Start out searching directory above current
     String curPath = "../";
 
@@ -316,8 +334,8 @@ public abstract class ContainerInstall {
     }
 
     // Return path to extensions plus hardcoded path from there to the WAR
-    return warModuleDir.getAbsolutePath()
-        + "/session-testing-war/build/libs/session-testing-war.war";
+    return warModuleDir.toPath().toAbsolutePath().normalize().resolve(
+        Paths.get("session-testing-war", "build", "libs", "session-testing-war.war"));
   }
 
   /**
@@ -327,58 +345,44 @@ public abstract class ContainerInstall {
    *        extract. Used as a search parameter to find the module archive.
    * @return The path to the non-archive (extracted) version of the module files
    */
-  private static String findAndExtractModule(String geodeModuleLocation, String moduleName)
+  private Path findAndExtractModule(Path defaultModulesDir, Path geodeModuleLocation,
+      String moduleName)
       throws IOException {
-    File modulesDir = new File(geodeModuleLocation);
-
-    logger.info("Trying to access build dir " + modulesDir);
+    logger.info("Trying to access build dir {}", geodeModuleLocation);
 
     // Search directory for tomcat module folder/zip
-    boolean archive = false;
-    File modulePath = null;
-    for (File file : modulesDir.listFiles()) {
-
-      if (file.getName().toLowerCase().contains(moduleName)) {
-        modulePath = file;
-
-        archive = !file.isDirectory();
-        if (!archive) {
-          break;
-        }
-      }
+    final Path moduleSource = Files.list(geodeModuleLocation)
+        .filter(p -> p.toString().toLowerCase().contains(moduleName))
+        .findFirst()
+        .map(Path::toAbsolutePath)
+        .orElseThrow(() -> new AssertionError("Could not find module " + moduleName));
+
+    if (isDirectory(moduleSource)) {
+      return moduleSource;
     }
 
-    assertThat(modulePath).describedAs("module path").isNotNull();
+    String moduleFileNameFull = moduleSource.getFileName().toString();
 
-    String extractedModulePath =
-        modulePath.getName().substring(0, modulePath.getName().length() - 4);
     // Get the name of the new module folder within the extraction directory
-    File newModuleFolder = new File(DEFAULT_MODULE_EXTRACTION_DIR + extractedModulePath);
+    final Path moduleDestinationDir = defaultModulesDir.resolve(getBaseName(moduleFileNameFull));
+
     // Remove any previous module folders extracted here
-    if (newModuleFolder.exists()) {
-      logger.info("Deleting previous modules directory " + newModuleFolder.getAbsolutePath());
-      FileUtils.deleteDirectory(newModuleFolder);
+    if (isDirectory(moduleDestinationDir)) {
+      logger.info("Deleting previous modules directory {}", moduleDestinationDir);
+      deleteDirectory(moduleDestinationDir.toFile());
     }
 
     // Unzip if it is a zip file
-    if (archive) {
-      if (!FilenameUtils.getExtension(modulePath.getAbsolutePath()).equals("zip")) {
-        throw new IOException("Bad module archive " + modulePath);
-      }
-
-      // Extract folder to location if not already there
-      if (!newModuleFolder.exists()) {
-        ZipUtils.unzip(modulePath.getAbsolutePath(), newModuleFolder.getAbsolutePath());
-      }
+    assertThat(moduleSource).isRegularFile();
 
-      modulePath = newModuleFolder;
+    if (!getExtension(moduleFileNameFull).equals("zip")) {
+      throw new IOException("Bad module archive " + moduleSource);
     }
 
-    // No module found within directory throw IOException
-    if (modulePath == null) {
-      throw new IOException("No module found in " + modulesDir);
-    }
-    return modulePath.getAbsolutePath();
+    // Extract folder to location if not already there
+    unzip(moduleSource.toString(), moduleDestinationDir.toString());
+
+    return moduleDestinationDir;
   }
 
   /**
@@ -391,9 +395,9 @@ public abstract class ContainerInstall {
    *        property value the current value. If false, replaces the current property value with the
    *        given property value
    */
-  static void editPropertyFile(String filePath, String propertyName, String propertyValue,
+  static void editPropertyFile(Path filePath, String propertyName, String propertyValue,
       boolean append) throws Exception {
-    FileInputStream input = new FileInputStream(filePath);
+    InputStream input = Files.newInputStream(filePath);
     Properties properties = new Properties();
     properties.load(input);
 
@@ -405,17 +409,17 @@ public abstract class ContainerInstall {
     }
 
     properties.setProperty(propertyName, val);
-    properties.store(new FileOutputStream(filePath), null);
+    properties.store(Files.newOutputStream(filePath), null);
 
     logger.info("Modified container Property file " + filePath);
   }
 
-  static void editXMLFile(String XMLPath, String tagId, String tagName,
+  static void editXMLFile(Path XMLPath, String tagId, String tagName,
       String parentTagName, HashMap<String, String> attributes) {
     editXMLFile(XMLPath, tagId, tagName, tagName, parentTagName, attributes, false);
   }
 
-  static void editXMLFile(String XMLPath, String tagName, String parentTagName,
+  static void editXMLFile(Path XMLPath, String tagName, String parentTagName,
       HashMap<String, String> attributes, boolean writeOnSimilarAttributeNames) {
     editXMLFile(XMLPath, null, tagName, tagName, parentTagName, attributes,
         writeOnSimilarAttributeNames);
@@ -441,7 +445,7 @@ public abstract class ContainerInstall {
    *        rather than adding a new element. If false, create a new XML element (unless tagId is
    *        not null).
    */
-  private static void editXMLFile(String XMLPath, String tagId, String tagName,
+  private static void editXMLFile(Path XMLPath, String tagId, String tagName,
       String replacementTagName, String parentTagName, HashMap<String, String> attributes,
       boolean writeOnSimilarAttributeNames) {
 
@@ -449,7 +453,7 @@ public abstract class ContainerInstall {
       // Get XML file to edit
       DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
       DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
-      Document doc = docBuilder.parse(XMLPath);
+      Document doc = docBuilder.parse(XMLPath.toFile());
 
       Node node = null;
       // Get node with specified tagId
@@ -500,12 +504,12 @@ public abstract class ContainerInstall {
       TransformerFactory transformerFactory = TransformerFactory.newInstance();
       Transformer transformer = transformerFactory.newTransformer();
       DOMSource source = new DOMSource(doc);
-      StreamResult result = new StreamResult(new File(XMLPath));
+      StreamResult result = new StreamResult(XMLPath.toFile());
       transformer.transform(source, result);
 
       logger.info("Modified container XML file " + XMLPath);
     } catch (Exception e) {
-      throw new RuntimeException("Unable to edit XML file", e);
+      throw new RuntimeException("Unable to edit XML file " + XMLPath, e);
     }
   }
 
diff --git a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/ServerContainer.java b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/ServerContainer.java
index 4bd53a6c3c..549c1a6cb5 100644
--- a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/ServerContainer.java
+++ b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/ServerContainer.java
@@ -14,8 +14,8 @@
  */
 package org.apache.geode.session.tests;
 
+import static java.nio.file.Files.copy;
 import static java.util.stream.Collectors.joining;
-import static org.apache.geode.session.tests.ContainerInstall.TMP_DIR;
 import static org.apache.geode.test.process.JavaModuleHelper.getJvmModuleOptions;
 
 import java.io.File;
@@ -33,6 +33,7 @@ import org.apache.logging.log4j.Logger;
 import org.codehaus.cargo.container.ContainerType;
 import org.codehaus.cargo.container.InstalledLocalContainer;
 import org.codehaus.cargo.container.State;
+import org.codehaus.cargo.container.configuration.Configuration;
 import org.codehaus.cargo.container.configuration.ConfigurationType;
 import org.codehaus.cargo.container.configuration.LocalConfiguration;
 import org.codehaus.cargo.container.deployable.WAR;
@@ -55,32 +56,30 @@ import org.apache.geode.logging.internal.log4j.api.LogService;
  * Subclasses provide setup and configuration of specific containers.
  */
 public abstract class ServerContainer {
-  private final File containerConfigHome;
+
+  protected static final Logger logger = LogService.getLogger();
+
+  private static final String DEFAULT_LOGGING_LEVEL = LoggingLevel.LOW.getLevel();
+
+  private final Path containerConfigHome;
   private final IntSupplier portSupplier;
   private final InstalledLocalContainer container;
   private final ContainerInstall install;
 
   private String locatorAddress;
   private int locatorPort;
-  private File warFile;
+  private Path warFile;
 
   public String description;
-  public File gemfireLogFile;
-  public File cacheXMLFile;
-  public File cargoLogDir;
-
-  public String loggingLevel;
+  public Path cacheXMLFile;
+  public Path cargoLogDir;
 
-  public HashMap<String, String> cacheProperties;
-  public HashMap<String, String> systemProperties;
+  private String loggingLevel;
 
-  public final String DEFAULT_CONF_DIR;
+  protected HashMap<String, String> cacheProperties;
+  protected HashMap<String, String> systemProperties;
 
-  public static final String DEFAULT_LOGGING_LEVEL = LoggingLevel.LOW.getLevel();
-  public static final String DEFAULT_LOG_DIR = "cargo_logs/";
-  public static final String DEFAULT_CONFIG_DIR = TMP_DIR + "/cargo_configs/";
-
-  public static final Logger logger = LogService.getLogger();
+  final Path defaultConfigDir;
 
   /**
    * Sets up the container using the given installation
@@ -91,6 +90,8 @@ public abstract class ServerContainer {
    * variable.
    *
    * @param install the installation with which to set up the container
+   * @param rootDir The root folder used by default for cargo logs, container configs and other
+   *        files and directories
    * @param containerConfigHome The folder that the container configuration folder should be setup
    *        in
    * @param containerDescriptors A string of extra descriptors for the container used in the
@@ -98,7 +99,7 @@ public abstract class ServerContainer {
    * @param portSupplier allocates ports for use by the container
    * @throws IOException if an exception is encountered
    */
-  public ServerContainer(ContainerInstall install, File containerConfigHome,
+  public ServerContainer(ContainerInstall install, Path rootDir, Path containerConfigHome,
       String containerDescriptors, IntSupplier portSupplier) throws IOException {
     this.install = install;
     this.portSupplier = portSupplier;
@@ -106,26 +107,30 @@ public abstract class ServerContainer {
     description = generateUniqueContainerDescription(containerDescriptors);
     // Setup logging
     loggingLevel = DEFAULT_LOGGING_LEVEL;
-    cargoLogDir = new File(DEFAULT_LOG_DIR + description);
-    cargoLogDir.mkdirs();
+    cargoLogDir = rootDir.resolve("cargo_logs").resolve(description);
+    Files.createDirectories(cargoLogDir);
 
     logger.info("Creating new container {}", description);
 
-    DEFAULT_CONF_DIR = install.getHome() + "/conf/";
+    defaultConfigDir = install.getHome().resolve("conf");
+
     // Use the default configuration home path if not passed a config home
+    Path defaultConfigDir = rootDir.resolve("cargo_configs");
+
     this.containerConfigHome = containerConfigHome == null
-        ? new File(DEFAULT_CONFIG_DIR + description) : containerConfigHome;
+        ? defaultConfigDir.resolve(description) : containerConfigHome;
 
     // Init the property lists
     cacheProperties = new HashMap<>();
     systemProperties = new HashMap<>();
     // Set WAR file to session testing war
-    warFile = new File(install.getWarFilePath());
+    warFile = install.getWarFilePath();
 
     // Create the Cargo Container instance wrapping our physical container
-    LocalConfiguration configuration = (LocalConfiguration) new DefaultConfigurationFactory()
+    Configuration configuration = new DefaultConfigurationFactory()
         .createConfiguration(install.getInstallId(), ContainerType.INSTALLED,
-            ConfigurationType.STANDALONE, this.containerConfigHome.getAbsolutePath());
+            ConfigurationType.STANDALONE, this.containerConfigHome.toString());
+
     // Set configuration/container logging level
     configuration.setProperty(GeneralPropertySet.LOGGING, loggingLevel);
     // Removes secureRandom generation so that container startup is much faster
@@ -133,33 +138,33 @@ public abstract class ServerContainer {
         "-Djava.security.egd=file:/dev/./urandom -Xmx256m -Xms64m");
 
     // Setup the gemfire log file for this container
-    gemfireLogFile = new File(cargoLogDir.getAbsolutePath() + "/gemfire.log");
-    gemfireLogFile.getParentFile().mkdirs();
-    setSystemProperty("log-file", gemfireLogFile.getAbsolutePath());
+    Path gemfireLogFile = cargoLogDir.resolve("gemfire.log");
+    Files.createDirectories(cargoLogDir);
+    setSystemProperty("log-file", gemfireLogFile.toString());
 
-    logger.info("Gemfire logs can be found in {}", gemfireLogFile.getAbsolutePath());
+    logger.info("Gemfire logs can be found in {}", gemfireLogFile);
 
     // Create the container
-    container = (InstalledLocalContainer) (new DefaultContainerFactory())
+    container = (InstalledLocalContainer) new DefaultContainerFactory()
         .createContainer(install.getInstallId(), ContainerType.INSTALLED, configuration);
     // Set container's home dir to where it was installed
-    container.setHome(install.getHome());
+    container.setHome(install.getHome().toString());
     // Set container output log to directory setup for it
-    container.setOutput(cargoLogDir.getAbsolutePath() + "/container.log");
+    container.setOutput(cargoLogDir.resolve("container.log").toString());
 
     // Set cacheXML file
-    File installXMLFile = install.getCacheXMLFile();
+    Path installXMLFile = install.getCacheXMLFile();
     // Sets the cacheXMLFile variable and adds the cache XML file server system property map
-    setCacheXMLFile(new File(cargoLogDir.getAbsolutePath() + "/" + installXMLFile.getName()));
+    setCacheXMLFile(cargoLogDir.resolve(installXMLFile.getFileName()));
     // Copy the cacheXML file to a new, unique location for this container
-    FileUtils.copyFile(installXMLFile, cacheXMLFile);
+    copy(installXMLFile, cacheXMLFile);
   }
 
   /*
    * Generates a unique, mostly human readable, description string of the container using the
    * installation's description, extraIdentifiers, and the current system nano time
    */
-  public String generateUniqueContainerDescription(String extraIdentifiers) {
+  private String generateUniqueContainerDescription(String extraIdentifiers) {
     return String.join("_", Arrays.asList(install.getInstallDescription(), extraIdentifiers,
         UUID.randomUUID().toString()));
   }
@@ -167,9 +172,9 @@ public abstract class ServerContainer {
   /**
    * Deploys the {@link #warFile} to the cargo container ({@link #container}).
    */
-  public void deployWar() {
+  protected void deployWar() {
     // Get the cargo war from the war file
-    WAR war = new WAR(warFile.getAbsolutePath());
+    WAR war = new WAR(warFile.toString());
     // Set context access to nothing
     war.setContext("");
     // Deploy the war the container's configuration
@@ -232,8 +237,8 @@ public abstract class ServerContainer {
 
   public void dumpLogs() {
     System.out.println("Logs for container " + this);
-    dumpLogsInDir(cargoLogDir.toPath());
-    dumpLogsInDir(containerConfigHome.toPath().resolve("logs"));
+    dumpLogsInDir(cargoLogDir);
+    dumpLogsInDir(containerConfigHome.resolve("logs"));
     dumpConfiguration();
   }
 
@@ -254,7 +259,7 @@ public abstract class ServerContainer {
     System.out.println(path.toAbsolutePath());
     System.out.println("-------------------------------------------");
     try {
-      Files.copy(path, System.out);
+      copy(path, System.out);
     } catch (IOException thrown) {
       System.out.println("Exception while dumping log file to stdout.");
       System.out.println("   File: " + path.toAbsolutePath());
@@ -302,7 +307,7 @@ public abstract class ServerContainer {
    * @param port the port that the locator is listening on
    * @throws IOException if an exception is encountered when updating the locator property file
    */
-  public void setLocator(String address, int port) throws IOException {
+  protected void setLocator(String address, int port) throws IOException {
     locatorAddress = address;
     locatorPort = port;
     updateLocator();
@@ -311,29 +316,29 @@ public abstract class ServerContainer {
   /*
    * Sets the container's cache XML file
    */
-  public void setCacheXMLFile(File cacheXMLFile) throws IOException {
-    setSystemProperty("cache-xml-file", cacheXMLFile.getAbsolutePath());
+  private void setCacheXMLFile(Path cacheXMLFile) throws IOException {
+    setSystemProperty("cache-xml-file", cacheXMLFile.toString());
     this.cacheXMLFile = cacheXMLFile;
   }
 
   /*
    * Set a geode session replication property
    */
-  public String setCacheProperty(String name, String value) throws IOException {
+  protected String setCacheProperty(String name, String value) throws IOException {
     return cacheProperties.put(name, value);
   }
 
   /*
    * Set geode distributed system property
    */
-  public String setSystemProperty(String name, String value) throws IOException {
+  private String setSystemProperty(String name, String value) throws IOException {
     return systemProperties.put(name, value);
   }
 
   /*
    * Sets the war file for this container to deploy and use
    */
-  public void setWarFile(File warFile) {
+  void setWarFile(Path warFile) {
     this.warFile = warFile;
   }
 
@@ -356,7 +361,7 @@ public abstract class ServerContainer {
     return install;
   }
 
-  public File getWarFile() {
+  public Path getWarFile() {
     return warFile;
   }
 
@@ -372,11 +377,11 @@ public abstract class ServerContainer {
     return container.getState();
   }
 
-  public String getCacheProperty(String name) {
+  protected String getCacheProperty(String name) {
     return cacheProperties.get(name);
   }
 
-  public String getSystemProperty(String name) {
+  protected String getSystemProperty(String name) {
     return systemProperties.get(name);
   }
 
@@ -452,8 +457,7 @@ public abstract class ServerContainer {
       attributes.put("host", locatorAddress);
       attributes.put("port", Integer.toString(locatorPort));
 
-      ContainerInstall.editXMLFile(cacheXMLFile.getAbsolutePath(), "locator", "pool",
-          attributes, true);
+      ContainerInstall.editXMLFile(cacheXMLFile, "locator", "pool", attributes, true);
     } else {
       setSystemProperty("locators", locatorAddress + "[" + locatorPort + "]");
     }
diff --git a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatContainer.java b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatContainer.java
index b92ba165e7..8907013da9 100644
--- a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatContainer.java
+++ b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatContainer.java
@@ -14,12 +14,13 @@
  */
 package org.apache.geode.session.tests;
 
-import java.io.File;
+import static java.nio.file.Files.copy;
+
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.HashMap;
 import java.util.function.IntSupplier;
 
-import org.apache.commons.io.FileUtils;
 import org.codehaus.cargo.container.configuration.FileConfig;
 import org.codehaus.cargo.container.configuration.StandaloneLocalConfiguration;
 import org.codehaus.cargo.container.tomcat.TomcatPropertySet;
@@ -34,14 +35,15 @@ import org.codehaus.cargo.util.XmlReplacement;
  */
 public class TomcatContainer extends ServerContainer {
 
-  private final File contextXMLFile;
-  private final File serverXMLFile;
+  private final Path contextXMLFile;
+  private final Path serverXMLFile;
 
-  public static final String DEFAULT_TOMCAT_CONFIG_XML_DIR = "conf/";
+  private static final String DEFAULT_TOMCAT_CONFIG_XML_DIR = "conf/";
 
-  public static final String DEFAULT_TOMCAT_XML_REPLACEMENT_DIR =
+  private static final String DEFAULT_TOMCAT_XML_REPLACEMENT_DIR =
       DEFAULT_TOMCAT_CONFIG_XML_DIR + "Catalina/localhost/";
-  public static final String DEFAULT_TOMCAT_CONTEXT_XML_REPLACEMENT_NAME = "context.xml.default";
+
+  private static final String DEFAULT_TOMCAT_CONTEXT_XML_REPLACEMENT_NAME = "context.xml.default";
 
   /*
    * Setup the Tomcat container
@@ -51,18 +53,19 @@ public class TomcatContainer extends ServerContainer {
    * properties, deploys the session testing WAR file to the Cargo container, and sets various
    * container properties (i.e. locator, local cache, etc.)
    */
-  public TomcatContainer(TomcatInstall install, File containerConfigHome,
+  TomcatContainer(TomcatInstall install, Path rootDir, Path containerConfigHome,
       String containerDescriptors, IntSupplier portSupplier) throws IOException {
-    super(install, containerConfigHome, containerDescriptors, portSupplier);
+    super(install, rootDir, containerConfigHome, containerDescriptors, portSupplier);
 
     // Setup container specific XML files
-    contextXMLFile = new File(cargoLogDir.getAbsolutePath() + "/context.xml");
-    serverXMLFile = new File(DEFAULT_CONF_DIR + "server.xml");
+    contextXMLFile = cargoLogDir.resolve("context.xml");
+    serverXMLFile = defaultConfigDir.resolve("server.xml");
 
     // Copy the default container context XML file from the install to the specified path
-    FileUtils.copyFile(new File(DEFAULT_CONF_DIR + "context.xml"), contextXMLFile);
+    copy(defaultConfigDir.resolve("context.xml"), contextXMLFile);
     // Set the container context XML file to the new location copied to above
-    setConfigFile(contextXMLFile.getAbsolutePath(), DEFAULT_TOMCAT_XML_REPLACEMENT_DIR,
+    setConfigFile(contextXMLFile,
+        DEFAULT_TOMCAT_XML_REPLACEMENT_DIR,
         DEFAULT_TOMCAT_CONTEXT_XML_REPLACEMENT_NAME);
 
     if (install.getConnectionType() == ContainerInstall.ConnectionType.CLIENT_SERVER ||
@@ -94,23 +97,25 @@ public class TomcatContainer extends ServerContainer {
   }
 
   /**
-   * Implements the {@link ServerContainer#writeSettings()} function in order to write the proper
+   * Implements the {@code ServerContainer#writeSettings()} function in order to write the proper
    * settings to the container
    *
-   * Method uses the {@link ContainerInstall#editXMLFile(String, String, String, String, HashMap)}
+   * <p>
+   * Method uses the {@link ContainerInstall#editXMLFile(Path, String, String, String, HashMap)}
    * to edit the {@link #contextXMLFile} with the {@link #cacheProperties}. Method uses
    * {@link #writePropertiesToConfig(StandaloneLocalConfiguration, String, String, HashMap)} to
    * write the {@link #systemProperties} to the {@link #serverXMLFile} using the container's
    * configuration (obtained from {@link #getConfiguration()}).
    */
   @Override
-  public void writeSettings() throws IOException {
+  public void writeSettings() {
     StandaloneLocalConfiguration config = (StandaloneLocalConfiguration) getConfiguration();
 
     // Edit the context XML file
-    ContainerInstall.editXMLFile(contextXMLFile.getAbsolutePath(), "Tomcat", "Manager", "Context",
+    ContainerInstall.editXMLFile(contextXMLFile, "Tomcat", "Manager", "Context",
         cacheProperties);
-    writePropertiesToConfig(config, DEFAULT_TOMCAT_CONFIG_XML_DIR + "/" + serverXMLFile.getName(),
+    writePropertiesToConfig(config,
+        DEFAULT_TOMCAT_CONFIG_XML_DIR + "/" + serverXMLFile.toFile().getName(),
         "//Server/Listener[@className='"
             + ((TomcatInstall) getInstall()).getServerLifeCycleListenerClass() + "']",
         systemProperties);
@@ -156,10 +161,10 @@ public class TomcatContainer extends ServerContainer {
    * @param configDirDest The name of the directory that the configuration file be placed in
    * @param configFileDestName The name of destination file for the new configuration file
    */
-  private void setConfigFile(String filePath, String configDirDest, String configFileDestName) {
+  private void setConfigFile(Path filePath, String configDirDest, String configFileDestName) {
     FileConfig configFile = new FileConfig();
 
-    configFile.setFile(filePath);
+    configFile.setFile(filePath.toString());
     configFile.setToDir(configDirDest);
     configFile.setToFile(configFileDestName);
     getConfiguration().setConfigFileProperty(configFile);
diff --git a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatInstall.java b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatInstall.java
index 3f785a24ee..bec094c415 100644
--- a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatInstall.java
+++ b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatInstall.java
@@ -14,19 +14,21 @@
  */
 package org.apache.geode.session.tests;
 
+import static java.nio.file.Files.createTempDirectory;
+
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.nio.file.StandardCopyOption;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.function.IntSupplier;
 import java.util.regex.Pattern;
 
-
 /**
- * Tomcat specific container installation class
- *
  * Provides logic for installation of tomcat. This makes the modifications to the tomcat install as
  * described in <a href=
  * "https://geode.apache.org/docs/guide/latest/tools_modules/http_session_mgmt/session_mgmt_tomcat.html">
@@ -36,6 +38,7 @@ public class TomcatInstall extends ContainerInstall {
   /**
    * Version of tomcat that this class will install
    *
+   * <p>
    * Includes the download URL for the each version, the version number associated with each
    * version, and other properties or XML attributes needed to setup tomcat containers within Cargo
    */
@@ -113,6 +116,8 @@ public class TomcatInstall extends ContainerInstall {
   /**
    * If you update this list method to return different dependencies, please also update the Tomcat
    * module documentation! The documentation can be found here:
+   *
+   * <p>
    * geode-docs/tools_modules/http_session_mgmt/tomcat_installing_the_module.html.md.erb
    */
   private static final String[] tomcatRequiredJars =
@@ -121,15 +126,37 @@ public class TomcatInstall extends ContainerInstall {
           "geode-membership", "geode-management", "geode-serialization", "geode-tcp-server",
           "javax.transaction-api", "jgroups", "log4j-api", "log4j-core", "log4j-jul", "micrometer",
           "shiro-core", "jetty-server", "jetty-util", "jetty-http", "jetty-io"};
+
   private final TomcatVersion version;
 
   private final CommitValve commitValve;
 
   public TomcatInstall(String name, TomcatVersion version, ConnectionType connectionType,
-      IntSupplier portSupplier,
+      IntSupplier portSupplier, CommitValve commitValve) throws Exception {
+    this(createTempDirectory("geode_container_install").toAbsolutePath(), name, version,
+        connectionType, DEFAULT_MODULE_PATH, GEODE_LIB_PATH, portSupplier, commitValve);
+  }
+
+  public TomcatInstall(Path rootDir, String name, TomcatVersion version,
+      ConnectionType connectionType,
+      IntSupplier portSupplier, CommitValve commitValve) throws Exception {
+    this(rootDir, name, version, connectionType, DEFAULT_MODULE_PATH, GEODE_LIB_PATH, portSupplier,
+        commitValve);
+  }
+
+  public TomcatInstall(String name, TomcatVersion version, ConnectionType connType,
+      String modulesJarLocation, String extraJarsPath, IntSupplier portSupplier,
+      CommitValve commitValve)
+      throws Exception {
+    this(createTempDirectory("geode_container_install").toAbsolutePath(), name, version, connType,
+        Paths.get(modulesJarLocation), Paths.get(extraJarsPath), portSupplier, commitValve);
+  }
+
+  public TomcatInstall(Path rootDir, String name, TomcatVersion version, ConnectionType connType,
+      String modulesJarLocation, String extraJarsPath, IntSupplier portSupplier,
       CommitValve commitValve)
       throws Exception {
-    this(name, version, connectionType, DEFAULT_MODULE_LOCATION, GEODE_BUILD_HOME_LIB,
+    this(rootDir, name, version, connType, Paths.get(modulesJarLocation), Paths.get(extraJarsPath),
         portSupplier, commitValve);
   }
 
@@ -137,12 +164,14 @@ public class TomcatInstall extends ContainerInstall {
    * Download and setup an installation tomcat using the {@link ContainerInstall} constructor and
    * some extra functions this class provides
    *
-   * Specifically, this function uses {@link #copyTomcatGeodeReqFiles(String, String)} to install
+   * <p>
+   * Specifically, this function uses {@link #copyTomcatGeodeReqFiles(Path, Path)} to install
    * geode session into Tomcat, {@link #setupDefaultSettings()} to modify the context and server XML
    * files within the installation's 'conf' folder, and {@link #updateProperties()} to set the jar
    * skipping properties needed to speedup container startup.
    *
-   *
+   * @param rootDir The root folder used by default for cargo logs, container configs and other
+   *        files and directories
    * @param name used to name install directory
    * @param version the version of Tomcat to use
    * @param connType Enum representing the connection type of this installation (either client
@@ -154,16 +183,17 @@ public class TomcatInstall extends ContainerInstall {
    *
    * @throws Exception if an exception is encountered
    */
-  public TomcatInstall(String name, TomcatVersion version, ConnectionType connType,
-      String modulesJarLocation, String extraJarsPath, IntSupplier portSupplier,
+  public TomcatInstall(Path rootDir, String name, TomcatVersion version, ConnectionType connType,
+      Path modulesJarLocation, Path extraJarsPath, IntSupplier portSupplier,
       CommitValve commitValve)
       throws Exception {
     // Does download and install from URL
-    super(name, version.getDownloadURL(), connType, "tomcat", modulesJarLocation, portSupplier);
+    super(rootDir, name, version.getDownloadURL(), connType, "tomcat", modulesJarLocation,
+        portSupplier);
 
     this.version = version;
     this.commitValve = commitValve;
-    modulesJarLocation = getModulePath() + "/lib/";
+    modulesJarLocation = Paths.get(getModulePath()).toAbsolutePath().resolve("lib");
 
     // Install geode sessions into tomcat install
     copyTomcatGeodeReqFiles(modulesJarLocation, extraJarsPath);
@@ -186,12 +216,12 @@ public class TomcatInstall extends ContainerInstall {
 
     // Set the session manager class within the context XML file
     attributes.put("className", getContextSessionManagerClass());
-    editXMLFile(getDefaultContextXMLFile().getAbsolutePath(), "Tomcat", "Manager", "Context",
+    editXMLFile(getDefaultContextXMLFile(), "Tomcat", "Manager", "Context",
         attributes);
 
     // Set the server lifecycle listener within the server XML file
     attributes.put("className", getServerLifeCycleListenerClass());
-    editXMLFile(getDefaultServerXMLFile().getAbsolutePath(), "Tomcat", "Listener", "Server",
+    editXMLFile(getDefaultServerXMLFile(), "Tomcat", "Listener", "Server",
         attributes);
   }
 
@@ -227,8 +257,8 @@ public class TomcatInstall extends ContainerInstall {
    *
    * @return the location of the context XML file in the installation's 'conf' directory
    */
-  public File getDefaultContextXMLFile() {
-    return new File(getHome() + "/conf/context.xml");
+  public Path getDefaultContextXMLFile() {
+    return getHome().resolve("conf").resolve("context.xml");
   }
 
   /**
@@ -236,13 +266,11 @@ public class TomcatInstall extends ContainerInstall {
    *
    * @return the location of the server XML file in the installation's 'conf' directory
    */
-  public File getDefaultServerXMLFile() {
-    return new File(getHome() + "/conf/server.xml");
+  public Path getDefaultServerXMLFile() {
+    return getHome().resolve("conf").resolve("server.xml");
   }
 
   /**
-   * Implements {@link ContainerInstall#getContextSessionManagerClass()}
-   *
    * Gets the TomcatDeltaSessionManager class associated with this {@link #version}. Use's the
    * {@link #version}'s toInteger function to do so.
    */
@@ -253,17 +281,16 @@ public class TomcatInstall extends ContainerInstall {
   }
 
   /**
-   * Implementation of {@link ContainerInstall#generateContainer(File, String)}, which generates a
-   * Tomcat specific container
-   *
    * Creates a {@link TomcatContainer} instance off of this installation.
    *
    * @param containerDescriptors Additional descriptors used to identify a container
    */
   @Override
-  public TomcatContainer generateContainer(File containerConfigHome, String containerDescriptors)
+  public TomcatContainer generateContainer(Path rootDir, Path containerConfigHome,
+      String containerDescriptors)
       throws IOException {
-    return new TomcatContainer(this, containerConfigHome, containerDescriptors, portSupplier());
+    return new TomcatContainer(this, rootDir, containerConfigHome, containerDescriptors,
+        portSupplier());
   }
 
   /**
@@ -276,9 +303,6 @@ public class TomcatInstall extends ContainerInstall {
     return version.getContainerId();
   }
 
-  /**
-   * @see ContainerInstall#getInstallDescription()
-   */
   @Override
   public String getInstallDescription() {
     return version.name() + "_" + getConnectionType().getName();
@@ -295,25 +319,26 @@ public class TomcatInstall extends ContainerInstall {
    * @throws IOException if the {@link #getModulePath()}, installation lib directory, or extra
    *         directory passed in contain no files.
    */
-  private void copyTomcatGeodeReqFiles(String moduleJarDir, String extraJarsPath)
+  private void copyTomcatGeodeReqFiles(Path moduleJarDir, Path extraJarsPath)
       throws IOException {
-    ArrayList<File> requiredFiles = new ArrayList<>();
+    List<File> requiredFiles = new ArrayList<>();
+
     // The library path for the current tomcat installation
-    String tomcatLibPath = getHome() + "/lib/";
+    Path tomcatLibPath = getHome().resolve("lib");
 
     // List of required jars and form version regexps from them
     String versionRegex = "-?[0-9]*.*\\.jar";
-    ArrayList<Pattern> patterns = new ArrayList<>(tomcatRequiredJars.length);
+    List<Pattern> patterns = new ArrayList<>(tomcatRequiredJars.length);
     for (String jar : tomcatRequiredJars) {
       patterns.add(Pattern.compile(jar + versionRegex));
     }
 
     // Don't need to copy any jars already in the tomcat install
-    File tomcatLib = new File(tomcatLibPath);
+    File tomcatLib = tomcatLibPath.toFile();
 
     // Find all jars in the tomcatModulePath and add them as required jars
     try {
-      for (File file : (new File(moduleJarDir)).listFiles()) {
+      for (File file : moduleJarDir.toFile().listFiles()) {
         if (file.isFile() && file.getName().endsWith(".jar")) {
           requiredFiles.add(file);
         }
@@ -325,7 +350,7 @@ public class TomcatInstall extends ContainerInstall {
 
     // Find all the required jars in the extraJarsPath
     try {
-      for (File file : (new File(extraJarsPath)).listFiles()) {
+      for (File file : extraJarsPath.toFile().listFiles()) {
         for (Pattern pattern : patterns) {
           if (pattern.matcher(file.getName()).find()) {
             requiredFiles.add(file);
@@ -342,7 +367,7 @@ public class TomcatInstall extends ContainerInstall {
       Files.copy(file.toPath(), tomcatLib.toPath().resolve(file.toPath().getFileName()),
           StandardCopyOption.REPLACE_EXISTING);
       logger.debug("Copied required jar from " + file.toPath() + " to "
-          + (new File(tomcatLibPath)).toPath().resolve(file.toPath().getFileName()));
+          + tomcatLibPath.resolve(file.toPath().getFileName()));
     }
 
     logger.info("Copied required jars into the Tomcat installation");
@@ -359,8 +384,8 @@ public class TomcatInstall extends ContainerInstall {
     }
 
     // Add the jars to skip to the catalina property file
-    editPropertyFile(getHome() + "/conf/catalina.properties", version.jarSkipPropertyName(),
-        jarsToSkip, true);
+    editPropertyFile(getHome().resolve("conf").resolve("catalina.properties"),
+        version.jarSkipPropertyName(), jarsToSkip, true);
   }
 
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/persistence/MissingDiskStoreAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/persistence/MissingDiskStoreAcceptanceTest.java
index b0717e547d..2c023d04b5 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/persistence/MissingDiskStoreAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/persistence/MissingDiskStoreAcceptanceTest.java
@@ -28,13 +28,13 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientCacheFactory;
 import org.apache.geode.test.assertj.LogFileAssert;
 import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class MissingDiskStoreAcceptanceTest {
@@ -55,20 +55,18 @@ public class MissingDiskStoreAcceptanceTest {
   private String startServer1Command;
   private String startServer2Command;
 
-  @Rule
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+  @Rule(order = 2)
   public ExecutorServiceRule executorServiceRule = new ExecutorServiceRule();
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
   @Before
   public void setUp() throws Exception {
-    locatorFolder = temporaryFolder.newFolder(LOCATOR_NAME).toPath().toAbsolutePath();
-    server1Folder = temporaryFolder.newFolder(SERVER_1_NAME).toPath().toAbsolutePath();
-    server2Folder = temporaryFolder.newFolder(SERVER_2_NAME).toPath().toAbsolutePath();
+    locatorFolder = folderRule.getFolder().toPath().resolve(LOCATOR_NAME).toAbsolutePath();
+    server1Folder = folderRule.getFolder().toPath().resolve(SERVER_1_NAME).toAbsolutePath();
+    server2Folder = folderRule.getFolder().toPath().resolve(SERVER_2_NAME).toAbsolutePath();
 
     int[] ports = getRandomAvailableTCPPorts(6);
     locatorPort = ports[0];
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/persistence/MissingDiskStoreAfterServerRestartAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/persistence/MissingDiskStoreAfterServerRestartAcceptanceTest.java
index 825c1adea0..2031889501 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/persistence/MissingDiskStoreAfterServerRestartAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/cache/persistence/MissingDiskStoreAfterServerRestartAcceptanceTest.java
@@ -15,7 +15,7 @@
 package org.apache.geode.cache.persistence;
 
 import static org.apache.geode.cache.Region.SEPARATOR;
-import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -28,9 +28,9 @@ import java.nio.file.StandardCopyOption;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.test.assertj.LogFileAssert;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class MissingDiskStoreAfterServerRestartAcceptanceTest {
@@ -45,36 +45,24 @@ public class MissingDiskStoreAfterServerRestartAcceptanceTest {
 
   private Path server4Folder;
   private Path server5Folder;
-  private TemporaryFolder temporaryFolder;
 
-  private int locatorPort;
-
-  private String startServer1Command;
-  private String startServer2Command;
-  private String startServer3Command;
-  private String startServer4Command;
   private String startServer5Command;
 
-  private String createRegionWithUnderscoreCommand;
   private String connectToLocatorCommand;
   private String queryCommand;
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   @Before
-  public void setUp() throws Exception {
-    temporaryFolder = gfshRule.getTemporaryFolder();
-    server4Folder = temporaryFolder.newFolder(SERVER_4_NAME).toPath().toAbsolutePath();
-    server5Folder = temporaryFolder.newFolder(SERVER_5_NAME).toPath().toAbsolutePath();
-
-    int[] ports = getRandomAvailableTCPPorts(6);
-    locatorPort = ports[0];
-    int server1Port = ports[1];
-    int server2Port = ports[2];
-    int server3Port = ports[3];
-    int server4Port = ports[4];
-    int server5Port = ports[5];
+  public void setUp() {
+    Path rootFolder = folderRule.getFolder().toPath();
+    server4Folder = rootFolder.resolve(SERVER_4_NAME);
+    server5Folder = rootFolder.resolve(SERVER_5_NAME);
+
+    int locatorPort = getRandomAvailableTCPPort();
 
     String startLocatorCommand = String.join(" ",
         "start locator",
@@ -82,39 +70,39 @@ public class MissingDiskStoreAfterServerRestartAcceptanceTest {
         "--port=" + locatorPort,
         "--locators=localhost[" + locatorPort + "]");
 
-    startServer1Command = String.join(" ",
+    String startServer1Command = String.join(" ",
         "start server",
         "--name=" + SERVER_1_NAME,
         "--locators=localhost[" + locatorPort + "]",
-        "--server-port=" + server1Port);
+        "--disable-default-server");
 
-    startServer2Command = String.join(" ",
+    String startServer2Command = String.join(" ",
         "start server",
         "--name=" + SERVER_2_NAME,
         "--locators=localhost[" + locatorPort + "]",
-        "--server-port=" + server2Port);
+        "--disable-default-server");
 
-    startServer3Command = String.join(" ",
+    String startServer3Command = String.join(" ",
         "start server",
         "--name=" + SERVER_3_NAME,
         "--locators=localhost[" + locatorPort + "]",
-        "--server-port=" + server3Port);
+        "--disable-default-server");
 
-    startServer4Command = String.join(" ",
+    String startServer4Command = String.join(" ",
         "start server",
         "--name=" + SERVER_4_NAME,
         "--dir=" + server4Folder,
         "--locators=localhost[" + locatorPort + "]",
-        "--server-port=" + server4Port);
+        "--disable-default-server");
 
     startServer5Command = String.join(" ",
         "start server",
         "--name=" + SERVER_5_NAME,
         "--dir=" + server5Folder,
         "--locators=localhost[" + locatorPort + "]",
-        "--server-port=" + server5Port);
+        "--disable-default-server");
 
-    createRegionWithUnderscoreCommand = String.join(" ",
+    String createRegionWithUnderscoreCommand = String.join(" ",
         "create region",
         "--name=" + REGION_NAME_WITH_UNDERSCORE,
         "--type=PARTITION_REDUNDANT_PERSISTENT",
@@ -124,7 +112,7 @@ public class MissingDiskStoreAfterServerRestartAcceptanceTest {
     connectToLocatorCommand = "connect --locator=localhost[" + locatorPort + "]";
 
     queryCommand =
-        "query --query=\'select * from " + SEPARATOR + REGION_NAME_WITH_UNDERSCORE + "\'";
+        "query --query='select * from " + SEPARATOR + REGION_NAME_WITH_UNDERSCORE + "'";
 
     gfshRule.execute(startLocatorCommand, startServer1Command, startServer2Command,
         startServer3Command, startServer4Command,
@@ -146,8 +134,7 @@ public class MissingDiskStoreAfterServerRestartAcceptanceTest {
     gfshRule.execute(startServer5Command);
 
     await().untilAsserted(() -> {
-      String waitingForMembersMessage = String.format(
-          "Server server5 startup completed in");
+      String waitingForMembersMessage = "Server server5 startup completed in";
 
       LogFileAssert.assertThat(server5Folder.resolve(SERVER_5_NAME + ".log").toFile())
           .exists()
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupNotificationTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupNotificationTest.java
index 15181ab1cc..a571df69fe 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupNotificationTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupNotificationTest.java
@@ -17,7 +17,6 @@ package org.apache.geode.launchers;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.File;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.concurrent.CompletionException;
@@ -27,7 +26,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
 import org.apache.geode.distributed.ServerLauncherCacheProvider;
@@ -35,34 +33,32 @@ import org.apache.geode.launchers.startuptasks.CompletingAndFailing;
 import org.apache.geode.launchers.startuptasks.Failing;
 import org.apache.geode.launchers.startuptasks.MultipleFailing;
 import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class ServerStartupNotificationTest {
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-  @Rule
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+  @Rule(order = 2)
   public TestName testName = new TestName();
-
-  @Rule
+  @Rule(order = 3)
   public ServiceJarRule serviceJarRule = new ServiceJarRule();
 
-  private File serverFolder;
+  private Path serverFolder;
   private String serverName;
 
   @Before
   public void setup() {
-    serverFolder = temporaryFolder.getRoot();
+    serverFolder = folderRule.getFolder().toPath().toAbsolutePath();
     serverName = testName.getMethodName();
   }
 
   @After
   public void stopServer() {
-    String stopServerCommand = "stop server --dir=" + serverFolder.getAbsolutePath();
+    String stopServerCommand = "stop server --dir=" + serverFolder.toFile();
     gfshRule.execute(stopServerCommand);
   }
 
@@ -71,12 +67,12 @@ public class ServerStartupNotificationTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=" + serverName,
-        "--dir=" + serverFolder.getAbsolutePath(),
+        "--dir=" + serverFolder,
         "--disable-default-server");
 
     gfshRule.execute(startServerCommand);
 
-    Path logFile = serverFolder.toPath().resolve(serverName + ".log");
+    Path logFile = serverFolder.resolve(serverName + ".log");
 
     Pattern expectedLogLine =
         Pattern.compile("^\\[info .*].*Server " + serverName + " startup completed in \\d+ ms");
@@ -93,13 +89,13 @@ public class ServerStartupNotificationTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=" + serverName,
-        "--dir=" + serverFolder.getAbsolutePath(),
+        "--dir=" + serverFolder,
         "--classpath=" + serviceJarPath,
         "--disable-default-server");
 
     gfshRule.execute(startServerCommand);
 
-    Path logFile = serverFolder.toPath().resolve(serverName + ".log");
+    Path logFile = serverFolder.resolve(serverName + ".log");
 
     Exception exception = Failing.EXCEPTION;
     String errorDetail = CompletionException.class.getName() + ": " +
@@ -121,13 +117,13 @@ public class ServerStartupNotificationTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=" + serverName,
-        "--dir=" + serverFolder.getAbsolutePath(),
+        "--dir=" + serverFolder,
         "--classpath=" + serviceJarPath,
         "--disable-default-server");
 
     gfshRule.execute(startServerCommand);
 
-    Path logFile = serverFolder.toPath().resolve(serverName + ".log");
+    Path logFile = serverFolder.resolve(serverName + ".log");
 
     Exception exception = MultipleFailing.EXCEPTION;
     String errorDetail = CompletionException.class.getName() + ": " +
@@ -149,13 +145,13 @@ public class ServerStartupNotificationTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=" + serverName,
-        "--dir=" + serverFolder.getAbsolutePath(),
+        "--dir=" + serverFolder,
         "--classpath=" + serviceJarPath,
         "--disable-default-server");
 
     gfshRule.execute(startServerCommand);
 
-    Path logFile = serverFolder.toPath().resolve(serverName + ".log");
+    Path logFile = serverFolder.resolve(serverName + ".log");
 
     Exception exception = CompletingAndFailing.EXCEPTION;
     String errorDetail = CompletionException.class.getName() + ": " +
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupOnlineTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupOnlineTest.java
index b23be923da..afeeaffd7b 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupOnlineTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupOnlineTest.java
@@ -12,9 +12,9 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.launchers;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -31,37 +31,33 @@ import javax.management.remote.JMXConnector;
 import javax.management.remote.JMXConnectorFactory;
 import javax.management.remote.JMXServiceURL;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
 import org.apache.geode.distributed.ServerLauncherCacheProvider;
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.launchers.startuptasks.WaitForFileToExist;
 import org.apache.geode.management.MemberMXBean;
 import org.apache.geode.rules.ServiceJarRule;
 import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class ServerStartupOnlineTest {
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public TestName testName = new TestName();
-
   @Rule
   public ServiceJarRule serviceJarRule = new ServiceJarRule();
-
   @Rule
   public ExecutorServiceRule executorServiceRule = new ExecutorServiceRule();
 
@@ -75,10 +71,10 @@ public class ServerStartupOnlineTest {
     Path serviceJarPath = serviceJarRule.createJarFor("ServerLauncherCacheProvider.jar",
         ServerLauncherCacheProvider.class, WaitForFileToExist.class);
 
-    serverFolder = temporaryFolder.getRoot().toPath().toAbsolutePath();
+    serverFolder = folderRule.getFolder().toPath().toAbsolutePath();
     serverName = testName.getMethodName();
 
-    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+    int[] ports = getRandomAvailableTCPPorts(2);
 
     int jmxHttpPort = ports[0];
     jmxRmiPort = ports[1];
@@ -102,7 +98,8 @@ public class ServerStartupOnlineTest {
   }
 
   @Test
-  public void startServerReturnsAfterStartupTaskCompletes() throws Exception {
+  public void startServerReturnsAfterStartupTaskCompletes()
+      throws InterruptedException, IOException {
     CompletableFuture<Void> startServerTask =
         executorServiceRule.runAsync(() -> gfshRule.execute(startServerCommand));
 
@@ -116,7 +113,8 @@ public class ServerStartupOnlineTest {
   }
 
   @Test
-  public void statusServerReportsStartingUntilStartupTaskCompletes() throws Exception {
+  public void statusServerReportsStartingUntilStartupTaskCompletes()
+      throws InterruptedException, IOException {
     CompletableFuture<Void> startServerTask =
         executorServiceRule.runAsync(() -> gfshRule.execute(startServerCommand));
 
@@ -141,7 +139,8 @@ public class ServerStartupOnlineTest {
   }
 
   @Test
-  public void memberMXBeanStatusReportsStartingUntilStartupTaskCompletes() throws Exception {
+  public void memberMXBeanStatusReportsStartingUntilStartupTaskCompletes()
+      throws InterruptedException, IOException {
     CompletableFuture<Void> startServerTask =
         executorServiceRule.runAsync(() -> gfshRule.execute(startServerCommand));
 
@@ -166,24 +165,20 @@ public class ServerStartupOnlineTest {
     });
   }
 
-  private String getServerStatusFromJmx() throws MalformedObjectNameException,
-      IOException {
+  private String getServerStatusFromJmx() throws MalformedObjectNameException, IOException {
     ObjectName objectName = ObjectName.getInstance("GemFire:type=Member,member=" + serverName);
     JMXServiceURL url =
         new JMXServiceURL("service:jmx:rmi:///jndi/rmi://localhost:" + jmxRmiPort + "/jmxrmi");
-    JMXConnector jmxConnector = JMXConnectorFactory.connect(url, null);
-    try {
+    try (JMXConnector jmxConnector = JMXConnectorFactory.connect(url, null)) {
       MBeanServerConnection mbeanServer = jmxConnector.getMBeanServerConnection();
       MemberMXBean memberMXBean =
           JMX.newMXBeanProxy(mbeanServer, objectName, MemberMXBean.class, false);
       String json = memberMXBean.status();
       return parseStatusFromJson(json);
-    } finally {
-      jmxConnector.close();
     }
   }
 
-  private String parseStatusFromJson(String json) throws IOException {
+  private String parseStatusFromJson(String json) throws JsonProcessingException {
     ObjectMapper mapper = new ObjectMapper();
     JsonNode jsonNode = mapper.readTree(json);
     return jsonNode.get("status").textValue();
@@ -194,8 +189,7 @@ public class ServerStartupOnlineTest {
     return gfshRule.execute(statusServerCommand).getOutputText();
   }
 
-  private void waitForStartServerCommandToHang()
-      throws InterruptedException {
+  private void waitForStartServerCommandToHang() throws InterruptedException {
     await().untilAsserted(() -> assertThat(serverFolder.resolve(serverName + ".log")).exists());
     // Without sleeping, this test can pass when it shouldn't.
     Thread.sleep(10_000);
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupRedundancyRecoveryNotificationTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupRedundancyRecoveryNotificationTest.java
index e801169082..2a35de7c89 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupRedundancyRecoveryNotificationTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupRedundancyRecoveryNotificationTest.java
@@ -19,7 +19,6 @@ import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTC
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.List;
@@ -31,9 +30,9 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class ServerStartupRedundancyRecoveryNotificationTest {
@@ -42,13 +41,14 @@ public class ServerStartupRedundancyRecoveryNotificationTest {
   private static final String SERVER_2_NAME = "server2";
   private static final String LOCATOR_NAME = "locator";
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public TestName testName = new TestName();
 
+  private Path rootFolder;
   private Path locatorFolder;
   private Path server1Folder;
   private Path server2Folder;
@@ -58,11 +58,11 @@ public class ServerStartupRedundancyRecoveryNotificationTest {
   private String regionNameTwo;
 
   @Before
-  public void redundantRegionThatRequiresRedundancyRecovery() throws IOException {
-    locatorFolder = temporaryFolder.newFolder(LOCATOR_NAME).toPath().toAbsolutePath();
-    server1Folder = temporaryFolder.newFolder(SERVER_1_NAME + "_before").toPath()
-        .toAbsolutePath();
-    server2Folder = temporaryFolder.newFolder(SERVER_2_NAME).toPath().toAbsolutePath();
+  public void redundantRegionThatRequiresRedundancyRecovery() {
+    rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+    locatorFolder = rootFolder.resolve(LOCATOR_NAME);
+    server1Folder = rootFolder.resolve(SERVER_1_NAME + "_before");
+    server2Folder = rootFolder.resolve(SERVER_2_NAME);
 
     locatorPort = getRandomAvailableTCPPort();
 
@@ -129,10 +129,9 @@ public class ServerStartupRedundancyRecoveryNotificationTest {
   }
 
   @Test
-  public void startupReportsOnlineOnlyAfterRedundancyRestored() throws IOException {
+  public void startupReportsOnlineOnlyAfterRedundancyRestored() {
     String connectCommand = "connect --locator=localhost[" + locatorPort + "]";
-    server1Folder =
-        temporaryFolder.newFolder(SERVER_1_NAME + "_test").toPath().toAbsolutePath();
+    server1Folder = rootFolder.resolve(SERVER_1_NAME + "_test");
     startServer1Command = String.join(" ",
         "start server",
         "--name=" + SERVER_1_NAME,
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupValueRecoveryNotificationTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupValueRecoveryNotificationTest.java
index 78f2e37711..3a1f0834d1 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupValueRecoveryNotificationTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/launchers/ServerStartupValueRecoveryNotificationTest.java
@@ -12,9 +12,10 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.launchers;
 
+import static java.nio.file.Files.createDirectory;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -30,29 +31,26 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
-import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class ServerStartupValueRecoveryNotificationTest {
 
-
   private static final String SERVER_1_NAME = "server1";
   private static final String LOCATOR_NAME = "locator";
   private static final String DISKSTORE_1 = "diskstore1";
   private static final String DISKSTORE_2 = "diskstore2";
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public TestName testName = new TestName();
 
+  private Path temporaryFolder;
   private Path locatorFolder;
   private Path server1Folder;
   private int locatorPort;
@@ -60,11 +58,19 @@ public class ServerStartupValueRecoveryNotificationTest {
 
   @Before
   public void persistentRegionThatRequiresValueRecovery() throws IOException {
-    locatorFolder = temporaryFolder.newFolder(LOCATOR_NAME).toPath().toAbsolutePath();
-    server1Folder = temporaryFolder.newFolder(SERVER_1_NAME).toPath().toAbsolutePath();
-    Path diskStore1Folder = temporaryFolder.newFolder(DISKSTORE_1).toPath().toAbsolutePath();
-    Path diskStore2Folder = temporaryFolder.newFolder(DISKSTORE_2).toPath().toAbsolutePath();
-    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(1);
+    temporaryFolder = folderRule.getFolder().toPath().toAbsolutePath();
+
+    locatorFolder = temporaryFolder.resolve(LOCATOR_NAME);
+    server1Folder = temporaryFolder.resolve(SERVER_1_NAME);
+    Path diskStore1Folder = temporaryFolder.resolve(DISKSTORE_1);
+    Path diskStore2Folder = temporaryFolder.resolve(DISKSTORE_2);
+
+    createDirectory(locatorFolder);
+    createDirectory(server1Folder);
+    createDirectory(diskStore1Folder);
+    createDirectory(diskStore2Folder);
+
+    int[] ports = getRandomAvailableTCPPorts(1);
     locatorPort = ports[0];
 
     String startLocatorCommand = String.join(" ",
@@ -134,9 +140,10 @@ public class ServerStartupValueRecoveryNotificationTest {
 
   @Test
   public void startupReportsOnlineOnlyAfterRedundancyRestored() throws IOException {
+    server1Folder = temporaryFolder.resolve(SERVER_1_NAME + "secondfolder");
+    createDirectory(server1Folder);
+
     String connectCommand = "connect --locator=localhost[" + locatorPort + "]";
-    server1Folder =
-        temporaryFolder.newFolder(SERVER_1_NAME + "secondfolder").toPath().toAbsolutePath();
     startServer1Command = String.join(" ",
         "start server",
         "--name=" + SERVER_1_NAME,
@@ -187,5 +194,4 @@ public class ServerStartupValueRecoveryNotificationTest {
               .matches(serverOnlinePattern.asPredicate(), serverOnlinePattern.pattern());
         });
   }
-
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/logging/LocatorWithCustomLogConfigAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/logging/LocatorWithCustomLogConfigAcceptanceTest.java
index a64c165521..dc68f9bef7 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/logging/LocatorWithCustomLogConfigAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/logging/LocatorWithCustomLogConfigAcceptanceTest.java
@@ -21,16 +21,17 @@ import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.apache.geode.test.util.ResourceUtils.createFileFromResource;
 import static org.apache.geode.test.util.ResourceUtils.getResource;
 
+import java.io.IOException;
 import java.nio.file.Path;
 
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
 import org.apache.geode.test.junit.categories.LoggingTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 @Category(LoggingTest.class)
@@ -51,42 +52,37 @@ public class LocatorWithCustomLogConfigAcceptanceTest {
   private Path locatorLogFile;
   private Path pulseLogFile;
   private Path customLogFile;
-  private TemporaryFolder temporaryFolder;
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public TestName testName = new TestName();
 
   @Before
-  public void setUpLogConfigFiles() {
-    temporaryFolder = gfshRule.getTemporaryFolder();
+  public void setUp() {
+    // set up log config files
+    workingDir = folderRule.getFolder().toPath().toAbsolutePath();
 
     configWithGeodePluginsFile = createFileFromResource(
-        getResource(CONFIG_WITH_GEODE_PLUGINS_FILE_NAME), temporaryFolder.getRoot(),
+        getResource(CONFIG_WITH_GEODE_PLUGINS_FILE_NAME), workingDir.toFile(),
         CONFIG_WITH_GEODE_PLUGINS_FILE_NAME)
             .toPath();
 
     configWithoutGeodePluginsFile = createFileFromResource(
-        getResource(CONFIG_WITHOUT_GEODE_PLUGINS_FILE_NAME), temporaryFolder.getRoot(),
+        getResource(CONFIG_WITHOUT_GEODE_PLUGINS_FILE_NAME), workingDir.toFile(),
         CONFIG_WITHOUT_GEODE_PLUGINS_FILE_NAME)
             .toPath();
-  }
-
-  @Before
-  public void setUpOutputFiles() {
-    temporaryFolder = gfshRule.getTemporaryFolder();
 
+    // set up output files
     locatorName = testName.getMethodName();
 
-    workingDir = temporaryFolder.getRoot().toPath().toAbsolutePath();
     locatorLogFile = workingDir.resolve(locatorName + ".log");
     pulseLogFile = workingDir.resolve("pulse.log");
     customLogFile = workingDir.resolve("custom.log");
-  }
 
-  @Before
-  public void setUpRandomPorts() {
+    // set up random ports
     int[] ports = getRandomAvailableTCPPorts(3);
 
     locatorPort = ports[0];
@@ -164,7 +160,7 @@ public class LocatorWithCustomLogConfigAcceptanceTest {
   }
 
   @Test
-  public void locatorLauncherUsesConfigFileInClasspathWithoutGeodePlugins() throws Exception {
+  public void locatorLauncherUsesConfigFileInClasspathWithoutGeodePlugins() throws IOException {
     copy(configWithoutGeodePluginsFile, workingDir.resolve("log4j2.xml"));
 
     String classpath = workingDir.toFile().getAbsolutePath();
@@ -243,7 +239,7 @@ public class LocatorWithCustomLogConfigAcceptanceTest {
   }
 
   @Test
-  public void locatorLauncherUsesConfigFileInClasspathWithGeodePlugins() throws Exception {
+  public void locatorLauncherUsesConfigFileInClasspathWithGeodePlugins() throws IOException {
     copy(configWithGeodePluginsFile, workingDir.resolve("log4j2.xml"));
 
     String classpath = workingDir.toFile().getAbsolutePath();
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/logging/ServerWithCustomLogConfigAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/logging/ServerWithCustomLogConfigAcceptanceTest.java
index 85bc5e02fa..b6c88b9c17 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/logging/ServerWithCustomLogConfigAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/logging/ServerWithCustomLogConfigAcceptanceTest.java
@@ -20,16 +20,17 @@ import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.apache.geode.test.util.ResourceUtils.createFileFromResource;
 import static org.apache.geode.test.util.ResourceUtils.getResource;
 
+import java.io.IOException;
 import java.nio.file.Path;
 
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
 import org.apache.geode.test.junit.categories.LoggingTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 @Category(LoggingTest.class)
@@ -46,35 +47,32 @@ public class ServerWithCustomLogConfigAcceptanceTest {
   private Path configWithoutGeodePluginsFile;
   private Path serverLogFile;
   private Path customLogFile;
-  private TemporaryFolder temporaryFolder;
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public TestName testName = new TestName();
 
   @Before
   public void setUpLogConfigFiles() {
-    temporaryFolder = gfshRule.getTemporaryFolder();
+    // set up log config files
+    workingDir = folderRule.getFolder().toPath().toAbsolutePath();
 
     configWithGeodePluginsFile = createFileFromResource(
-        getResource(CONFIG_WITH_GEODE_PLUGINS_FILE_NAME), temporaryFolder.getRoot(),
+        getResource(CONFIG_WITH_GEODE_PLUGINS_FILE_NAME), workingDir.toFile(),
         CONFIG_WITH_GEODE_PLUGINS_FILE_NAME)
             .toPath();
 
     configWithoutGeodePluginsFile = createFileFromResource(
-        getResource(CONFIG_WITHOUT_GEODE_PLUGINS_FILE_NAME), temporaryFolder.getRoot(),
+        getResource(CONFIG_WITHOUT_GEODE_PLUGINS_FILE_NAME), workingDir.toFile(),
         CONFIG_WITHOUT_GEODE_PLUGINS_FILE_NAME)
             .toPath();
-  }
-
-  @Before
-  public void setUpOutputFiles() {
-    temporaryFolder = gfshRule.getTemporaryFolder();
 
+    // set up output files
     serverName = testName.getMethodName();
 
-    workingDir = temporaryFolder.getRoot().toPath().toAbsolutePath();
     serverLogFile = workingDir.resolve(serverName + ".log");
     customLogFile = workingDir.resolve("custom.log");
   }
@@ -127,7 +125,7 @@ public class ServerWithCustomLogConfigAcceptanceTest {
   }
 
   @Test
-  public void serverLauncherUsesConfigFileInClasspathWithoutGeodePlugins() throws Exception {
+  public void serverLauncherUsesConfigFileInClasspathWithoutGeodePlugins() throws IOException {
     copy(configWithoutGeodePluginsFile, workingDir.resolve("log4j2.xml"));
 
     String classpath = workingDir.toFile().getAbsolutePath();
@@ -181,7 +179,7 @@ public class ServerWithCustomLogConfigAcceptanceTest {
   }
 
   @Test
-  public void serverLauncherUsesConfigFileInClasspathWithGeodePlugins() throws Exception {
+  public void serverLauncherUsesConfigFileInClasspathWithGeodePlugins() throws IOException {
     copy(configWithGeodePluginsFile, workingDir.resolve("log4j2.xml"));
 
     String classpath = workingDir.toFile().getAbsolutePath();
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/ConfigureEvictionThroughGfsh.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/ConfigureEvictionThroughGfsh.java
index 1c7b476563..cabedabde0 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/ConfigureEvictionThroughGfsh.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/ConfigureEvictionThroughGfsh.java
@@ -16,36 +16,47 @@
 package org.apache.geode.management.internal.cli.commands;
 
 import static org.apache.geode.cache.Region.SEPARATOR;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.io.IOException;
 
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
 import org.apache.geode.test.compiler.JarBuilder;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
-// GEODE-1897 Users should be able to configure eviction through gfsh
 public class ConfigureEvictionThroughGfsh {
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
+  private int locatorPort;
 
-  @Test
-  public void configureEvictionByEntryCount() throws Exception {
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
+  @Before
+  public void setUp() {
+    locatorPort = getRandomAvailableTCPPort();
+  }
 
+  @Test
+  public void configureEvictionByEntryCount() {
     GfshExecution execution = GfshScript
-        .of("start locator --name=locator", "start server --name=server --server-port=0",
+        .of("start locator --name=locator --port=" + locatorPort,
+            "start server --name=server --disable-default-server",
             "create region --name=region1 --eviction-action=local-destroy --eviction-entry-count=1000 --type=REPLICATE",
             "create region --name=region2 --eviction-action=overflow-to-disk --eviction-entry-count=1000 --type=REPLICATE",
             "create region --name=region3 --eviction-action=overflow-to-disk --eviction-entry-count=1000 --type=REPLICATE_PERSISTENT",
             "create region --name=region4 --eviction-action=local-destroy --eviction-entry-count=1000 --type=LOCAL",
             "create region --name=region5 --eviction-action=overflow-to-disk --eviction-entry-count=1000 --type=LOCAL")
-        .execute(gfsh);
+        .execute(gfshRule);
 
     assertThat(execution.getOutputText())
         .contains("Region \"" + SEPARATOR + "region1\" created on \"server\"");
@@ -59,54 +70,70 @@ public class ConfigureEvictionThroughGfsh {
         .contains("Region \"" + SEPARATOR + "region5\" created on \"server\"");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]",
+        .of("connect --locator=localhost[" + locatorPort + "]",
             "create region --name=region6 --eviction-action=local-destroy --eviction-entry-count=1000 --type=REPLICATE_PERSISTENT")
-        .expectFailure().execute(gfsh);
-    assertThat(execution.getOutputText()).contains(
-        "An Eviction Controller with local destroy eviction action is incompatible with");
+        .expectFailure()
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .contains("An Eviction Controller with local destroy eviction action is incompatible with");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region1").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region1")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region1")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region1")
         .containsPattern("eviction-action\\s+| local-destroy")
         .containsPattern("eviction-maximum-value\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region2").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region2")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region2")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region2")
         .containsPattern("eviction-action\\s+| overflow-to-disk")
         .containsPattern("eviction-maximum-value\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region3").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region3")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region3")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region3")
         .containsPattern("eviction-action\\s+| overflow-to-disk")
         .containsPattern("eviction-maximum-value\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region4").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region4")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region4")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region4")
         .containsPattern("eviction-action\\s+| local-destroy")
         .containsPattern("eviction-maximum-value\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region5").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region5")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region5")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region5")
         .containsPattern("eviction-action\\s+| overflow-to-disk")
         .containsPattern("eviction-maximum-value\\s+ | 1000");
-
   }
 
   @Test
-  public void configureEvictionByMaxMemory() throws Exception {
+  public void configureEvictionByMaxMemory() {
     GfshExecution execution = GfshScript
-        .of("start locator --name=locator", "start server --name=server --server-port=0",
+        .of("start locator --name=locator --port=" + locatorPort,
+            "start server --name=server --disable-default-server",
             "create region --name=region1 --eviction-action=local-destroy --eviction-max-memory=1000 --type=REPLICATE",
             "create region --name=region2 --eviction-action=overflow-to-disk --eviction-max-memory=1000 --type=REPLICATE",
             "create region --name=region3 --eviction-action=overflow-to-disk --eviction-max-memory=1000 --type=REPLICATE_PERSISTENT",
             "create region --name=region4 --eviction-action=local-destroy --eviction-max-memory=1000 --type=LOCAL",
             "create region --name=region5 --eviction-action=overflow-to-disk --eviction-max-memory=1000 --type=LOCAL")
-        .execute(gfsh);
+        .execute(gfshRule);
     assertThat(execution.getOutputText())
         .contains("Region \"" + SEPARATOR + "region1\" created on \"server\"");
     assertThat(execution.getOutputText())
@@ -119,59 +146,65 @@ public class ConfigureEvictionThroughGfsh {
         .contains("Region \"" + SEPARATOR + "region5\" created on \"server\"");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]",
+        .of("connect --locator=localhost[" + locatorPort + "]",
             "create region --name=region6 --eviction-action=local-destroy --eviction-max-memory=1000 --type=REPLICATE_PERSISTENT")
-        .expectFailure().execute(gfsh);
-    assertThat(execution.getOutputText()).contains(
-        "An Eviction Controller with local destroy eviction action is incompatible with");
+        .expectFailure()
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .contains("An Eviction Controller with local destroy eviction action is incompatible with");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region1").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region1")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region1")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region1")
         .containsPattern("eviction-action\\s+| local-destroy")
         .containsPattern("eviction-max-memory\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region2").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region2")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region2")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region2")
         .containsPattern("eviction-action\\s+| overflow-to-disk")
         .containsPattern("eviction-max-memory\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region3").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region3")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region3")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region3")
         .containsPattern("eviction-action\\s+| overflow-to-disk")
         .containsPattern("eviction-max-memory\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region4").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region4")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region4")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region4")
         .containsPattern("eviction-action\\s+| local-destroy")
         .containsPattern("eviction-max-memory\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region5").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region5")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region5")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region5")
         .containsPattern("eviction-action\\s+| overflow-to-disk")
         .containsPattern("eviction-max-memory\\s+ | 1000");
   }
 
-  private File createJar() throws IOException {
-    File jarToDeploy = new File(gfsh.getTemporaryFolder().getRoot(), "ourJar.jar");
-
-    String classContents =
-        "import org.apache.geode.cache.util.ObjectSizer; import org.apache.geode.cache.Declarable;public class MySizer implements ObjectSizer, Declarable { public int sizeof(Object o) { return 10; } }";
-
-    JarBuilder jarBuilder = new JarBuilder();
-    jarBuilder.buildJar(jarToDeploy, classContents);
-
-    return jarToDeploy;
-  }
-
   @Test
-  public void configureEvictionByObjectSizer() throws Exception {
+  public void configureEvictionByObjectSizer()
+      throws IOException {
     GfshExecution execution = GfshScript
-        .of("start locator --name=locator", "start server --name=server --server-port=0",
+        .of("start locator --name=locator --port=" + locatorPort,
+            "start server --name=server --disable-default-server",
             "sleep --time=1",
             "deploy --jar=" + createJar().getAbsolutePath(),
             "create region --name=region1 --eviction-action=local-destroy --eviction-max-memory=1000 --eviction-object-sizer=MySizer --type=REPLICATE",
@@ -179,7 +212,7 @@ public class ConfigureEvictionThroughGfsh {
             "create region --name=region3 --eviction-action=overflow-to-disk --eviction-max-memory=1000 --eviction-object-sizer=MySizer --type=REPLICATE_PERSISTENT",
             "create region --name=region4 --eviction-action=local-destroy --eviction-max-memory=1000 --eviction-object-sizer=MySizer --type=LOCAL",
             "create region --name=region5 --eviction-action=overflow-to-disk --eviction-max-memory=1000 --eviction-object-sizer=MySizer --type=LOCAL")
-        .execute(gfsh);
+        .execute(gfshRule);
 
     assertThat(execution.getOutputText())
         .contains("Region \"" + SEPARATOR + "region1\" created on \"server\"");
@@ -193,41 +226,68 @@ public class ConfigureEvictionThroughGfsh {
         .contains("Region \"" + SEPARATOR + "region5\" created on \"server\"");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]",
+        .of("connect --locator=localhost[" + locatorPort + "]",
             "create region --name=region6 --eviction-action=local-destroy --eviction-max-memory=1000 --eviction-object-sizer=MySizer --type=REPLICATE_PERSISTENT")
-        .expectFailure().execute(gfsh);
-    assertThat(execution.getOutputText()).contains(
-        "An Eviction Controller with local destroy eviction action is incompatible with");
+        .expectFailure()
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .contains("An Eviction Controller with local destroy eviction action is incompatible with");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region1").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region1")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region1")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region1")
         .containsPattern("eviction-action\\s+| local-destroy")
         .containsPattern("eviction-max-memory\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region2").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region2")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region2")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region2")
         .containsPattern("eviction-action\\s+| overflow-to-disk")
         .containsPattern("eviction-max-memory\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region3").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region3")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region3")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region3")
         .containsPattern("eviction-action\\s+| overflow-to-disk")
         .containsPattern("eviction-max-memory\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region4").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region4")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region4")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region4")
         .containsPattern("eviction-action\\s+| local-destroy")
         .containsPattern("eviction-max-memory\\s+ | 1000");
 
     execution = GfshScript
-        .of("connect --locator=localhost[10334]", "describe region --name=region5").execute(gfsh);
-    assertThat(execution.getOutputText()).containsPattern("region5")
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "describe region --name=region5")
+        .execute(gfshRule);
+    assertThat(execution.getOutputText())
+        .containsPattern("region5")
         .containsPattern("eviction-action\\s+| overflow-to-disk")
         .containsPattern("eviction-max-memory\\s+ | 1000");
+  }
+
+  private File createJar() throws IOException {
+    File jarToDeploy = folderRule.getFolder().toPath().resolve("ourJar.jar").toFile();
 
+    String classContents =
+        "import org.apache.geode.cache.util.ObjectSizer; import org.apache.geode.cache.Declarable;public class MySizer implements ObjectSizer, Declarable { public int sizeof(Object o) { return 10; } }";
+
+    JarBuilder jarBuilder = new JarBuilder();
+    jarBuilder.buildJar(jarToDeploy, classContents);
+
+    return jarToDeploy;
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/DeployWithLargeJarTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/DeployWithLargeJarTest.java
index d121950e8b..900e0e6f8a 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/DeployWithLargeJarTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/DeployWithLargeJarTest.java
@@ -15,30 +15,62 @@
 
 package org.apache.geode.management.internal.cli.commands;
 
+import static java.util.Objects.requireNonNull;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+import static org.apache.geode.internal.lang.SystemUtils.isWindows;
+
 import java.io.File;
+import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.stream.Collectors;
 
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
+import org.apache.geode.test.junit.rules.RequiresGeodeHome;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class DeployWithLargeJarTest {
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
+  private int locatorPort;
+
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
+  @Before
+  public void setUp() {
+    locatorPort = getRandomAvailableTCPPort();
+  }
 
   @Test
-  public void deployLargeSetOfJars() throws Exception {
-    File libDir = gfsh.getGfshPath().getParent().getParent().resolve("lib").toFile();
-    String commonLibs = Arrays.stream(libDir.listFiles(x -> x.getName().startsWith("commons")))
-        .map(File::getAbsolutePath).collect(Collectors.joining(","));
-    GfshExecution execution = GfshScript.of("start locator --name=locator --max-heap=128m",
-        "start server --name=server --max-heap=128m --server-port=0", "sleep --time=1",
-        "deploy --jars=" + commonLibs).execute(gfsh);
+  public void deployLargeSetOfJars() {
+    File libDir = findGfsh().getParent().getParent().resolve("lib").toFile();
+
+    String commonLibs = Arrays
+        .stream(requireNonNull(libDir.listFiles(x -> x.getName().startsWith("commons"))))
+        .map(File::getAbsolutePath)
+        .collect(Collectors.joining(","));
+
+    GfshExecution execution = GfshScript
+        .of("start locator --name=locator --max-heap=128m --port=" + locatorPort,
+            "start server --name=server --max-heap=128m --disable-default-server",
+            "sleep --time=1",
+            "deploy --jars=" + commonLibs)
+        .execute(gfshRule);
   }
 
+  private static Path findGfsh() {
+    Path geodeHome = new RequiresGeodeHome().getGeodeHome().toPath();
+
+    if (isWindows()) {
+      return geodeHome.resolve("bin").resolve("gfsh.bat");
+    }
+    return geodeHome.resolve("bin").resolve("gfsh");
+  }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/DestroyIndexIfExistsTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/DestroyIndexIfExistsTest.java
index d0aadf3e1d..00cb432901 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/DestroyIndexIfExistsTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/DestroyIndexIfExistsTest.java
@@ -12,30 +12,49 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
 import static org.assertj.core.api.Assertions.assertThat;
 
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class DestroyIndexIfExistsTest {
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
+  private int locatorPort;
 
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  @Test
-  public void destroyIndexIfExists() throws Exception {
-    GfshExecution execution =
-        GfshScript.of("start locator --name=locator", "start server --name=server --server-port=0",
-            "sleep --time=1", "destroy index --name=i1 --if-exists=true").execute(gfsh);
+  @Before
+  public void setUp() {
+    locatorPort = getRandomAvailableTCPPort();
+  }
 
-    assertThat(execution.getOutputText()).contains("IGNORED", "Index named \"i1\" not found");
+  @Test
+  public void destroyIndexIfExists()
+      throws IOException, ExecutionException, InterruptedException, TimeoutException {
+    GfshExecution execution = GfshScript
+        .of("start locator --name=locator --port=" + locatorPort,
+            "start server --name=server --server-port=0",
+            "sleep --time=1",
+            "destroy index --name=i1 --if-exists=true")
+        .execute(gfshRule);
+
+    assertThat(execution.getOutputText())
+        .contains("IGNORED", "Index named \"i1\" not found");
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/GfshStartLocatorLogAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/GfshStartLocatorLogAcceptanceTest.java
index ac97fa1784..f9394adc8c 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/GfshStartLocatorLogAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/GfshStartLocatorLogAcceptanceTest.java
@@ -14,9 +14,14 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
 import static org.apache.geode.logging.internal.Configuration.STARTUP_CONFIGURATION;
+import static org.apache.geode.test.assertj.LogFileAssert.assertThat;
 
 import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
 
 import org.junit.Before;
 import org.junit.Rule;
@@ -24,9 +29,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.internal.logging.Banner;
-import org.apache.geode.test.assertj.LogFileAssert;
 import org.apache.geode.test.junit.categories.GfshTest;
 import org.apache.geode.test.junit.categories.LoggingTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
@@ -36,24 +41,36 @@ public class GfshStartLocatorLogAcceptanceTest {
 
   private File logFile;
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   @Before
-  public void setUp() {
-    GfshExecution gfshExecution = GfshScript.of("start locator").execute(gfshRule);
-    File[] files = gfshExecution.getWorkingDir().listFiles();
-    String logName = files[0].getAbsolutePath() + "/" + files[0].getName() + ".log";
-    logFile = new File(logName);
+  public void setUp()
+      throws IOException, ExecutionException, InterruptedException, TimeoutException {
+    int locatorPort = getRandomAvailableTCPPort();
+
+    GfshExecution execution = GfshScript
+        .of("start locator --name=locator --port=" + locatorPort)
+        .execute(gfshRule);
+
+    logFile = execution
+        .getSubDir("locator")
+        .resolve("locator.log")
+        .toAbsolutePath()
+        .toFile();
   }
 
   @Test
   public void bannerIsLoggedOnlyOnce() {
-    LogFileAssert.assertThat(logFile).containsOnlyOnce(Banner.BannerHeader.displayValues());
+    assertThat(logFile)
+        .containsOnlyOnce(Banner.BannerHeader.displayValues());
   }
 
   @Test
   public void startupConfigIsLoggedOnlyOnce() {
-    LogFileAssert.assertThat(logFile).containsOnlyOnce(STARTUP_CONFIGURATION);
+    assertThat(logFile)
+        .containsOnlyOnce(STARTUP_CONFIGURATION);
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/ImportClusterConfigTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/ImportClusterConfigTest.java
index 834a4bb3a0..d9fa9a59d7 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/ImportClusterConfigTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/ImportClusterConfigTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.geode.management.internal.cli.commands;
 
 import static org.assertj.core.api.Assertions.assertThat;
@@ -28,31 +27,38 @@ import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class ImportClusterConfigTest extends ClusterConfigTestBase {
+
   private static final String locatorName = "locator";
   private static final String serverNotShutDownName = "serverNotShutDown";
 
   @Rule
-  public GfshRule gfsh = new GfshRule();
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   @Test
   public void importWouldNotShutDownServer() {
     GfshExecution startCluster = GfshScript
         .of("start locator --name=" + locatorName,
-            "start server --name=" + serverNotShutDownName + " --server-port=0")
-        .withName("startCluster").execute(gfsh);
-    assertThat(startCluster.getOutputText()).contains(locatorName + " is currently online")
+            "start server --name=" + serverNotShutDownName + " --disable-default-server")
+        .withName("startCluster")
+        .execute(gfshRule);
+    assertThat(startCluster.getOutputText())
+        .contains(locatorName + " is currently online")
         .contains(serverNotShutDownName + " is currently online");
 
     GfshExecution importConfiguration = GfshScript
         .of("connect", "import cluster-configuration --zip-file-name=" + clusterConfigZipPath)
-        .withName("importConfiguration").execute(gfsh);
+        .withName("importConfiguration")
+        .execute(gfshRule);
     assertThat(importConfiguration.getOutputText())
         .contains("Cluster configuration successfully imported")
         .contains("Configure the servers in 'cluster' group: ");
 
-    GfshExecution listMembers =
-        GfshScript.of("connect", "list members").withName("listMembers").execute(gfsh);
-    assertThat(listMembers.getOutputText()).contains("serverNotShutDown");
+    GfshExecution listMembers = GfshScript
+        .of("connect",
+            "list members")
+        .withName("listMembers")
+        .execute(gfshRule);
+    assertThat(listMembers.getOutputText())
+        .contains("serverNotShutDown");
   }
-
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/LogsAndDescribeConfigAreFullyRedactedAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/LogsAndDescribeConfigAreFullyRedactedAcceptanceTest.java
index e569709307..30f2c4a0b2 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/LogsAndDescribeConfigAreFullyRedactedAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/LogsAndDescribeConfigAreFullyRedactedAcceptanceTest.java
@@ -17,25 +17,28 @@ package org.apache.geode.management.internal.cli.commands;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.apache.geode.examples.security.ExampleSecurityManager.SECURITY_JSON;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
 import static org.apache.geode.test.util.ResourceUtils.createFileFromResource;
 import static org.apache.geode.test.util.ResourceUtils.getResource;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Path;
 import java.util.Properties;
 
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.examples.security.ExampleSecurityManager;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.test.assertj.LogFileAssert;
 import org.apache.geode.test.junit.categories.LoggingTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.RequiresGeodeHome;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
@@ -54,25 +57,29 @@ public class LogsAndDescribeConfigAreFullyRedactedAcceptanceTest {
 
   private static final String PASSWORD = "abcdefg";
 
-  @Rule
+  private int locatorPort;
+
+  @Rule(order = 0)
   public RequiresGeodeHome geodeHome = new RequiresGeodeHome();
-  @Rule
-  public GfshRule gfsh = new GfshRule();
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  @Rule(order = 1)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 2)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   @Before
-  public void createDirectoriesAndFiles() throws Exception {
-    File geodePropertiesFile = temporaryFolder.newFile("geode.properties");
-    File securityPropertiesFile = temporaryFolder.newFile("security.properties");
+  public void createDirectoriesAndFiles() throws IOException {
+    Path rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+
+    Path geodePropertiesFile = rootFolder.resolve("geode.properties");
+    Path securityPropertiesFile = rootFolder.resolve("security.properties");
 
     Properties geodeProperties = new Properties();
     geodeProperties.setProperty(LOG_LEVEL, "debug");
     geodeProperties.setProperty("security-username", "propertyFileUser");
     geodeProperties.setProperty("security-password", PASSWORD + "-propertyFile");
 
-    try (FileOutputStream fileOutputStream = new FileOutputStream(geodePropertiesFile)) {
-      geodeProperties.store(fileOutputStream, null);
+    try (FileOutputStream fos = new FileOutputStream(geodePropertiesFile.toFile())) {
+      geodeProperties.store(fos, null);
     }
 
     Properties securityProperties = new Properties();
@@ -81,41 +88,46 @@ public class LogsAndDescribeConfigAreFullyRedactedAcceptanceTest {
     securityProperties.setProperty("security-file-username", "securityPropertyFileUser");
     securityProperties.setProperty("security-file-password", PASSWORD + "-securityPropertyFile");
 
-    try (FileOutputStream fileOutputStream = new FileOutputStream(securityPropertiesFile)) {
-      securityProperties.store(fileOutputStream, null);
+    try (FileOutputStream fos = new FileOutputStream(securityPropertiesFile.toFile())) {
+      securityProperties.store(fos, null);
     }
 
     // The json is in the root resource directory.
-    createFileFromResource(getResource("/security.json"), temporaryFolder.getRoot(),
+    createFileFromResource(getResource("/security.json"), rootFolder.toFile(),
         "security.json");
 
+    locatorPort = getRandomAvailableTCPPort();
+
     String startLocatorCmd = new CommandStringBuilder("start locator")
         .addOption("name", "test-locator")
-        .addOption("properties-file", geodePropertiesFile.getAbsolutePath())
-        .addOption("security-properties-file", securityPropertiesFile.getAbsolutePath())
+        .addOption("port", String.valueOf(locatorPort))
+        .addOption("properties-file", geodePropertiesFile.toString())
+        .addOption("security-properties-file", securityPropertiesFile.toString())
         .addOption("J", "-Dsecure-username-jd=user-jd")
         .addOption("J", "-Dsecure-password-jd=password-jd")
-        .addOption("classpath", temporaryFolder.getRoot().getAbsolutePath())
+        .addOption("classpath", rootFolder.toString())
         .getCommandString();
 
     String startServerCmd = new CommandStringBuilder("start server")
         .addOption("name", "test-server")
+        .addOption("locators", "localhost[" + locatorPort + "]")
+        .addOption("disable-default-server", "true")
         .addOption("user", "viaStartMemberOptions")
         .addOption("password", PASSWORD + "-viaStartMemberOptions")
-        .addOption("properties-file", geodePropertiesFile.getAbsolutePath())
-        .addOption("security-properties-file", securityPropertiesFile.getAbsolutePath())
+        .addOption("properties-file", geodePropertiesFile.toString())
+        .addOption("security-properties-file", securityPropertiesFile.toString())
         .addOption("J", "-Dsecure-username-jd=user-jd")
         .addOption("J", "-Dsecure-password-jd=" + PASSWORD + "-password-jd")
-        .addOption("server-port", "0")
-        .addOption("classpath", temporaryFolder.getRoot().getAbsolutePath())
+        .addOption("classpath", rootFolder.toString())
         .getCommandString();
 
-    gfsh.execute(startLocatorCmd, startServerCmd);
+    gfshRule.execute(startLocatorCmd, startServerCmd);
   }
 
   @Test
   public void logsDoNotContainStringThatShouldBeRedacted() {
-    File dir = gfsh.getTemporaryFolder().getRoot();
+    Path rootFolder = folderRule.getFolder().toPath();
+    File dir = rootFolder.toFile();
     File[] logFiles = dir.listFiles((d, name) -> name.endsWith(".log"));
 
     for (File logFile : logFiles) {
@@ -126,17 +138,24 @@ public class LogsAndDescribeConfigAreFullyRedactedAcceptanceTest {
   @Test
   public void describeConfigRedactsJvmArguments() {
     String connectCommand = new CommandStringBuilder("connect")
+        .addOption("locator", "localhost[" + locatorPort + "]")
         .addOption("user", "viaStartMemberOptions")
-        .addOption("password", PASSWORD + "-viaStartMemberOptions").getCommandString();
+        .addOption("password", PASSWORD + "-viaStartMemberOptions")
+        .getCommandString();
 
     String describeLocatorConfigCommand = new CommandStringBuilder("describe config")
-        .addOption("hide-defaults", "false").addOption("member", "test-locator").getCommandString();
+        .addOption("hide-defaults", "false")
+        .addOption("member", "test-locator")
+        .getCommandString();
 
     String describeServerConfigCommand = new CommandStringBuilder("describe config")
-        .addOption("hide-defaults", "false").addOption("member", "test-server").getCommandString();
+        .addOption("hide-defaults", "false")
+        .addOption("member", "test-server")
+        .getCommandString();
 
     GfshExecution execution =
-        gfsh.execute(connectCommand, describeLocatorConfigCommand, describeServerConfigCommand);
+        gfshRule.execute(connectCommand, describeLocatorConfigCommand,
+            describeServerConfigCommand);
     assertThat(execution.getOutputText()).doesNotContain(PASSWORD);
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/PutCommandWithJsonTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/PutCommandWithJsonTest.java
index 394f355296..8e684d4fbd 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/PutCommandWithJsonTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/PutCommandWithJsonTest.java
@@ -14,48 +14,59 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
 import org.apache.geode.test.compiler.JarBuilder;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class PutCommandWithJsonTest {
 
-  private File jarToDeploy;
+  private Path jarToDeploy;
+  private int locatorPort;
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   @Before
   public void setup() throws IOException {
-    jarToDeploy = new File(gfsh.getTemporaryFolder().getRoot(), "ourJar.jar");
+    Path rootFolder = folderRule.getFolder().toPath();
+    jarToDeploy = rootFolder.resolve("ourJar.jar");
 
     String classContents =
         "public class Customer implements java.io.Serializable {private String name; public void setName(String name){this.name=name;}}";
     JarBuilder jarBuilder = new JarBuilder();
-    jarBuilder.buildJar(jarToDeploy, classContents);
+    jarBuilder.buildJar(jarToDeploy.toFile(), classContents);
+
+    locatorPort = getRandomAvailableTCPPort();
   }
 
   @Test
-  public void putWithJsonString() throws Exception {
+  public void putWithJsonString() {
     GfshExecution execution = GfshScript
-        .of("start locator --name=locator", "start server --name=server --server-port=0",
+        .of("start locator --name=locator --port=" + locatorPort,
+            "start server --name=server --disable-default-server --locators=localhost["
+                + locatorPort + "]",
             "sleep --time=1",
-            "deploy --jar=" + jarToDeploy.getAbsolutePath(),
+            "deploy --jar=" + jarToDeploy,
             "create region --name=region --type=REPLICATE", "sleep --time=1",
             "put --region=region --key=key --value=('name':'Jinmei') --value-class=Customer")
-        .execute(gfsh);
+        .execute(gfshRule);
 
-    assertThat(execution.getOutputText()).doesNotContain("Couldn't convert JSON to Object");
-    assertThat(execution.getOutputText()).contains("Value Class : Customer");
+    assertThat(execution.getOutputText())
+        .contains("Value Class : Customer")
+        .doesNotContain("Couldn't convert JSON to Object");
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartLocatorAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartLocatorAcceptanceTest.java
index 7c1894f96d..2e4365fb11 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartLocatorAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartLocatorAcceptanceTest.java
@@ -14,43 +14,61 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
 import static org.assertj.core.api.Assertions.assertThat;
 
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class StartLocatorAcceptanceTest {
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+
+  private int locatorPort;
+
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
+  @Before
+  public void setUp() {
+    locatorPort = getRandomAvailableTCPPort();
+  }
 
   @Test
-  public void startLocatorWithAutoConnectShouldBeConnectedAndRetrieveClusterConfigurationStatus()
-      throws Exception {
-    GfshExecution execution = GfshScript.of("start locator --name=locator1").execute(gfshRule);
-    assertThat(execution.getOutputText()).contains("Successfully connected to: JMX Manager");
+  public void startLocatorWithAutoConnectShouldBeConnectedAndRetrieveClusterConfigurationStatus() {
+    GfshExecution execution = GfshScript
+        .of("start locator --name=locator1 --port=" + locatorPort)
+        .execute(gfshRule);
+
     assertThat(execution.getOutputText())
+        .contains("Successfully connected to: JMX Manager")
         .contains("Cluster configuration service is up and running.");
   }
 
   @Test
-  public void startLocatorWithConnectFalseShouldNotBeConnectedAndNotRetrieveClusterConfigurationStatus()
-      throws Exception {
-    GfshExecution execution =
-        GfshScript.of("start locator --name=locator1 --connect=false").execute(gfshRule);
-    assertThat(execution.getOutputText()).doesNotContain("Successfully connected to: JMX Manager");
+  public void startLocatorWithConnectFalseShouldNotBeConnectedAndNotRetrieveClusterConfigurationStatus() {
+    GfshExecution execution = GfshScript
+        .of("start locator --name=locator1 --connect=false --port=" + locatorPort)
+        .execute(gfshRule);
+
     assertThat(execution.getOutputText())
+        .doesNotContain("Successfully connected to: JMX Manager")
         .doesNotContain("Cluster configuration service is up and running.");
   }
 
   @Test
-  public void startLocatorWithSecurityManagerShouldNotBeConnected() throws Exception {
+  public void startLocatorWithSecurityManagerShouldNotBeConnected() {
     GfshExecution execution = GfshScript
-        .of("start locator --name=locator1 --J=-Dgemfire.security-manager=org.apache.geode.examples.SimpleSecurityManager")
+        .of("start locator --name=locator1 --J=-Dgemfire.security-manager=org.apache.geode.examples.SimpleSecurityManager --port="
+            + locatorPort)
         .execute(gfshRule);
+
     assertThat(execution.getOutputText())
         .contains("Unable to auto-connect (Security Manager may be enabled)");
   }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartServerCommandAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartServerCommandAcceptanceTest.java
index b955bf7ee4..fb9a665bcf 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartServerCommandAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StartServerCommandAcceptanceTest.java
@@ -14,28 +14,28 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.nio.file.Path;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.LineIterator;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.TestName;
 
 import org.apache.geode.cache.server.CacheServer;
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.cache.xmlcache.CacheCreation;
 import org.apache.geode.internal.cache.xmlcache.CacheXmlGenerator;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
-import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
 
 /**
  * Start server parameters should take precedence over cache.xml file and cluster configuration
@@ -43,24 +43,32 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
  */
 public class StartServerCommandAcceptanceTest {
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  private static final String SERVER_NAME = "server";
 
-  @Rule
-  public TestName testName = new SerializableTestName();
+  private Path rootFolder;
 
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule(FolderRule.Policy.KEEP_ALWAYS);
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
+  @Before
+  public void setUp() {
+    rootFolder = folderRule.getFolder().toPath();
+  }
 
   @Test
-  public void startStandaloneServerWithParametersShouldOverrideCacheXmlConfiguration()
-      throws IOException {
-    File logFile = temporaryFolder.newFile(testName.getMethodName() + ".log");
-    File cacheXmlFile = temporaryFolder.newFile(testName.getMethodName() + "Cache.xml");
+  public void parametersOverrideCacheXml() throws IOException {
+    Path logFile = rootFolder.resolve(SERVER_NAME + ".log");
+    Path cacheXmlFile = rootFolder.resolve(SERVER_NAME + "Cache.xml");
+
+    int[] ports = getRandomAvailableTCPPorts(2);
+    int serverPortInXml = ports[0];
+    int serverPortInGfsh = ports[1];
 
     CacheCreation creation = new CacheCreation();
     CacheServer server = creation.addCacheServer();
-    server.setPort(40404);
+    server.setPort(serverPortInXml);
     server.setBindAddress(null);
     server.setHostnameForClients(null);
     server.setMaxConnections(800);
@@ -68,77 +76,91 @@ public class StartServerCommandAcceptanceTest {
     server.setMaximumMessageCount(230000);
     server.setMessageTimeToLive(180);
     server.setSocketBufferSize(32768);
-    PrintWriter pw = new PrintWriter(new FileWriter(cacheXmlFile), true);
-    CacheXmlGenerator.generate(creation, pw);
-    pw.close();
-
-    Integer serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    String startServerCommand =
-        "start server --max-threads=100 --max-connections=1200 --max-message-count=5000 --message-time-to-live=360 --socket-buffer-size=16384 --server-port="
-            + serverPort + " --name=" + testName.getMethodName() + " --cache-xml-file="
-            + cacheXmlFile.getAbsolutePath() + " --J=-Dgemfire.log-file=" + logFile
-                .getAbsolutePath();
-
-    GfshExecution execution = GfshScript.of(startServerCommand).execute(gfshRule);
+
+    try (PrintWriter pw = new PrintWriter(new FileWriter(cacheXmlFile.toFile()), true)) {
+      CacheXmlGenerator.generate(creation, pw);
+    }
+
+    String startServerCommand = String.join(" ",
+        "start server",
+        "--max-threads=100",
+        "--max-connections=1200",
+        "--max-message-count=5000",
+        "--message-time-to-live=360",
+        "--socket-buffer-size=16384",
+        "--server-port=" + serverPortInGfsh,
+        "--name=" + SERVER_NAME,
+        "--cache-xml-file=" + cacheXmlFile,
+        "--J=-Dgemfire.log-file=" + logFile);
+
+    GfshExecution execution = GfshScript
+        .of(startServerCommand)
+        .execute(gfshRule);
     assertThat(execution.getOutputText())
-        .containsPattern("Server .* " + testName.getMethodName() + " is currently online.");
+        .containsPattern("Server .* " + SERVER_NAME + " is currently online.");
 
     // Assert Server Properties.
     Boolean configurationLineFound = Boolean.FALSE;
-    LineIterator lineIterator = FileUtils.lineIterator(logFile);
+    LineIterator lineIterator = FileUtils.lineIterator(logFile.toFile());
     while (lineIterator.hasNext()) {
       String line = lineIterator.nextLine();
       if (line.contains("CacheServer Configuration:")) {
         configurationLineFound = Boolean.TRUE;
-        assertThat(line.contains("max-threads=100")).isTrue();
-        assertThat(line.contains("port=" + serverPort)).isTrue();
-        assertThat(line.contains("max-connections=1200")).isTrue();
-        assertThat(line.contains("message-time-to-live=360")).isTrue();
-        assertThat(line.contains("socket-buffer-size=16384")).isTrue();
-        assertThat(line.contains("maximum-message-count=5000")).isTrue();
+        assertThat(line).contains("max-threads=100");
+        assertThat(line).contains("port=" + serverPortInGfsh);
+        assertThat(line).contains("max-connections=1200");
+        assertThat(line).contains("message-time-to-live=360");
+        assertThat(line).contains("socket-buffer-size=16384");
+        assertThat(line).contains("maximum-message-count=5000");
       }
     }
 
     assertThat(configurationLineFound).isTrue();
   }
 
-
   @Test
-  public void startServerWithParametersWhenClusterConfigurationServiceIsEnabledShouldOverrideDefaults()
-      throws IOException {
-    Integer serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    File logFile = temporaryFolder.newFile(testName.getMethodName() + ".log");
-
-    String startServerCommand =
-        "start server --max-threads=50 --max-connections=200 --max-message-count=500 --message-time-to-live=120 --socket-buffer-size=8192 --server-port="
-            + serverPort + " --use-cluster-configuration=true --name=" + testName.getMethodName()
-            + " --J=-Dgemfire.log-file=" + logFile.getAbsolutePath();
+  public void usesClusterConfigurationIfEnabled() throws IOException {
+    int serverPort = getRandomAvailableTCPPort();
+    Path logFile = rootFolder.resolve(SERVER_NAME + ".log");
+
+    String startServerCommand = String.join(" ",
+        "start server",
+        "--max-threads=50",
+        "--max-connections=200",
+        "--max-message-count=500",
+        "--message-time-to-live=120",
+        "--socket-buffer-size=8192",
+        "--server-port=" + serverPort,
+        "--use-cluster-configuration=true",
+        "--name=" + SERVER_NAME,
+        "--J=-Dgemfire.log-file=" + logFile);
 
     // Start Locator, configure PDX (just to have a non-empty cluster-configuration) and start
     // server.
     GfshExecution startClusterExecution = GfshScript
-        .of("start locator --name=locator1 --connect=true --enable-cluster-configuration=true",
-            "configure pdx --read-serialized=true", startServerCommand)
+        .of("start locator --name=locator --connect=true --enable-cluster-configuration=true",
+            "configure pdx --read-serialized=true",
+            startServerCommand)
         .execute(gfshRule);
 
     assertThat(startClusterExecution.getOutputText())
         .contains("Successfully connected to: JMX Manager")
         .contains("Cluster configuration for group 'cluster' is updated")
-        .containsPattern("Server .* " + testName.getMethodName() + " is currently online.");
+        .containsPattern("Server .* " + SERVER_NAME + " is currently online.");
 
     // Assert Server Properties.
-    Boolean configurationLineFound = Boolean.FALSE;
-    LineIterator lineIterator = FileUtils.lineIterator(logFile);
+    boolean configurationLineFound = false;
+    LineIterator lineIterator = FileUtils.lineIterator(logFile.toFile());
     while (lineIterator.hasNext()) {
       String line = lineIterator.nextLine();
       if (line.contains("CacheServer Configuration:")) {
-        configurationLineFound = Boolean.TRUE;
-        assertThat(line.contains("max-threads=50")).isTrue();
-        assertThat(line.contains("port=" + serverPort)).isTrue();
-        assertThat(line.contains("max-connections=200")).isTrue();
-        assertThat(line.contains("message-time-to-live=120")).isTrue();
-        assertThat(line.contains("socket-buffer-size=8192")).isTrue();
-        assertThat(line.contains("maximum-message-count=500")).isTrue();
+        configurationLineFound = true;
+        assertThat(line).contains("max-threads=50");
+        assertThat(line).contains("port=" + serverPort);
+        assertThat(line).contains("max-connections=200");
+        assertThat(line).contains("message-time-to-live=120");
+        assertThat(line).contains("socket-buffer-size=8192");
+        assertThat(line).contains("maximum-message-count=500");
       }
     }
 
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StatusLocatorRealGfshTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StatusLocatorRealGfshTest.java
index b8d62459e9..004913d780 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StatusLocatorRealGfshTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StatusLocatorRealGfshTest.java
@@ -14,28 +14,53 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class StatusLocatorRealGfshTest {
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+
+  private int locatorPort;
+
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
+  @Before
+  public void setUp() {
+    locatorPort = getRandomAvailableTCPPort();
+  }
 
   @Test
-  public void statusLocatorSucceedsWhenConnected() throws Exception {
-    GfshScript.of("start locator --name=locator1").execute(gfshRule);
+  public void statusLocatorSucceedsWhenConnected() {
+    GfshScript
+        .of("start locator --name=locator1 --port=" + locatorPort)
+        .execute(gfshRule);
 
-    GfshScript.of("connect", "status locator --name=locator1").execute(gfshRule);
+    GfshScript
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "status locator --name=locator1")
+        .execute(gfshRule);
   }
 
   @Test
-  public void statusLocatorFailsWhenNotConnected() throws Exception {
-    GfshScript.of("start locator --name=locator1").withName("start-locator").execute(gfshRule);
+  public void statusLocatorFailsWhenNotConnected() {
+    GfshScript
+        .of("start locator --name=locator1 --port=" + locatorPort)
+        .withName("start-locator")
+        .execute(gfshRule);
 
-    GfshScript.of("status locator --name=locator1").withName("status-locator").expectFailure()
+    GfshScript
+        .of("status locator --name=locator1")
+        .withName("status-locator")
+        .expectFailure()
         .execute(gfshRule);
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerAcceptanceTest.java
index c7568b15f2..5a83558539 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerAcceptanceTest.java
@@ -14,36 +14,58 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class StopServerAcceptanceTest {
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  private int locatorPort;
 
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   @Before
-  public void startCluster() {
-    gfshRule.execute("start locator --name=locator", "start server --name=server --server-port=0");
+  public void setUp()
+      throws IOException, ExecutionException, InterruptedException, TimeoutException {
+    locatorPort = getRandomAvailableTCPPort();
+
+    gfshRule.execute(GfshScript
+        .of("start locator --name=locator --port=" + locatorPort,
+            "start server --name=server --disable-default-server --locators=localhost["
+                + locatorPort + "]"));
   }
 
   @Test
-  public void canStopServerByNameWhenConnectedOverJmx() throws Exception {
-    gfshRule.execute("connect", "stop server --name=server");
+  public void canStopServerByNameWhenConnectedOverJmx() {
+    gfshRule.execute(GfshScript
+        .of("connect --locator=localhost[" + locatorPort + "]",
+            "stop server --name=server"));
   }
 
   @Test
-  public void canStopServerByNameWhenConnectedOverHttp() throws Exception {
-    gfshRule.execute("connect --use-http", "stop server --name=server");
+  public void canStopServerByNameWhenConnectedOverHttp() {
+    gfshRule.execute(GfshScript
+        .of("connect --use-http --locator=localhost[" + locatorPort + "]",
+            "stop server --name=server"));
   }
 
   @Test
-  public void cannotStopServerByNameWhenNotConnected() throws Exception {
-    gfshRule.execute(GfshScript.of("stop server --name=server").expectFailure());
+  public void cannotStopServerByNameWhenNotConnected() {
+    gfshRule.execute(GfshScript
+        .of("stop server --name=server")
+        .expectFailure());
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerWithSecurityAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerWithSecurityAcceptanceTest.java
index 99a2c0037e..76d401c6ce 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerWithSecurityAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerWithSecurityAcceptanceTest.java
@@ -19,6 +19,8 @@ import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Path;
 import java.util.Properties;
 
 import org.junit.Before;
@@ -27,48 +29,49 @@ import org.junit.Test;
 
 import org.apache.geode.examples.SimpleSecurityManager;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class StopServerWithSecurityAcceptanceTest {
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  private static final Properties securityProps = new Properties();
-
-  static {
-    securityProps.setProperty(SECURITY_MANAGER, SimpleSecurityManager.class.getName());
-    securityProps.setProperty("security-username", "cluster");
-    securityProps.setProperty("security-password", "cluster");
-  }
+  private final Properties securityProps = newSecurityProperties();
 
   private File securityPropertiesFile;
 
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
   @Before
-  public void before() throws Exception {
-    securityPropertiesFile = gfshRule.getTemporaryFolder().newFile("security.properties");
-    securityProps.store(new FileOutputStream(securityPropertiesFile), null);
+  public void setUp() throws IOException {
+    Path rootFolder = folderRule.getFolder().toPath();
+    securityPropertiesFile = rootFolder.resolve("security.properties").toFile();
+    try (FileOutputStream out = new FileOutputStream(securityPropertiesFile)) {
+      securityProps.store(out, null);
+    }
   }
 
   @Test
-  public void cannotStopServerAsDataReaderOverHttp() throws Exception {
+  public void cannotStopServerAsDataReaderOverHttp() {
     startCluster();
 
     GfshExecution stopServer = dataReaderCannotStopServer(true);
-    assertThat(stopServer.getOutputText()).contains("dataReader not authorized for CLUSTER:READ");
+    assertThat(stopServer.getOutputText())
+        .contains("dataReader not authorized for CLUSTER:READ");
   }
 
   @Test
-  public void canStopServerAsClusterAdminOverHttp() throws Exception {
+  public void canStopServerAsClusterAdminOverHttp() {
     startCluster();
 
     clusterAdminCanStopServer(true);
   }
 
   @Test
-  public void cannotStopServerAsDataReaderOverJmx() throws Exception {
+  public void cannotStopServerAsDataReaderOverJmx() {
     startCluster();
 
     GfshExecution stopServer = dataReaderCannotStopServer(false);
@@ -76,14 +79,14 @@ public class StopServerWithSecurityAcceptanceTest {
   }
 
   @Test
-  public void canStopServerAsClusterAdminOverJmx() throws Exception {
+  public void canStopServerAsClusterAdminOverJmx() {
     startCluster();
 
     clusterAdminCanStopServer(false);
   }
 
   @Test
-  public void cannotStopServerAsClusterReaderOverJmx() throws Exception {
+  public void cannotStopServerAsClusterReaderOverJmx() {
     startCluster();
 
     GfshExecution stopServer = clusterReaderCannotStopServer(false);
@@ -92,7 +95,7 @@ public class StopServerWithSecurityAcceptanceTest {
   }
 
   @Test
-  public void cannotStopServerAsClusterReaderOverHttp() throws Exception {
+  public void cannotStopServerAsClusterReaderOverHttp() {
     startCluster();
 
     GfshExecution stopServer = clusterReaderCannotStopServer(true);
@@ -100,34 +103,57 @@ public class StopServerWithSecurityAcceptanceTest {
         .contains("clusterRead not authorized for CLUSTER:MANAGE");
   }
 
-  private GfshExecution startCluster() {
-    String startLocator = new CommandStringBuilder("start locator").addOption("name", "locator")
-        .addOption("security-properties-file", securityPropertiesFile.getAbsolutePath()).toString();
+  private Properties newSecurityProperties() {
+    Properties properties = new Properties();
+    properties.setProperty(SECURITY_MANAGER, SimpleSecurityManager.class.getName());
+    properties.setProperty("security-username", "cluster");
+    properties.setProperty("security-password", "cluster");
+    return properties;
+  }
 
-    String startServer = new CommandStringBuilder("start server").addOption("name", "server")
-        .addOption("server-port", "0")
+  private GfshExecution startCluster() {
+    String startLocator = new CommandStringBuilder("start locator")
+        .addOption("name", "locator")
+        .addOption("security-properties-file", securityPropertiesFile.getAbsolutePath())
+        .toString();
+
+    String startServer = new CommandStringBuilder("start server")
+        .addOption("name", "server")
+        .addOption("disable-default-server", "true")
         .addOption("security-properties-file", securityPropertiesFile.getAbsolutePath()).toString();
 
-    return GfshScript.of(startLocator, startServer).withName("cluster-setup").execute(gfshRule);
+    return GfshScript
+        .of(startLocator,
+            startServer)
+        .withName("cluster-setup")
+        .execute(gfshRule);
   }
 
   private GfshExecution dataReaderCannotStopServer(boolean useHttp) {
-    return GfshScript.of(connectCommand("dataReader", useHttp), "stop server --name=server")
+    return GfshScript
+        .of(connectCommand("dataReader", useHttp),
+            "stop server --name=server")
         .expectFailure().execute(gfshRule);
   }
 
   private GfshExecution clusterAdminCanStopServer(boolean useHttp) {
-    return GfshScript.of(connectCommand("cluster", useHttp), "stop server --name=server")
+    return GfshScript
+        .of(connectCommand("cluster", useHttp),
+            "stop server --name=server")
         .execute(gfshRule);
   }
 
   private GfshExecution clusterReaderCannotStopServer(boolean useHttp) {
-    return GfshScript.of(connectCommand("clusterRead", useHttp), "stop server --name=server")
-        .expectFailure().execute(gfshRule);
+    return GfshScript
+        .of(connectCommand("clusterRead", useHttp),
+            "stop server --name=server")
+        .expectFailure()
+        .execute(gfshRule);
   }
 
   private String connectCommand(String permission, boolean useHttp) {
-    CommandStringBuilder cmd = new CommandStringBuilder("connect").addOption("user", permission)
+    CommandStringBuilder cmd = new CommandStringBuilder("connect")
+        .addOption("user", permission)
         .addOption("password", permission);
     if (useHttp) {
       cmd.addOption("use-http");
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java
index e2ebd32a3f..189e3cd005 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java
@@ -14,25 +14,35 @@
  */
 package org.apache.geode.management.internal.cli.shell;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class GfshDisconnectWithinScript {
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
+  private int locatorPort;
 
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   @Before
-  public void setup() {
-    gfsh.execute("start locator");
+  public void setUp() {
+    locatorPort = getRandomAvailableTCPPort();
+    gfshRule.execute("start locator --port=" + locatorPort);
   }
 
   @Test
   public void disconnectInScriptDoesNotRaiseNPE() {
-    gfsh.execute("connect", "disconnect", "echo \"Disconnect command resolved without issue.\"");
+    gfshRule.execute(
+        "connect --locator=localhost[" + locatorPort + "]",
+        "disconnect",
+        "echo \"Disconnect command resolved without issue.\"");
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/StatusLocatorExitCodeAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/StatusLocatorExitCodeAcceptanceTest.java
index 883dc70852..b48420bf8d 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/StatusLocatorExitCodeAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/StatusLocatorExitCodeAcceptanceTest.java
@@ -14,23 +14,21 @@
  */
 package org.apache.geode.management.internal.cli.shell;
 
-import static java.util.Arrays.stream;
-import static org.apache.geode.management.internal.cli.shell.DirectoryTree.printDirectoryTree;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+import static org.apache.geode.internal.process.ProcessType.LOCATOR;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.File;
-import java.io.IOException;
 import java.nio.file.Path;
+import java.nio.file.Paths;
 
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.ExitCode;
-import org.apache.geode.internal.process.PidFile;
 import org.apache.geode.test.junit.categories.GfshTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
@@ -43,57 +41,39 @@ public class StatusLocatorExitCodeAcceptanceTest {
 
   private static final String LOCATOR_NAME = "myLocator";
 
-  private static int locatorPort;
-  private static Path toolsJar;
-  private static int locatorPid;
-  private static Path locatorDir;
-  private static Path rootPath;
-  private static String connectCommand;
+  private int locatorPort;
+  private int locatorPid;
+  private Path locatorDir;
+  private String connectCommand;
 
-  @ClassRule
-  public static GfshRule gfshRule = new GfshRule();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  @BeforeClass
-  public static void startLocator() throws IOException {
-    rootPath = gfshRule.getTemporaryFolder().getRoot().toPath();
-    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+  @Before
+  public void startLocator() {
+    locatorPort = getRandomAvailableTCPPort();
 
-    GfshExecution execution = GfshScript.of(
-        "start locator --name=" + LOCATOR_NAME + " --port=" + locatorPort)
+    GfshExecution execution = GfshScript
+        .of("start locator --name=" + LOCATOR_NAME + " --port=" + locatorPort)
         .execute(gfshRule);
 
     assertThat(execution.getProcess().exitValue())
         .isZero();
 
-    locatorPid = readPidFile(LOCATOR_NAME, "locator.pid");
-    locatorDir = rootPath.resolve(LOCATOR_NAME).toAbsolutePath();
+    locatorDir = execution.getSubDir(LOCATOR_NAME);
+    locatorPid = LOCATOR.readPid(locatorDir);
 
     connectCommand = "connect --locator=[" + locatorPort + "]";
   }
 
-  @BeforeClass
-  public static void setUpJavaTools() {
-    String javaHome = System.getProperty("java.home");
-    assertThat(javaHome)
-        .as("System.getProperty(\"java.home\")")
-        .isNotNull();
-
-    Path javaHomeFile = new File(javaHome).toPath();
-    assertThat(javaHomeFile)
-        .as(javaHomeFile + ": " + printDirectoryTree(javaHomeFile.toFile()))
-        .exists();
-
-    String toolsPath = javaHomeFile.toFile().getName().equalsIgnoreCase("jre")
-        ? ".." + File.separator + "lib" + File.separator + "tools.jar"
-        : "lib" + File.separator + "tools.jar";
-    toolsJar = javaHomeFile.resolve(toolsPath);
-  }
-
   @Test
   public void statusCommandWithInvalidPortShouldFail() {
     String commandWithBadPort = "status locator --port=-10";
 
-    GfshScript.of(commandWithBadPort)
+    GfshScript
+        .of(commandWithBadPort)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -103,7 +83,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void statusCommandWithInvalidOptionValueShouldFail() {
     String commandWithBadPid = "status locator --pid=-1";
 
-    GfshScript.of(commandWithBadPid)
+    GfshScript
+        .of(commandWithBadPid)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -113,7 +94,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void statusCommandWithIncorrectHostShouldFail() {
     String commandWithWrongHostname = "status locator --host=someIncorrectHostname";
 
-    GfshScript.of(commandWithWrongHostname)
+    GfshScript
+        .of(commandWithWrongHostname)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -121,10 +103,11 @@ public class StatusLocatorExitCodeAcceptanceTest {
 
   @Test
   public void statusCommandWithIncorrectPortShouldFail() {
-    int incorrectPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    int incorrectPort = getRandomAvailableTCPPort();
     String commandWithWrongPort = "status locator --port=" + incorrectPort;
 
-    GfshScript.of(commandWithWrongPort)
+    GfshScript
+        .of(commandWithWrongPort)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -134,7 +117,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void statusCommandWithIncorrectDirShouldFail() {
     String commandWithWrongDir = "status locator --dir=.";
 
-    GfshScript.of(commandWithWrongDir)
+    GfshScript
+        .of(commandWithWrongDir)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -144,7 +128,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void statusCommandWithIncorrectNameShouldFail() {
     String commandWithWrongName = "status locator --name=some-locator-name";
 
-    GfshScript.of(commandWithWrongName)
+    GfshScript
+        .of(commandWithWrongName)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -154,7 +139,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void onlineStatusCommandShouldSucceedWhenConnected_locator_name() {
     String statusCommand = "status locator --name=" + LOCATOR_NAME;
 
-    GfshScript.of(connectCommand, statusCommand)
+    GfshScript
+        .of(connectCommand, statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -164,7 +150,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void onlineStatusCommandShouldSucceedWhenConnected_locator_port() {
     String statusCommand = "status locator --port=" + locatorPort;
 
-    GfshScript.of(connectCommand, statusCommand)
+    GfshScript
+        .of(connectCommand, statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -174,7 +161,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void onlineStatusCommandShouldSucceedWhenConnected_locator_host_and_port() {
     String statusCommand = "status locator --host=localhost --port=" + locatorPort;
 
-    GfshScript.of(connectCommand, statusCommand)
+    GfshScript
+        .of(connectCommand, statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -184,7 +172,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void onlineStatusCommandShouldFailWhenConnectedNonDefaultPort_locator_host() {
     String statusCommand = "status locator --host=localhost";
 
-    GfshScript.of(connectCommand, statusCommand)
+    GfshScript
+        .of(connectCommand, statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -194,7 +183,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void offlineStatusCommandShouldSucceedWhenConnected_locator_dir() {
     String statusCommand = "status locator --dir=" + locatorDir;
 
-    GfshScript.of(connectCommand, statusCommand)
+    GfshScript
+        .of(connectCommand, statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -204,7 +194,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void onlineStatusCommandShouldFailWhenNotConnected_locator_name() {
     String statusCommand = "status locator --name=" + LOCATOR_NAME;
 
-    GfshScript.of(statusCommand)
+    GfshScript
+        .of(statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -215,7 +206,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
     // --host defaults to localhost, so `status locator --port=xxx` should still succeed.
     String statusCommand = "status locator --port=" + locatorPort;
 
-    GfshScript.of(statusCommand)
+    GfshScript
+        .of(statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -225,9 +217,10 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void offlineStatusCommandShouldSucceedWhenConnected_locator_pid() {
     String statusCommand = "status locator --pid=" + locatorPid;
 
-    GfshScript.of(connectCommand, statusCommand)
+    GfshScript
+        .of(connectCommand, statusCommand)
         .withName("test-frame")
-        .addToClasspath(toolsJar.toFile().getAbsolutePath())
+        .addToClasspath(toolsJar())
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
   }
@@ -238,7 +231,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
     // --port=xxx` should succeed
     String statusCommand = "status locator --host=localhost --port=" + locatorPort;
 
-    GfshScript.of(statusCommand)
+    GfshScript
+        .of(statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -248,7 +242,8 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void offlineStatusCommandShouldSucceedEvenWhenNotConnected_locator_dir() {
     String statusCommand = "status locator --dir=" + locatorDir;
 
-    GfshScript.of(statusCommand)
+    GfshScript
+        .of(statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -258,27 +253,28 @@ public class StatusLocatorExitCodeAcceptanceTest {
   public void offlineStatusCommandShouldSucceedEvenWhenNotConnected_locator_pid() {
     String statusCommand = "status locator --pid=" + locatorPid;
 
-    GfshScript.of(statusCommand)
+    GfshScript
+        .of(statusCommand)
         .withName("test-frame")
-        .addToClasspath(toolsJar.toFile().getAbsolutePath())
+        .addToClasspath(toolsJar())
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
   }
 
-  private static int readPidFile(String memberName, String pidFileEndsWith) throws IOException {
-    File directory = rootPath.resolve(memberName).toFile();
-    File[] files = directory.listFiles();
-
-    assertThat(files)
-        .as(String.format("Expected directory ('%s') for member '%s'.", directory, memberName))
+  private static String toolsJar() {
+    String javaHome = System.getProperty("java.home");
+    assertThat(javaHome)
+        .as("java.home system property")
         .isNotNull();
 
-    File pidFile = stream(files)
-        .filter(file -> file.getName().endsWith(pidFileEndsWith))
-        .findFirst()
-        .orElseThrow(() -> new RuntimeException(String
-            .format("Expected member '%s' to have pid file but could not find it.", memberName)));
+    Path javaHomePath = Paths.get(javaHome).toAbsolutePath();
+    assertThat(javaHomePath)
+        .as("java.home path")
+        .exists();
+
+    boolean isJre = javaHomePath.getFileName().toString().equalsIgnoreCase("jre");
+    Path jdkPath = javaHomePath.resolve(isJre ? ".." : ".").normalize();
 
-    return new PidFile(pidFile).readPid();
+    return jdkPath.resolve("lib").resolve("tools.jar").toString();
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/StatusServerExitCodeAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/StatusServerExitCodeAcceptanceTest.java
index e880166080..4899afe4da 100755
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/StatusServerExitCodeAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/StatusServerExitCodeAcceptanceTest.java
@@ -14,23 +14,22 @@
  */
 package org.apache.geode.management.internal.cli.shell;
 
-import static java.util.Arrays.stream;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+import static org.apache.geode.internal.process.ProcessType.SERVER;
 import static org.apache.geode.management.internal.cli.shell.DirectoryTree.printDirectoryTree;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
-import java.io.IOException;
 import java.nio.file.Path;
 
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.ExitCode;
-import org.apache.geode.internal.process.PidFile;
 import org.apache.geode.test.junit.categories.GfshTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
@@ -44,37 +43,36 @@ public class StatusServerExitCodeAcceptanceTest {
   private static final String LOCATOR_NAME = "myLocator";
   private static final String SERVER_NAME = "myServer";
 
-  private static int locatorPort;
-  private static Path toolsJar;
-  private static int serverPid;
-  private static Path serverDir;
-  private static Path rootPath;
-  private static String connectCommand;
+  private Path toolsJar;
+  private int serverPid;
+  private Path serverDir;
+  private String connectCommand;
 
-  @ClassRule
-  public static GfshRule gfshRule = new GfshRule();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  @BeforeClass
-  public static void startCluster() throws IOException {
-    rootPath = gfshRule.getTemporaryFolder().getRoot().toPath();
-    locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+  @Before
+  public void startCluster() {
+    int locatorPort = getRandomAvailableTCPPort();
 
-    GfshExecution execution = GfshScript.of(
-        "start locator --name=" + LOCATOR_NAME + " --port=" + locatorPort,
-        "start server --disable-default-server --name=" + SERVER_NAME)
+    GfshExecution execution = GfshScript
+        .of("start locator --name=" + LOCATOR_NAME + " --port=" + locatorPort,
+            "start server --disable-default-server --name=" + SERVER_NAME)
         .execute(gfshRule);
 
     assertThat(execution.getProcess().exitValue())
         .isZero();
 
-    serverPid = readPidFile(SERVER_NAME, "server.pid");
-    serverDir = rootPath.resolve(SERVER_NAME).toAbsolutePath();
+    serverDir = execution.getWorkingDir().toPath().resolve(SERVER_NAME);
+    serverPid = SERVER.readPid(serverDir);
 
     connectCommand = "connect --locator=[" + locatorPort + "]";
   }
 
-  @BeforeClass
-  public static void setUpJavaTools() {
+  @Before
+  public void setUpJavaTools() {
     String javaHome = System.getProperty("java.home");
     assertThat(javaHome)
         .as("System.getProperty(\"java.home\")")
@@ -95,7 +93,8 @@ public class StatusServerExitCodeAcceptanceTest {
   public void statusCommandWithInvalidOptionValueShouldFail() {
     String commandWithBadPid = "status server --pid=-1";
 
-    GfshScript.of(commandWithBadPid)
+    GfshScript
+        .of(commandWithBadPid)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -105,7 +104,8 @@ public class StatusServerExitCodeAcceptanceTest {
   public void statusCommandWithIncorrectDirShouldFail() {
     String commandWithWrongDir = "status server --dir=.";
 
-    GfshScript.of(commandWithWrongDir)
+    GfshScript
+        .of(commandWithWrongDir)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -115,7 +115,8 @@ public class StatusServerExitCodeAcceptanceTest {
   public void statusCommandWithIncorrectNameShouldFail() {
     String commandWithWrongName = "status server --name=some-server-name";
 
-    GfshScript.of(commandWithWrongName)
+    GfshScript
+        .of(commandWithWrongName)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -125,7 +126,8 @@ public class StatusServerExitCodeAcceptanceTest {
   public void statusCommandWithIncorrectPidShouldFail() {
     String commandWithWrongPid = "status server --pid=100";
 
-    GfshScript.of(commandWithWrongPid)
+    GfshScript
+        .of(commandWithWrongPid)
         .withName("test-frame")
         .addToClasspath(toolsJar.toFile().getAbsolutePath())
         .expectExitCode(ExitCode.FATAL.getValue())
@@ -136,7 +138,8 @@ public class StatusServerExitCodeAcceptanceTest {
   public void onlineStatusCommandShouldFailWhenNotConnected_server_name() {
     String statusCommand = "status server --name=" + SERVER_NAME;
 
-    GfshScript.of(statusCommand)
+    GfshScript
+        .of(statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.FATAL.getValue())
         .execute(gfshRule);
@@ -146,7 +149,8 @@ public class StatusServerExitCodeAcceptanceTest {
   public void onlineStatusCommandShouldSucceedWhenConnected_server_name() {
     String statusCommand = "status server --name=" + SERVER_NAME;
 
-    GfshScript.of(connectCommand, statusCommand)
+    GfshScript
+        .of(connectCommand, statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -156,7 +160,8 @@ public class StatusServerExitCodeAcceptanceTest {
   public void offlineStatusCommandShouldSucceedWhenConnected_server_dir() {
     String statusCommand = "status server --dir=" + serverDir;
 
-    GfshScript.of(connectCommand, statusCommand)
+    GfshScript
+        .of(connectCommand, statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -166,7 +171,8 @@ public class StatusServerExitCodeAcceptanceTest {
   public void offlineStatusCommandShouldSucceedWhenConnected_server_pid() {
     String statusCommand = "status server --pid=" + serverPid;
 
-    GfshScript.of(connectCommand, statusCommand)
+    GfshScript
+        .of(connectCommand, statusCommand)
         .withName("test-frame")
         .addToClasspath(toolsJar.toFile().getAbsolutePath())
         .expectExitCode(ExitCode.NORMAL.getValue())
@@ -177,7 +183,8 @@ public class StatusServerExitCodeAcceptanceTest {
   public void offlineStatusCommandShouldSucceedEvenWhenNotConnected_server_dir() {
     String statusCommand = "status server --dir=" + serverDir;
 
-    GfshScript.of(statusCommand)
+    GfshScript
+        .of(statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .execute(gfshRule);
@@ -187,27 +194,11 @@ public class StatusServerExitCodeAcceptanceTest {
   public void offlineStatusCommandShouldSucceedEvenWhenNotConnected_server_pid() {
     String statusCommand = "status server --pid=" + serverPid;
 
-    GfshScript.of(statusCommand)
+    GfshScript
+        .of(statusCommand)
         .withName("test-frame")
         .expectExitCode(ExitCode.NORMAL.getValue())
         .addToClasspath(toolsJar.toFile().getAbsolutePath())
         .execute(gfshRule);
   }
-
-  private static int readPidFile(String memberName, String pidFileEndsWith) throws IOException {
-    File directory = rootPath.resolve(memberName).toFile();
-    File[] files = directory.listFiles();
-
-    assertThat(files)
-        .as(String.format("Expected directory ('%s') for member '%s'.", directory, memberName))
-        .isNotNull();
-
-    File pidFile = stream(files)
-        .filter(file -> file.getName().endsWith(pidFileEndsWith))
-        .findFirst()
-        .orElseThrow(() -> new RuntimeException(String
-            .format("Expected member '%s' to have pid file but could not find it.", memberName)));
-
-    return new PidFile(pidFile).readPid();
-  }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/RegionManagementAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/RegionManagementAcceptanceTest.java
index 01e1801684..b46eb70884 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/RegionManagementAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/RegionManagementAcceptanceTest.java
@@ -14,26 +14,48 @@
  */
 package org.apache.geode.management.internal.rest;
 
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.GeodeDevRestClient;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class RegionManagementAcceptanceTest {
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
+  private int locatorPort;
+  private int httpPort;
+
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
+  @Before
+  public void setUp() {
+    int[] ports = getRandomAvailableTCPPorts(2);
+    locatorPort = ports[0];
+    httpPort = ports[1];
+  }
 
   @Test
-  public void sanityCheck() throws Exception {
-    GfshScript.of("start locator --port=0").execute(gfsh);
+  public void sanityCheck() {
+    GfshScript
+        .of("start locator --port=" + locatorPort + " --J=-Dgemfire.http-service-port=" + httpPort)
+        .execute(gfshRule);
 
     // verify the management rest api is started correctly
     GeodeDevRestClient client =
-        new GeodeDevRestClient("/management/v1", "localhost", 7070, false);
+        new GeodeDevRestClient("/management/v1", "localhost", httpPort, false);
 
-    client.doGetAndAssert("/ping").hasStatusCode(200).hasResponseBody().isEqualTo("pong");
+    client
+        .doGetAndAssert("/ping")
+        .hasStatusCode(200)
+        .hasResponseBody()
+        .isEqualTo("pong");
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/StandaloneClientManagementAPIAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/StandaloneClientManagementAPIAcceptanceTest.java
index dec4914809..295974e0fe 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/StandaloneClientManagementAPIAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/StandaloneClientManagementAPIAcceptanceTest.java
@@ -14,7 +14,11 @@
  */
 package org.apache.geode.management.internal.rest;
 
+import static java.lang.System.lineSeparator;
+import static java.nio.file.Files.createDirectories;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.getTimeout;
 import static org.apache.geode.test.util.ResourceUtils.createTempFileFromResource;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -26,23 +30,22 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
 import java.util.stream.Collectors;
 
 import org.junit.After;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
 import org.junit.runners.Parameterized.UseParametersRunnerFactory;
 
-import org.apache.geode.internal.AvailablePortHelper;
-import org.apache.geode.test.awaitility.GeodeAwaitility;
 import org.apache.geode.test.compiler.JarBuilder;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
@@ -53,14 +56,6 @@ import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactor
 @UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class StandaloneClientManagementAPIAcceptanceTest {
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
-
-  @Rule
-  public TemporaryFolder tempDir = new TemporaryFolder();
-
-  private static String trustStorePath;
-
   @Parameters
   public static Collection<Boolean> data() {
     return Arrays.asList(true, false);
@@ -69,72 +64,94 @@ public class StandaloneClientManagementAPIAcceptanceTest {
   @Parameter
   public Boolean useSsl;
 
+  private String trustStorePath;
   private ProcessLogger clientProcessLogger;
+  private Path rootFolder;
+
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
+  @Before
+  public void setUp() {
+    rootFolder = folderRule.getFolder().toPath();
 
-  @BeforeClass
-  public static void beforeClass() {
     /*
      * This file was generated with:
      * keytool -genkey -dname "CN=localhost" -alias self -validity 3650 -keyalg EC \
      * -keystore trusted.keystore -keypass password -storepass password \
      * -ext san=ip:127.0.0.1,dns:localhost -storetype jks
      */
-    trustStorePath =
-        createTempFileFromResource(StandaloneClientManagementAPIAcceptanceTest.class,
-            "/ssl/trusted.keystore").getAbsolutePath();
-    assertThat(trustStorePath).as("java file resource not found").isNotBlank();
+    trustStorePath = createTempFileFromResource(
+        StandaloneClientManagementAPIAcceptanceTest.class, "/ssl/trusted.keystore")
+            .getAbsolutePath();
+    assertThat(trustStorePath)
+        .as("java file resource not found")
+        .isNotBlank();
   }
 
   @After
-  public void tearDown() throws Exception {
-    clientProcessLogger.awaitTermination(GeodeAwaitility.getTimeout().toMillis(), MILLISECONDS);
+  public void tearDown() throws InterruptedException, ExecutionException, TimeoutException {
+    clientProcessLogger.awaitTermination(getTimeout().toMillis(), MILLISECONDS);
     clientProcessLogger.close();
   }
 
   @Test
-  public void clientCreatesRegionUsingClusterManagementService() throws Exception {
+  public void clientCreatesRegionUsingClusterManagementService()
+      throws IOException, InterruptedException {
     JarBuilder jarBuilder = new JarBuilder();
-    String filePath =
-        createTempFileFromResource(getClass(), "/ManagementClientCreateRegion.java")
-            .getAbsolutePath();
-    assertThat(filePath).as("java file resource not found").isNotBlank();
+    String filePath = createTempFileFromResource(
+        getClass(), "/ManagementClientCreateRegion.java").getAbsolutePath();
+    assertThat(filePath)
+        .as("java file resource not found")
+        .isNotBlank();
 
-    File outputJar = new File(tempDir.getRoot(), "output.jar");
+    File outputJar = new File(rootFolder.toFile(), "output.jar");
     jarBuilder.buildJar(outputJar, new File(filePath));
 
-    int[] availablePorts = AvailablePortHelper.getRandomAvailableTCPPorts(3);
+    int[] availablePorts = getRandomAvailableTCPPorts(3);
     int locatorPort = availablePorts[0];
     int httpPort = availablePorts[1];
     int jmxPort = availablePorts[2];
-    GfshExecution startCluster =
-        GfshScript.of(
-            String.format(
-                "start locator --port=%d --http-service-port=%d --J=-Dgemfire.JMX_MANAGER_PORT=%d %s",
-                locatorPort, httpPort, jmxPort, getSslParameters()),
-            String.format("start server --locators=localhost[%d] --server-port=0", locatorPort))
-            .withName("startCluster").execute(gfsh);
 
+    GfshExecution startCluster = GfshScript
+        .of(String.format(
+            "start locator --port=%d --http-service-port=%d --J=-Dgemfire.JMX_MANAGER_PORT=%d %s",
+            locatorPort, httpPort, jmxPort, getSslParameters()),
+            String.format("start server --locators=localhost[%d] --server-port=0", locatorPort))
+        .withName("startCluster")
+        .execute(gfshRule);
 
     assertThat(startCluster.getProcess().exitValue())
-        .as("Cluster did not start correctly").isEqualTo(0);
+        .as("Cluster did not start correctly")
+        .isEqualTo(0);
 
     Process process = launchClientProcess(outputJar, httpPort);
 
-    boolean exited = process.waitFor(30, TimeUnit.SECONDS);
-    assertThat(exited).as("Process did not exit within 10 seconds").isTrue();
-    assertThat(process.exitValue()).as("Process did not exit with 0 return code").isEqualTo(0);
+    boolean exited = process.waitFor(getTimeout().toMillis(), MILLISECONDS);
+    assertThat(exited)
+        .as("Process did not exit within 10 seconds")
+        .isTrue();
+    assertThat(process.exitValue())
+        .as("Process did not exit with 0 return code")
+        .isEqualTo(0);
 
     GfshExecution listRegionsResult = GfshScript
         .of(String.format("connect --locator=localhost[%d]", locatorPort), "list regions")
-        .withName("listRegions").execute(gfsh);
-    assertThat(listRegionsResult.getOutputText()).contains("REGION1");
+        .withName("listRegions")
+        .execute(gfshRule);
+    assertThat(listRegionsResult.getOutputText())
+        .contains("REGION1");
   }
 
   private Process launchClientProcess(File outputJar, int httpPort) throws IOException {
     Path javaBin = Paths.get(System.getProperty("java.home"), "bin", "java");
 
-    ProcessBuilder pBuilder = new ProcessBuilder();
-    pBuilder.directory(tempDir.newFolder());
+    Path clientFolder = createDirectories(rootFolder.resolve("client"));
+
+    ProcessBuilder processBuilder = new ProcessBuilder();
+    processBuilder.directory(clientFolder.toFile());
 
     StringBuilder classPath = new StringBuilder();
     for (String module : Arrays.asList(
@@ -172,13 +189,13 @@ public class StandaloneClientManagementAPIAcceptanceTest {
     command.add("ManagementClientCreateRegion");
     command.add("REGION1");
     command.add(useSsl.toString());
-    command.add("" + httpPort);
+    command.add(String.valueOf(httpPort));
 
-    pBuilder.command(command);
+    processBuilder.command(command);
 
-    System.out.format("Launching client command: %s\n", command);
+    System.out.format("Launching client command: %s%s", command, lineSeparator());
 
-    Process process = pBuilder.start();
+    Process process = processBuilder.start();
     clientProcessLogger = new ProcessLogger(process, "clientCreateRegion");
     clientProcessLogger.start();
     return process;
@@ -202,12 +219,10 @@ public class StandaloneClientManagementAPIAcceptanceTest {
     String classPath = Arrays.stream(classPathValue
         .split(File.pathSeparator))
         .filter(x -> x.contains(module))
-        // && (x.endsWith("/classes") || x.endsWith("/classes/java/main")
-        // || x.endsWith("/resources") || x.endsWith("/resources/main")
-        // || x.endsWith(".jar")))
         .collect(Collectors.joining(File.pathSeparator));
 
-    assertThat(classPath).as("no classes found for module: " + module)
+    assertThat(classPath)
+        .as("no classes found for module: " + module)
         .isNotBlank();
 
     return classPath;
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/CacheGetsTimerTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/CacheGetsTimerTest.java
index 8a4a5801db..6ce050122e 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/CacheGetsTimerTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/CacheGetsTimerTest.java
@@ -21,6 +21,7 @@ import static org.apache.geode.cache.execute.FunctionService.onServer;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.test.compiler.ClassBuilder.writeJarFromClasses;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.catchThrowable;
 
@@ -35,9 +36,9 @@ import java.util.Properties;
 
 import io.micrometer.core.instrument.Timer;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.client.ClientCache;
@@ -51,23 +52,29 @@ import org.apache.geode.security.AuthInitialize;
 import org.apache.geode.security.AuthenticationFailedException;
 import org.apache.geode.security.NotAuthorizedException;
 import org.apache.geode.security.ResourcePermission;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class CacheGetsTimerTest {
+
   private int locatorPort;
   private ClientCache clientCache;
   private Region<Object, Object> replicateRegion;
   private Region<Object, Object> partitionRegion;
+  private Path rootFolder;
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public ServiceJarRule serviceJarRule = new ServiceJarRule();
 
+  @Before
+  public void setUp() {
+    rootFolder = folderRule.getFolder().toPath();
+  }
+
   @After
   public void tearDown() {
     if (clientCache != null) {
@@ -324,15 +331,14 @@ public class CacheGetsTimerTest {
     Path serviceJarPath = serviceJarRule.createJarFor("metrics-publishing-service.jar",
         MetricsPublishingService.class, SimpleMetricsPublishingService.class);
 
-    Path helpersJarPath = temporaryFolder.getRoot().toPath()
-        .resolve("helpers.jar").toAbsolutePath();
+    Path helpersJarPath = rootFolder.resolve("helpers.jar");
     writeJarFromClasses(helpersJarPath.toFile(), TimerValue.class,
         FetchCacheGetsTimerValues.class, DenyAllDataRead.class, ClientSecurityConfig.class);
 
     String startLocatorCommand = String.join(" ",
         "start locator",
         "--name=" + "locator",
-        "--dir=" + temporaryFolder.newFolder("locator").getAbsolutePath(),
+        "--dir=" + rootFolder.resolve("locator"),
         "--port=" + locatorPort,
         "--http-service-port=0",
         "--J=-Dgemfire.jmx-manager-port=" + locatorJmxPort,
@@ -342,7 +348,7 @@ public class CacheGetsTimerTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=" + serverName,
-        "--dir=" + temporaryFolder.newFolder(serverName).getAbsolutePath(),
+        "--dir=" + rootFolder.resolve(serverName),
         "--server-port=" + serverPort,
         "--locators=localhost[" + locatorPort + "]",
         "--classpath=" + serviceJarPath + pathSeparatorChar + helpersJarPath);
@@ -394,7 +400,7 @@ public class CacheGetsTimerTest {
 
   private File createSecurityPropertiesFile() throws IOException {
     Properties securityProperties = ClientSecurityConfig.securityProperties();
-    File securityPropertiesFile = gfshRule.getTemporaryFolder().newFile("security.properties");
+    File securityPropertiesFile = rootFolder.resolve("security.properties").toFile();
     securityProperties.store(new FileOutputStream(securityPropertiesFile), null);
     return securityPropertiesFile;
   }
@@ -420,11 +426,9 @@ public class CacheGetsTimerTest {
   }
 
   private List<TimerValue> allTimerValuesForRegion(Region<?, ?> region) {
-    @SuppressWarnings("unchecked")
-    List<List<TimerValue>> timerValuesFromAllServers =
-        (List<List<TimerValue>>) onServer(clientCache)
-            .execute(new FetchCacheGetsTimerValues())
-            .getResult();
+    List<List<TimerValue>> timerValuesFromAllServers = uncheckedCast(onServer(clientCache)
+        .execute(new FetchCacheGetsTimerValues())
+        .getResult());
 
     assertThat(timerValuesFromAllServers)
         .hasSize(1);
@@ -435,10 +439,10 @@ public class CacheGetsTimerTest {
   }
 
   static class TimerValue implements Serializable {
-    final long count;
-    final double totalTime;
-    final String region;
-    final String result;
+    private final long count;
+    private final double totalTime;
+    private final String region;
+    private final String result;
 
     TimerValue(long count, double totalTime, String region, String result) {
       this.count = count;
@@ -519,7 +523,7 @@ public class CacheGetsTimerTest {
       return securityProperties();
     }
 
-    static Properties securityProperties() {
+    private static Properties securityProperties() {
       Properties securityProperties = new Properties();
       securityProperties.setProperty(SECURITY_MANAGER, DenyAllDataRead.class.getName());
       securityProperties.setProperty("security-username", "user");
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/GatewayReceiverMetricsTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/GatewayReceiverMetricsTest.java
index ee569633d3..5158938161 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/GatewayReceiverMetricsTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/GatewayReceiverMetricsTest.java
@@ -12,17 +12,17 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.metrics;
 
+import static java.nio.file.Files.createDirectories;
+import static java.util.Arrays.asList;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.apache.geode.test.compiler.ClassBuilder.writeJarFromClasses;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.file.Path;
-import java.util.Arrays;
 import java.util.Collection;
 
 import io.micrometer.core.instrument.Counter;
@@ -31,44 +31,49 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionContext;
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.rules.ServiceJarRule;
 import org.apache.geode.test.junit.categories.MetricsTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 @Category(MetricsTest.class)
 public class GatewayReceiverMetricsTest {
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-  @Rule
-  public ServiceJarRule serviceJarRule = new ServiceJarRule();
-
   private static final String SENDER_LOCATOR_NAME = "sender-locator";
   private static final String RECEIVER_LOCATOR_NAME = "receiver-locator";
   private static final String SENDER_SERVER_NAME = "sender-server";
   private static final String RECEIVER_SERVER_NAME = "receiver-server";
   private static final String REGION_NAME = "region";
   private static final String GFSH_COMMAND_SEPARATOR = " ";
-  private String senderLocatorFolder;
-  private String receiverLocatorFolder;
-  private String senderServerFolder;
-  private String receiverServerFolder;
+
+  private Path rootFolder;
+  private Path senderLocatorFolder;
+  private Path receiverLocatorFolder;
+  private Path senderServerFolder;
+  private Path receiverServerFolder;
   private int receiverLocatorPort;
   private int senderLocatorPort;
 
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+  @Rule
+  public ServiceJarRule serviceJarRule = new ServiceJarRule();
+
   @Before
   public void startClusters() throws IOException {
-    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(6);
+    rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+    senderLocatorFolder = createDirectories(rootFolder.resolve(SENDER_LOCATOR_NAME));
+    receiverLocatorFolder = createDirectories(rootFolder.resolve(RECEIVER_LOCATOR_NAME));
+    senderServerFolder = createDirectories(rootFolder.resolve(SENDER_SERVER_NAME));
+    receiverServerFolder = createDirectories(rootFolder.resolve(RECEIVER_SERVER_NAME));
+
+    int[] ports = getRandomAvailableTCPPorts(6);
 
     receiverLocatorPort = ports[0];
     senderLocatorPort = ports[1];
@@ -80,11 +85,6 @@ public class GatewayReceiverMetricsTest {
     int senderSystemId = 2;
     int receiverSystemId = 1;
 
-    senderLocatorFolder = newFolder(SENDER_LOCATOR_NAME);
-    receiverLocatorFolder = newFolder(RECEIVER_LOCATOR_NAME);
-    senderServerFolder = newFolder(SENDER_SERVER_NAME);
-    receiverServerFolder = newFolder(RECEIVER_SERVER_NAME);
-
     String startSenderLocatorCommand = String.join(GFSH_COMMAND_SEPARATOR,
         "start locator",
         "--name=" + SENDER_LOCATOR_NAME,
@@ -163,7 +163,7 @@ public class GatewayReceiverMetricsTest {
         createReceiverRegionCommand);
 
     // Deploy function to members
-    String functionJarPath =
+    Path functionJarPath =
         newJarForFunctionClass(GetEventsReceivedCountFunction.class, "function.jar");
     String deployCommand = "deploy --jar=" + functionJarPath;
     String listFunctionsCommand = "list functions";
@@ -178,8 +178,8 @@ public class GatewayReceiverMetricsTest {
     String stopReceiverLocatorCommand = "stop locator --dir=" + receiverLocatorFolder;
     String stopSenderLocatorCommand = "stop locator --dir=" + senderLocatorFolder;
 
-    gfshRule.execute(stopReceiverServerCommand, stopSenderServerCommand, stopReceiverLocatorCommand,
-        stopSenderLocatorCommand);
+    gfshRule.execute(stopReceiverServerCommand, stopSenderServerCommand,
+        stopReceiverLocatorCommand, stopSenderLocatorCommand);
   }
 
   @Test
@@ -209,12 +209,12 @@ public class GatewayReceiverMetricsTest {
         "connect --locator=localhost[" + receiverLocatorPort + "]";
     String executeFunctionCommand = "execute function --id=" + GetEventsReceivedCountFunction.ID;
 
-    Collection<String> gatewayEventsExpectedToReceive =
-        Arrays.asList(doPutCommand, doRemoveCommand);
+    Collection<String> gatewayEventsExpectedToReceive = asList(doPutCommand, doRemoveCommand);
 
     await().untilAsserted(() -> {
-      String output =
-          gfshRule.execute(connectToReceiverLocatorCommand, executeFunctionCommand).getOutputText();
+      String output = gfshRule
+          .execute(connectToReceiverLocatorCommand, executeFunctionCommand)
+          .getOutputText();
 
       assertThat(output.trim())
           .as("Returned count of events received.")
@@ -222,18 +222,14 @@ public class GatewayReceiverMetricsTest {
     });
   }
 
-  private String newFolder(String folderName) throws IOException {
-    return temporaryFolder.newFolder(folderName).getAbsolutePath();
-  }
-
-  private String newJarForFunctionClass(Class clazz, String jarName) throws IOException {
-    File jar = temporaryFolder.newFile(jarName);
-    writeJarFromClasses(jar, clazz);
-    return jar.getAbsolutePath();
+  private Path newJarForFunctionClass(Class<?> clazz, String jarName) throws IOException {
+    Path jarPath = rootFolder.resolve(jarName);
+    writeJarFromClasses(jarPath.toFile(), clazz);
+    return jarPath;
   }
 
   public static class GetEventsReceivedCountFunction implements Function<Void> {
-    static final String ID = "GetEventsReceivedCountFunction";
+    private static final String ID = "GetEventsReceivedCountFunction";
 
     @Override
     public void execute(FunctionContext<Void> context) {
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/MemberTypeCommonTagsTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/MemberTypeCommonTagsTest.java
index c6bd396006..06248cdd41 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/MemberTypeCommonTagsTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/MemberTypeCommonTagsTest.java
@@ -20,6 +20,7 @@ import static org.apache.geode.cache.execute.FunctionService.onServer;
 import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.test.compiler.ClassBuilder.writeJarFromClasses;
 import static org.apache.geode.test.micrometer.MicrometerAssertions.assertThat;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.IOException;
@@ -35,7 +36,6 @@ import io.micrometer.core.instrument.Tag;
 import io.micrometer.core.instrument.simple.SimpleMeterRegistry;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
@@ -49,21 +49,21 @@ import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class MemberTypeCommonTagsTest {
+
   private Path locatorFolder;
   private Path serverFolder;
   private Pool serverPool;
   private ClientCache clientCache;
   private Cache cache;
 
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public ServiceJarRule serviceJarRule = new ServiceJarRule();
 
@@ -118,7 +118,7 @@ public class MemberTypeCommonTagsTest {
   }
 
   private DistributedMember startLocator() throws IOException {
-    locatorFolder = temporaryFolder.getRoot().toPath().toAbsolutePath();
+    locatorFolder = folderRule.getFolder().toPath().toAbsolutePath();
 
     int[] ports = getRandomAvailableTCPPorts(2);
 
@@ -165,7 +165,7 @@ public class MemberTypeCommonTagsTest {
   }
 
   private void startServer(boolean withLocator) throws IOException {
-    serverFolder = temporaryFolder.getRoot().toPath().toAbsolutePath();
+    serverFolder = folderRule.getFolder().toPath().toAbsolutePath();
 
     int[] availablePorts = getRandomAvailableTCPPorts(2);
 
@@ -209,11 +209,10 @@ public class MemberTypeCommonTagsTest {
     gfshRule.execute(stopServerCommand);
   }
 
-  private String memberTypeTag(Execution execution) {
-    @SuppressWarnings("unchecked")
-    List<String> results = (List<String>) execution
+  private String memberTypeTag(Execution<?, ?, ?> execution) {
+    List<String> results = uncheckedCast(execution
         .execute(new GetMemberTypeTag())
-        .getResult();
+        .getResult());
     return results.get(0);
   }
 
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/MicrometerBinderTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/MicrometerBinderTest.java
index 1640c15bef..91fc5b4b35 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/MicrometerBinderTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/MicrometerBinderTest.java
@@ -16,6 +16,7 @@ package org.apache.geode.metrics;
 
 import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.test.compiler.ClassBuilder.writeJarFromClasses;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.IOException;
@@ -31,7 +32,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientCacheFactory;
@@ -42,6 +42,7 @@ import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class MicrometerBinderTest {
@@ -51,27 +52,26 @@ public class MicrometerBinderTest {
   private Pool serverPool;
   private Execution<String, Boolean, List<Boolean>> functionExecution;
 
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public ServiceJarRule serviceJarRule = new ServiceJarRule();
 
   @Before
   public void startServer() throws IOException {
-    serverFolder = temporaryFolder.getRoot().toPath().toAbsolutePath();
+    Path rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+    serverFolder = rootFolder;
+
+    Path serviceJarPath = serviceJarRule.createJarFor("metrics-publishing-service.jar",
+        MetricsPublishingService.class, SimpleMetricsPublishingService.class);
 
     int[] ports = getRandomAvailableTCPPorts(2);
 
     int serverPort = ports[0];
     int jmxRmiPort = ports[1];
 
-    Path serviceJarPath = serviceJarRule.createJarFor("metrics-publishing-service.jar",
-        MetricsPublishingService.class, SimpleMetricsPublishingService.class);
-
     String startServerCommand = String.join(" ",
         "start server",
         "--name=server",
@@ -100,10 +100,7 @@ public class MicrometerBinderTest {
         .addServer("localhost", serverPort)
         .create("server-pool");
 
-    @SuppressWarnings("unchecked")
-    Execution<String, Boolean, List<Boolean>> functionExecution =
-        (Execution<String, Boolean, List<Boolean>>) FunctionService.onServer(serverPool);
-    this.functionExecution = functionExecution;
+    functionExecution = uncheckedCast(FunctionService.onServer(serverPool));
   }
 
   @After
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/RegionEntriesGaugeTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/RegionEntriesGaugeTest.java
index 40d985c68b..901e4f0aed 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/RegionEntriesGaugeTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/RegionEntriesGaugeTest.java
@@ -16,6 +16,7 @@
 package org.apache.geode.metrics;
 
 import static java.lang.Integer.parseInt;
+import static java.nio.file.Files.createDirectories;
 import static java.util.Arrays.asList;
 import static java.util.stream.Collectors.toList;
 import static org.apache.geode.cache.RegionShortcut.LOCAL;
@@ -29,7 +30,6 @@ import static org.apache.geode.test.compiler.ClassBuilder.writeJarFromClasses;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.BufferedReader;
-import java.io.File;
 import java.io.IOException;
 import java.io.StringReader;
 import java.nio.file.Path;
@@ -45,7 +45,6 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.client.ClientCache;
@@ -57,18 +56,17 @@ import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.rules.ServiceJarRule;
 import org.apache.geode.test.junit.categories.MetricsTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 @Category(MetricsTest.class)
 public class RegionEntriesGaugeTest {
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public ServiceJarRule serviceJarRule = new ServiceJarRule();
 
@@ -76,13 +74,22 @@ public class RegionEntriesGaugeTest {
   private String connectToLocatorCommand;
   private String locatorString;
   private Pool server1Pool;
+  private Path rootFolder;
   private Path serviceJarPath;
 
   @Before
-  public void startMembers() throws Exception {
+  public void startMembers() throws IOException {
+    rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+
     serviceJarPath = serviceJarRule.createJarFor("metrics-publishing-service.jar",
         MetricsPublishingService.class, SimpleMetricsPublishingService.class);
+
+    Path folderForLocator = createDirectories(rootFolder.resolve("locator"));
+    Path folderForServer1 = createDirectories(rootFolder.resolve("server1"));
+    Path folderForServer2 = createDirectories(rootFolder.resolve("server2"));
+
     int[] availablePorts = getRandomAvailableTCPPorts(4);
+
     int locatorPort = availablePorts[0];
     int locatorJmxPort = availablePorts[1];
     int serverPort1 = availablePorts[2];
@@ -90,14 +97,10 @@ public class RegionEntriesGaugeTest {
 
     locatorString = "localhost[" + locatorPort + "]";
 
-    File folderForLocator = temporaryFolder.newFolder("locator");
-    File folderForServer1 = temporaryFolder.newFolder("server1");
-    File folderForServer2 = temporaryFolder.newFolder("server2");
-
     String startLocatorCommand = String.join(" ",
         "start locator",
         "--name=" + "locator",
-        "--dir=" + folderForLocator.getAbsolutePath(),
+        "--dir=" + folderForLocator,
         "--port=" + locatorPort,
         "--http-service-port=0",
         "--J=-Dgemfire.jmx-manager-port=" + locatorJmxPort);
@@ -109,8 +112,7 @@ public class RegionEntriesGaugeTest {
 
     connectToLocatorCommand = "connect --locator=" + locatorString;
 
-    Path functionJarPath =
-        temporaryFolder.getRoot().toPath().resolve("function.jar").toAbsolutePath();
+    Path functionJarPath = rootFolder.resolve("function.jar").toAbsolutePath();
     writeJarFromClasses(functionJarPath.toFile(), GetMemberRegionEntriesGaugeFunction.class);
 
     String deployCommand = "deploy --jar=" + functionJarPath.toAbsolutePath();
@@ -154,27 +156,27 @@ public class RegionEntriesGaugeTest {
 
     String getGaugeValueCommand = memberRegionEntryGaugeValueCommand(regionName);
 
-    await()
-        .untilAsserted(() -> {
-          GfshExecution execution = gfshRule.execute(connectToLocatorCommand, getGaugeValueCommand);
-          OptionalInt server1EntryCount = linesOf(execution.getOutputText())
-              .filter(s -> s.startsWith("server1"))
-              .mapToInt(RegionEntriesGaugeTest::extractEntryCount)
-              .findFirst();
+    await().untilAsserted(() -> {
+      GfshExecution execution =
+          gfshRule.execute(connectToLocatorCommand, getGaugeValueCommand);
+      OptionalInt server1EntryCount = linesOf(execution.getOutputText())
+          .filter(s -> s.startsWith("server1"))
+          .mapToInt(RegionEntriesGaugeTest::extractEntryCount)
+          .findFirst();
 
-          assertThat(server1EntryCount)
-              .as("Number of entries reported by server1")
-              .hasValue(expectedNumberOfEntries);
+      assertThat(server1EntryCount)
+          .as("Number of entries reported by server1")
+          .hasValue(expectedNumberOfEntries);
 
-          String server2Response = linesOf(execution.getOutputText())
-              .filter(s -> s.startsWith("server2"))
-              .findFirst()
-              .orElse("No response from server2");
+      String server2Response = linesOf(execution.getOutputText())
+          .filter(s -> s.startsWith("server2"))
+          .findFirst()
+          .orElse("No response from server2");
 
-          assertThat(server2Response)
-              .as("server2 response from entry count function")
-              .endsWith("[Meter not found.]");
-        });
+      assertThat(server2Response)
+          .as("server2 response from entry count function")
+          .endsWith("[Meter not found.]");
+    });
   }
 
   @Test
@@ -245,7 +247,7 @@ public class RegionEntriesGaugeTest {
 
     int server3Port = AvailablePortHelper.getRandomAvailableTCPPort();
 
-    File folderForServer3 = temporaryFolder.newFolder("server3");
+    Path folderForServer3 = createDirectories(rootFolder.resolve("server3"));
 
     String startServer3Command = startServerCommand("server3", server3Port, folderForServer3);
     gfshRule.execute(connectToLocatorCommand, startServer3Command);
@@ -338,12 +340,12 @@ public class RegionEntriesGaugeTest {
         .create(regionName);
   }
 
-  private String startServerCommand(String serverName, int serverPort, File folderForServer) {
+  private String startServerCommand(String serverName, int serverPort, Path folderForServer) {
     return String.join(" ",
         "start server",
         "--name=" + serverName,
         "--groups=" + serverName,
-        "--dir=" + folderForServer.getAbsolutePath(),
+        "--dir=" + folderForServer,
         "--server-port=" + serverPort,
         "--locators=" + locatorString,
         "--classpath=" + serviceJarPath);
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerClusterTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerClusterTest.java
index 0a02a371f4..dea24d9db4 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerClusterTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerClusterTest.java
@@ -14,23 +14,23 @@
  */
 package org.apache.geode.metrics.function.executions;
 
-
 import static java.io.File.pathSeparatorChar;
 import static java.lang.Boolean.TRUE;
 import static java.util.stream.Collectors.toList;
 import static org.apache.geode.test.compiler.ClassBuilder.writeJarFromClasses;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.IOException;
 import java.nio.file.Path;
 import java.time.Duration;
+import java.util.Collection;
 import java.util.List;
 
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.client.ClientCache;
@@ -45,6 +45,7 @@ import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.metrics.MetricsPublishingService;
 import org.apache.geode.metrics.SimpleMetricsPublishingService;
 import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 /**
@@ -59,37 +60,38 @@ public class FunctionExecutionsTimerClusterTest {
   private Pool multiServerPool;
   private Region<Object, Object> replicateRegion;
   private Region<Object, Object> partitionRegion;
+  private Path rootFolder;
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public ServiceJarRule serviceJarRule = new ServiceJarRule();
 
   @Before
   public void setUp() throws IOException {
-    int[] availablePorts = AvailablePortHelper.getRandomAvailableTCPPorts(4);
-
-    locatorPort = availablePorts[0];
-    int locatorJmxPort = availablePorts[1];
-    int server1Port = availablePorts[2];
-    int server2Port = availablePorts[3];
+    rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
 
     Path serviceJarPath = serviceJarRule.createJarFor("metrics-publishing-service.jar",
         MetricsPublishingService.class, SimpleMetricsPublishingService.class);
 
-    Path functionsJarPath = temporaryFolder.getRoot().toPath()
+    Path functionsJarPath = folderRule.getFolder().toPath()
         .resolve("functions.jar").toAbsolutePath();
     writeJarFromClasses(functionsJarPath.toFile(), GetFunctionExecutionTimerValues.class,
         FunctionToTimeWithResult.class, ExecutionsTimerValues.class, ThreadSleep.class);
 
+    int[] availablePorts = AvailablePortHelper.getRandomAvailableTCPPorts(4);
+
+    locatorPort = availablePorts[0];
+    int locatorJmxPort = availablePorts[1];
+    int server1Port = availablePorts[2];
+    int server2Port = availablePorts[3];
+
     String startLocatorCommand = String.join(" ",
         "start locator",
         "--name=" + "locator",
-        "--dir=" + temporaryFolder.newFolder("locator").getAbsolutePath(),
+        "--dir=" + rootFolder.resolve("locator"),
         "--port=" + locatorPort,
         "--http-service-port=0",
         "--J=-Dgemfire.jmx-manager-port=" + locatorJmxPort);
@@ -165,7 +167,7 @@ public class FunctionExecutionsTimerClusterTest {
 
   @Test
   public void timersRecordCountAndTotalTime_ifFunctionExecutedOnReplicateRegion() {
-    FunctionToTimeWithResult function = new FunctionToTimeWithResult();
+    Function<String[]> function = new FunctionToTimeWithResult();
     Duration functionDuration = Duration.ofSeconds(1);
     executeFunctionOnReplicateRegion(function, functionDuration);
 
@@ -177,12 +179,12 @@ public class FunctionExecutionsTimerClusterTest {
 
     assertThat(getAggregateTotalTime(values))
         .as("Total time of function executions across all servers")
-        .isBetween((double) functionDuration.toNanos(), ((double) functionDuration.toNanos()) * 2);
+        .isBetween((double) functionDuration.toNanos(), (double) functionDuration.toNanos() * 2);
   }
 
   @Test
   public void timersRecordCountAndTotalTime_ifFunctionExecutedOnReplicateRegionMultipleTimes() {
-    FunctionToTimeWithResult function = new FunctionToTimeWithResult();
+    Function<String[]> function = new FunctionToTimeWithResult();
     Duration functionDuration = Duration.ofSeconds(1);
     int numberOfExecutions = 10;
 
@@ -192,7 +194,7 @@ public class FunctionExecutionsTimerClusterTest {
 
     List<ExecutionsTimerValues> values = getAllExecutionsTimerValues(function.getId());
 
-    double expectedMinimumTotalTime = ((double) functionDuration.toNanos()) * numberOfExecutions;
+    double expectedMinimumTotalTime = (double) functionDuration.toNanos() * numberOfExecutions;
     double expectedMaximumTotalTime = expectedMinimumTotalTime * 2;
 
     assertThat(getAggregateCount(values))
@@ -206,7 +208,7 @@ public class FunctionExecutionsTimerClusterTest {
 
   @Test
   public void timersRecordCountAndTotalTime_ifFunctionExecutedOnPartitionRegionMultipleTimes() {
-    FunctionToTimeWithResult function = new FunctionToTimeWithResult();
+    Function<String[]> function = new FunctionToTimeWithResult();
     Duration functionDuration = Duration.ofSeconds(1);
     int numberOfExecutions = 10;
 
@@ -217,7 +219,7 @@ public class FunctionExecutionsTimerClusterTest {
     List<ExecutionsTimerValues> server1Values = getServer1ExecutionsTimerValues(function.getId());
     List<ExecutionsTimerValues> server2Values = getServer2ExecutionsTimerValues(function.getId());
 
-    double expectedMinimumTotalTime = ((double) functionDuration.toNanos()) * numberOfExecutions;
+    double expectedMinimumTotalTime = (double) functionDuration.toNanos() * numberOfExecutions;
     double expectedMaximumTotalTime = expectedMinimumTotalTime * 2;
 
     assertThat(getAggregateCount(server1Values))
@@ -238,13 +240,12 @@ public class FunctionExecutionsTimerClusterTest {
   }
 
   private String startServerCommand(String serverName, int serverPort, Path serviceJarPath,
-      Path functionsJarPath)
-      throws IOException {
+      Path functionsJarPath) {
     return String.join(" ",
         "start server",
         "--name=" + serverName,
         "--groups=" + serverName,
-        "--dir=" + temporaryFolder.newFolder(serverName).getAbsolutePath(),
+        "--dir=" + rootFolder.resolve(serverName),
         "--server-port=" + serverPort,
         "--locators=localhost[" + locatorPort + "]",
         "--classpath=" + serviceJarPath + pathSeparatorChar + functionsJarPath);
@@ -262,9 +263,8 @@ public class FunctionExecutionsTimerClusterTest {
 
   private void executeFunctionOnRegion(Function<? super String[]> function, Duration duration,
       Region<?, ?> region) {
-    @SuppressWarnings("unchecked")
     Execution<String[], Object, List<Object>> execution =
-        (Execution<String[], Object, List<Object>>) FunctionService.onRegion(region);
+        uncheckedCast(FunctionService.onRegion(region));
 
     execution
         .setArguments(new String[] {String.valueOf(duration.toMillis()), TRUE.toString()})
@@ -286,10 +286,8 @@ public class FunctionExecutionsTimerClusterTest {
 
   private List<ExecutionsTimerValues> getExecutionsTimerValuesFromPool(String functionId,
       Pool pool) {
-    @SuppressWarnings("unchecked")
     Execution<Void, List<ExecutionsTimerValues>, List<List<ExecutionsTimerValues>>> functionExecution =
-        (Execution<Void, List<ExecutionsTimerValues>, List<List<ExecutionsTimerValues>>>) FunctionService
-            .onServers(pool);
+        uncheckedCast(FunctionService.onServers(pool));
 
     List<List<ExecutionsTimerValues>> timerValuesForEachServer = functionExecution
         .execute(new GetFunctionExecutionTimerValues())
@@ -301,11 +299,11 @@ public class FunctionExecutionsTimerClusterTest {
         .collect(toList());
   }
 
-  private static Long getAggregateCount(List<ExecutionsTimerValues> values) {
+  private static Long getAggregateCount(Collection<ExecutionsTimerValues> values) {
     return values.stream().map(x -> x.count).reduce(0L, Long::sum);
   }
 
-  private static Double getAggregateTotalTime(List<ExecutionsTimerValues> values) {
+  private static Double getAggregateTotalTime(Collection<ExecutionsTimerValues> values) {
     return values.stream().map(x -> x.totalTime).reduce(0.0, Double::sum);
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerLonerTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerLonerTest.java
index 7c4a0b0527..2a4edc7616 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerLonerTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerLonerTest.java
@@ -14,12 +14,14 @@
  */
 package org.apache.geode.metrics.function.executions;
 
-
 import static java.io.File.pathSeparatorChar;
 import static java.lang.Boolean.TRUE;
 import static java.lang.String.valueOf;
+import static java.nio.file.Files.createDirectories;
 import static java.util.stream.Collectors.toList;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.test.compiler.ClassBuilder.writeJarFromClasses;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.catchThrowable;
 
@@ -32,7 +34,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientCacheFactory;
@@ -41,11 +42,13 @@ import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.metrics.MetricsPublishingService;
 import org.apache.geode.metrics.SimpleMetricsPublishingService;
 import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.junit.rules.FolderRule;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 /**
  * Acceptance tests for function executions timer on a loner server with no locator
@@ -64,30 +67,30 @@ public class FunctionExecutionsTimerLonerTest {
   private String startServerCommandWithStatsDisabled;
   private String startServerCommandWithTimeStatsDisabled;
   private String stopServerCommand;
+  private Path rootFolder;
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public ServiceJarRule serviceJarRule = new ServiceJarRule();
 
   @Before
   public void setUp() throws IOException {
-    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+    rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+
+    int[] ports = getRandomAvailableTCPPorts(2);
 
     serverPort = ports[0];
     jmxRmiPort = ports[1];
 
-    serverFolder = temporaryFolder.newFolder("server").toPath().toAbsolutePath();
+    serverFolder = createDirectories(rootFolder.resolve("server"));
 
     serviceJarPath = serviceJarRule.createJarFor("services.jar",
         MetricsPublishingService.class, SimpleMetricsPublishingService.class);
 
-    functionHelpersJarPath =
-        temporaryFolder.getRoot().toPath().resolve("function-helpers.jar").toAbsolutePath();
+    functionHelpersJarPath = rootFolder.resolve("function-helpers.jar");
     writeJarFromClasses(functionHelpersJarPath.toFile(), FunctionToTimeWithResult.class,
         GetFunctionExecutionTimerValues.class, ExecutionsTimerValues.class, ThreadSleep.class);
 
@@ -243,7 +246,8 @@ public class FunctionExecutionsTimerLonerTest {
 
   private void stopServer() {
     closeClientAndPool();
-    gfshRule.execute(stopServerCommand);
+    GfshExecution execution = GfshScript.of(stopServerCommand).execute(gfshRule);
+    execution.serverStopper().awaitStop(serverFolder);
   }
 
   private void startServerWithStatsEnabled() {
@@ -270,8 +274,7 @@ public class FunctionExecutionsTimerLonerTest {
 
   @SuppressWarnings("SameParameterValue")
   private <T> void deployFunction(Class<? extends Function<T>> functionClass) {
-    Path functionJarPath = temporaryFolder.getRoot().toPath()
-        .resolve(functionClass.getSimpleName() + ".jar").toAbsolutePath();
+    Path functionJarPath = rootFolder.resolve(functionClass.getSimpleName() + ".jar");
 
     Throwable thrown =
         catchThrowable(() -> writeJarFromClasses(functionJarPath.toFile(), functionClass));
@@ -302,9 +305,8 @@ public class FunctionExecutionsTimerLonerTest {
 
   private void executeFunction(Function<? super String[]> function, Duration duration,
       boolean successful) {
-    @SuppressWarnings("unchecked")
     Execution<String[], Object, List<Object>> execution =
-        (Execution<String[], Object, List<Object>>) FunctionService.onServer(serverPool);
+        uncheckedCast(FunctionService.onServer(serverPool));
 
     execution
         .setArguments(new String[] {valueOf(duration.toMillis()), valueOf(successful)})
@@ -314,9 +316,8 @@ public class FunctionExecutionsTimerLonerTest {
 
   @SuppressWarnings("SameParameterValue")
   private void executeFunctionById(String functionId, Duration duration) {
-    @SuppressWarnings("unchecked")
     Execution<String[], Object, List<Object>> execution =
-        (Execution<String[], Object, List<Object>>) FunctionService.onServer(serverPool);
+        uncheckedCast(FunctionService.onServer(serverPool));
 
     Throwable thrown = catchThrowable(() -> execution
         .setArguments(new String[] {valueOf(duration.toMillis()), TRUE.toString()})
@@ -350,10 +351,8 @@ public class FunctionExecutionsTimerLonerTest {
   }
 
   private List<ExecutionsTimerValues> getExecutionsTimerValuesFor(String functionId) {
-    @SuppressWarnings("unchecked")
     Execution<Void, List<ExecutionsTimerValues>, List<List<ExecutionsTimerValues>>> functionExecution =
-        (Execution<Void, List<ExecutionsTimerValues>, List<List<ExecutionsTimerValues>>>) FunctionService
-            .onServer(serverPool);
+        uncheckedCast(FunctionService.onServer(serverPool));
 
     List<List<ExecutionsTimerValues>> results = functionExecution
         .execute(new GetFunctionExecutionTimerValues())
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerNoResultTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerNoResultTest.java
index 72192e65cc..760d49bf3b 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerNoResultTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/metrics/function/executions/FunctionExecutionsTimerNoResultTest.java
@@ -14,9 +14,9 @@
  */
 package org.apache.geode.metrics.function.executions;
 
-
 import static java.io.File.pathSeparatorChar;
 import static java.util.stream.Collectors.toList;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.apache.geode.test.compiler.ClassBuilder.writeJarFromClasses;
 import static org.assertj.core.api.Assertions.assertThat;
@@ -26,12 +26,10 @@ import java.nio.file.Path;
 import java.time.Duration;
 import java.util.List;
 
-import org.assertj.core.api.Assertions;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionService;
@@ -40,10 +38,10 @@ import org.apache.geode.cache.client.ClientCacheFactory;
 import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.metrics.MetricsPublishingService;
 import org.apache.geode.metrics.SimpleMetricsPublishingService;
 import org.apache.geode.rules.ServiceJarRule;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 /**
@@ -57,19 +55,20 @@ public class FunctionExecutionsTimerNoResultTest {
   private Region<Object, Object> partitionRegion;
   private FunctionToTimeWithoutResult functionWithNoResult;
   private Duration functionDuration;
+  private Path rootFolder;
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
   @Rule
   public ServiceJarRule serviceJarRule = new ServiceJarRule();
 
   @Before
   public void setUp() throws IOException {
-    int[] availablePorts = AvailablePortHelper.getRandomAvailableTCPPorts(3);
+    rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+
+    int[] availablePorts = getRandomAvailableTCPPorts(3);
 
     locatorPort = availablePorts[0];
     int locatorJmxPort = availablePorts[1];
@@ -78,8 +77,7 @@ public class FunctionExecutionsTimerNoResultTest {
     Path serviceJarPath = serviceJarRule.createJarFor("metrics-publishing-service.jar",
         MetricsPublishingService.class, SimpleMetricsPublishingService.class);
 
-    Path functionsJarPath = temporaryFolder.getRoot().toPath()
-        .resolve("functions.jar").toAbsolutePath();
+    Path functionsJarPath = rootFolder.resolve("functions.jar");
     writeJarFromClasses(functionsJarPath.toFile(),
         GetFunctionExecutionTimerValues.class, FunctionToTimeWithoutResult.class,
         ExecutionsTimerValues.class, ThreadSleep.class);
@@ -87,7 +85,7 @@ public class FunctionExecutionsTimerNoResultTest {
     String startLocatorCommand = String.join(" ",
         "start locator",
         "--name=" + "locator",
-        "--dir=" + temporaryFolder.newFolder("locator").getAbsolutePath(),
+        "--dir=" + rootFolder.resolve("locator"),
         "--port=" + locatorPort,
         "--http-service-port=0",
         "--J=-Dgemfire.jmx-manager-port=" + locatorJmxPort);
@@ -196,11 +194,11 @@ public class FunctionExecutionsTimerNoResultTest {
     await().untilAsserted(() -> {
       ExecutionsTimerValues value = failureTimerValue();
 
-      Assertions.assertThat(value.count)
+      assertThat(value.count)
           .as("Number of failed executions")
           .isEqualTo(1);
 
-      Assertions.assertThat(value.totalTime)
+      assertThat(value.totalTime)
           .as("Total time of failed executions")
           .isGreaterThan(functionDuration.toNanos());
     });
@@ -241,13 +239,12 @@ public class FunctionExecutionsTimerNoResultTest {
   }
 
   private String startServerCommand(String serverName, int serverPort, Path serviceJarPath,
-      Path functionsJarPath)
-      throws IOException {
+      Path functionsJarPath) {
     return String.join(" ",
         "start server",
         "--name=" + serverName,
         "--groups=" + serverName,
-        "--dir=" + temporaryFolder.newFolder(serverName).getAbsolutePath(),
+        "--dir=" + rootFolder.resolve(serverName),
         "--server-port=" + serverPort,
         "--locators=localhost[" + locatorPort + "]",
         "--classpath=" + serviceJarPath + pathSeparatorChar + functionsJarPath);
@@ -290,7 +287,7 @@ public class FunctionExecutionsTimerNoResultTest {
         .filter(v -> v.succeeded == isSuccessful)
         .collect(toList());
 
-    Assertions.assertThat(values)
+    assertThat(values)
         .hasSize(1);
 
     return values.get(0);
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/modules/DeployJarAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/modules/DeployJarAcceptanceTest.java
index 9258505d6b..5f321fce5a 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/modules/DeployJarAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/modules/DeployJarAcceptanceTest.java
@@ -16,253 +16,369 @@
  */
 package org.apache.geode.modules;
 
-
+import static java.nio.file.Files.createDirectories;
 import static org.apache.geode.test.util.ResourceUtils.createTempFileFromResource;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
 
 import org.junit.After;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
+import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.test.compiler.JarBuilder;
+import org.apache.geode.test.junit.rules.FolderRule;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
 public class DeployJarAcceptanceTest {
 
-  @ClassRule
-  public static GfshRule gfshRule = new GfshRule();
-
-  @ClassRule
-  public static TemporaryFolder stagingTempDir = new TemporaryFolder();
-
-  private static File jarFile;
-  private static File jarFileV2;
-  private static File anotherJarFile;
-
-  @BeforeClass
-  public static void setup() throws IOException {
-    File stagingDir = stagingTempDir.newFolder("staging");
-    jarFile = new File(stagingDir, "myJar-1.0.jar");
-    jarFileV2 = new File(stagingDir, "myJar-2.0.jar");
-    anotherJarFile = new File(stagingDir, "anotherJar-1.0.jar");
+  private int locatorPort;
+  private Path jarFile;
+  private Path jarFileV2;
+  private Path anotherJarFile;
+  private Path stagingTempDir;
+
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
+  @Before
+  public void setUp()
+      throws IOException, ExecutionException, InterruptedException, TimeoutException {
+    Path rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+    stagingTempDir = rootFolder;
+
+    Path stagingDir = createDirectories(rootFolder.resolve("staging"));
+    jarFile = stagingDir.resolve("myJar-1.0.jar");
+    jarFileV2 = stagingDir.resolve("myJar-2.0.jar");
+    anotherJarFile = stagingDir.resolve("anotherJar-1.0.jar");
     JarBuilder jarBuilder = new JarBuilder();
-    jarBuilder.buildJarFromClassNames(jarFile, "SomeClass");
-    jarBuilder.buildJarFromClassNames(jarFileV2, "SomeClass", "SomeClassVersionTwo");
-    jarBuilder.buildJarFromClassNames(anotherJarFile, "SomeOtherClass");
+    jarBuilder.buildJarFromClassNames(jarFile.toFile(), "SomeClass");
+    jarBuilder.buildJarFromClassNames(jarFileV2.toFile(), "SomeClass", "SomeClassVersionTwo");
+    jarBuilder.buildJarFromClassNames(anotherJarFile.toFile(), "SomeOtherClass");
+
+    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+    locatorPort = ports[0];
+    int jmxPort = ports[1];
 
     GfshScript
-        .of("start locator --name=locator", "configure pdx --read-serialized=true",
-            "start server --name=server --locators=localhost[10334]")
+        .of(String.format(
+            "start locator --name=locator --port=%d --http-service-port=0 --J=-Dgemfire.jmx-manager-port=%d",
+            locatorPort, jmxPort),
+            "configure pdx --read-serialized=true",
+            String.format(
+                "start server --name=server --locators=localhost[%d] --disable-default-server",
+                locatorPort))
         .execute(gfshRule);
   }
 
   @After
-  public void teardown() {
-    System.out.println(GfshScript.of(getLocatorGFSHConnectionString(), "undeploy")
-        .execute(gfshRule).getOutputText());
-  }
-
-  private String getLocatorGFSHConnectionString() {
-    return "connect --locator=localhost[10334]";
+  public void tearDown()
+      throws IOException, ExecutionException, InterruptedException, TimeoutException {
+    System.out.println(GfshScript
+        .of(connectCommand(),
+            "undeploy")
+        .execute(gfshRule)
+        .getOutputText());
   }
 
   @Test
-  public void testDeployJar() throws IOException {
-    GfshScript.of(getLocatorGFSHConnectionString(),
-        "deploy --jar=" + jarFile.getCanonicalPath()).execute(gfshRule);
+  public void testDeployJar() {
+    GfshScript
+        .of(connectCommand(),
+            "deploy --jar=" + jarFile)
+        .execute(gfshRule);
 
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(), "list deployed")
-        .execute(gfshRule).getOutputText()).contains(jarFile.getName()).contains("JAR Location");
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list deployed")
+        .execute(gfshRule).getOutputText())
+            .contains(jarFile.toFile().getName())
+            .contains("JAR Location");
   }
 
   @Test
-  public void testDeployExistingJar() throws IOException {
-    GfshScript.of(getLocatorGFSHConnectionString(),
-        "deploy --jar=" + jarFile.getCanonicalPath()).execute(gfshRule);
+  public void testDeployExistingJar() {
+    GfshScript
+        .of(connectCommand(),
+            "deploy --jar=" + jarFile)
+        .execute(gfshRule);
 
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(), "list deployed")
-        .execute(gfshRule).getOutputText()).contains(jarFile.getName()).contains("JAR Location");
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list deployed")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains(jarFile.toFile().getName())
+            .contains("JAR Location");
 
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(),
-        "deploy --jar=" + jarFile.getCanonicalPath()).execute(gfshRule).getOutputText())
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "deploy --jar=" + jarFile)
+        .execute(gfshRule)
+        .getOutputText())
             .contains("Already deployed");
 
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(), "list deployed")
-        .execute(gfshRule).getOutputText()).contains(jarFile.getName()).contains("JAR Location");
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list deployed")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains(jarFile.toFile().getName())
+            .contains("JAR Location");
   }
 
   @Test
-  public void testUndeployJar() throws IOException {
-    GfshScript.of(getLocatorGFSHConnectionString(),
-        "deploy --jar=" + jarFile.getCanonicalPath()).execute(gfshRule);
-
-    assertThat(
-        GfshScript.of(getLocatorGFSHConnectionString(),
-            "undeploy --jar=" + jarFile.getName())
-            .execute(gfshRule).getOutputText()).contains(jarFile.getName())
-                .contains("Un-Deployed From JAR Location");
-
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(), "list deployed")
-        .execute(gfshRule).getOutputText()).doesNotContain(jarFile.getName());
+  public void testUndeployJar() {
+    GfshScript
+        .of(connectCommand(),
+            "deploy --jar=" + jarFile)
+        .execute(gfshRule);
+
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "undeploy --jar=" + jarFile.toFile().getName())
+        .execute(gfshRule)
+        .getOutputText())
+            .contains(jarFile.toFile().getName())
+            .contains("Un-Deployed From JAR Location");
+
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list deployed")
+        .execute(gfshRule)
+        .getOutputText())
+            .doesNotContain(jarFile.toFile().getName());
   }
 
   @Test
   public void testUndeployWithNothingDeployed() {
-    assertThat(
-        GfshScript.of(getLocatorGFSHConnectionString(),
-            "undeploy --jar=" + jarFile.getName())
-            .execute(gfshRule).getOutputText()).contains(jarFile.getName() + " not deployed");
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "undeploy --jar=" + jarFile.toFile().getName())
+        .execute(gfshRule).getOutputText())
+            .contains(jarFile.toFile().getName() + " not deployed");
   }
 
   @Test
-  public void testRedeployNewJar() throws IOException {
-    GfshScript.of(getLocatorGFSHConnectionString(),
-        "deploy --jar=" + jarFile.getCanonicalPath()).execute(gfshRule);
+  public void testRedeployNewJar() {
+    GfshScript
+        .of(connectCommand(),
+            "deploy --jar=" + jarFile)
+        .execute(gfshRule);
 
-    assertThat(
-        GfshScript.of(getLocatorGFSHConnectionString(),
-            "undeploy --jar=" + jarFile.getName())
-            .execute(gfshRule).getOutputText()).contains(jarFile.getName())
-                .contains("Un-Deployed From JAR Location");
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "undeploy --jar=" + jarFile.toFile().getName())
+        .execute(gfshRule)
+        .getOutputText())
+            .contains(jarFile.toFile().getName())
+            .contains("Un-Deployed From JAR Location");
 
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(), "list deployed")
-        .execute(gfshRule).getOutputText()).doesNotContain(jarFile.getName());
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list deployed")
+        .execute(gfshRule)
+        .getOutputText())
+            .doesNotContain(jarFile.toFile().getName());
 
     GfshScript
-        .of(getLocatorGFSHConnectionString(),
-            "deploy --jar=" + anotherJarFile.getCanonicalPath())
+        .of(connectCommand(),
+            "deploy --jar=" + anotherJarFile)
         .execute(gfshRule);
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(), "list deployed")
-        .execute(gfshRule).getOutputText()).contains(anotherJarFile.getName());
+
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list deployed")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains(anotherJarFile.toFile().getName());
   }
 
   @Test
-  public void testUpdateJar() throws IOException {
-    GfshScript.of(getLocatorGFSHConnectionString(),
-        "deploy --jar=" + jarFile.getCanonicalPath()).execute(gfshRule);
+  public void testUpdateJar() {
+    GfshScript
+        .of(connectCommand(),
+            "deploy --jar=" + jarFile)
+        .execute(gfshRule);
 
-    GfshScript.of(getLocatorGFSHConnectionString(),
-        "deploy --jar=" + jarFileV2.getCanonicalPath()).execute(gfshRule);
+    GfshScript
+        .of(connectCommand(),
+            "deploy --jar=" + jarFileV2)
+        .execute(gfshRule);
 
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(),
-        "list deployed").execute(gfshRule).getOutputText()).contains(jarFileV2.getName())
-            .doesNotContain(jarFile.getName());
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list deployed")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains(jarFileV2.toFile().getName())
+            .doesNotContain(jarFile.toFile().getName());
   }
 
   @Test
-  public void testDeployMultipleJars() throws IOException {
-    GfshScript.of(getLocatorGFSHConnectionString(),
-        "deploy --jar=" + jarFile.getCanonicalPath(),
-        "deploy --jar=" + anotherJarFile.getCanonicalPath()).execute(gfshRule);
-
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(),
-        "list deployed").execute(gfshRule).getOutputText()).contains(jarFile.getName())
-            .contains(anotherJarFile.getName());
+  public void testDeployMultipleJars() {
+    GfshScript
+        .of(connectCommand(),
+            "deploy --jar=" + jarFile,
+            "deploy --jar=" + anotherJarFile)
+        .execute(gfshRule);
+
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list deployed")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains(jarFile.toFile().getName())
+            .contains(anotherJarFile.toFile().getName());
   }
 
   @Test
   public void testDeployFunction() throws IOException {
-    JarBuilder jarBuilder = new JarBuilder();
-    File source = loadTestResource("/example/test/function/ExampleFunction.java");
+    Path source = loadTestResource("/example/test/function/ExampleFunction.java");
+    Path outputJar = stagingTempDir.resolve("function.jar").toAbsolutePath();
 
-    File outputJar = new File(stagingTempDir.newFolder(), "function.jar");
-    jarBuilder.buildJar(outputJar, source);
+    JarBuilder jarBuilder = new JarBuilder();
+    jarBuilder.buildJar(outputJar.toFile(), source.toFile());
 
-    GfshScript.of(getLocatorGFSHConnectionString(), "deploy --jars=" + outputJar.getCanonicalPath())
+    GfshScript
+        .of(connectCommand(),
+            "deploy --jars=" + outputJar)
         .execute(gfshRule);
 
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(), "list functions").execute(gfshRule)
-        .getOutputText()).contains("ExampleFunction");
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list functions")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains("ExampleFunction");
 
-    assertThat(
-        GfshScript.of(getLocatorGFSHConnectionString(), "execute function --id=ExampleFunction")
-            .execute(gfshRule)
-            .getOutputText()).contains("SUCCESS");
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "execute function --id=ExampleFunction")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains("SUCCESS");
   }
 
   @Test
   public void testDeployAndUndeployFunction() throws IOException {
-    JarBuilder jarBuilder = new JarBuilder();
-    File source = loadTestResource("/example/test/function/ExampleFunction.java");
+    Path source = loadTestResource("/example/test/function/ExampleFunction.java");
+    Path outputJar = stagingTempDir.resolve("function.jar");
 
-    File outputJar = new File(stagingTempDir.newFolder(), "function.jar");
-    jarBuilder.buildJar(outputJar, source);
+    JarBuilder jarBuilder = new JarBuilder();
+    jarBuilder.buildJar(outputJar.toFile(), source.toFile());
 
-    GfshScript.of(getLocatorGFSHConnectionString(), "deploy --jars=" + outputJar.getCanonicalPath())
+    GfshScript
+        .of(connectCommand(),
+            "deploy --jars=" + outputJar)
         .execute(gfshRule);
 
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(), "list functions").execute(gfshRule)
-        .getOutputText()).contains("ExampleFunction");
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list functions")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains("ExampleFunction");
 
-    assertThat(
-        GfshScript.of(getLocatorGFSHConnectionString(), "execute function --id=ExampleFunction")
-            .execute(gfshRule)
-            .getOutputText()).contains("SUCCESS");
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "execute function --id=ExampleFunction")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains("SUCCESS");
 
     GfshScript
-        .of(getLocatorGFSHConnectionString(), "undeploy --jars=" + outputJar.getName())
+        .of(connectCommand(),
+            "undeploy --jars=" + outputJar.toFile().getName())
         .execute(gfshRule);
 
-    assertThat(GfshScript.of(getLocatorGFSHConnectionString(), "list functions").execute(gfshRule)
-        .getOutputText()).doesNotContain("ExampleFunction");
-
-
+    assertThat(GfshScript
+        .of(connectCommand(),
+            "list functions")
+        .execute(gfshRule)
+        .getOutputText())
+            .doesNotContain("ExampleFunction");
   }
 
   @Test
   public void testDeployPojo() throws IOException {
+    Path functionSource = loadTestResource("/example/test/function/PojoFunction.java");
+    Path pojoSource = loadTestResource("/example/test/pojo/ExamplePojo.java");
+    Path outputJar = stagingTempDir.resolve("functionAndPojo.jar");
+
     JarBuilder jarBuilder = new JarBuilder();
-    File functionSource = loadTestResource("/example/test/function/PojoFunction.java");
-    File pojoSource = loadTestResource("/example/test/pojo/ExamplePojo.java");
+    jarBuilder.buildJar(outputJar.toFile(), pojoSource.toFile(), functionSource.toFile());
 
-    File outputJar = new File(stagingTempDir.newFolder(), "functionAndPojo.jar");
-    jarBuilder.buildJar(outputJar, pojoSource, functionSource);
+    Path folderForExampleDiskStore =
+        createDirectories(stagingTempDir.resolve("folderForExampleDiskStore")).toAbsolutePath();
 
     System.out.println(GfshScript
-        .of(getLocatorGFSHConnectionString(),
-            "create disk-store --name=ExampleDiskStore --dir="
-                + stagingTempDir.newFolder().getCanonicalPath())
-        .execute(gfshRule).getOutputText());
+        .of(connectCommand(),
+            "create disk-store --name=ExampleDiskStore --dir=" + folderForExampleDiskStore)
+        .execute(gfshRule)
+        .getOutputText());
 
     System.out.println(GfshScript
-        .of(getLocatorGFSHConnectionString(),
+        .of(connectCommand(),
             "create region --name=/ExampleRegion --type=REPLICATE_PERSISTENT --disk-store=ExampleDiskStore")
         .execute(gfshRule).getOutputText());
 
     System.out.println(GfshScript
-        .of(getLocatorGFSHConnectionString(), "deploy --jars=" + outputJar.getAbsolutePath())
+        .of(connectCommand(),
+            "deploy --jars=" + outputJar)
         .execute(gfshRule));
 
-    System.out.println(
-        GfshScript.of(getLocatorGFSHConnectionString(), "execute function --id=PojoFunction")
-            .execute(gfshRule).getOutputText());
+    System.out.println(GfshScript
+        .of(connectCommand(),
+            "execute function --id=PojoFunction")
+        .execute(gfshRule)
+        .getOutputText());
 
     assertThat(GfshScript
-        .of(getLocatorGFSHConnectionString(), "query --query='SELECT * FROM /ExampleRegion'")
-        .execute(gfshRule).getOutputText()).contains("John");
+        .of(connectCommand(),
+            "query --query='SELECT * FROM /ExampleRegion'")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains("John");
+
+    // 1: issue stop command
+    GfshExecution execution = GfshScript
+        .of(connectCommand(),
+            "stop server --name=server")
+        .execute(gfshRule);
 
-    GfshScript.of(getLocatorGFSHConnectionString(), "stop server --name=server").execute(gfshRule);
+    // 2: await termination of process
+    execution.serverStopper().awaitStop("server");
 
-    GfshScript.of(getLocatorGFSHConnectionString(),
-        "start server --name=server --locators=localhost[10334]  --server-port=40404 --http-service-port=9090 --start-rest-api")
+    GfshScript
+        .of(connectCommand(),
+            String.format(
+                "start server --name=server --locators=localhost[%d] --disable-default-server",
+                locatorPort))
         .execute(gfshRule);
 
     assertThat(GfshScript
-        .of(getLocatorGFSHConnectionString(), "query --query='SELECT * FROM /ExampleRegion'")
-        .execute(gfshRule).getOutputText()).contains("John");
+        .of(connectCommand(),
+            "query --query='SELECT * FROM /ExampleRegion'")
+        .execute(gfshRule)
+        .getOutputText())
+            .contains("John");
   }
 
-  private File loadTestResource(String fileName) {
-    String filePath =
-        createTempFileFromResource(getClass(), fileName).getAbsolutePath();
-    assertThat(filePath).isNotNull();
+  private String connectCommand() {
+    return String.format("connect --locator=localhost[%d]", locatorPort);
+  }
 
-    return new File(filePath);
+  private Path loadTestResource(String fileName) {
+    return createTempFileFromResource(getClass(), fileName).toPath().toAbsolutePath();
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/GfshRuleExampleTest.java
similarity index 52%
copy from geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerAcceptanceTest.java
copy to geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/GfshRuleExampleTest.java
index c7568b15f2..3bc699f3f7 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/commands/StopServerAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/GfshRuleExampleTest.java
@@ -12,38 +12,45 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.management.internal.cli.commands;
+package org.apache.geode.rules;
+
+import static java.lang.String.format;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 
-public class StopServerAcceptanceTest {
+public class GfshRuleExampleTest {
 
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  private int locatorPort;
+  private int httpPort;
+  private int jmxPort;
 
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   @Before
-  public void startCluster() {
-    gfshRule.execute("start locator --name=locator", "start server --name=server --server-port=0");
-  }
-
-  @Test
-  public void canStopServerByNameWhenConnectedOverJmx() throws Exception {
-    gfshRule.execute("connect", "stop server --name=server");
-  }
-
-  @Test
-  public void canStopServerByNameWhenConnectedOverHttp() throws Exception {
-    gfshRule.execute("connect --use-http", "stop server --name=server");
+  public void setUp() {
+    int[] ports = getRandomAvailableTCPPorts(3);
+    locatorPort = ports[0];
+    httpPort = ports[1];
+    jmxPort = ports[2];
   }
 
   @Test
-  public void cannotStopServerByNameWhenNotConnected() throws Exception {
-    gfshRule.execute(GfshScript.of("stop server --name=server").expectFailure());
+  public void test() {
+    GfshScript
+        .of(format(
+            "start locator --name=locator --port=%d --http-service-port=%d --J=-Dgeode.jmx-manager-port=%d",
+            locatorPort, httpPort, jmxPort),
+            "start server --name=server --disable-default-server")
+        .execute(gfshRule);
   }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartLocatorGlobalSerialFilterAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartLocatorGlobalSerialFilterAcceptanceTest.java
index e4cc254d02..e5fd9d8cac 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartLocatorGlobalSerialFilterAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartLocatorGlobalSerialFilterAcceptanceTest.java
@@ -24,15 +24,18 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.RequiresGeodeHome;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class StartLocatorGlobalSerialFilterAcceptanceTest {
 
-  @Rule
+  @Rule(order = 0)
   public RequiresGeodeHome requiresGeodeHome = new RequiresGeodeHome();
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  @Rule(order = 1)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 2)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   private Path locatorFolder;
   private int locatorPort;
@@ -41,7 +44,7 @@ public class StartLocatorGlobalSerialFilterAcceptanceTest {
 
   @Before
   public void setUpFiles() {
-    locatorFolder = gfshRule.getTemporaryFolder().getRoot().toPath().toAbsolutePath();
+    locatorFolder = folderRule.getFolder().toPath().toAbsolutePath();
     locatorLogFile = locatorFolder.resolve("locator.log");
   }
 
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartLocatorJmxSerialFilterAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartLocatorJmxSerialFilterAcceptanceTest.java
index f6d50c2f36..ed43ac0c9e 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartLocatorJmxSerialFilterAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartLocatorJmxSerialFilterAcceptanceTest.java
@@ -29,6 +29,7 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.RequiresGeodeHome;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
@@ -36,10 +37,12 @@ public class StartLocatorJmxSerialFilterAcceptanceTest {
 
   private static final String PROPERTY_NAME = "jmx.remote.rmi.server.serial.filter.pattern";
 
-  @Rule
+  @Rule(order = 0)
   public RequiresGeodeHome requiresGeodeHome = new RequiresGeodeHome();
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  @Rule(order = 1)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 2)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   private Path locatorFolder;
   private int locatorPort;
@@ -48,7 +51,7 @@ public class StartLocatorJmxSerialFilterAcceptanceTest {
 
   @Before
   public void setUpFiles() {
-    locatorFolder = gfshRule.getTemporaryFolder().getRoot().toPath().toAbsolutePath();
+    locatorFolder = folderRule.getFolder().toPath().toAbsolutePath();
     locatorLogFile = locatorFolder.resolve("locator.log");
   }
 
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartServerGlobalSerialFilterAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartServerGlobalSerialFilterAcceptanceTest.java
index 06bd7e2987..a59f855c1a 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartServerGlobalSerialFilterAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartServerGlobalSerialFilterAcceptanceTest.java
@@ -17,7 +17,6 @@ package org.apache.geode.serialization.filter;
 import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 
-import java.io.File;
 import java.nio.file.Path;
 
 import org.junit.Before;
@@ -25,22 +24,25 @@ import org.junit.Rule;
 import org.junit.Test;
 
 import org.apache.geode.test.assertj.LogFileAssert;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.RequiresGeodeHome;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class StartServerGlobalSerialFilterAcceptanceTest {
 
-  @Rule
+  @Rule(order = 0)
   public RequiresGeodeHome requiresGeodeHome = new RequiresGeodeHome();
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  @Rule(order = 1)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 2)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  private File serverFolder;
+  private Path serverFolder;
   private int jmxPort;
 
   @Before
   public void setServerFolder() {
-    serverFolder = gfshRule.getTemporaryFolder().getRoot();
+    serverFolder = folderRule.getFolder().toPath().toAbsolutePath();
   }
 
   @Before
@@ -53,7 +55,7 @@ public class StartServerGlobalSerialFilterAcceptanceTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=server",
-        "--dir=" + serverFolder.getAbsolutePath(),
+        "--dir=" + serverFolder,
         "--disable-default-server",
         "--J=-Dgemfire.enable-cluster-configuration=false",
         "--J=-Dgemfire.http-service-port=0",
@@ -63,7 +65,7 @@ public class StartServerGlobalSerialFilterAcceptanceTest {
 
     gfshRule.execute(startServerCommand);
 
-    Path serverLogFile = serverFolder.toPath().resolve("server.log");
+    Path serverLogFile = serverFolder.resolve("server.log");
     await().untilAsserted(() -> {
       LogFileAssert.assertThat(serverLogFile.toFile()).exists()
           .doesNotContain("Global serial filter is now configured.")
@@ -76,7 +78,7 @@ public class StartServerGlobalSerialFilterAcceptanceTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=server",
-        "--dir=" + serverFolder.getAbsolutePath(),
+        "--dir=" + serverFolder,
         "--disable-default-server",
         "--J=-Dgemfire.enable-cluster-configuration=false",
         "--J=-Dgemfire.http-service-port=0",
@@ -87,7 +89,7 @@ public class StartServerGlobalSerialFilterAcceptanceTest {
 
     gfshRule.execute(startServerCommand);
 
-    Path serverLogFile = serverFolder.toPath().resolve("server.log");
+    Path serverLogFile = serverFolder.resolve("server.log");
     await().untilAsserted(() -> {
       LogFileAssert.assertThat(serverLogFile.toFile()).exists()
           .doesNotContain("Global serial filter is now configured.")
@@ -104,7 +106,7 @@ public class StartServerGlobalSerialFilterAcceptanceTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=server",
-        "--dir=" + serverFolder.getAbsolutePath(),
+        "--dir=" + serverFolder,
         "--disable-default-server",
         "--J=-Dgemfire.enable-cluster-configuration=false",
         "--J=-Dgemfire.http-service-port=0",
@@ -115,7 +117,7 @@ public class StartServerGlobalSerialFilterAcceptanceTest {
 
     gfshRule.execute(startServerCommand);
 
-    Path serverLogFile = serverFolder.toPath().resolve("server.log");
+    Path serverLogFile = serverFolder.resolve("server.log");
     await().untilAsserted(() -> {
       LogFileAssert.assertThat(serverLogFile.toFile()).exists()
           .contains("Global serial filter is now configured.")
@@ -128,7 +130,7 @@ public class StartServerGlobalSerialFilterAcceptanceTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=server",
-        "--dir=" + serverFolder.getAbsolutePath(),
+        "--dir=" + serverFolder,
         "--disable-default-server",
         "--J=-Dgemfire.enable-cluster-configuration=false",
         "--J=-Dgemfire.http-service-port=0",
@@ -140,7 +142,7 @@ public class StartServerGlobalSerialFilterAcceptanceTest {
 
     gfshRule.execute(startServerCommand);
 
-    Path serverLogFile = serverFolder.toPath().resolve("server.log");
+    Path serverLogFile = serverFolder.resolve("server.log");
     await().untilAsserted(() -> {
       LogFileAssert.assertThat(serverLogFile.toFile()).exists()
           .doesNotContain("Global serial filter is now configured.")
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartServerJmxSerialFilterAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartServerJmxSerialFilterAcceptanceTest.java
index bf03b22c71..34c82e877b 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartServerJmxSerialFilterAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/serialization/filter/StartServerJmxSerialFilterAcceptanceTest.java
@@ -29,6 +29,7 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.RequiresGeodeHome;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
@@ -36,10 +37,12 @@ public class StartServerJmxSerialFilterAcceptanceTest {
 
   private static final String PROPERTY_NAME = "jmx.remote.rmi.server.serial.filter.pattern";
 
-  @Rule
+  @Rule(order = 0)
   public RequiresGeodeHome requiresGeodeHome = new RequiresGeodeHome();
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  @Rule(order = 1)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 2)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   private Path serverFolder;
   private int jmxPort;
@@ -47,7 +50,7 @@ public class StartServerJmxSerialFilterAcceptanceTest {
 
   @Before
   public void setUpFiles() {
-    serverFolder = gfshRule.getTemporaryFolder().getRoot().toPath().toAbsolutePath();
+    serverFolder = folderRule.getFolder().toPath().toAbsolutePath();
     serverLogFile = serverFolder.resolve("server.log");
   }
 
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/ssl/CertificateRotationTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/ssl/CertificateRotationTest.java
index 8ea214d4f1..82e1179742 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/ssl/CertificateRotationTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/ssl/CertificateRotationTest.java
@@ -14,6 +14,8 @@
  */
 package org.apache.geode.ssl;
 
+import static java.nio.file.Files.createDirectories;
+import static java.nio.file.Files.createFile;
 import static java.util.regex.Pattern.compile;
 import static java.util.regex.Pattern.quote;
 import static org.apache.geode.cache.client.ClientRegionShortcut.PROXY;
@@ -23,7 +25,6 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatCode;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
-import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.net.InetAddress;
@@ -40,7 +41,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.client.ClientCache;
@@ -48,12 +48,14 @@ import org.apache.geode.cache.client.ClientCacheFactory;
 import org.apache.geode.cache.ssl.CertStores;
 import org.apache.geode.cache.ssl.CertificateBuilder;
 import org.apache.geode.cache.ssl.CertificateMaterial;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 /**
  * This test creates a cluster and a client with SSL enabled for all components and client
  * authentication enabled.
  *
+ * <p>
  * It verifies that the cluster certificate, the client certificate, and the CA certificate can be
  * rotated without having to restart the client or the members.
  */
@@ -64,26 +66,27 @@ public class CertificateRotationTest {
   private static final Pattern updatedKeyManager = compile("Updated KeyManager");
   private static final Pattern updatedTrustManager = compile("Updated TrustManager");
 
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-  @Rule
-  public GfshRule gfshRule = new GfshRule();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
   private CertificateMaterial caCert;
 
   private String[] memberNames;
   private int locatorPort;
   private int locatorHttpPort;
-  private File clusterKeyStore;
-  private File clusterTrustStore;
-  private File clusterSecurityProperties;
+  private Path clusterKeyStore;
+  private Path clusterTrustStore;
+  private Path clusterSecurityProperties;
 
   private ClientCache client;
   private Region<String, String> region;
-  private File clientKeyStore;
-  private File clientTrustStore;
-  private File clientLogFile;
+  private Path clientKeyStore;
+  private Path clientTrustStore;
+  private Path clientLogFile;
+
+  private Path rootFolder;
 
   /**
    * The test setup creates a cluster with 1 locator and 2 servers, a client cache, and a CA
@@ -92,7 +95,9 @@ public class CertificateRotationTest {
    * client has a certificate signed by the same CA and also trusts the CA certificate.
    */
   @Before
-  public void setUp() throws Exception {
+  public void setUp() throws IOException, GeneralSecurityException, InterruptedException {
+    rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+
     caCert = new CertificateBuilder()
         .commonName("ca")
         .isCA()
@@ -116,7 +121,8 @@ public class CertificateRotationTest {
    * connection.
    */
   @Test
-  public void rotateClusterCertificate() throws Exception {
+  public void rotateClusterCertificate()
+      throws GeneralSecurityException, IOException {
     CertificateMaterial newClusterCert = new CertificateBuilder()
         .commonName("cluster")
         .issuedBy(caCert)
@@ -124,7 +130,7 @@ public class CertificateRotationTest {
         .sanIpAddress(InetAddress.getByName("127.0.0.1"))
         .generate();
 
-    writeCertsToKeyStore(clusterKeyStore.toPath(), newClusterCert);
+    writeCertsToKeyStore(clusterKeyStore, newClusterCert);
     waitForMembersToLogMessage(updatedKeyManager);
 
     assertThatCode(() -> region.put("foo", "bar"))
@@ -137,7 +143,8 @@ public class CertificateRotationTest {
    * connection.
    */
   @Test
-  public void rotateClientCertificate() throws Exception {
+  public void rotateClientCertificate()
+      throws GeneralSecurityException, IOException {
     CertificateMaterial newClientCert = new CertificateBuilder()
         .commonName("client")
         .issuedBy(caCert)
@@ -145,7 +152,7 @@ public class CertificateRotationTest {
         .sanIpAddress(InetAddress.getByName("127.0.0.1"))
         .generate();
 
-    writeCertsToKeyStore(clientKeyStore.toPath(), newClientCert);
+    writeCertsToKeyStore(clientKeyStore, newClientCert);
     waitForClientToLogMessage(updatedKeyManager);
 
     assertThatCode(() -> region.put("foo", "bar"))
@@ -159,7 +166,8 @@ public class CertificateRotationTest {
    * CA certificate removed.
    */
   @Test
-  public void rotateCaCertificate() throws Exception {
+  public void rotateCaCertificate()
+      throws GeneralSecurityException, IOException {
     /*
      * First, create a new CA certificate and add it to both the cluster's and the client's trust
      * stores. The trust stores will contain both the old and the new CA certificates.
@@ -170,8 +178,8 @@ public class CertificateRotationTest {
         .isCA()
         .generate();
 
-    writeCertsToTrustStore(clusterTrustStore.toPath(), caCert, newCaCert);
-    writeCertsToTrustStore(clientTrustStore.toPath(), caCert, newCaCert);
+    writeCertsToTrustStore(clusterTrustStore, caCert, newCaCert);
+    writeCertsToTrustStore(clientTrustStore, caCert, newCaCert);
 
     waitForMembersToLogMessage(updatedTrustManager);
     waitForClientToLogMessage(updatedTrustManager);
@@ -195,8 +203,8 @@ public class CertificateRotationTest {
         .sanIpAddress(InetAddress.getByName("127.0.0.1"))
         .generate();
 
-    writeCertsToKeyStore(clusterKeyStore.toPath(), newClusterCert);
-    writeCertsToKeyStore(clientKeyStore.toPath(), newClientCert);
+    writeCertsToKeyStore(clusterKeyStore, newClusterCert);
+    writeCertsToKeyStore(clientKeyStore, newClientCert);
 
     waitForMembersToLogMessage(updatedKeyManager);
     waitForClientToLogMessage(updatedKeyManager);
@@ -205,8 +213,8 @@ public class CertificateRotationTest {
      * Finally, remove the old CA certificate from both the cluster's and the client's trust stores.
      */
 
-    writeCertsToTrustStore(clusterTrustStore.toPath(), newCaCert);
-    writeCertsToTrustStore(clientTrustStore.toPath(), newCaCert);
+    writeCertsToTrustStore(clusterTrustStore, newCaCert);
+    writeCertsToTrustStore(clientTrustStore, newCaCert);
 
     for (String name : memberNames) {
       await().untilAsserted(() -> assertThat(logsForMember(name))
@@ -229,14 +237,15 @@ public class CertificateRotationTest {
    * being dynamically updated.
    */
   @Test
-  public void untrustedCertificateThrows() throws Exception {
+  public void untrustedCertificateThrows()
+      throws GeneralSecurityException, IOException {
     CertificateMaterial selfSignedCert = new CertificateBuilder()
         .commonName("client")
         .sanDnsName("localhost")
         .sanIpAddress(InetAddress.getByName("127.0.0.1"))
         .generate();
 
-    writeCertsToKeyStore(clientKeyStore.toPath(), selfSignedCert);
+    writeCertsToKeyStore(clientKeyStore, selfSignedCert);
     waitForClientToLogMessage(updatedKeyManager);
 
     assertThatThrownBy(() -> region.put("foo", "bar"))
@@ -281,11 +290,11 @@ public class CertificateRotationTest {
   }
 
   private Stream<String> logsForClient() throws IOException {
-    return Files.lines(clientLogFile.toPath());
+    return Files.lines(clientLogFile);
   }
 
   private Stream<String> logsForMember(String name) throws IOException {
-    Path logFile = temporaryFolder.getRoot().toPath().resolve(name).resolve(name + ".log");
+    Path logFile = rootFolder.resolve(name).resolve(name + ".log");
     return Files.lines(logFile);
   }
 
@@ -297,22 +306,22 @@ public class CertificateRotationTest {
         .sanIpAddress(InetAddress.getByName("127.0.0.1"))
         .generate();
 
-    clientKeyStore = temporaryFolder.newFile("client-keystore.jks");
-    writeCertsToKeyStore(clientKeyStore.toPath(), clientCert);
+    clientKeyStore = createFile(rootFolder.resolve("client-keystore.jks"));
+    writeCertsToKeyStore(clientKeyStore, clientCert);
 
-    clientTrustStore = temporaryFolder.newFile("client-truststore.jks");
-    writeCertsToTrustStore(clientTrustStore.toPath(), caCert);
+    clientTrustStore = createFile(rootFolder.resolve("client-truststore.jks"));
+    writeCertsToTrustStore(clientTrustStore, caCert);
 
-    File clientSecurityProperties = temporaryFolder.newFile("client-security.properties");
+    Path clientSecurityProperties = createFile(rootFolder.resolve("client-security.properties"));
     Properties properties = CertStores.propertiesWith("all", "any", "any",
         clientTrustStore, dummyStorePass, clientKeyStore, dummyStorePass, true, true);
-    properties.store(new FileOutputStream(clientSecurityProperties), "");
+    properties.store(new FileOutputStream(clientSecurityProperties.toFile()), "");
 
-    clientLogFile = temporaryFolder.newFile("client.log");
+    clientLogFile = createFile(rootFolder.resolve("client.log"));
 
     client = new ClientCacheFactory(properties)
         .addPoolLocator("localhost", locatorPort)
-        .set("log-file", clientLogFile.getAbsolutePath())
+        .set("log-file", clientLogFile.toString())
         // prevent the client from creating a connection until the first cache operation
         .setPoolMinConnections(0)
         .create();
@@ -320,8 +329,8 @@ public class CertificateRotationTest {
     region = client.<String, String>createClientRegionFactory(PROXY)
         .create(regionName);
 
-    waitForClientToLogMessage(compile(quote("Started watching " + clientKeyStore.getPath())));
-    waitForClientToLogMessage(compile(quote("Started watching " + clientTrustStore.getPath())));
+    waitForClientToLogMessage(compile(quote("Started watching " + clientKeyStore)));
+    waitForClientToLogMessage(compile(quote("Started watching " + clientTrustStore)));
 
     /*
      * This sleep is needed to ensure that any updates to the key or trust store file are detected
@@ -339,16 +348,16 @@ public class CertificateRotationTest {
         .sanIpAddress(InetAddress.getByName("127.0.0.1"))
         .generate();
 
-    clusterKeyStore = temporaryFolder.newFile("cluster-keystore.jks");
-    writeCertsToKeyStore(clusterKeyStore.toPath(), clusterCert);
+    clusterKeyStore = createFile(rootFolder.resolve("cluster-keystore.jks"));
+    writeCertsToKeyStore(clusterKeyStore, clusterCert);
 
-    clusterTrustStore = temporaryFolder.newFile("cluster-truststore.jks");
-    writeCertsToTrustStore(clusterTrustStore.toPath(), caCert);
+    clusterTrustStore = createFile(rootFolder.resolve("cluster-truststore.jks"));
+    writeCertsToTrustStore(clusterTrustStore, caCert);
 
-    clusterSecurityProperties = temporaryFolder.newFile("cluster-security.properties");
+    clusterSecurityProperties = createFile(rootFolder.resolve("cluster-security.properties"));
     Properties properties = CertStores.propertiesWith("all", "any", "any",
         clusterTrustStore, dummyStorePass, clusterKeyStore, dummyStorePass, true, true);
-    properties.store(new FileOutputStream(clusterSecurityProperties), "");
+    properties.store(new FileOutputStream(clusterSecurityProperties.toFile()), "");
 
     memberNames = new String[] {"locator", "server1", "server2"};
 
@@ -359,7 +368,7 @@ public class CertificateRotationTest {
   }
 
   private void startLocator(String name) throws IOException {
-    File dir = temporaryFolder.newFolder(name);
+    Path dir = createDirectories(rootFolder.resolve(name));
 
     int[] availablePorts = getRandomAvailableTCPPorts(3);
     locatorPort = availablePorts[0];
@@ -370,18 +379,18 @@ public class CertificateRotationTest {
         "start locator",
         "--connect=false",
         "--name=" + name,
-        "--dir=" + dir.getAbsolutePath(),
+        "--dir=" + dir,
         "--bind-address=127.0.0.1",
         "--port=" + locatorPort,
         "--http-service-port=" + locatorHttpPort,
         "--J=-Dgemfire.jmx-manager-port=" + locatorJmxPort,
-        "--security-properties-file=" + clusterSecurityProperties.getAbsolutePath());
+        "--security-properties-file=" + clusterSecurityProperties);
 
     gfshRule.execute(startLocatorCommand);
   }
 
   private void startServer(String name) throws IOException {
-    File dir = temporaryFolder.newFolder(name);
+    Path dir = createDirectories(rootFolder.resolve(name));
 
     int[] availablePorts = getRandomAvailableTCPPorts(1);
     int port = availablePorts[0];
@@ -391,11 +400,11 @@ public class CertificateRotationTest {
     String startServerCommand = String.join(" ",
         "start server",
         "--name=" + name,
-        "--dir=" + dir.getAbsolutePath(),
+        "--dir=" + dir,
         "--bind-address=127.0.0.1",
         "--server-port=" + port,
         "--locators=" + locatorString,
-        "--security-properties-file=" + clusterSecurityProperties.getAbsolutePath());
+        "--security-properties-file=" + clusterSecurityProperties);
 
     gfshRule.execute(startServerCommand);
   }
@@ -406,7 +415,7 @@ public class CertificateRotationTest {
         "--use-http",
         "--use-ssl",
         "--url=https://localhost:" + locatorHttpPort + "/geode-mgmt/v1",
-        "--security-properties-file=" + clusterSecurityProperties.getAbsolutePath());
+        "--security-properties-file=" + clusterSecurityProperties);
 
     String createRegionCommand = String.join(" ",
         "create region",
@@ -422,7 +431,7 @@ public class CertificateRotationTest {
         "--use-http",
         "--use-ssl",
         "--url=https://localhost:" + locatorHttpPort + "/geode-mgmt/v1",
-        "--security-properties-file=" + clusterSecurityProperties.getAbsolutePath());
+        "--security-properties-file=" + clusterSecurityProperties);
 
     String shutdownCommand = "shutdown --include-locators=true";
     gfshRule.execute(connectToLocatorCommand, shutdownCommand);
diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/CargoTestBase.java b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/CargoTestBase.java
index f456c79669..861aafb09a 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/CargoTestBase.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/CargoTestBase.java
@@ -448,7 +448,7 @@ public abstract class CargoTestBase {
 
   private void checkLogs() {
     for (int i = 0; i < manager.numContainers(); i++) {
-      File cargo_dir = manager.getContainer(i).cargoLogDir;
+      File cargo_dir = manager.getContainer(i).cargoLogDir.toFile();
       LogChecker.checkLogs(cargo_dir);
     }
   }
diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/GenericAppServerContainer.java b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/GenericAppServerContainer.java
index d4d955bcf7..90e5ed5090 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/GenericAppServerContainer.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/GenericAppServerContainer.java
@@ -17,11 +17,11 @@ package org.apache.geode.session.tests;
 import static java.lang.System.lineSeparator;
 import static java.nio.charset.Charset.defaultCharset;
 import static org.apache.commons.io.FileUtils.readLines;
-import static org.apache.geode.session.tests.ContainerInstall.GEODE_BUILD_HOME;
-import static org.apache.geode.session.tests.ContainerInstall.TMP_DIR;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.function.IntSupplier;
@@ -46,8 +46,6 @@ public class GenericAppServerContainer extends ServerContainer {
   private final File modifyWarScript;
   private final File modifyWarScriptLog;
 
-  private static final String DEFAULT_GENERIC_APPSERVER_WAR_DIR = TMP_DIR + "/cargo_wars/";
-
   /**
    * Setup the generic appserver container
    *
@@ -56,9 +54,10 @@ public class GenericAppServerContainer extends ServerContainer {
    * creating a temporary WAR file to use, deploys the war to the Cargo container, and sets various
    * container properties (i.e. locator, local cache, etc.)
    */
-  public GenericAppServerContainer(GenericAppServerInstall install, File containerConfigHome,
+  public GenericAppServerContainer(GenericAppServerInstall install, Path rootDir,
+      Path containerConfigHome,
       String containerDescriptors, IntSupplier portSupplier) throws IOException {
-    super(install, containerConfigHome, containerDescriptors, portSupplier);
+    super(install, rootDir, containerConfigHome, containerDescriptors, portSupplier);
 
     // Setup modify war script file so that it is executable and easily findable
     modifyWarScript = new File(install.getModulePath() + "/bin/modify_war");
@@ -72,9 +71,9 @@ public class GenericAppServerContainer extends ServerContainer {
     Assume.assumeFalse(System.getProperty("os.name").toLowerCase().contains("win"));
 
     // Create temp war file to use
-    File warDir = new File(DEFAULT_GENERIC_APPSERVER_WAR_DIR);
-    warDir.mkdirs();
-    setWarFile(File.createTempFile(description, ".war", warDir));
+    Path defaultGenericAppserverWarDir = rootDir.resolve("cargo_wars");
+    Files.createDirectories(defaultGenericAppserverWarDir);
+    setWarFile(Files.createFile(defaultGenericAppserverWarDir.resolve(description + ".war")));
 
     // Deploy war file to container configuration
     deployWar();
@@ -107,13 +106,13 @@ public class GenericAppServerContainer extends ServerContainer {
     command.add(modifyWarScript.getAbsolutePath());
     // Path to the WAR file to modify
     command.add("-w");
-    command.add(install.getWarFilePath());
+    command.add(install.getWarFilePath().toString());
     // Get connection type for the WAR (peer-to-peer or client-server)
     command.add("-t");
     command.add(install.getConnectionType().getName());
     // Path to the modified version of the origin WAR file
     command.add("-o");
-    command.add(getWarFile().getAbsolutePath());
+    command.add(getWarFile().toString());
     // Add all the cache properties setup to the WAR file
     for (String property : cacheProperties.keySet()) {
       command.add("-p");
@@ -140,7 +139,7 @@ public class GenericAppServerContainer extends ServerContainer {
   private void modifyWarFile() throws IOException, InterruptedException {
     // Build the environment to run the command
     ProcessBuilder builder = new ProcessBuilder();
-    builder.environment().put("GEODE", GEODE_BUILD_HOME);
+    builder.environment().put("GEODE", ContainerInstall.GEODE_HOME_PATH.toString());
     builder.inheritIO();
     // Setup the environment builder with the command
     builder.command(buildCommand());
diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/GenericAppServerInstall.java b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/GenericAppServerInstall.java
index be5b8035f2..42bd6e7eec 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/GenericAppServerInstall.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/GenericAppServerInstall.java
@@ -14,8 +14,9 @@
  */
 package org.apache.geode.session.tests;
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.function.IntSupplier;
 
 /**
@@ -70,14 +71,21 @@ public class GenericAppServerInstall extends ContainerInstall {
   private final GenericAppServerVersion version;
 
   public GenericAppServerInstall(String name, GenericAppServerVersion version,
-      ConnectionType connType, IntSupplier portSupplier) throws IOException, InterruptedException {
-    super(name, version.getDownloadURL(), connType, "appserver", portSupplier);
+      ConnectionType connType, IntSupplier portSupplier) throws IOException {
+    this(Files.createTempDirectory("geode_container_install").toAbsolutePath(), name, version,
+        connType, portSupplier);
+  }
+
+  public GenericAppServerInstall(Path rootDir, String name, GenericAppServerVersion version,
+      ConnectionType connType, IntSupplier portSupplier) throws IOException {
+    super(rootDir, name, version.getDownloadURL(), connType, "appserver", portSupplier);
 
     this.version = version;
   }
 
   /**
-   * Implementation of {@link ContainerInstall#generateContainer(File, String)}, which generates a
+   * Implementation of {@link ContainerInstall#generateContainer(Path, Path, String)}, which
+   * generates a
    * generic appserver specific container
    *
    * Creates a {@link GenericAppServerContainer} instance off of this installation.
@@ -85,9 +93,10 @@ public class GenericAppServerInstall extends ContainerInstall {
    * @param containerDescriptors Additional descriptors used to identify a container
    */
   @Override
-  public GenericAppServerContainer generateContainer(File containerConfigHome,
+  public GenericAppServerContainer generateContainer(Path rootDir,
+      Path containerConfigHome,
       String containerDescriptors) throws IOException {
-    return new GenericAppServerContainer(this, containerConfigHome, containerDescriptors,
+    return new GenericAppServerContainer(this, rootDir, containerConfigHome, containerDescriptors,
         portSupplier());
   }
 
diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/Tomcat8ClientServerCustomCacheXmlTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/Tomcat8ClientServerCustomCacheXmlTest.java
index d34db0c750..67488fe071 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/Tomcat8ClientServerCustomCacheXmlTest.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/Tomcat8ClientServerCustomCacheXmlTest.java
@@ -29,7 +29,7 @@ public class Tomcat8ClientServerCustomCacheXmlTest extends Tomcat8ClientServerTe
       regionAttributes.put("name", "gemfire_modules_sessions");
 
       ContainerInstall.editXMLFile(
-          container.cacheXMLFile.getAbsolutePath(),
+          container.cacheXMLFile,
           null,
           "region",
           "client-cache",
diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/TomcatClientServerTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/TomcatClientServerTest.java
index 92797964b5..43b3ddcd47 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/TomcatClientServerTest.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/session/tests/TomcatClientServerTest.java
@@ -15,6 +15,7 @@
 package org.apache.geode.session.tests;
 
 import java.io.File;
+import java.nio.file.Path;
 
 import org.junit.After;
 import org.junit.Before;
@@ -58,8 +59,8 @@ public abstract class TomcatClientServerTest extends CargoTestBase {
 
   private String startAServer(int serverNumber) {
     // List of all the jars for tomcat to put on the server classpath
-    String libDirJars = install.getHome() + "/lib/*";
-    String binDirJars = install.getHome() + "/bin/*";
+    Path libDir = install.getHome().resolve("lib");
+    Path binDir = install.getHome().resolve("bin");
 
     // Set server name based on the test about to be run
     String serverName =
@@ -72,7 +73,7 @@ public abstract class TomcatClientServerTest extends CargoTestBase {
     command.addOption(CliStrings.START_SERVER__SERVER_PORT, String.valueOf(locatorPortSuggestion));
     // Add Tomcat jars to server classpath
     command.addOption(CliStrings.START_SERVER__CLASSPATH,
-        binDirJars + File.pathSeparator + libDirJars);
+        binDir + "/*" + File.pathSeparator + libDir + "/*");
     command.addOption(CliStrings.START_SERVER__LOCATORS,
         locatorVM.invoke(() -> ClusterStartupRule.getLocator().asString()));
     command.addOption(CliStrings.START_SERVER__J, "-Dgemfire.member-timeout=60000");
diff --git a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/DeploymentManagementUpgradeTest.java b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/DeploymentManagementUpgradeTest.java
index 4f44278023..3f95214b58 100644
--- a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/DeploymentManagementUpgradeTest.java
+++ b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/DeploymentManagementUpgradeTest.java
@@ -15,93 +15,116 @@
 
 package org.apache.geode.management;
 
+import static java.nio.file.Files.createDirectory;
+import static java.util.stream.Collectors.toList;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
 import static org.apache.geode.test.junit.assertions.ClusterManagementListResultAssert.assertManagementListResult;
-import static org.apache.geode.test.junit.rules.gfsh.GfshRule.startLocatorCommand;
+import static org.apache.geode.test.version.TestVersions.atLeast;
+import static org.apache.geode.test.version.VmConfigurations.hasGeodeVersion;
 
 import java.io.File;
 import java.io.IOException;
 import java.util.Collection;
-import java.util.List;
 
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.junit.runners.Parameterized.UseParametersRunnerFactory;
 
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.management.api.ClusterManagementService;
 import org.apache.geode.management.cluster.client.ClusterManagementServiceBuilder;
 import org.apache.geode.management.configuration.Deployment;
 import org.apache.geode.test.compiler.JarBuilder;
 import org.apache.geode.test.junit.categories.BackwardCompatibilityTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecutor;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.apache.geode.test.version.TestVersion;
-import org.apache.geode.test.version.VersionManager;
+import org.apache.geode.test.version.VmConfiguration;
+import org.apache.geode.test.version.VmConfigurations;
 
-@Category({BackwardCompatibilityTest.class})
+@Category(BackwardCompatibilityTest.class)
 @RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+@UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class DeploymentManagementUpgradeTest {
-  private final String oldVersion;
 
-  @Parameterized.Parameters(name = "{0}")
-  public static Collection<String> data() {
-    List<String> result = VersionManager.getInstance().getVersionsWithoutCurrent();
-    result.removeIf(s -> TestVersion.compare(s, "1.10.0") < 0);
-    return result;
+  @Parameters(name = "{0}")
+  public static Collection<VmConfiguration> data() {
+    return VmConfigurations.upgrades().stream()
+        .filter(hasGeodeVersion(atLeast(TestVersion.valueOf("1.10.0"))))
+        .collect(toList());
   }
 
-  public DeploymentManagementUpgradeTest(String version) {
-    oldVersion = version;
-    oldGfsh = new GfshRule(oldVersion);
-  }
+  private static final String HOSTNAME = "localhost";
+
+  private final VmConfiguration sourceVmConfiguration;
 
-  @Rule
-  public GfshRule oldGfsh;
+  private File clusterJar;
+  private GfshExecutor oldGfsh;
+  private GfshExecutor currentGfsh;
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
+  public DeploymentManagementUpgradeTest(VmConfiguration sourceVmConfiguration) {
+    this.sourceVmConfiguration = sourceVmConfiguration;
+  }
 
-  @ClassRule
-  public static TemporaryFolder tempFolder = new TemporaryFolder();
-  private static File stagingDir, clusterJar;
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  @BeforeClass
-  public static void beforeClass() throws Exception {
+  @Before
+  public void setUp() throws IOException {
     // prepare the jars to be deployed
-    stagingDir = tempFolder.newFolder("staging");
-    clusterJar = new File(stagingDir, "cluster.jar");
+    File stagingDir = createDirectory(folderRule.getFolder().toPath().resolve("staging")).toFile();
+    clusterJar = stagingDir.toPath().resolve("cluster.jar").toFile();
     JarBuilder jarBuilder = new JarBuilder();
     jarBuilder.buildJarFromClassNames(clusterJar, "Class1");
+
+    oldGfsh = gfshRule.executor().withVmConfiguration(sourceVmConfiguration).build();
+    currentGfsh = gfshRule.executor().build();
   }
 
   @Test
-  public void newLocatorCanReadOldConfigurationData() throws IOException {
-    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(3);
+  public void newLocatorCanReadOldConfigurationData() {
+    int[] ports = getRandomAvailableTCPPorts(3);
     int httpPort = ports[0];
     int locatorPort = ports[1];
     int jmxPort = ports[2];
-    GfshExecution execute =
-        GfshScript.of(startLocatorCommand("test", "localhost", locatorPort, jmxPort, httpPort, 0))
-            .and("deploy --jar=" + clusterJar.getAbsolutePath())
-            .and("shutdown --include-locators")
-            .execute(oldGfsh);
+    GfshExecution execute = GfshScript
+        .of(startLocatorCommand("test", locatorPort, jmxPort, httpPort, 0))
+        .and("deploy --jar=" + clusterJar.getAbsolutePath())
+        .and("shutdown --include-locators")
+        .execute(oldGfsh);
 
     // use the latest gfsh to start the locator in the same working dir
-    GfshScript.of(startLocatorCommand("test", "localhost", locatorPort, jmxPort, httpPort, 0))
-        .execute(gfsh, execute.getWorkingDir());
+    GfshScript
+        .of(startLocatorCommand("test", locatorPort, jmxPort, httpPort, 0))
+        .execute(currentGfsh, execute.getWorkingDir());
 
     ClusterManagementService cms = new ClusterManagementServiceBuilder()
         .setPort(httpPort)
         .build();
-    assertManagementListResult(cms.list(new Deployment())).isSuccessful()
-        .hasConfigurations().hasSize(1);
+    assertManagementListResult(cms.list(new Deployment()))
+        .isSuccessful()
+        .hasConfigurations()
+        .hasSize(1);
+  }
+
+  private static String startLocatorCommand(String name, int port, int jmxPort, int httpPort,
+      int connectedLocatorPort) {
+    String startLocatorCommand =
+        "start locator --name=%s --port=%d --http-service-port=%d --J=-Dgemfire.jmx-manager-port=%d";
+    if (connectedLocatorPort > 0) {
+      return String.format(startLocatorCommand + " --locators=%s[%d]",
+          name, port, httpPort, jmxPort, HOSTNAME, connectedLocatorPort);
+    }
+    return String.format(startLocatorCommand, name, port, httpPort, jmxPort);
   }
 }
diff --git a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/OperationManagementUpgradeTest.java b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/OperationManagementUpgradeTest.java
index f579e0e056..34d877a55c 100644
--- a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/OperationManagementUpgradeTest.java
+++ b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/OperationManagementUpgradeTest.java
@@ -15,20 +15,24 @@
 
 package org.apache.geode.management;
 
-import static org.apache.geode.test.junit.rules.gfsh.GfshRule.startLocatorCommand;
-import static org.apache.geode.test.junit.rules.gfsh.GfshRule.startServerCommand;
+import static java.util.stream.Collectors.toList;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.version.TestVersions.greaterThan;
+import static org.apache.geode.test.version.VmConfigurations.hasGeodeVersion;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.util.Collection;
-import java.util.List;
 
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.junit.runners.Parameterized.UseParametersRunnerFactory;
 
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.management.api.ClusterManagementOperationResult;
 import org.apache.geode.management.api.ClusterManagementResult;
 import org.apache.geode.management.api.ClusterManagementService;
@@ -36,46 +40,61 @@ import org.apache.geode.management.cluster.client.ClusterManagementServiceBuilde
 import org.apache.geode.management.operation.RebalanceOperation;
 import org.apache.geode.management.runtime.RebalanceResult;
 import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.internal.DUnitLauncher;
+import org.apache.geode.test.dunit.rules.DistributedRule;
 import org.apache.geode.test.junit.categories.BackwardCompatibilityTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecutor;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.apache.geode.test.version.TestVersion;
-import org.apache.geode.test.version.VersionManager;
+import org.apache.geode.test.version.VmConfiguration;
+import org.apache.geode.test.version.VmConfigurations;
 
-@Category({BackwardCompatibilityTest.class})
+@Category(BackwardCompatibilityTest.class)
 @RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+@UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class OperationManagementUpgradeTest {
-  private final String oldVersion;
-  private final VM vm;
-
-  @Parameterized.Parameters(name = "{0}")
-  public static Collection<String> data() {
-    List<String> result = VersionManager.getInstance().getVersionsWithoutCurrent();
-    result.removeIf(s -> TestVersion.compare(s, "1.13.0") < 0);
-    return result;
+
+  @Parameters(name = "{0}")
+  public static Collection<VmConfiguration> data() {
+    return VmConfigurations.upgrades().stream()
+        .filter(hasGeodeVersion(greaterThan(TestVersion.valueOf("1.13.0"))))
+        .collect(toList());
   }
 
-  public OperationManagementUpgradeTest(String version) {
-    oldVersion = version;
-    oldGfsh = new GfshRule(oldVersion);
-    DUnitLauncher.launchIfNeeded(false);
-    // get the vm with the same version of the oldGfsh
-    vm = VM.getVM(oldVersion, 0);
+  private static final String HOSTNAME = "localhost";
+
+  private final VmConfiguration sourceVmConfiguration;
+
+  private GfshExecutor currentGfsh;
+  private GfshExecutor oldGfsh;
+  private VM vm;
+
+  public OperationManagementUpgradeTest(VmConfiguration sourceVmConfiguration) {
+    this.sourceVmConfiguration = sourceVmConfiguration;
   }
 
-  @Rule
-  public GfshRule oldGfsh;
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+  @Rule(order = 2)
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Before
+  public void setUp() {
+    currentGfsh = gfshRule.executor().build();
+    oldGfsh = gfshRule.executor().withVmConfiguration(sourceVmConfiguration).build();
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
+    // get the vm with the same version of the oldGfsh
+    vm = getVM(sourceVmConfiguration, 0);
+  }
 
   @Test
   public void newLocatorCanReadOldConfigurationData() {
-    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(7);
+    int[] ports = getRandomAvailableTCPPorts(7);
     int locatorPort1 = ports[0];
     int jmxPort1 = ports[1];
     int httpPort1 = ports[2];
@@ -83,19 +102,16 @@ public class OperationManagementUpgradeTest {
     int jmxPort2 = ports[4];
     int httpPort2 = ports[5];
     int serverPort = ports[6];
-    final String hostname = "localhost";
-    GfshExecution execute =
-        GfshScript
-            .of(startLocatorCommand("locator1", hostname, locatorPort1, jmxPort1, httpPort1, 0))
-            .and(startLocatorCommand("locator2", hostname, locatorPort2, jmxPort2, httpPort2,
-                locatorPort1))
-            .and(startServerCommand("server", hostname, serverPort, locatorPort1))
-            .execute(oldGfsh);
+    GfshExecution execute = GfshScript
+        .of(startLocatorCommand("locator1", locatorPort1, jmxPort1, httpPort1, 0))
+        .and(startLocatorCommand("locator2", locatorPort2, jmxPort2, httpPort2, locatorPort1))
+        .and(startServerCommand("server", serverPort, locatorPort1))
+        .execute(oldGfsh);
 
     String operationId = vm.invoke(() -> {
       // start a cms client that connects to locator1's http port
       ClusterManagementService cms = new ClusterManagementServiceBuilder()
-          .setHost(hostname)
+          .setHost(HOSTNAME)
           .setPort(httpPort1)
           .build();
 
@@ -107,15 +123,15 @@ public class OperationManagementUpgradeTest {
     });
 
     // stop locator1
-    oldGfsh.stopLocator(execute, "locator1");
+    execute.locatorStopper().stop("locator1");
     // use new gfsh to start locator1, make sure new locator can start
-    GfshScript.of(startLocatorCommand("locator1", hostname, locatorPort1, jmxPort1, httpPort1,
-        locatorPort2))
-        .execute(gfsh, execute.getWorkingDir());
+    GfshScript
+        .of(startLocatorCommand("locator1", locatorPort1, jmxPort1, httpPort1, locatorPort2))
+        .execute(currentGfsh, execute.getWorkingDir());
 
     // use the new cms client
     ClusterManagementService cms = new ClusterManagementServiceBuilder()
-        .setHost(hostname)
+        .setHost(HOSTNAME)
         .setPort(httpPort1)
         .build();
     ClusterManagementOperationResult<RebalanceOperation, RebalanceResult> operationResult =
@@ -123,4 +139,20 @@ public class OperationManagementUpgradeTest {
     System.out.println(operationResult);
     assertThat(operationResult.getStatusCode()).isEqualTo(ClusterManagementResult.StatusCode.OK);
   }
+
+  private static String startServerCommand(String name, int port, int connectedLocatorPort) {
+    return String.format("start server --name=%s --server-port=%d --locators=%s[%d]",
+        name, port, HOSTNAME, connectedLocatorPort);
+  }
+
+  private static String startLocatorCommand(String name, int port, int jmxPort, int httpPort,
+      int connectedLocatorPort) {
+    String startLocatorCommand =
+        "start locator --name=%s --port=%d --http-service-port=%d --J=-Dgemfire.jmx-manager-port=%d";
+    if (connectedLocatorPort > 0) {
+      return String.format(startLocatorCommand + " --locators=%s[%d]",
+          name, port, httpPort, jmxPort, HOSTNAME, connectedLocatorPort);
+    }
+    return String.format(startLocatorCommand, name, port, httpPort, jmxPort);
+  }
 }
diff --git a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/RollingUpgradeWithGfshDUnitTest.java b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/RollingUpgradeWithGfshDUnitTest.java
index 3c2f91b06a..f58819291c 100644
--- a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/RollingUpgradeWithGfshDUnitTest.java
+++ b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/RollingUpgradeWithGfshDUnitTest.java
@@ -14,124 +14,154 @@
  */
 package org.apache.geode.management;
 
-import static org.apache.geode.test.junit.rules.gfsh.GfshRule.startLocatorCommand;
-import static org.apache.geode.test.junit.rules.gfsh.GfshRule.startServerCommand;
+import static java.util.stream.Collectors.toList;
+import static org.apache.geode.test.version.TestVersions.atLeast;
+import static org.apache.geode.test.version.VmConfigurations.hasGeodeVersion;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.io.IOException;
 import java.util.Collection;
-import java.util.List;
 
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.junit.runners.Parameterized.UseParametersRunnerFactory;
 
 import org.apache.geode.internal.UniquePortSupplier;
 import org.apache.geode.test.compiler.ClassBuilder;
 import org.apache.geode.test.junit.categories.BackwardCompatibilityTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecutor;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.apache.geode.test.version.TestVersion;
-import org.apache.geode.test.version.VersionManager;
+import org.apache.geode.test.version.VmConfiguration;
+import org.apache.geode.test.version.VmConfigurations;
 
 /**
  * This test iterates through the versions of Geode and executes client compatibility with
  * the current version of Geode.
  */
-@Category({BackwardCompatibilityTest.class})
+@Category(BackwardCompatibilityTest.class)
 @RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+@UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class RollingUpgradeWithGfshDUnitTest {
-  private final UniquePortSupplier portSupplier = new UniquePortSupplier();
-  private final String oldVersion;
 
-  @Parameterized.Parameters(name = "{0}")
-  public static Collection<String> data() {
-    List<String> result = VersionManager.getInstance().getVersionsWithoutCurrent();
-    result.removeIf(s -> TestVersion.compare(s, "1.10.0") < 0);
-    return result;
+  @Parameters(name = "{0}")
+  public static Collection<VmConfiguration> data() {
+    return VmConfigurations.upgrades().stream()
+        .filter(hasGeodeVersion(atLeast(TestVersion.valueOf("1.10.0"))))
+        .collect(toList());
   }
 
-  @Rule
-  public GfshRule oldGfsh;
+  private static final String HOSTNAME = "localhost";
+
+  private final UniquePortSupplier portSupplier = new UniquePortSupplier();
+
+  private final VmConfiguration sourceVmConfiguration;
 
-  @Rule
-  public GfshRule currentGfsh = new GfshRule();
+  private GfshExecutor currentGfsh;
+  private GfshExecutor oldGfsh;
 
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  public RollingUpgradeWithGfshDUnitTest(String version) {
-    oldVersion = version;
-    oldGfsh = new GfshRule(oldVersion);
+  public RollingUpgradeWithGfshDUnitTest(VmConfiguration sourceVmConfiguration) {
+    this.sourceVmConfiguration = sourceVmConfiguration;
+  }
+
+  @Before
+  public void setUp() {
+    currentGfsh = gfshRule.executor().build();
+    oldGfsh = gfshRule.executor().withVmConfiguration(sourceVmConfiguration).build();
   }
 
   @Test
-  public void testRollingUpgradeWithDeployment() throws Exception {
+  public void testRollingUpgradeWithDeployment() throws IOException {
     int locatorPort = portSupplier.getAvailablePort();
     int locatorJmxPort = portSupplier.getAvailablePort();
     int locator2Port = portSupplier.getAvailablePort();
     int locator2JmxPort = portSupplier.getAvailablePort();
     int server1Port = portSupplier.getAvailablePort();
     int server2Port = portSupplier.getAvailablePort();
-    final String hostname = "localhost";
-
-    GfshExecution startupExecution =
-        GfshScript.of(startLocatorCommand("loc1", hostname, locatorPort, locatorJmxPort, 0,
-            -1))
-            .and(startLocatorCommand("loc2", hostname, locator2Port, locator2JmxPort, 0,
-                locatorPort))
-            .and(startServerCommand("server1", hostname, server1Port, locatorPort))
-            .and(startServerCommand("server2", hostname, server2Port, locatorPort))
-            .and(deployDirCommand())
-            .execute(oldGfsh);
+
+    GfshExecution startupExecution = GfshScript
+        .of(startLocatorCommand("loc1", locatorPort, locatorJmxPort, 0, -1))
+        .and(startLocatorCommand("loc2", locator2Port, locator2JmxPort, 0, locatorPort))
+        .and(startServerCommand("server1", server1Port, locatorPort))
+        .and(startServerCommand("server2", server2Port, locatorPort))
+        .and(deployDirCommand())
+        .execute(oldGfsh);
 
     // doing rolling upgrades
-    oldGfsh.stopLocator(startupExecution, "loc1");
+    startupExecution.locatorStopper().stop("loc1");
     GfshScript
-        .of(startLocatorCommand("loc1", hostname, locatorPort, locatorJmxPort, 0,
-            locator2Port))
+        .of(startLocatorCommand("loc1", locatorPort, locatorJmxPort, 0, locator2Port))
         .execute(currentGfsh);
     verifyListDeployed(locatorPort);
 
-    oldGfsh.stopLocator(startupExecution, "loc2");
+    startupExecution.locatorStopper().stop("loc2");
     GfshScript
-        .of(startLocatorCommand("loc2", hostname, locator2Port, locator2JmxPort, 0,
-            locatorPort))
+        .of(startLocatorCommand("loc2", locator2Port, locator2JmxPort, 0, locatorPort))
         .execute(currentGfsh);
     verifyListDeployed(locator2Port);
 
     // make sure servers can do rolling upgrade too
-    oldGfsh.stopServer(startupExecution, "server1");
-    GfshScript.of(startServerCommand("server1", hostname, server1Port, locatorPort))
+    startupExecution.serverStopper().stop("server1");
+    GfshScript
+        .of(startServerCommand("server1", server1Port, locatorPort))
         .execute(currentGfsh);
 
-    oldGfsh.stopServer(startupExecution, "server2");
-    GfshScript.of(startServerCommand("server2", hostname, server2Port, locatorPort))
+    startupExecution.serverStopper().stop("server2");
+    GfshScript
+        .of(startServerCommand("server2", server2Port, locatorPort))
         .execute(currentGfsh);
   }
 
   private void verifyListDeployed(int locatorPort) {
-    GfshExecution list_deployed = GfshScript.of("connect --locator=localhost[" + locatorPort + "]")
-        .and("list deployed").execute(currentGfsh);
-    assertThat(list_deployed.getOutputText()).contains("DeployCommandsDUnit1.jar")
-        .contains("server1").contains("server2");
+    GfshExecution list_deployed = GfshScript
+        .of("connect --locator=" + HOSTNAME + "[" + locatorPort + "]")
+        .and("list deployed")
+        .execute(currentGfsh);
+    assertThat(list_deployed.getOutputText())
+        .contains("DeployCommandsDUnit1.jar")
+        .contains("server1")
+        .contains("server2");
     currentGfsh.execute("disconnect");
   }
 
   private String deployDirCommand() throws IOException {
     ClassBuilder classBuilder = new ClassBuilder();
-    File jarsDir = tempFolder.newFolder();
+    File jarsDir = folderRule.getFolder().toPath().toFile();
     String jarName1 = "DeployCommandsDUnit1.jar";
     File jar1 = new File(jarsDir, jarName1);
     String class1 = "DeployCommandsDUnitA";
     classBuilder.writeJarFromName(class1, jar1);
     return "deploy --dir=" + jarsDir.getAbsolutePath();
   }
+
+  private static String startServerCommand(String name, int port, int connectedLocatorPort) {
+    return String.format("start server --name=%s --server-port=%d --locators=%s[%d]",
+        name, port, HOSTNAME, connectedLocatorPort);
+  }
+
+  private static String startLocatorCommand(String name, int port, int jmxPort, int httpPort,
+      int connectedLocatorPort) {
+    String startLocatorCommand =
+        "start locator --name=%s --port=%d --http-service-port=%d --J=-Dgemfire.jmx-manager-port=%d";
+    if (connectedLocatorPort > 0) {
+      return String.format(startLocatorCommand + " --locators=%s[%d]",
+          name, port, httpPort, jmxPort, HOSTNAME, connectedLocatorPort);
+    }
+    return String.format(startLocatorCommand, name, port, httpPort, jmxPort);
+  }
 }
diff --git a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/RollingUpgradeWithSslDUnitTest.java b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/RollingUpgradeWithSslDUnitTest.java
index 17c85b551b..5329927437 100644
--- a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/RollingUpgradeWithSslDUnitTest.java
+++ b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/RollingUpgradeWithSslDUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.management;
 
+import static java.util.stream.Collectors.toList;
 import static org.apache.geode.distributed.ConfigurationProperties.BIND_ADDRESS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENABLED_COMPONENTS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENDPOINT_IDENTIFICATION_ENABLED;
@@ -24,80 +25,91 @@ import static org.apache.geode.distributed.ConfigurationProperties.SSL_REQUIRE_A
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_PASSWORD;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_TYPE;
-import static org.apache.geode.test.junit.rules.gfsh.GfshRule.startLocatorCommand;
-import static org.apache.geode.test.junit.rules.gfsh.GfshRule.startServerCommand;
+import static org.apache.geode.test.version.TestVersions.atLeast;
+import static org.apache.geode.test.version.VmConfigurations.hasGeodeVersion;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.net.InetAddress;
-import java.net.UnknownHostException;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.security.GeneralSecurityException;
 import java.util.Collection;
-import java.util.List;
 import java.util.Properties;
 
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.junit.runners.Parameterized.UseParametersRunnerFactory;
 
 import org.apache.geode.cache.ssl.CertStores;
 import org.apache.geode.cache.ssl.CertificateBuilder;
 import org.apache.geode.cache.ssl.CertificateMaterial;
 import org.apache.geode.internal.UniquePortSupplier;
 import org.apache.geode.test.junit.categories.BackwardCompatibilityTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecutor;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.apache.geode.test.version.TestVersion;
-import org.apache.geode.test.version.VersionManager;
+import org.apache.geode.test.version.VmConfiguration;
+import org.apache.geode.test.version.VmConfigurations;
 
 /**
  * This test iterates through the versions of Geode and executes client compatibility with
  * the current version of Geode.
  */
-@Category({BackwardCompatibilityTest.class})
+@Category(BackwardCompatibilityTest.class)
 @RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+@UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class RollingUpgradeWithSslDUnitTest {
-  private final UniquePortSupplier portSupplier = new UniquePortSupplier();
-  private final String hostName;
-  private final String keyStoreFileName;
-  private final String trustStoreFileName;
-  private File securityPropertiesFile;
 
-  @Parameterized.Parameters(name = "{0}")
-  public static Collection<String> data() {
-    final List<String> result = VersionManager.getInstance().getVersionsWithoutCurrent();
-    result.removeIf(s -> TestVersion.compare(s, "1.10.0") < 0);
-    return result;
+  @Parameters(name = "{0}")
+  public static Collection<VmConfiguration> data() {
+    return VmConfigurations.upgrades().stream()
+        .filter(hasGeodeVersion(atLeast(TestVersion.valueOf("1.10.0"))))
+        .collect(toList());
   }
 
-  @Rule
-  public GfshRule oldGfsh;
+  private final UniquePortSupplier portSupplier = new UniquePortSupplier();
+
+  private final VmConfiguration sourceVmConfiguration;
 
-  @Rule
-  public GfshRule currentGfsh;
+  private String hostName;
+  private String keyStoreFileName;
+  private String trustStoreFileName;
+  private File securityPropertiesFile;
+  private GfshExecutor oldGfsh;
+  private GfshExecutor currentGfsh;
+  private Path tempFolder;
+
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder();
+  public RollingUpgradeWithSslDUnitTest(VmConfiguration vmConfiguration) {
+    sourceVmConfiguration = vmConfiguration;
+  }
 
-  public RollingUpgradeWithSslDUnitTest(String version) throws UnknownHostException {
-    oldGfsh = new GfshRule(version);
-    currentGfsh = new GfshRule();
+  @Before
+  public void setUp() throws IOException, GeneralSecurityException {
+    oldGfsh = gfshRule.executor().withVmConfiguration(sourceVmConfiguration).build();
+    currentGfsh = gfshRule.executor().build();
     hostName = InetAddress.getLocalHost().getCanonicalHostName();
     keyStoreFileName = hostName + "-keystore.jks";
     trustStoreFileName = hostName + "-truststore.jks";
-  }
 
-  @Before
-  public void before() throws IOException, GeneralSecurityException {
+    tempFolder = folderRule.getFolder().toPath();
+
     generateStores();
     /*
      * We must use absolute paths for truststore and keystore in properties file and
@@ -107,14 +119,15 @@ public class RollingUpgradeWithSslDUnitTest {
      */
     final Properties properties = generateSslProperties();
 
-    securityPropertiesFile = tempFolder.newFile("gfsecurity.properties");
+    securityPropertiesFile = tempFolder.resolve("gfsecurity.properties").toFile();
+    Files.createFile(securityPropertiesFile.toPath());
     final FileOutputStream fileOutputStream =
         new FileOutputStream(securityPropertiesFile.getAbsolutePath());
     properties.store(fileOutputStream, "");
   }
 
   @Test
-  public void testRollingUpgradeWithDeployment() throws Exception {
+  public void testRollingUpgradeWithDeployment() {
     final int locatorPort = portSupplier.getAvailablePort();
     final int locatorJmxPort = portSupplier.getAvailablePort();
     final int locator2Port = portSupplier.getAvailablePort();
@@ -129,7 +142,7 @@ public class RollingUpgradeWithSslDUnitTest {
                 locatorPort))
             .and(startServerCommandWithConfig("server1", server1Port, locatorPort))
             .and(startServerCommandWithConfig("server2", server2Port, locatorPort))
-            .execute(oldGfsh, tempFolder.getRoot());
+            .execute(oldGfsh, tempFolder);
 
     initializeRegion(locatorPort);
     causeP2PTraffic(locatorPort);
@@ -152,19 +165,18 @@ public class RollingUpgradeWithSslDUnitTest {
   }
 
   private void upgradeLocator(String name, int locatorPort, int locatorJmxPort,
-      int connectedLocatorPort,
-      GfshExecution startupExecution) {
-    oldGfsh.stopLocator(startupExecution, name);
+      int connectedLocatorPort, GfshExecution startupExecution) {
+    startupExecution.locatorStopper().stop(name);
     GfshScript
         .of(startLocatorCommandWithConfig(name, locatorPort, locatorJmxPort, connectedLocatorPort))
-        .execute(currentGfsh, tempFolder.getRoot());
+        .execute(currentGfsh, tempFolder);
   }
 
   private void upgradeServer(String name, int serverPort, int locatorPort,
       GfshExecution startupExecution) {
-    oldGfsh.stopServer(startupExecution, name);
+    startupExecution.serverStopper().stop(name);
     GfshScript.of(startServerCommandWithConfig(name, serverPort, locatorPort))
-        .execute(currentGfsh, tempFolder.getRoot());
+        .execute(currentGfsh, tempFolder);
   }
 
   private Properties generateSslProperties() {
@@ -175,11 +187,11 @@ public class RollingUpgradeWithSslDUnitTest {
     properties.setProperty(SSL_ENABLED_COMPONENTS, "cluster,server");
     properties.setProperty(SSL_ENDPOINT_IDENTIFICATION_ENABLED, "true");
 
-    properties.setProperty(SSL_KEYSTORE, tempFolder.getRoot() + "/" + keyStoreFileName);
+    properties.setProperty(SSL_KEYSTORE, tempFolder + "/" + keyStoreFileName);
     properties.setProperty(SSL_KEYSTORE_TYPE, "jks");
     properties.setProperty(SSL_KEYSTORE_PASSWORD, "geode");
 
-    properties.setProperty(SSL_TRUSTSTORE, tempFolder.getRoot() + "/" + trustStoreFileName);
+    properties.setProperty(SSL_TRUSTSTORE, tempFolder + "/" + trustStoreFileName);
     properties.setProperty(SSL_TRUSTSTORE_TYPE, "jks");
     properties.setProperty(SSL_TRUSTSTORE_PASSWORD, "geode");
 
@@ -190,7 +202,7 @@ public class RollingUpgradeWithSslDUnitTest {
     final GfshExecution members =
         GfshScript.of("connect --locator=" + hostName + "[" + locatorPort + "]")
             .and("list members")
-            .execute(currentGfsh, tempFolder.getRoot());
+            .execute(currentGfsh, tempFolder);
 
     assertThat(members.getOutputText())
         .contains("locator1")
@@ -198,7 +210,7 @@ public class RollingUpgradeWithSslDUnitTest {
         .contains("server1")
         .contains("server2");
 
-    GfshScript.of("disconnect").execute(currentGfsh, tempFolder.getRoot());
+    GfshScript.of("disconnect").execute(currentGfsh, tempFolder);
   }
 
   private String startServerCommandWithConfig(String server, int serverPort, int locatorPort) {
@@ -206,11 +218,9 @@ public class RollingUpgradeWithSslDUnitTest {
   }
 
   private String startLocatorCommandWithConfig(String name, final int locatorPort,
-      final int locatorJmxPort,
-      final int connectedLocatorPort) {
+      final int locatorJmxPort, final int connectedLocatorPort) {
     return startLocatorCommand(name, hostName, locatorPort, locatorJmxPort, 0, connectedLocatorPort)
-        +
-        additionalParameters();
+        + additionalParameters();
   }
 
   private String additionalParameters() {
@@ -227,11 +237,11 @@ public class RollingUpgradeWithSslDUnitTest {
         GfshScript.of("connect --locator=" + hostName + "[" + locatorPort + "]")
             .and("create region --name=region1 --type=REPLICATE")
             .and("list regions")
-            .execute(currentGfsh, tempFolder.getRoot());
+            .execute(currentGfsh, tempFolder);
 
     assertThat(getResponse.getOutputText()).contains("region1");
 
-    GfshScript.of("disconnect").execute(currentGfsh, tempFolder.getRoot());
+    GfshScript.of("disconnect").execute(currentGfsh, tempFolder);
   }
 
   private void causeP2PTraffic(int locatorPort) {
@@ -239,14 +249,14 @@ public class RollingUpgradeWithSslDUnitTest {
         GfshScript.of("connect --locator=" + hostName + "[" + locatorPort + "]")
             .and("put --key='123abc' --value='Hello World!!' --region=region1")
             .and("get --key='123abc' --region=region1")
-            .execute(currentGfsh, tempFolder.getRoot());
+            .execute(currentGfsh, tempFolder);
 
     assertThat(getResponse.getOutputText()).contains("Hello World!!");
 
-    GfshScript.of("disconnect").execute(currentGfsh, tempFolder.getRoot());
+    GfshScript.of("disconnect").execute(currentGfsh, tempFolder);
   }
 
-  public void generateStores() throws IOException, GeneralSecurityException {
+  private void generateStores() throws IOException, GeneralSecurityException {
     final String algorithm = "SHA256withRSA";
     final CertificateMaterial ca = new CertificateBuilder(365, algorithm)
         .commonName("Test CA")
@@ -263,14 +273,33 @@ public class RollingUpgradeWithSslDUnitTest {
     store.withCertificate("geode", certificate);
     store.trust("ca", ca);
 
-    final File keyStoreFile = new File(tempFolder.getRoot(), keyStoreFileName);
+    final File keyStoreFile = new File(tempFolder.toFile(), keyStoreFileName);
     keyStoreFile.createNewFile();
     store.createKeyStore(keyStoreFile.getAbsolutePath(), "geode");
     System.out.println("Keystore created: " + keyStoreFile.getAbsolutePath());
 
-    final File trustStoreFile = new File(tempFolder.getRoot(), trustStoreFileName);
+    final File trustStoreFile = new File(tempFolder.toFile(), trustStoreFileName);
     trustStoreFile.createNewFile();
     store.createTrustStore(trustStoreFile.getPath(), "geode");
     System.out.println("Truststore created: " + trustStoreFile.getAbsolutePath());
   }
+
+  private static String startServerCommand(String name, String hostname, int port,
+      int connectedLocatorPort) {
+    return "start server --name=" + name
+        + " --server-port=" + port
+        + " --locators=" + hostname + "[" + connectedLocatorPort + "]";
+  }
+
+  private static String startLocatorCommand(String name, String hostname, int port, int jmxPort,
+      int httpPort, int connectedLocatorPort) {
+    String command = "start locator --name=" + name
+        + " --port=" + port
+        + " --http-service-port=" + httpPort;
+    if (connectedLocatorPort > 0) {
+      command += " --locators=" + hostname + "[" + connectedLocatorPort + "]";
+    }
+    command += " --J=-Dgemfire.jmx-manager-port=" + jmxPort;
+    return command;
+  }
 }
diff --git a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/internal/cli/commands/ConnectCommandUpgradeTest.java b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/internal/cli/commands/ConnectCommandUpgradeTest.java
index d99b86aaca..e4f606d585 100644
--- a/geode-assembly/src/upgradeTest/java/org/apache/geode/management/internal/cli/commands/ConnectCommandUpgradeTest.java
+++ b/geode-assembly/src/upgradeTest/java/org/apache/geode/management/internal/cli/commands/ConnectCommandUpgradeTest.java
@@ -16,76 +16,96 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import static java.util.stream.Collectors.toList;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.test.version.TestVersions.greaterThan;
+import static org.apache.geode.test.version.VmConfigurations.hasGeodeVersion;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assume.assumeFalse;
+import static org.assertj.core.api.Assumptions.assumeThat;
 
 import java.util.Collection;
-import java.util.List;
 
 import org.apache.commons.lang3.JavaVersion;
-import org.apache.commons.lang3.SystemUtils;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
-import org.apache.geode.test.dunit.rules.ClusterStartupRule;
-import org.apache.geode.test.dunit.rules.MemberVM;
 import org.apache.geode.test.junit.categories.GfshTest;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshExecution;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecutor;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 import org.apache.geode.test.version.TestVersion;
-import org.apache.geode.test.version.VersionManager;
+import org.apache.geode.test.version.VmConfiguration;
+import org.apache.geode.test.version.VmConfigurations;
 
 @Category(GfshTest.class)
 @RunWith(Parameterized.class)
 public class ConnectCommandUpgradeTest {
 
-  private String oldVersion;
-
-  public ConnectCommandUpgradeTest(String oldVersion) {
-    this.oldVersion = oldVersion;
+  @Parameters(name = "Locator: {0}")
+  public static Collection<VmConfiguration> data() {
+    return VmConfigurations.upgrades().stream()
+        .filter(hasGeodeVersion(greaterThan(TestVersion.valueOf("1.7.0"))))
+        .collect(toList());
   }
 
-  @Parameterized.Parameters(name = "Locator Version: {0}")
-  public static Collection<String> data() {
-    List<String> result = VersionManager.getInstance().getVersionsWithoutCurrent();
-    return result;
+  private final VmConfiguration sourceVmConfiguration;
+
+  private GfshExecutor oldGfsh;
+  private GfshExecutor currentGfsh;
+
+  public ConnectCommandUpgradeTest(VmConfiguration vmConfiguration) {
+    sourceVmConfiguration = vmConfiguration;
   }
 
-  @Rule
-  public ClusterStartupRule clusterStartupRule = new ClusterStartupRule();
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  @Rule
-  public GfshRule gfshDefault = new GfshRule();
+  @Before
+  public void setUp() {
+    currentGfsh = gfshRule.executor().build();
+    oldGfsh = gfshRule.executor().withVmConfiguration(sourceVmConfiguration).build();
+  }
 
   @Test
   public void useCurrentGfshToConnectToOlderLocator() {
-    assumeFalse(
-        "this test can only be run with pre-9 jdk since it needs to run older version of gfsh",
-        SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9));
+    assumeThat(JavaVersion.JAVA_RECENT).isLessThanOrEqualTo(JavaVersion.JAVA_1_8);
 
-    MemberVM oldVersionLocator = clusterStartupRule.startLocatorVM(0, oldVersion);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    int locatorPort = ports[0];
+    int jmxPort = ports[1];
 
-    if (TestVersion.compare(oldVersion, "1.10.0") < 0) { // New version gfsh could not connect to
-                                                         // locators with version below 1.10.0
+    GfshScript
+        .of(startLocatorCommand("test", locatorPort, jmxPort))
+        .execute(oldGfsh);
+
+    // New version gfsh could not connect to locators with version below 1.10.0
+    if (sourceVmConfiguration.geodeVersion().lessThan(TestVersion.valueOf("1.10.0"))) {
       GfshExecution connect = GfshScript
-          .of("connect --locator=localhost[" + oldVersionLocator.getPort() + "]")
+          .of("connect --locator=localhost[" + locatorPort + "]")
           .expectFailure()
-          .execute(gfshDefault);
+          .execute(currentGfsh);
 
       assertThat(connect.getOutputText())
           .contains("Cannot use a")
           .contains("gfsh client to connect to")
           .contains("cluster.");
+    }
 
-    } else { // From 1.10.0 new version gfsh are able to connect to old version locators
+    // From 1.10.0 new version gfsh are able to connect to old version locators
+    else {
       GfshExecution connect = GfshScript
-          .of("connect --locator=localhost[" + oldVersionLocator.getPort() + "]")
+          .of("connect --locator=localhost[" + locatorPort + "]")
           .expectExitCode(0)
-          .execute(gfshDefault);
+          .execute(currentGfsh);
 
       assertThat(connect.getOutputText())
           .contains("Successfully connected to:");
@@ -94,16 +114,27 @@ public class ConnectCommandUpgradeTest {
 
   @Test
   public void invalidHostname() {
-    MemberVM oldVersionLocator = clusterStartupRule.startLocatorVM(0, oldVersion);
+    int[] ports = getRandomAvailableTCPPorts(2);
+    int locatorPort = ports[0];
+    int jmxPort = ports[1];
+    GfshScript
+        .of(startLocatorCommand("test", locatorPort, jmxPort))
+        .execute(oldGfsh);
 
     GfshExecution connect = GfshScript
         .of("connect --locator=\"invalid host name[52326]\"")
         .expectFailure()
-        .execute(gfshDefault);
+        .execute(currentGfsh);
 
     assertThat(connect.getOutputText())
         .doesNotContain("UnknownHostException")
         .doesNotContain("nodename nor servname")
         .contains("can't be reached. Hostname or IP address could not be found.");
   }
+
+  private static String startLocatorCommand(String name, int port, int jmxPort) {
+    String startLocatorCommand =
+        "start locator --name=%s --port=%d --http-service-port=%d --J=-Dgemfire.jmx-manager-port=%d";
+    return String.format(startLocatorCommand, name, port, 0, jmxPort);
+  }
 }
diff --git a/geode-assembly/src/upgradeTest/java/org/apache/geode/session/tests/Tomcat8ClientServerRollingUpgradeTest.java b/geode-assembly/src/upgradeTest/java/org/apache/geode/session/tests/Tomcat8ClientServerRollingUpgradeTest.java
index 1c2b8797d0..593cc97fa9 100644
--- a/geode-assembly/src/upgradeTest/java/org/apache/geode/session/tests/Tomcat8ClientServerRollingUpgradeTest.java
+++ b/geode-assembly/src/upgradeTest/java/org/apache/geode/session/tests/Tomcat8ClientServerRollingUpgradeTest.java
@@ -14,109 +14,120 @@
  */
 package org.apache.geode.session.tests;
 
+import static java.util.stream.Collectors.joining;
+import static java.util.stream.Collectors.toList;
+import static org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast;
+import static org.apache.geode.internal.GemFireVersion.getGemFireVersion;
+import static org.apache.geode.test.version.TestVersions.atLeast;
+import static org.apache.geode.test.version.VmConfigurations.hasGeodeVersion;
 import static org.junit.Assert.assertEquals;
 
 import java.io.File;
 import java.io.IOException;
 import java.net.URISyntaxException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.Collection;
-import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Stream;
 
 import org.apache.commons.lang3.JavaVersion;
-import org.apache.commons.lang3.SystemUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.junit.runners.Parameterized.UseParametersRunnerFactory;
 
 import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.UniquePortSupplier;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.management.internal.i18n.CliStrings;
 import org.apache.geode.test.junit.categories.BackwardCompatibilityTest;
+import org.apache.geode.test.junit.rules.FolderRule;
+import org.apache.geode.test.junit.rules.RequiresGeodeHome;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecutor;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshScript;
 import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.apache.geode.test.version.TestVersion;
 import org.apache.geode.test.version.VersionManager;
+import org.apache.geode.test.version.VmConfiguration;
+import org.apache.geode.test.version.VmConfigurations;
 
 /**
  * This test iterates through the versions of Geode and executes session client compatibility with
  * the current version of Geode.
  */
-@Category({BackwardCompatibilityTest.class})
+@Category(BackwardCompatibilityTest.class)
 @RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+@UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class Tomcat8ClientServerRollingUpgradeTest {
-  private final UniquePortSupplier portSupplier = new UniquePortSupplier();
-  private final String oldVersion;
-  private String locatorDir;
-  private String server1Dir;
-  private String server2Dir;
-
-  @Parameterized.Parameters(name = "{0}")
-  public static Collection<String> data() {
-    List<String> result = VersionManager.getInstance().getVersionsWithoutCurrent();
-    String minimumVersion =
-        SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9) ? "1.8.0" : "1.7.0";
-    result.removeIf(s -> TestVersion.compare(s, minimumVersion) < 0);
-    return result;
-  }
 
-  @Rule
-  public transient GfshRule oldGfsh;
-
-  @Rule
-  public final transient GfshRule currentGfsh = new GfshRule();
-
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder();
+  @Parameters(name = "{0}")
+  public static Collection<VmConfiguration> data() {
+    String minimumVersion = isJavaVersionAtLeast(JavaVersion.JAVA_9) ? "1.8.0" : "1.7.0";
+    return VmConfigurations.upgrades().stream()
+        .filter(hasGeodeVersion(atLeast(TestVersion.valueOf(minimumVersion))))
+        .collect(toList());
+  }
 
-  @Rule
-  public transient TestName testName = new TestName();
+  private final UniquePortSupplier portSupplier = new UniquePortSupplier();
+  private final VmConfiguration sourceVmConfiguration;
+  private Path locatorDir;
+  private Path server1Dir;
+  private Path server2Dir;
+  private GfshExecutor oldGfsh;
 
-  protected transient Client client;
-  protected transient ContainerManager manager;
+  private GfshExecutor currentGfsh;
 
+  protected Client client;
+  protected ContainerManager manager;
 
-  protected TomcatInstall tomcat8AndOldModules;
-  protected TomcatInstall tomcat8AndCurrentModules;
+  private TomcatInstall tomcat8AndOldModules;
+  private TomcatInstall tomcat8AndCurrentModules;
 
-  protected int locatorPort;
-  protected int locatorJmxPort;
+  private int locatorPort;
+  private int locatorJmxPort;
 
-  protected String classPathTomcat8AndCurrentModules;
+  private String classPathTomcat8AndCurrentModules;
   private String classPathTomcat8AndOldModules;
 
-  public Tomcat8ClientServerRollingUpgradeTest(String version) {
-    oldVersion = version;
-    oldGfsh = new GfshRule(oldVersion);
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+  @Rule
+  public TestName testName = new TestName();
+
+  public Tomcat8ClientServerRollingUpgradeTest(VmConfiguration vmConfiguration) {
+    sourceVmConfiguration = vmConfiguration;
   }
 
-  protected void startServer(String name, String classPath, int locatorPort, GfshRule gfsh,
-      String serverDir) throws Exception {
+  private void startServer(String name, String classPath, int locatorPort, GfshExecutor gfsh,
+      Path serverDir) {
     CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_SERVER);
     command.addOption(CliStrings.START_SERVER__NAME, name);
     command.addOption(CliStrings.START_SERVER__SERVER_PORT, "0");
     command.addOption(CliStrings.START_SERVER__CLASSPATH, classPath);
     command.addOption(CliStrings.START_SERVER__LOCATORS, "localhost[" + locatorPort + "]");
-    command.addOption(CliStrings.START_SERVER__DIR, serverDir);
+    command.addOption(CliStrings.START_SERVER__DIR, serverDir.toString());
+
     gfsh.execute(GfshScript.of(command.toString()).expectExitCode(0));
   }
 
-  protected void startLocator(String name, String classPath, int port, GfshRule gfsh,
-      String locatorDir) throws Exception {
+  private void startLocator(String name, String classPath, int port, GfshExecutor gfsh,
+      Path locatorDir) {
     CommandStringBuilder locStarter = new CommandStringBuilder(CliStrings.START_LOCATOR);
     locStarter.addOption(CliStrings.START_LOCATOR__MEMBER_NAME, name);
     locStarter.addOption(CliStrings.START_LOCATOR__CLASSPATH, classPath);
     locStarter.addOption(CliStrings.START_LOCATOR__PORT, Integer.toString(port));
-    locStarter.addOption(CliStrings.START_LOCATOR__DIR, locatorDir);
+    locStarter.addOption(CliStrings.START_LOCATOR__DIR, locatorDir.toString());
     locStarter.addOption(CliStrings.START_LOCATOR__HTTP_SERVICE_PORT, "0");
     locStarter.addOption(CliStrings.START_LOCATOR__J,
         "-Dgemfire.jmx-manager-port=" + locatorJmxPort);
@@ -125,21 +136,32 @@ public class Tomcat8ClientServerRollingUpgradeTest {
 
   @Before
   public void setup() throws Exception {
-    VersionManager versionManager = VersionManager.getInstance();
-    String installLocation = versionManager.getInstall(oldVersion);
-    File oldBuild = new File(installLocation);
-    File oldModules = new File(installLocation + "/tools/Modules/");
+    currentGfsh = gfshRule.executor().build();
+    oldGfsh = gfshRule.executor().withVmConfiguration(sourceVmConfiguration).build();
+
+    String version = sourceVmConfiguration.geodeVersion().toString();
+    Path installLocation;
+    if (version == null || VersionManager.isCurrentVersion(version)) {
+      installLocation = new RequiresGeodeHome().getGeodeHome().toPath();
+    } else {
+      installLocation = Paths.get(VersionManager.getInstance().getInstall(version));
+    }
+
+    File oldBuild = installLocation.toFile();
+    File oldModules = installLocation.resolve("tools").resolve("Modules").toFile();
 
+    Path tempFolder = folderRule.getFolder().toPath();
 
     tomcat8AndOldModules =
-        new TomcatInstall("Tomcat8AndOldModules", TomcatInstall.TomcatVersion.TOMCAT8,
+        new TomcatInstall(tempFolder, "Tomcat8AndOldModules", TomcatInstall.TomcatVersion.TOMCAT8,
             ContainerInstall.ConnectionType.CLIENT_SERVER,
             oldModules.getAbsolutePath(),
             oldBuild.getAbsolutePath() + "/lib",
             portSupplier::getAvailablePort, TomcatInstall.CommitValve.DEFAULT);
 
     tomcat8AndCurrentModules =
-        new TomcatInstall("Tomcat8AndCurrentModules", TomcatInstall.TomcatVersion.TOMCAT8,
+        new TomcatInstall(tempFolder, "Tomcat8AndCurrentModules",
+            TomcatInstall.TomcatVersion.TOMCAT8,
             ContainerInstall.ConnectionType.CLIENT_SERVER,
             portSupplier::getAvailablePort, TomcatInstall.CommitValve.DEFAULT);
 
@@ -158,42 +180,26 @@ public class Tomcat8ClientServerRollingUpgradeTest {
     manager = new ContainerManager();
     // Due to parameterization of the test name, the URI would be malformed. Instead, it strips off
     // the [] symbols
-    manager.setTestName(testName.getMethodName().replace("[", "").replace("]", ""));
+    manager.setTestName(testName.getMethodName().replaceAll("[\\[\\] ,]+", ""));
 
-    locatorDir = tempFolder.newFolder("loc").getPath();
-    server1Dir = tempFolder.newFolder("server1").getPath();
-    server2Dir = tempFolder.newFolder("server2").getPath();
+    locatorDir = tempFolder.resolve("loc");
+    server1Dir = tempFolder.resolve("server1");
+    server2Dir = tempFolder.resolve("server2");
   }
 
   /**
    * Stops all containers that were previously started and cleans up their configurations
    */
   @After
-  public void stop() throws Exception {
+  public void stop() throws IOException {
     manager.stopAllActiveContainers();
     manager.cleanUp();
-
-    CommandStringBuilder connect = new CommandStringBuilder(CliStrings.CONNECT)
-        .addOption(CliStrings.CONNECT__LOCATOR, "localhost[" + locatorPort + "]");
-
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.SHUTDOWN);
-    command.addOption(CliStrings.INCLUDE_LOCATORS, "true");
-    final GfshScript script = GfshScript.of(connect.toString(), command.toString());
-    try {
-      oldGfsh.execute(script);
-    } catch (Throwable e) {
-      // ignore
-    }
-
-    try {
-      currentGfsh.execute(script);
-    } catch (Throwable e) {
-      // ignore
-    }
   }
 
   @Test
-  public void canDoARollingUpgradeOfGeodeServersWithSessionModules() throws Exception {
+  public void canDoARollingUpgradeOfGeodeServersWithSessionModules()
+      throws IOException, ExecutionException, InterruptedException, TimeoutException,
+      URISyntaxException {
 
     startLocator("loc", classPathTomcat8AndOldModules, locatorPort, oldGfsh, locatorDir);
     startServer("server1", classPathTomcat8AndOldModules, locatorPort, oldGfsh, server1Dir);
@@ -244,7 +250,7 @@ public class Tomcat8ClientServerRollingUpgradeTest {
     verifySessionReplication();
   }
 
-  private void createRegion(GfshRule gfsh) {
+  private void createRegion(GfshExecutor gfsh) {
     CommandStringBuilder connect = new CommandStringBuilder(CliStrings.CONNECT)
         .addOption(CliStrings.CONNECT__LOCATOR, "localhost[" + locatorPort + "]");
 
@@ -260,15 +266,15 @@ public class Tomcat8ClientServerRollingUpgradeTest {
     gfsh.execute(script);
   }
 
-  private void stopLocator(GfshRule gfsh, String locatorDir) {
+  private void stopLocator(GfshExecutor gfsh, Path locatorDir) {
     CommandStringBuilder command = new CommandStringBuilder(CliStrings.STOP_LOCATOR)
-        .addOption(CliStrings.STOP_LOCATOR__DIR, locatorDir);
+        .addOption(CliStrings.STOP_LOCATOR__DIR, locatorDir.toString());
     gfsh.execute(command.toString());
   }
 
-  private void stopServer(GfshRule gfsh, String serverDir) {
+  private void stopServer(GfshExecutor gfsh, Path serverDir) {
     CommandStringBuilder command = new CommandStringBuilder(CliStrings.STOP_SERVER)
-        .addOption(CliStrings.STOP_SERVER__DIR, serverDir);
+        .addOption(CliStrings.STOP_SERVER__DIR, serverDir.toString());
     gfsh.execute(command.toString());
   }
 
@@ -301,16 +307,12 @@ public class Tomcat8ClientServerRollingUpgradeTest {
    * @return Paths to required jars
    */
   private String getClassPathTomcat8AndOldModules() {
-    final String[] requiredClasspathJars = {
-        "/lib/geode-modules-" + oldVersion + ".jar",
-        "/lib/geode-modules-tomcat8-" + oldVersion + ".jar",
-        "/lib/servlet-api.jar",
-        "/lib/catalina.jar",
-        "/lib/tomcat-util.jar",
-        "/bin/tomcat-juli.jar"
-    };
+    TestVersion geodeVersion = sourceVmConfiguration.geodeVersion();
+    if (geodeVersion.equals(TestVersion.current())) {
+      return getClassPathTomcat8AndCurrentModules();
+    }
 
-    return getRequiredClasspathJars(tomcat8AndOldModules.getHome(), requiredClasspathJars);
+    return getRequiredClasspathJars(tomcat8AndOldModules.getHome(), geodeVersion.toString());
   }
 
   /**
@@ -324,31 +326,18 @@ public class Tomcat8ClientServerRollingUpgradeTest {
    * @return Paths to required jars
    */
   private String getClassPathTomcat8AndCurrentModules() {
-    String currentVersion = GemFireVersion.getGemFireVersion();
+    return getRequiredClasspathJars(tomcat8AndCurrentModules.getHome(), getGemFireVersion());
+  }
 
-    final String[] requiredClasspathJars = {
-        "/lib/geode-modules-" + currentVersion + ".jar",
-        "/lib/geode-modules-tomcat8-" + currentVersion + ".jar",
+  private String getRequiredClasspathJars(final Path installDir, final String version) {
+    return Stream.of(
+        "/lib/geode-modules-" + version + ".jar",
+        "/lib/geode-modules-tomcat8-" + version + ".jar",
         "/lib/servlet-api.jar",
         "/lib/catalina.jar",
         "/lib/tomcat-util.jar",
-        "/bin/tomcat-juli.jar"
-    };
-
-    return getRequiredClasspathJars(tomcat8AndCurrentModules.getHome(), requiredClasspathJars);
-  }
-
-  private String getRequiredClasspathJars(final String tomcat8AndRequiredModules,
-      final String[] requiredClasspathJars) {
-    StringBuilder completeJarList = new StringBuilder();
-    for (String requiredJar : requiredClasspathJars) {
-      completeJarList.append(tomcat8AndRequiredModules)
-          .append(requiredJar)
-          .append(File.pathSeparator);
-    }
-
-    completeJarList.deleteCharAt(completeJarList.length() - 1);
-
-    return completeJarList.toString();
+        "/bin/tomcat-juli.jar")
+        .map(installDir.toString()::concat)
+        .collect(joining(File.pathSeparator));
   }
 }
diff --git a/geode-assembly/src/upgradeTest/java/org/apache/geode/session/tests/TomcatSessionBackwardsCompatibilityTestBase.java b/geode-assembly/src/upgradeTest/java/org/apache/geode/session/tests/TomcatSessionBackwardsCompatibilityTestBase.java
index b712f28c42..88bd3b7de3 100644
--- a/geode-assembly/src/upgradeTest/java/org/apache/geode/session/tests/TomcatSessionBackwardsCompatibilityTestBase.java
+++ b/geode-assembly/src/upgradeTest/java/org/apache/geode/session/tests/TomcatSessionBackwardsCompatibilityTestBase.java
@@ -62,7 +62,7 @@ public abstract class TomcatSessionBackwardsCompatibilityTestBase {
         // Skip versions older than 1.2
         .filter(hasGeodeVersion(TestVersions.atLeast(TestVersion.valueOf("1.2.0"))))
         // Skip Java upgrades
-        .filter(hasGeodeVersion(TestVersions.lessThan(TestVersion.CURRENT_VERSION)))
+        .filter(hasGeodeVersion(TestVersions.lessThan(TestVersion.current())))
         .map(VmConfiguration::geodeVersion)
         .map(String::valueOf)
         .collect(toList());
diff --git a/geode-assembly/src/upgradeTest/java/org/apache/geode/test/junit/rules/GfshExecutorVersionTest.java b/geode-assembly/src/upgradeTest/java/org/apache/geode/test/junit/rules/GfshExecutorVersionTest.java
new file mode 100644
index 0000000000..a84b47cf44
--- /dev/null
+++ b/geode-assembly/src/upgradeTest/java/org/apache/geode/test/junit/rules/GfshExecutorVersionTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.junit.rules;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.internal.util.ProductVersionUtil;
+import org.apache.geode.test.junit.categories.GfshTest;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecutor;
+import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+
+@Category(GfshTest.class)
+public class GfshExecutorVersionTest {
+
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
+
+  @Test
+  public void contextUsesCurrentGeodeVersionByDefault() {
+    String currentVersion = ProductVersionUtil.getDistributionVersion().getVersion();
+
+    GfshExecutor executor = gfshRule.executor().build();
+
+    assertThat(executor.execute("version").getOutputText()).contains(currentVersion);
+  }
+
+  @Test
+  public void contextUsesSpecifiedGeodeVersion() {
+    String specifiedVersion = "1.3.0";
+
+    GfshExecutor executor = gfshRule.executor().withGeodeVersion(specifiedVersion).build();
+
+    assertThat(executor.execute("version").getOutputText()).contains(specifiedVersion);
+  }
+}
diff --git a/geode-assembly/src/upgradeTest/java/org/apache/geode/test/junit/rules/GfshRuleUpgradeTest.java b/geode-assembly/src/upgradeTest/java/org/apache/geode/test/junit/rules/GfshRuleUpgradeTest.java
deleted file mode 100644
index 40326aac5d..0000000000
--- a/geode-assembly/src/upgradeTest/java/org/apache/geode/test/junit/rules/GfshRuleUpgradeTest.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.test.junit.rules;
-
-import static org.apache.geode.internal.lang.SystemUtils.isWindows;
-import static org.assertj.core.api.Assertions.assertThat;
-
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.test.junit.categories.GfshTest;
-import org.apache.geode.test.junit.rules.gfsh.GfshRule;
-import org.apache.geode.test.version.VersionManager;
-
-@Category(GfshTest.class)
-public class GfshRuleUpgradeTest {
-
-  private static final String GEODE_HOME = System.getenv("GEODE_HOME");
-  private static final String GFSH_OLD_VERSION = "1.3.0";
-
-  private Path gfshCurrent;
-  private Path gfsh130;
-
-  @Rule
-  public GfshRule gfsh130Rule = new GfshRule(GFSH_OLD_VERSION);
-  @Rule
-  public GfshRule gfshCurrentRule = new GfshRule();
-
-  @Before
-  public void findGfshCurrentPath() {
-    Path geodeHomePath = Paths.get(GEODE_HOME).toAbsolutePath();
-    assertThat(geodeHomePath)
-        .as("System.getenv(\"GEODE_HOME\")")
-        .exists();
-
-    String gfsh = isWindows() ? "gfsh.bat" : "gfsh";
-    gfshCurrent = Paths.get(GEODE_HOME, "bin", gfsh);
-    assertThat(gfshCurrent)
-        .as("GEODE_HOME/bin/" + gfsh + " exists")
-        .exists();
-  }
-
-  @Before
-  public void findGfsh130Path() {
-    Path geode130Home = Paths.get(VersionManager.getInstance().getInstall(GFSH_OLD_VERSION));
-    assertThat(geode130Home)
-        .as("VersionManager installation for " + GFSH_OLD_VERSION)
-        .exists();
-
-    String gfsh = isWindows() ? "gfsh.bat" : "gfsh";
-    gfsh130 = Paths.get(geode130Home.toString(), "bin", gfsh);
-    assertThat(gfsh130)
-        .as("geode130Home/bin/" + gfsh + " exists")
-        .exists();
-  }
-
-  @Test
-  public void gfshCurrentRuleUsesCurrentGfsh() {
-    assertThat(gfshCurrentRule.getGfshPath())
-        .isEqualTo(gfshCurrent);
-  }
-
-  @Test
-  public void gfsh130RuleUses130Gfsh() {
-    assertThat(gfsh130Rule.getGfshPath())
-        .isEqualTo(gfsh130);
-  }
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
index 3d3e225e49..51a5bec87f 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
@@ -14,12 +14,14 @@
  */
 package org.apache.geode.management.internal.configuration;
 
+import static java.nio.file.Files.createFile;
 import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.io.FileWriter;
+import java.io.IOException;
 
 import org.junit.Before;
 import org.junit.Rule;
@@ -30,7 +32,6 @@ import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
 import org.apache.geode.test.junit.rules.GfshCommandRule;
 
-
 public class ClusterConfigDeployJarDUnitTest extends ClusterConfigTestBase {
 
   private String clusterJar;
@@ -44,10 +45,10 @@ public class ClusterConfigDeployJarDUnitTest extends ClusterConfigTestBase {
   public GfshCommandRule gfshConnector = new GfshCommandRule();
 
   @Before
-  public void before() throws Exception {
-    clusterJar = createJarFileWithClass("Cluster", "cluster.jar", temporaryFolder.getRoot());
-    group1Jar = createJarFileWithClass("Group1", "group1.jar", temporaryFolder.getRoot());
-    group2Jar = createJarFileWithClass("Group2", "group2.jar", temporaryFolder.getRoot());
+  public void before() throws IOException {
+    clusterJar = createJarFileWithClass("Cluster", "cluster.jar", rootFolder.toFile());
+    group1Jar = createJarFileWithClass("Group1", "group1.jar", rootFolder.toFile());
+    group2Jar = createJarFileWithClass("Group2", "group2.jar", rootFolder.toFile());
   }
 
   @Test
@@ -148,7 +149,7 @@ public class ClusterConfigDeployJarDUnitTest extends ClusterConfigTestBase {
     gfshConnector.connect(locator);
     assertThat(gfshConnector.isConnected()).isTrue();
 
-    File junkFile = temporaryFolder.newFile("junk");
+    File junkFile = createFile(rootFolder.resolve("junk").toAbsolutePath()).toFile();
     FileWriter writer = new FileWriter(junkFile);
     writer.write("this is not a real jar");
     writer.close();
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
index 3e101d023e..fa12141098 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
@@ -19,8 +19,6 @@ import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.File;
-
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -31,7 +29,6 @@ import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
 import org.apache.geode.test.junit.rules.GfshCommandRule;
 
-
 public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
 
   private static final ClusterConfig INITIAL_CONFIG = new ClusterConfig(new ConfigGroup("cluster"));
@@ -54,7 +51,7 @@ public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
   }
 
   @Test
-  public void testImportWithRunningServerWithRegion() throws Exception {
+  public void testImportWithRunningServerWithRegion() {
     MemberVM server1 = lsRule.startServerVM(1, serverProps, locatorVM.getPort());
     // create another server as well
     MemberVM server2 = lsRule.startServerVM(2, serverProps, locatorVM.getPort());
@@ -73,7 +70,7 @@ public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
   }
 
   @Test
-  public void testImportWithRunningServer() throws Exception {
+  public void testImportWithRunningServer() throws ClassNotFoundException {
     MemberVM server1 = lsRule.startServerVM(1, serverProps, locatorVM.getPort());
 
     serverProps.setProperty("groups", "group2");
@@ -117,7 +114,7 @@ public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
   }
 
   @Test
-  public void testImportClusterConfig() throws Exception {
+  public void testImportClusterConfig() throws ClassNotFoundException {
     gfshConnector
         .executeAndAssertThat(
             "import cluster-configuration --zip-file-name=" + clusterConfigZipPath)
@@ -125,7 +122,7 @@ public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
 
     // Make sure that a backup of the old clusterConfig was created
     assertThat(locatorVM.getWorkingDir().listFiles())
-        .filteredOn((File file) -> file.getName().contains("cluster_config")).hasSize(2);
+        .filteredOn(file -> file.getName().contains("cluster_config")).hasSize(2);
 
     CONFIG_FROM_ZIP.verify(locatorVM);
 
@@ -145,7 +142,7 @@ public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
   }
 
   @Test
-  public void testImportWithMultipleLocators() throws Exception {
+  public void testImportWithMultipleLocators() throws ClassNotFoundException {
     locatorProps.setProperty(LOCATORS, "localhost[" + locatorVM.getPort() + "]");
     MemberVM locator1 = lsRule.startLocatorVM(1, locatorProps);
 
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
index 27315bad56..6e3d45cf38 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
@@ -20,6 +20,7 @@ import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOAD_CLUSTER_CONFIGURATION_FROM_DIR;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.Properties;
 
 import org.junit.Before;
@@ -30,7 +31,6 @@ import org.apache.geode.management.internal.configuration.utils.ZipUtils;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
 
-
 public class ClusterConfigStartMemberDUnitTest extends ClusterConfigTestBase {
 
   private MemberVM locator;
@@ -39,18 +39,18 @@ public class ClusterConfigStartMemberDUnitTest extends ClusterConfigTestBase {
   public ClusterStartupRule lsRule = new ClusterStartupRule();
 
   @Before
-  public void before() throws Exception {
+  public void before() throws IOException {
     locator = startLocatorWithLoadCCFromDir();
   }
 
   @Test
-  public void testStartLocator() throws Exception {
+  public void testStartLocator() throws ClassNotFoundException {
     MemberVM secondLocator = lsRule.startLocatorVM(1, locator.getPort());
     REPLICATED_CONFIG_FROM_ZIP.verify(secondLocator);
   }
 
   @Test
-  public void testStartServerWithSingleGroup() throws Exception {
+  public void testStartServerWithSingleGroup() throws ClassNotFoundException {
     ClusterConfig expectedNoGroupConfig = new ClusterConfig(CLUSTER);
     ClusterConfig expectedGroup1Config = new ClusterConfig(CLUSTER, GROUP1);
     ClusterConfig expectedGroup2Config = new ClusterConfig(CLUSTER, GROUP2);
@@ -68,7 +68,7 @@ public class ClusterConfigStartMemberDUnitTest extends ClusterConfigTestBase {
   }
 
   @Test
-  public void testStartServerWithMultipleGroup() throws Exception {
+  public void testStartServerWithMultipleGroup() throws ClassNotFoundException {
     ClusterConfig expectedGroup1And2Config = new ClusterConfig(CLUSTER, GROUP1, GROUP2);
 
     serverProps.setProperty(GROUPS, "group1,group2");
@@ -77,7 +77,7 @@ public class ClusterConfigStartMemberDUnitTest extends ClusterConfigTestBase {
     expectedGroup1And2Config.verify(server);
   }
 
-  private MemberVM startLocatorWithLoadCCFromDir() throws Exception {
+  private MemberVM startLocatorWithLoadCCFromDir() throws IOException {
     File locatorDir = new File(lsRule.getWorkingDirRoot(), "vm0");
     File configDir = new File(locatorDir, "cluster_config");
 
@@ -93,8 +93,6 @@ public class ClusterConfigStartMemberDUnitTest extends ClusterConfigTestBase {
     properties.setProperty(ENABLE_CLUSTER_CONFIGURATION, "true");
     properties.setProperty(LOAD_CLUSTER_CONFIGURATION_FROM_DIR, "true");
 
-    MemberVM locator = lsRule.startLocatorVM(0, properties);
-
-    return locator;
+    return lsRule.startLocatorVM(0, properties);
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/process/AttachProcessUtils.java b/geode-core/src/main/java/org/apache/geode/internal/process/AttachProcessUtils.java
index 919b92bb7c..a7495125be 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/process/AttachProcessUtils.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/process/AttachProcessUtils.java
@@ -19,14 +19,12 @@ import static org.apache.commons.lang3.Validate.isTrue;
 import com.sun.tools.attach.VirtualMachine;
 import com.sun.tools.attach.VirtualMachineDescriptor;
 
-import org.apache.geode.internal.process.ProcessUtils.InternalProcessUtils;
-
 /**
  * Implementation of the {@link ProcessUtils} SPI that uses the JDK Attach API.
  *
  * @since GemFire 8.0
  */
-class AttachProcessUtils implements InternalProcessUtils {
+class AttachProcessUtils implements ProcessUtilsProvider {
 
   @Override
   public boolean isProcessAlive(final int pid) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/process/FileControllableProcess.java b/geode-core/src/main/java/org/apache/geode/internal/process/FileControllableProcess.java
index bdbc9d3c60..c4aa134764 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/process/FileControllableProcess.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/process/FileControllableProcess.java
@@ -73,6 +73,9 @@ public class FileControllableProcess implements ControllableProcess {
     this.stopRequestFileWatchdog = stopRequestFileWatchdog;
     this.statusRequestFileWatchdog = statusRequestFileWatchdog;
 
+    logger.info("Creating {} for stop and status support in {}.", getClass().getSimpleName(),
+        directory);
+
     deleteFiles(directory, processType);
     stopRequestFileWatchdog.start();
     statusRequestFileWatchdog.start();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/process/LocalProcessLauncher.java b/geode-core/src/main/java/org/apache/geode/internal/process/LocalProcessLauncher.java
index 00f1e4a88c..1f93e069e4 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/process/LocalProcessLauncher.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/process/LocalProcessLauncher.java
@@ -21,6 +21,9 @@ import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
 import org.apache.geode.util.internal.GeodeGlossary;
 
 /**
@@ -42,6 +45,8 @@ import org.apache.geode.util.internal.GeodeGlossary;
  */
 class LocalProcessLauncher {
 
+  private static final Logger logger = LogService.getLogger();
+
   static final String PROPERTY_IGNORE_IS_PID_ALIVE =
       GeodeGlossary.GEMFIRE_PREFIX + "test.LocalProcessLauncher.ignoreIsPidAlive";
 
@@ -127,6 +132,8 @@ class LocalProcessLauncher {
     File tempPidFile = new File(pidFile.getParent(), pidFile.getName() + ".tmp");
     tempPidFile.createNewFile();
 
+    logger.info("Writing process id {} to {}.", pid, pidFile);
+
     try (FileWriter writer = new FileWriter(tempPidFile)) {
       writer.write(String.valueOf(pid));
       writer.flush();
@@ -140,7 +147,7 @@ class LocalProcessLauncher {
     int otherPid = 0;
     try {
       otherPid = ProcessUtils.readPid(pidFile);
-    } catch (NumberFormatException | IOException ignore) {
+    } catch (NumberFormatException ignore) {
       // suppress
     }
     return otherPid;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/process/NativeProcessUtils.java b/geode-core/src/main/java/org/apache/geode/internal/process/NativeProcessUtils.java
index 4016141011..5a14351c04 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/process/NativeProcessUtils.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/process/NativeProcessUtils.java
@@ -17,7 +17,6 @@ package org.apache.geode.internal.process;
 import static org.apache.commons.lang3.Validate.isTrue;
 
 import org.apache.geode.annotations.Immutable;
-import org.apache.geode.internal.process.ProcessUtils.InternalProcessUtils;
 import org.apache.geode.internal.shared.NativeCalls;
 
 /**
@@ -25,11 +24,15 @@ import org.apache.geode.internal.shared.NativeCalls;
  *
  * @since GemFire 8.0
  */
-class NativeProcessUtils implements InternalProcessUtils {
+public class NativeProcessUtils implements ProcessUtilsProvider {
 
   @Immutable
   private static final NativeCalls nativeCalls = NativeCalls.getInstance();
 
+  public static ProcessUtilsProvider create() {
+    return new NativeProcessUtils();
+  }
+
   @Override
   public boolean isProcessAlive(final int pid) {
     isTrue(pid > 0, "Invalid pid '" + pid + "' specified");
diff --git a/geode-core/src/main/java/org/apache/geode/internal/process/ProcessType.java b/geode-core/src/main/java/org/apache/geode/internal/process/ProcessType.java
index 233efda2eb..7ae9c4cd2f 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/process/ProcessType.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/process/ProcessType.java
@@ -16,6 +16,8 @@ package org.apache.geode.internal.process;
 
 import static org.apache.commons.lang3.StringUtils.EMPTY;
 
+import java.nio.file.Path;
+
 import org.apache.geode.util.internal.GeodeGlossary;
 
 /**
@@ -63,6 +65,10 @@ public enum ProcessType {
         + '.' + SUFFIX_STATUS;
   }
 
+  public int readPid(Path workingDir) {
+    return ProcessUtils.readPid(workingDir.resolve(getPidFileName()));
+  }
+
   @Override
   public String toString() {
     return name;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtils.java b/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtils.java
index 6aa474bef9..4edae5218a 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtils.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtils.java
@@ -22,7 +22,9 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.lang.management.ManagementFactory;
+import java.nio.file.Path;
 
 import org.apache.geode.annotations.Immutable;
 
@@ -34,7 +36,7 @@ import org.apache.geode.annotations.Immutable;
 public class ProcessUtils {
 
   @Immutable
-  private static final InternalProcessUtils internal = initializeInternalProcessUtils();
+  private static final ProcessUtilsProvider internal = initializeInternalProcessUtils();
 
   private ProcessUtils() {
     // nothing
@@ -125,12 +127,18 @@ public class ProcessUtils {
     return internal.killProcess(pid);
   }
 
-  public static int readPid(final File pidFile) throws IOException {
+  public static int readPid(final Path pidFile) {
+    return readPid(pidFile.toFile());
+  }
+
+  public static int readPid(final File pidFile) {
     notNull(pidFile, "Invalid pidFile '" + pidFile + "' specified");
     isTrue(pidFile.exists(), "Nonexistent pidFile '" + pidFile + "' specified");
 
     try (BufferedReader reader = new BufferedReader(new FileReader(pidFile))) {
       return Integer.parseInt(reader.readLine());
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
     }
   }
 
@@ -149,7 +157,7 @@ public class ProcessUtils {
     return internal.isAttachApiAvailable();
   }
 
-  private static InternalProcessUtils initializeInternalProcessUtils() {
+  private static ProcessUtilsProvider initializeInternalProcessUtils() {
     // 1) prefer Attach because it filters out non-JVM processes
     try {
       Class.forName("com.sun.tools.attach.VirtualMachine");
@@ -174,7 +182,7 @@ public class ProcessUtils {
     }
 
     // 3) consider logging warning and then proceed with no-op
-    return new InternalProcessUtils() {
+    return new ProcessUtilsProvider() {
       @Override
       public boolean isProcessAlive(final int pid) {
         return false;
@@ -197,17 +205,4 @@ public class ProcessUtils {
     };
   }
 
-  /**
-   * Defines the SPI for ProcessUtils
-   */
-  interface InternalProcessUtils {
-
-    boolean isProcessAlive(final int pid);
-
-    boolean killProcess(final int pid);
-
-    boolean isAvailable();
-
-    boolean isAttachApiAvailable();
-  }
 }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java b/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtilsProvider.java
similarity index 60%
copy from geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java
copy to geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtilsProvider.java
index e2ebd32a3f..20ec4c7211 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/process/ProcessUtilsProvider.java
@@ -12,27 +12,18 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.management.internal.cli.shell;
+package org.apache.geode.internal.process;
 
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-
-import org.apache.geode.test.junit.rules.gfsh.GfshRule;
-
-public class GfshDisconnectWithinScript {
+/**
+ * Defines the SPI for ProcessUtils
+ */
+public interface ProcessUtilsProvider {
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
+  boolean isProcessAlive(final int pid);
 
+  boolean killProcess(final int pid);
 
-  @Before
-  public void setup() {
-    gfsh.execute("start locator");
-  }
+  boolean isAvailable();
 
-  @Test
-  public void disconnectInScriptDoesNotRaiseNPE() {
-    gfsh.execute("connect", "disconnect", "echo \"Disconnect command resolved without issue.\"");
-  }
+  boolean isAttachApiAvailable();
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/process/NativeProcessUtilsTest.java b/geode-core/src/test/java/org/apache/geode/internal/process/NativeProcessUtilsTest.java
index 65ab647d22..a73fec05b6 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/process/NativeProcessUtilsTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/process/NativeProcessUtilsTest.java
@@ -37,7 +37,6 @@ public class NativeProcessUtilsTest {
   private static final int PREFERRED_FAKE_PID = 42;
 
   private int actualPid;
-  private int fakePid;
   private NativeProcessUtils nativeProcessUtils;
 
   @Rule
@@ -46,28 +45,32 @@ public class NativeProcessUtilsTest {
   @Before
   public void before() throws Exception {
     actualPid = identifyPid();
-    fakePid = new AvailablePid().findAvailablePid(PREFERRED_FAKE_PID);
     nativeProcessUtils = new NativeProcessUtils();
   }
 
   @Test
-  public void isAttachApiAvailable_returnsFalse() throws Exception {
+  public void isAttachApiAvailable_returnsFalse() {
     assertThat(nativeProcessUtils.isAttachApiAvailable()).isFalse();
   }
 
   @Test
-  public void isAvailable_returnsTrue() throws Exception {
+  public void isAvailable_returnsTrue() {
     assertThat(nativeProcessUtils.isAvailable()).isTrue();
   }
 
   @Test
-  public void isProcessAlive_livePid_returnsTrue() throws Exception {
-    assertThat(nativeProcessUtils.isProcessAlive(actualPid)).isTrue();
+  public void isProcessAlive_livePid_returnsTrue() {
+    assertThat(nativeProcessUtils.isProcessAlive(actualPid))
+        .as("is process identified by " + actualPid + " alive?")
+        .isTrue();
   }
 
   @Test
   @Retry(3)
   public void isProcessAlive_deadPid_returnsFalse() throws Exception {
-    assertThat(nativeProcessUtils.isProcessAlive(fakePid)).isFalse();
+    int fakePid = new AvailablePid().findAvailablePid(PREFERRED_FAKE_PID);
+    assertThat(nativeProcessUtils.isProcessAlive(fakePid))
+        .as("is process identified by " + fakePid + " alive?")
+        .isFalse();
   }
 }
diff --git a/geode-core/src/upgradeTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java b/geode-core/src/upgradeTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java
index 04dd2f4f6a..ba0e355c42 100644
--- a/geode-core/src/upgradeTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java
+++ b/geode-core/src/upgradeTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java
@@ -281,7 +281,7 @@ public class ClusterCommunicationsDUnitTest implements Serializable {
   public void performARollingUpgrade() {
     Optional<VmConfiguration> sourceConfiguration = VmConfigurations.upgrades().stream()
         // Skip the configurations with the current Geode
-        .filter(hasGeodeVersion(TestVersions.lessThan(TestVersion.CURRENT_VERSION)))
+        .filter(hasGeodeVersion(TestVersions.lessThan(TestVersion.current())))
         // Get the configuration with the latest Geode
         .max(comparing(VmConfiguration::geodeVersion));
     assumeThat(sourceConfiguration)
diff --git a/geode-core/src/upgradeTest/java/org/apache/geode/internal/net/SocketCreatorUpgradeTest.java b/geode-core/src/upgradeTest/java/org/apache/geode/internal/net/SocketCreatorUpgradeTest.java
index 9f94377c65..21d00624b5 100644
--- a/geode-core/src/upgradeTest/java/org/apache/geode/internal/net/SocketCreatorUpgradeTest.java
+++ b/geode-core/src/upgradeTest/java/org/apache/geode/internal/net/SocketCreatorUpgradeTest.java
@@ -12,15 +12,11 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.internal.net;
 
-import static java.lang.Integer.parseInt;
 import static java.lang.String.format;
 import static java.lang.System.getenv;
-import static java.nio.charset.Charset.defaultCharset;
 import static java.util.concurrent.CompletableFuture.runAsync;
-import static org.apache.commons.io.FileUtils.readFileToString;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_CLIENT_PROTOCOLS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENABLED_COMPONENTS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENDPOINT_IDENTIFICATION_ENABLED;
@@ -38,7 +34,6 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assumptions.assumeThat;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.net.InetAddress;
@@ -52,11 +47,9 @@ import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
-import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
@@ -65,7 +58,8 @@ import org.apache.geode.cache.ssl.CertStores;
 import org.apache.geode.cache.ssl.CertificateBuilder;
 import org.apache.geode.cache.ssl.CertificateMaterial;
 import org.apache.geode.internal.UniquePortSupplier;
-import org.apache.geode.internal.shared.NativeCalls;
+import org.apache.geode.test.junit.rules.FolderRule;
+import org.apache.geode.test.junit.rules.gfsh.GfshExecutor;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 import org.apache.geode.test.version.TestVersion;
 import org.apache.geode.test.version.VersionManager;
@@ -83,35 +77,31 @@ public class SocketCreatorUpgradeTest {
   private static final String LOCATOR_1 = "locator1";
   private static final String LOCATOR_2 = "locator2";
 
-  private final TestVersion version;
-
-  private final String startLocator1;
-  private final String startLocator2;
-  private final String startLocator1New;
-  private final String startLocator2New;
-  private final String stopLocator1;
-  private final String stopLocator2;
-
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder();
+  private final String version;
+  private final TestVersion testVersion;
 
-  @Rule
-  public final GfshRule gfshOldGeodeOldJava;
+  private String startLocator1;
+  private String startLocator2;
+  private String startLocator1New;
+  private String startLocator2New;
+  private String stopLocator1;
+  private String stopLocator2;
 
-  @Rule
-  public final GfshRule gfshOldGeodeNewJava;
+  @Rule(order = 0)
+  public final FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public final GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  @Rule
-  public final GfshRule gfshNewGeodeOldJava;
+  private GfshExecutor gfshOldGeodeOldJava;
+  private GfshExecutor gfshOldGeodeNewJava;
+  private GfshExecutor gfshNewGeodeOldJava;
+  private GfshExecutor gfshNewGeodeNewJava;
 
-  @Rule
-  public final GfshRule gfshNewGeodeNewJava;
-
-  private final File root;
-  private final File keyStoreFile;
-  private final File trustStoreFile;
-  private final File securityPropertiesFile;
-  private final File newSecurityPropertiesFile;
+  private Path root;
+  private File keyStoreFile;
+  private File trustStoreFile;
+  private File securityPropertiesFile;
+  private File newSecurityPropertiesFile;
 
   private File locator1LogFile;
   private File locator2LogFile;
@@ -124,18 +114,30 @@ public class SocketCreatorUpgradeTest {
     return result;
   }
 
-  public SocketCreatorUpgradeTest(final String version) throws IOException,
-      GeneralSecurityException {
-
-    this.version = TestVersion.valueOf(version);
+  public SocketCreatorUpgradeTest(final String version) {
+    this.version = version;
+    testVersion = TestVersion.valueOf(version);
+  }
 
+  @Before
+  public void setUp() throws IOException, GeneralSecurityException {
     final Path oldJavaHome = Paths.get(getenv("JAVA_HOME_8u265"));
     final Path newJavaHome = Paths.get(getenv("JAVA_HOME_8u272"));
 
-    gfshOldGeodeOldJava = new GfshRule(version, oldJavaHome);
-    gfshOldGeodeNewJava = new GfshRule(version, newJavaHome);
-    gfshNewGeodeOldJava = new GfshRule(oldJavaHome);
-    gfshNewGeodeNewJava = new GfshRule(newJavaHome);
+    gfshOldGeodeOldJava = gfshRule.executor()
+        .withGeodeVersion(version)
+        .withJavaHome(oldJavaHome)
+        .build();
+    gfshOldGeodeNewJava = gfshRule.executor()
+        .withGeodeVersion(version)
+        .withJavaHome(newJavaHome)
+        .build();
+    gfshNewGeodeOldJava = gfshRule.executor()
+        .withJavaHome(oldJavaHome)
+        .build();
+    gfshNewGeodeNewJava = gfshRule.executor()
+        .withJavaHome(newJavaHome)
+        .build();
 
     final UniquePortSupplier portSupplier = new UniquePortSupplier();
     final int locator1Port = portSupplier.getAvailablePort();
@@ -143,44 +145,35 @@ public class SocketCreatorUpgradeTest {
     final int locator2Port = portSupplier.getAvailablePort();
     final int locator2JmxPort = portSupplier.getAvailablePort();
 
-    tempFolder.create();
-    root = tempFolder.getRoot();
-    keyStoreFile = tempFolder.newFile();
-    trustStoreFile = tempFolder.newFile();
-    securityPropertiesFile = tempFolder.newFile();
-    newSecurityPropertiesFile = tempFolder.newFile();
+    root = folderRule.getFolder().toPath();
+
+    keyStoreFile = Files.createFile(root.resolve("keyStoreFile")).toFile();
+    trustStoreFile = Files.createFile(root.resolve("trustStoreFile")).toFile();
+    securityPropertiesFile = Files.createFile(root.resolve("security.properties")).toFile();
+    newSecurityPropertiesFile = Files.createFile(root.resolve("newSecurity.properties")).toFile();
 
     final String hostName = InetAddress.getLocalHost().getCanonicalHostName();
     generateKeyAndTrustStore(hostName, keyStoreFile, trustStoreFile);
 
     startLocator1 = startLocator(LOCATOR_1, hostName, locator1Port, locator1JmxPort,
-        securityPropertiesFile, locator2Port, this.version);
+        securityPropertiesFile, locator2Port, testVersion);
     startLocator2 = startLocator(LOCATOR_2, hostName, locator2Port, locator2JmxPort,
-        securityPropertiesFile, locator1Port, this.version);
+        securityPropertiesFile, locator1Port, testVersion);
 
     startLocator1New = startLocator(LOCATOR_1, hostName, locator1Port, locator1JmxPort,
-        newSecurityPropertiesFile, locator2Port, this.version);
+        newSecurityPropertiesFile, locator2Port, testVersion);
     startLocator2New = startLocator(LOCATOR_2, hostName, locator2Port, locator2JmxPort,
-        newSecurityPropertiesFile, locator1Port, this.version);
+        newSecurityPropertiesFile, locator1Port, testVersion);
 
     stopLocator1 = stopLocator(LOCATOR_1);
     stopLocator2 = stopLocator(LOCATOR_2);
-  }
 
-  @Before
-  public void before() {
-    locator1LogFile = root.toPath().resolve(LOCATOR_1).resolve(LOCATOR_1 + ".log")
+    locator1LogFile = root.resolve(LOCATOR_1).resolve(LOCATOR_1 + ".log")
         .toAbsolutePath().toFile();
-    locator2LogFile = root.toPath().resolve(LOCATOR_2).resolve(LOCATOR_2 + ".log")
+    locator2LogFile = root.resolve(LOCATOR_2).resolve(LOCATOR_2 + ".log")
         .toAbsolutePath().toFile();
   }
 
-  @After
-  public void after() {
-    killLocator(root, LOCATOR_1);
-    killLocator(root, LOCATOR_2);
-  }
-
   @Test
   public void upgradingToNewGeodeOnOldJavaWithProtocolsAny() throws IOException {
     generateSecurityProperties(PROTOCOL_ANY, securityPropertiesFile, keyStoreFile, trustStoreFile);
@@ -197,7 +190,7 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void startingOldGeodeWithProtocolsTLSv1_2() throws IOException {
-    assumeThat(version)
+    assumeThat(testVersion)
         .as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
         .satisfiesAnyOf(
             v -> assertThat(v).isLessThanOrEqualTo(TestVersion.valueOf("1.12.0")),
@@ -212,7 +205,7 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void startingOldGeode1_12_1_UpTo1_13_0WithProtocolsTLSv1_2Hangs() throws IOException {
-    assumeThat(version).as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
+    assumeThat(testVersion).as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
         .isGreaterThanOrEqualTo(TestVersion.valueOf("1.12.1"))
         .isLessThan(TestVersion.valueOf("1.13.0"));
 
@@ -228,7 +221,7 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void upgradingToNewGeodeOnOldJavaWithProtocolsTLSv1_2() throws IOException {
-    assumeThat(version).as("TODO")
+    assumeThat(testVersion).as("TODO")
         .isLessThanOrEqualTo(TestVersion.valueOf("1.12.0"));
     generateSecurityProperties(PROTOCOL_TLSv1_2, securityPropertiesFile, keyStoreFile,
         trustStoreFile);
@@ -245,9 +238,9 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void upgradingToNewGeodeOnOldJavaWithProtocolsTLSv1_2Hangs() throws IOException {
-    assumeThat(version).as("Geode 1.12.0 and older can upgrade.")
+    assumeThat(testVersion).as("Geode 1.12.0 and older can upgrade.")
         .isGreaterThan(TestVersion.valueOf("1.12.0"));
-    assumeThat(version).as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
+    assumeThat(testVersion).as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
         .isGreaterThanOrEqualTo(TestVersion.valueOf("1.13.0"));
 
     generateSecurityProperties(PROTOCOL_TLSv1_2, securityPropertiesFile, keyStoreFile,
@@ -263,7 +256,7 @@ public class SocketCreatorUpgradeTest {
   @Test
   public void upgradingToNewGeodeOnOldJavaWithProtocolsTLSv1_2WithNewProperties()
       throws IOException {
-    assumeThat(version)
+    assumeThat(testVersion)
         .as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
         .satisfiesAnyOf(
             v -> assertThat(v).isLessThanOrEqualTo(TestVersion.valueOf("1.12.0")),
@@ -288,7 +281,7 @@ public class SocketCreatorUpgradeTest {
   @Test
   public void upgradingToNewGeodeOnOldJavaWithProtocolsTLSv1_2ThroughNewProperties()
       throws IOException {
-    assumeThat(version)
+    assumeThat(testVersion)
         .as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
         .satisfiesAnyOf(
             v -> assertThat(v).isLessThanOrEqualTo(TestVersion.valueOf("1.12.0")),
@@ -342,7 +335,7 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void upgradingToNewJavaOnOldGeodeWithProtocolsAny() throws IOException {
-    assumeThat(version).as("Only Geode older than 1.13.0 can directly upgrade Java version.")
+    assumeThat(testVersion).as("Only Geode older than 1.13.0 can directly upgrade Java version.")
         .isLessThan(TestVersion.valueOf("1.13.0"));
 
     generateSecurityProperties(PROTOCOL_ANY, securityPropertiesFile, keyStoreFile, trustStoreFile);
@@ -359,7 +352,7 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void upgradingToNewJavaOnOldGeodeWithProtocolsAnyHangs() throws IOException {
-    assumeThat(version).as("Geode older than 1.13.0 can directly upgrade Java version.")
+    assumeThat(testVersion).as("Geode older than 1.13.0 can directly upgrade Java version.")
         .isGreaterThanOrEqualTo(TestVersion.valueOf("1.13.0"));
 
     generateSecurityProperties(PROTOCOL_ANY, securityPropertiesFile, keyStoreFile, trustStoreFile);
@@ -373,7 +366,7 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void upgradingToNewJavaOnOldGeodeWithProtocolsTLSv1_2() throws IOException {
-    assumeThat(version).as("Only Geode older than 1.12.1 can directly upgrade Java version.")
+    assumeThat(testVersion).as("Only Geode older than 1.12.1 can directly upgrade Java version.")
         .isLessThan(TestVersion.valueOf("1.12.1"));
 
     generateSecurityProperties(PROTOCOL_ANY, securityPropertiesFile, keyStoreFile, trustStoreFile);
@@ -390,9 +383,9 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void upgradingToNewJavaOnOldGeodeWithProtocolsTLSv1_2Hangs() throws IOException {
-    assumeThat(version).as("Geode 1.12.0 and older can upgrade.")
+    assumeThat(testVersion).as("Geode 1.12.0 and older can upgrade.")
         .isGreaterThan(TestVersion.valueOf("1.12.0"));
-    assumeThat(version)
+    assumeThat(testVersion)
         .as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
         .satisfiesAnyOf(
             v -> assertThat(v).isLessThanOrEqualTo(TestVersion.valueOf("1.12.0")),
@@ -410,7 +403,7 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void upgradingToNewJavaOnOldGeodeWithProtocolsTLSv1_2_SSLv2Hello() throws IOException {
-    assumeThat(version).as("Only Geode older than 1.13.0 can directly upgrade Java version.")
+    assumeThat(testVersion).as("Only Geode older than 1.13.0 can directly upgrade Java version.")
         .isLessThan(TestVersion.valueOf("1.13.0"));
 
     generateSecurityProperties(PROTOCOL_TLSv1_2_SSLv2Hello, securityPropertiesFile, keyStoreFile,
@@ -429,7 +422,7 @@ public class SocketCreatorUpgradeTest {
   @Test
   public void upgradingToNewJavaOnOldGeodeWithProtocolsTLSv1_2_SSLv2HelloHangs()
       throws IOException {
-    assumeThat(version).as("Geode older than 1.13.0 can directly upgrade Java version.")
+    assumeThat(testVersion).as("Geode older than 1.13.0 can directly upgrade Java version.")
         .isGreaterThanOrEqualTo(TestVersion.valueOf("1.13.0"));
 
     generateSecurityProperties(PROTOCOL_TLSv1_2_SSLv2Hello, securityPropertiesFile, keyStoreFile,
@@ -458,9 +451,9 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void upgradingToNewGeodeAndNewJavaWithProtocolsTLSv1_2Hangs() throws IOException {
-    assumeThat(version).as("Geode 1.12.0 and older can upgrade.")
+    assumeThat(testVersion).as("Geode 1.12.0 and older can upgrade.")
         .isGreaterThan(TestVersion.valueOf("1.12.0"));
-    assumeThat(version).as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
+    assumeThat(testVersion).as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
         .isGreaterThanOrEqualTo(TestVersion.valueOf("1.13.0"));
 
     generateSecurityProperties(PROTOCOL_TLSv1_2, securityPropertiesFile, keyStoreFile,
@@ -475,7 +468,7 @@ public class SocketCreatorUpgradeTest {
 
   @Test
   public void upgradingToNewGeodeAndNewJavaWithProtocolsTLSv1_2() throws IOException {
-    assumeThat(version)
+    assumeThat(testVersion)
         .as("Only Geode older than 1.12.1 can directly upgrade Geode and Java version.")
         .isLessThan(TestVersion.valueOf("1.12.1"));
 
@@ -494,7 +487,7 @@ public class SocketCreatorUpgradeTest {
   @Test
   public void upgradingToNewGeodeAndNewJavaWithProtocolsTLSv1_2WithNewProperties()
       throws IOException {
-    assumeThat(version)
+    assumeThat(testVersion)
         .as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
         .satisfiesAnyOf(
             v -> assertThat(v).isLessThanOrEqualTo(TestVersion.valueOf("1.12.0")),
@@ -519,7 +512,7 @@ public class SocketCreatorUpgradeTest {
   @Test
   public void upgradingToNewGeodeAndNewJavaWithProtocolsTLSv1_2ThroughNewProperties()
       throws IOException {
-    assumeThat(version)
+    assumeThat(testVersion)
         .as("Geode between [1.12.1, 1.3.0) can't connect p2p with just TLSv1.2")
         .satisfiesAnyOf(
             v -> assertThat(v).isLessThanOrEqualTo(TestVersion.valueOf("1.12.0")),
@@ -584,7 +577,7 @@ public class SocketCreatorUpgradeTest {
     return format("stop locator --dir=%s", name);
   }
 
-  public static void generateKeyAndTrustStore(final String hostName, final File keyStoreFile,
+  private static void generateKeyAndTrustStore(final String hostName, final File keyStoreFile,
       final File trustStoreFile) throws IOException, GeneralSecurityException {
     final CertificateMaterial ca = new CertificateBuilder(EXPIRATION, ALGORITHM)
         .commonName("Test CA")
@@ -616,8 +609,7 @@ public class SocketCreatorUpgradeTest {
       final String serverProtocols, final File securityPropertiesFile, final File keyStoreFile,
       final File trustStoreFile) throws IOException {
     generateSecurityProperties(null, clientProtocols, serverProtocols, securityPropertiesFile,
-        keyStoreFile,
-        trustStoreFile);
+        keyStoreFile, trustStoreFile);
   }
 
   private static void generateSecurityProperties(final String protocols,
@@ -648,20 +640,4 @@ public class SocketCreatorUpgradeTest {
 
     properties.store(new FileWriter(securityPropertiesFile), null);
   }
-
-  private static void killByPidFile(final Path pidFile) {
-    try {
-      final int pid = parseInt(readFileToString(pidFile.toFile(), defaultCharset()));
-      NativeCalls.getInstance().killProcess(pid);
-      Files.delete(pidFile);
-    } catch (FileNotFoundException ignore) {
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-  }
-
-  private static void killLocator(final File root, final String name) {
-    killByPidFile(root.toPath().resolve(name).resolve("vf.gf.locator.pid"));
-  }
-
 }
diff --git a/geode-dunit/src/main/java/org/apache/geode/management/internal/configuration/ClusterConfigTestBase.java b/geode-dunit/src/main/java/org/apache/geode/management/internal/configuration/ClusterConfigTestBase.java
index 67a94e623e..76188f362d 100644
--- a/geode-dunit/src/main/java/org/apache/geode/management/internal/configuration/ClusterConfigTestBase.java
+++ b/geode-dunit/src/main/java/org/apache/geode/management/internal/configuration/ClusterConfigTestBase.java
@@ -13,14 +13,16 @@
  * the License.
  *
  */
-
 package org.apache.geode.management.internal.configuration;
 
+import static java.nio.file.Files.createDirectories;
+import static java.nio.file.Files.createFile;
 import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
 import static org.apache.geode.distributed.ConfigurationProperties.USE_CLUSTER_CONFIGURATION;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.Properties;
 
 import org.apache.commons.io.FileUtils;
@@ -29,37 +31,55 @@ import org.junit.Rule;
 
 import org.apache.geode.management.internal.configuration.utils.ZipUtils;
 import org.apache.geode.test.compiler.ClassBuilder;
-import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+import org.apache.geode.test.junit.rules.FolderRule;
 
 public abstract class ClusterConfigTestBase {
-  public String clusterConfigZipPath;
-
-  public static final ConfigGroup CLUSTER = new ConfigGroup("cluster").regions("regionForCluster")
-      .jars("cluster.jar").maxLogFileSize("5000")
-      .configFiles("cluster.properties", "cluster.xml");
-  public static final ConfigGroup GROUP1 = new ConfigGroup("group1").regions("regionForGroup1")
-      .jars("group1.jar").maxLogFileSize("6000").configFiles("group1.properties", "group1.xml");
-  public static final ConfigGroup GROUP2 = new ConfigGroup("group2").regions("regionForGroup2")
-      .jars("group2.jar").maxLogFileSize("7000").configFiles("group2.properties", "group2.xml");
 
-  public static final ClusterConfig CONFIG_FROM_ZIP = new ClusterConfig(CLUSTER, GROUP1, GROUP2);
+  protected String clusterConfigZipPath;
+  protected Path rootFolder;
 
-  public static final ClusterConfig REPLICATED_CONFIG_FROM_ZIP = new ClusterConfig(
-      new ConfigGroup("cluster").maxLogFileSize("5000").jars("cluster.jar")
+  public static final ConfigGroup CLUSTER = new ConfigGroup("cluster")
+      .regions("regionForCluster")
+      .jars("cluster.jar")
+      .maxLogFileSize("5000")
+      .configFiles("cluster.properties", "cluster.xml");
+  public static final ConfigGroup GROUP1 = new ConfigGroup("group1")
+      .regions("regionForGroup1")
+      .jars("group1.jar")
+      .maxLogFileSize("6000")
+      .configFiles("group1.properties", "group1.xml");
+  public static final ConfigGroup GROUP2 = new ConfigGroup("group2")
+      .regions("regionForGroup2")
+      .jars("group2.jar")
+      .maxLogFileSize("7000")
+      .configFiles("group2.properties", "group2.xml");
+
+  protected static final ClusterConfig CONFIG_FROM_ZIP = new ClusterConfig(CLUSTER, GROUP1, GROUP2);
+
+  protected static final ClusterConfig REPLICATED_CONFIG_FROM_ZIP = new ClusterConfig(
+      new ConfigGroup("cluster")
+          .maxLogFileSize("5000")
+          .jars("cluster.jar")
           .regions("regionForCluster"),
-      new ConfigGroup("group1").maxLogFileSize("6000").jars("group1.jar")
+      new ConfigGroup("group1")
+          .maxLogFileSize("6000")
+          .jars("group1.jar")
           .regions("regionForGroup1"),
-      new ConfigGroup("group2").maxLogFileSize("7000").jars("group2.jar")
+      new ConfigGroup("group2")
+          .maxLogFileSize("7000")
+          .jars("group2.jar")
           .regions("regionForGroup2"));
 
   @Rule
-  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+  public FolderRule folderRule = new FolderRule();
 
   protected Properties locatorProps;
   protected Properties serverProps;
 
   @Before
   public void beforeClusterConfigTestBase() throws Exception {
+    rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
+
     clusterConfigZipPath = buildClusterZipFile();
     locatorProps = new Properties();
     serverProps = new Properties();
@@ -70,7 +90,7 @@ public abstract class ClusterConfigTestBase {
   }
 
   private String buildClusterZipFile() throws Exception {
-    File clusterConfigDir = temporaryFolder.newFolder("cluster_config");
+    File clusterConfigDir = createDirectories(rootFolder.resolve("cluster_config")).toFile();
 
     File clusterDir = new File(clusterConfigDir, "cluster");
     String clusterXml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"no\"?>\n"
@@ -103,8 +123,7 @@ public abstract class ClusterConfigTestBase {
     writeFile(group2Dir, "group2.properties", "log-file-size-limit=7000");
     createJarFileWithClass("Group2", "group2.jar", group2Dir);
 
-
-    File clusterConfigZip = new File(temporaryFolder.newFolder(), "cluster_config.zip");
+    File clusterConfigZip = createFile(rootFolder.resolve("cluster_config.zip")).toFile();
     ZipUtils.zipDirectory(clusterConfigDir.getCanonicalPath(), clusterConfigZip.getCanonicalPath());
 
     FileUtils.deleteDirectory(clusterConfigDir);
diff --git a/geode-gfsh/src/acceptanceTest/java/org/apache/geode/gfsh/GfshWithSslAcceptanceTest.java b/geode-gfsh/src/acceptanceTest/java/org/apache/geode/gfsh/GfshWithSslAcceptanceTest.java
index ba0db7313f..619655513f 100644
--- a/geode-gfsh/src/acceptanceTest/java/org/apache/geode/gfsh/GfshWithSslAcceptanceTest.java
+++ b/geode-gfsh/src/acceptanceTest/java/org/apache/geode/gfsh/GfshWithSslAcceptanceTest.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.gfsh;
 
 import static java.lang.String.format;
@@ -28,63 +27,62 @@ import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTOR
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_PASSWORD;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_TYPE;
 
-import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.net.InetAddress;
+import java.nio.file.Path;
 import java.security.GeneralSecurityException;
 import java.util.Properties;
 
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
 import org.apache.geode.cache.ssl.CertStores;
 import org.apache.geode.cache.ssl.CertificateBuilder;
 import org.apache.geode.cache.ssl.CertificateMaterial;
 import org.apache.geode.internal.UniquePortSupplier;
+import org.apache.geode.test.junit.rules.FolderRule;
 import org.apache.geode.test.junit.rules.gfsh.GfshRule;
 
 public class GfshWithSslAcceptanceTest {
+
   private static final String CERTIFICATE_ALGORITHM = "SHA256withRSA";
   private static final int CERTIFICATE_EXPIRATION_IN_DAYS = 1;
   private static final String STORE_PASSWORD = "geode";
   private static final String STORE_TYPE = "jks";
 
-  private final String startLocator;
-  private final String connect;
+  @Rule(order = 0)
+  public FolderRule folderRule = new FolderRule();
+  @Rule(order = 1)
+  public GfshRule gfshRule = new GfshRule(folderRule::getFolder);
 
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder();
+  private String startLocator;
+  private String connect;
+  private Path keyStoreFile;
+  private Path trustStoreFile;
+  private Path securityPropertiesFile;
 
-  @Rule
-  public final GfshRule gfsh;
+  @Before
+  public void setUp() throws IOException, GeneralSecurityException {
+    Path rootFolder = folderRule.getFolder().toPath().toAbsolutePath();
 
-  private final File keyStoreFile;
-  private final File trustStoreFile;
-  private final File securityPropertiesFile;
+    keyStoreFile = rootFolder.resolve("keyStoreFile");
+    trustStoreFile = rootFolder.resolve("trustStoreFile");
+    securityPropertiesFile = rootFolder.resolve("securityPropertiesFile");
 
-  public GfshWithSslAcceptanceTest() throws IOException,
-      GeneralSecurityException {
-    gfsh = new GfshRule();
+    final String hostName = InetAddress.getLocalHost().getCanonicalHostName();
+    generateKeyAndTrustStore(hostName, keyStoreFile, trustStoreFile);
 
     final UniquePortSupplier portSupplier = new UniquePortSupplier();
     final int port = portSupplier.getAvailablePort();
 
-    tempFolder.create();
-    keyStoreFile = tempFolder.newFile();
-    trustStoreFile = tempFolder.newFile();
-    securityPropertiesFile = tempFolder.newFile();
-
-    final String hostName = InetAddress.getLocalHost().getCanonicalHostName();
-    generateKeyAndTrustStore(hostName, keyStoreFile, trustStoreFile);
-
     startLocator = format(
         "start locator --connect=false --http-service-port=0 --name=locator --bind-address=%s --port=%d --J=-Dgemfire.jmx-manager-port=%d --security-properties-file=%s",
         hostName, port, portSupplier.getAvailablePort(),
-        securityPropertiesFile.getAbsolutePath());
+        securityPropertiesFile);
     connect = format("connect --locator=%s[%d] --security-properties-file=%s", hostName, port,
-        securityPropertiesFile.getAbsolutePath());
+        securityPropertiesFile);
   }
 
   @Test
@@ -92,21 +90,12 @@ public class GfshWithSslAcceptanceTest {
     generateSecurityProperties(true, securityPropertiesFile, keyStoreFile,
         trustStoreFile);
 
-    gfsh.execute(startLocator);
-    gfsh.execute(connect);
-  }
-
-  // @Test
-  public void gfshCanConnectViaSslWithEndpointIdentificationDisabled() throws IOException {
-    generateSecurityProperties(false, securityPropertiesFile, keyStoreFile,
-        trustStoreFile);
-
-    gfsh.execute(startLocator);
-    gfsh.execute(connect);
+    gfshRule.execute(startLocator);
+    gfshRule.execute(connect);
   }
 
-  public static void generateKeyAndTrustStore(final String hostName, final File keyStoreFile,
-      final File trustStoreFile) throws IOException, GeneralSecurityException {
+  private static void generateKeyAndTrustStore(final String hostName, final Path keyStoreFile,
+      final Path trustStoreFile) throws IOException, GeneralSecurityException {
     final CertificateMaterial ca =
         new CertificateBuilder(CERTIFICATE_EXPIRATION_IN_DAYS, CERTIFICATE_ALGORITHM)
             .commonName("Test CA")
@@ -124,12 +113,12 @@ public class GfshWithSslAcceptanceTest {
     store.withCertificate("geode", certificate);
     store.trust("ca", ca);
 
-    store.createKeyStore(keyStoreFile.getAbsolutePath(), STORE_PASSWORD);
-    store.createTrustStore(trustStoreFile.getAbsolutePath(), STORE_PASSWORD);
+    store.createKeyStore(keyStoreFile, STORE_PASSWORD);
+    store.createTrustStore(trustStoreFile, STORE_PASSWORD);
   }
 
   private static void generateSecurityProperties(final boolean endpointIdentificationEnabled,
-      final File securityPropertiesFile, final File keyStoreFile, final File trustStoreFile)
+      final Path securityPropertiesFile, final Path keyStoreFile, final Path trustStoreFile)
       throws IOException {
     final Properties properties = new Properties();
 
@@ -139,15 +128,14 @@ public class GfshWithSslAcceptanceTest {
         valueOf(endpointIdentificationEnabled));
     properties.setProperty(SSL_PROTOCOLS, "any");
 
-    properties.setProperty(SSL_KEYSTORE, keyStoreFile.getAbsolutePath());
+    properties.setProperty(SSL_KEYSTORE, keyStoreFile.toString());
     properties.setProperty(SSL_KEYSTORE_TYPE, STORE_TYPE);
     properties.setProperty(SSL_KEYSTORE_PASSWORD, STORE_PASSWORD);
 
-    properties.setProperty(SSL_TRUSTSTORE, trustStoreFile.getAbsolutePath());
+    properties.setProperty(SSL_TRUSTSTORE, trustStoreFile.toString());
     properties.setProperty(SSL_TRUSTSTORE_TYPE, STORE_TYPE);
     properties.setProperty(SSL_TRUSTSTORE_PASSWORD, STORE_PASSWORD);
 
-    properties.store(new FileWriter(securityPropertiesFile), null);
+    properties.store(new FileWriter(securityPropertiesFile.toFile()), null);
   }
-
 }
diff --git a/geode-junit/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt b/geode-junit/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
index 28b0f22776..081e659180 100644
--- a/geode-junit/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
+++ b/geode-junit/src/integrationTest/resources/org/apache/geode/codeAnalysis/excludedClasses.txt
@@ -1,3 +1,4 @@
 org/apache/geode/cache/query/data/PortfolioNoDS
 org/apache/geode/cache/query/data/PortfolioPdx
 org/apache/geode/cache/query/data/PortfolioPdx$Day
+org/apache/geode/test/junit/rules/ResourceReporterRule$Policy
diff --git a/geode-junit/src/main/java/org/apache/geode/cache/ssl/CertStores.java b/geode-junit/src/main/java/org/apache/geode/cache/ssl/CertStores.java
index b82257af0c..073597947b 100644
--- a/geode-junit/src/main/java/org/apache/geode/cache/ssl/CertStores.java
+++ b/geode-junit/src/main/java/org/apache/geode/cache/ssl/CertStores.java
@@ -31,6 +31,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.file.Files;
+import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.security.GeneralSecurityException;
 import java.security.KeyStore;
@@ -110,6 +111,14 @@ public class CertStores {
         keyStoreFile, keyStorePassword, requireAuth, endPointIdentification);
   }
 
+  public static Properties propertiesWith(String components, String protocols, String ciphers,
+      Path trustStoreFile, String trustStorePassword, Path keyStoreFile, String keyStorePassword,
+      boolean requireAuth, boolean endPointVerification) {
+    return propertiesWith(components, protocols, ciphers, trustStoreFile.toFile(),
+        trustStorePassword, keyStoreFile.toFile(), keyStorePassword, requireAuth,
+        endPointVerification);
+  }
+
   public static Properties propertiesWith(String components, String protocols, String ciphers,
       File trustStoreFile, String trustStorePassword, File keyStoreFile, String keyStorePassword,
       boolean requireAuth, boolean endPointVerification) {
@@ -133,8 +142,13 @@ public class CertStores {
 
   public void createTrustStore(String filename, String password)
       throws GeneralSecurityException, IOException {
+    createTrustStore(Paths.get(filename), password);
+  }
+
+  public void createTrustStore(Path filePath, String password)
+      throws GeneralSecurityException, IOException {
     KeyStore ks = KeyStore.getInstance("JKS");
-    try (InputStream in = Files.newInputStream(Paths.get(filename))) {
+    try (InputStream in = Files.newInputStream(filePath)) {
       ks.load(in, password.toCharArray());
     } catch (IOException e) {
       ks = createEmptyKeyStore();
@@ -143,13 +157,18 @@ public class CertStores {
       ks.setCertificateEntry(cert.getKey(), cert.getValue().getCertificate());
     }
 
-    try (OutputStream out = Files.newOutputStream(Paths.get(filename))) {
+    try (OutputStream out = Files.newOutputStream(filePath)) {
       ks.store(out, password.toCharArray());
     }
   }
 
   public void createKeyStore(String filename, String password)
       throws GeneralSecurityException, IOException {
+    createKeyStore(Paths.get(filename), password);
+  }
+
+  public void createKeyStore(Path filePath, String password)
+      throws GeneralSecurityException, IOException {
     KeyStore ks = createEmptyKeyStore();
 
     for (Map.Entry<String, CertificateMaterial> entry : keyStoreEntries.entrySet()) {
@@ -163,7 +182,7 @@ public class CertStores {
       ks.setKeyEntry(entry.getKey(), cert.getPrivateKey(), password.toCharArray(),
           chain.toArray(new Certificate[] {}));
     }
-    try (OutputStream out = Files.newOutputStream(Paths.get(filename))) {
+    try (OutputStream out = Files.newOutputStream(filePath)) {
       ks.store(out, password.toCharArray());
     }
   }
diff --git a/geode-junit/src/main/java/org/apache/geode/test/junit/rules/Folder.java b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/Folder.java
new file mode 100644
index 0000000000..539bb6419f
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/Folder.java
@@ -0,0 +1,51 @@
+/*
+ * 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.junit.rules;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.commons.io.FileUtils.deleteDirectory;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+
+public class Folder {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final Path root;
+
+  public Folder(Path root) throws IOException {
+    requireNonNull(root);
+    this.root = Files.createDirectories(root.toAbsolutePath().normalize());
+  }
+
+  public Path toPath() {
+    return root;
+  }
+
+  public void delete() throws IOException {
+    try {
+      deleteDirectory(root.toFile());
+    } catch (IOException | UncheckedIOException e) {
+      logger.error("Unable to delete directory " + root.toFile(), e);
+    }
+  }
+}
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/FolderFactory.java
similarity index 54%
copy from geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java
copy to geode-junit/src/main/java/org/apache/geode/test/junit/rules/FolderFactory.java
index e2ebd32a3f..905119ce3d 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/cli/shell/GfshDisconnectWithinScript.java
+++ b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/FolderFactory.java
@@ -12,27 +12,27 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.management.internal.cli.shell;
+package org.apache.geode.test.junit.rules;
 
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Paths;
 
-import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+import org.junit.runner.Description;
 
-public class GfshDisconnectWithinScript {
+public class FolderFactory {
 
-  @Rule
-  public GfshRule gfsh = new GfshRule();
-
-
-  @Before
-  public void setup() {
-    gfsh.execute("start locator");
+  public static Folder create(Description description) {
+    String className = description.getTestClass().getSimpleName();
+    String methodName = sanitizeForFolderName(description.getMethodName());
+    try {
+      return new Folder(Paths.get(className, methodName));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
   }
 
-  @Test
-  public void disconnectInScriptDoesNotRaiseNPE() {
-    gfsh.execute("connect", "disconnect", "echo \"Disconnect command resolved without issue.\"");
+  private static String sanitizeForFolderName(String methodName) {
+    return methodName.replaceAll("[ ,]+", "-");
   }
 }
diff --git a/geode-junit/src/main/java/org/apache/geode/test/junit/rules/FolderRule.java b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/FolderRule.java
new file mode 100644
index 0000000000..43fba6739b
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/FolderRule.java
@@ -0,0 +1,97 @@
+/*
+ * 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.junit.rules;
+
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.AssumptionViolatedException;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.MultipleFailureException;
+import org.junit.runners.model.Statement;
+
+public class FolderRule implements TestRule {
+
+  public enum Policy {
+    DELETE_ON_PASS,
+    KEEP_ALWAYS
+  }
+
+  private final List<Throwable> errors = new ArrayList<>();
+
+  private final Policy policy;
+
+  private Folder folder;
+
+  public FolderRule() {
+    this(Policy.DELETE_ON_PASS);
+  }
+
+  public FolderRule(Policy policy) {
+    this.policy = policy;
+  }
+
+  public Folder getFolder() {
+    return folder;
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    return new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+        createFolder(description);
+        try {
+          base.evaluate();
+        } catch (MultipleFailureException e) {
+          errors.addAll(e.getFailures());
+        } catch (Throwable e) {
+          errors.add(e);
+        } finally {
+          try {
+            if (policy == Policy.DELETE_ON_PASS && passed()) {
+              await()
+                  .ignoreExceptionsInstanceOf(IOException.class)
+                  .untilAsserted(() -> folder.delete());
+            }
+          } catch (Throwable e) {
+            errors.add(e);
+          }
+        }
+
+        MultipleFailureException.assertEmpty(errors);
+      }
+    };
+  }
+
+  private boolean passed() {
+    return errors.stream().allMatch(t -> t instanceof AssumptionViolatedException);
+  }
+
+  private void createFolder(Description description) throws IOException {
+    String className = description.getTestClass().getSimpleName();
+    String methodName = sanitizeForFolderName(description.getMethodName());
+    folder = new Folder(Paths.get(className, methodName));
+  }
+
+  private String sanitizeForFolderName(String methodName) {
+    return methodName.replaceAll("[ ,]+", "-");
+  }
+}
diff --git a/geode-junit/src/main/java/org/apache/geode/test/junit/rules/ResourceReporterRule.java b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/ResourceReporterRule.java
new file mode 100644
index 0000000000..99c3ef5d5a
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/ResourceReporterRule.java
@@ -0,0 +1,100 @@
+/*
+ * 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.junit.rules;
+
+import static java.util.Collections.synchronizedList;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Consumer;
+
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.MultipleFailureException;
+import org.junit.runners.model.Statement;
+
+
+public class ResourceReporterRule implements TestRule {
+
+  private final List<Throwable> errors = synchronizedList(new ArrayList<>());
+  private final List<Runnable> actions = synchronizedList(new ArrayList<>());
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    return new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+        try {
+          // initializeFolder();
+          base.evaluate();
+        } catch (MultipleFailureException e) {
+          // errors.addAll(e.getFailures());
+        } catch (Throwable e) {
+          // errors.add(e);
+        } finally {
+          try {
+            // cleanupGfshContexts();
+          } catch (Throwable e) {
+            // errors.add(e);
+          }
+        }
+
+        MultipleFailureException.assertEmpty(errors);
+      }
+    };
+  }
+
+  private enum Policy {
+    BEFORE,
+    AFTER,
+    BOTH
+  }
+
+  public ResourceReporterRule.Builder builder() {
+    return null; // return new ResourceReporterRule.Builder(e - > errors.add(e));
+  }
+
+  public static class Builder {
+
+    private final Map<Policy, List<Runnable>> actions = new HashMap<>();
+    private final Consumer<Throwable> thrown;
+
+    private Path dir;
+
+    Builder(Consumer<Throwable> thrown) {
+      this.thrown = thrown;
+    }
+
+    public Builder doBefore(Runnable action) {
+      return this;
+    }
+
+    public Builder doAfter(Runnable action) {
+      return this;
+    }
+
+    public Builder doBeforeAndAfter(Runnable action) {
+      return this;
+    }
+
+    public ResourceReporterRule build(Path dir) {
+      this.dir = dir;
+      return new ResourceReporterRule();
+    }
+  }
+}
diff --git a/geode-junit/src/main/java/org/apache/geode/test/junit/rules/gfsh/GfshContext.java b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/gfsh/GfshContext.java
new file mode 100644
index 0000000000..6551e17081
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/junit/rules/gfsh/GfshContext.java
@@ -0,0 +1,305 @@
+/*
+ * 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.junit.rules.gfsh;
+
+import static java.io.File.pathSeparator;
+import static java.nio.file.Files.createDirectories;
+import static java.nio.file.Files.exists;
+import static java.util.Collections.emptySet;
+import static java.util.Collections.synchronizedList;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.geode.internal.lang.SystemUtils.isWindows;
+import static org.apache.geode.internal.process.ProcessType.LOCATOR;
+import static org.apache.geode.internal.process.ProcessType.SERVER;
+import static org.apache.geode.internal.process.ProcessUtils.identifyPidAsUnchecked;
+import static org.apache.geode.internal.process.ProcessUtils.isProcessAlive;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Consumer;
+import java.util.function.IntConsumer;
+
+import org.apache.geode.internal.process.NativeProcessUtils;
+import org.apache.geode.internal.process.ProcessType;
+import org.apache.geode.internal.process.ProcessUtils;
+import org.apache.geode.internal.process.ProcessUtilsProvider;
+import org.apache.geode.test.junit.rules.RequiresGeodeHome;
+import org.apache.geode.test.version.VersionManager;
+import org.apache.geode.test.version.VmConfiguration;
+
+public class GfshContext implements GfshExecutor {
+
+  private final List<GfshExecution> gfshExecutions = synchronizedList(new ArrayList<>());
+  private final List<String> jvmOptions;
+
+  private final Consumer<Throwable> thrown;
+  private final IntConsumer processKiller;
+
+  private final Path javaHome;
+  private final Path gfshPath;
+  private final Path dir;
+
+  private GfshContext(Builder builder) {
+    thrown = builder.thrown;
+    javaHome = builder.javaHome;
+    gfshPath = findGfsh(builder.geodeVersion);
+    dir = builder.dir;
+    processKiller = builder.processKiller;
+    jvmOptions = new ArrayList<>(builder.jvmOptions);
+  }
+
+  void killProcesses() {
+    // kill all server processes
+    getPidFiles(SERVER).stream()
+        .map(ProcessUtils::readPid)
+        .forEach(this::killProcess);
+
+    // kill all locator processes
+    getPidFiles(LOCATOR).stream()
+        .map(ProcessUtils::readPid)
+        .forEach(this::killProcess);
+
+    // kill all gfsh processes
+    gfshExecutions
+        .forEach(this::killProcess);
+  }
+
+  @Override
+  public GfshExecution execute(String... commands) {
+    return execute(GfshScript.of(commands));
+  }
+
+  @Override
+  public GfshExecution execute(GfshScript gfshScript) {
+    try {
+      Path scriptPath = dir.resolve(gfshScript.getName());
+      File workingDir = createDirectories(scriptPath).toFile();
+      return execute(workingDir, gfshScript);
+    } catch (IOException e) {
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override
+  public GfshExecution execute(File workingDir, String... commands) {
+    return execute(workingDir.toPath(), GfshScript.of(commands));
+  }
+
+  @Override
+  public GfshExecution execute(File workingDir, GfshScript gfshScript) {
+    return execute(workingDir.toPath(), gfshScript);
+  }
+
+  @Override
+  public GfshExecution execute(Path workingDir, String... commands) {
+    return execute(workingDir, GfshScript.of(commands));
... 1344 lines suppressed ...