You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bu...@apache.org on 2020/06/08 21:14:52 UTC

[geode-benchmarks] branch develop updated: TLS SNI support via "withSniProxy" Option (#129)

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

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


The following commit(s) were added to refs/heads/develop by this push:
     new 46e8469  TLS SNI support via "withSniProxy" Option (#129)
46e8469 is described below

commit 46e84693e0abf8c33c717c41dd770f0a2f93640c
Author: Bill Burcham <bi...@gmail.com>
AuthorDate: Mon Jun 8 14:14:41 2020 -0700

    TLS SNI support via "withSniProxy" Option (#129)
---
 README.md                                          |  32 +++-
 geode-benchmarks/build.gradle                      |   6 +
 .../java/org/apache/geode/benchmark/Config.java    |  53 ++++++
 .../benchmark/parameters/GcLoggingParameters.java  |   6 +-
 .../geode/benchmark/parameters/GcParameters.java   |  10 +-
 .../benchmark/parameters/GeodeProperties.java      |   6 +-
 .../geode/benchmark/parameters/HeapParameters.java |   4 +-
 .../geode/benchmark/parameters/JvmParameters.java  |   4 +-
 .../benchmark/parameters/ProfilerParameters.java   |   4 +-
 .../apache/geode/benchmark/parameters/Utils.java   |  34 +++-
 .../tasks/DefineHostNamingsOffPlatformTask.java    |  69 ++++++++
 .../apache/geode/benchmark/tasks/LocatorUtil.java  |   4 +-
 .../geode/benchmark/tasks/PrePopulateRegion.java   |  12 +-
 .../benchmark/tasks/PrePopulateRegionLong.java     |  12 +-
 .../ProcessControl.java}                           |  24 +--
 .../apache/geode/benchmark/tasks/StartClient.java  |  32 +++-
 .../geode/benchmark/tasks/StartClientSNI.java      |  97 +++++++++++
 .../apache/geode/benchmark/tasks/StartLocator.java |  10 ++
 .../{LocatorUtil.java => StartLocatorSNI.java}     |  29 +++-
 .../apache/geode/benchmark/tasks/StartServer.java  |  17 +-
 .../{LocatorUtil.java => StartServerSNI.java}      |  28 +++-
 .../geode/benchmark/tasks/StartSniProxy.java       | 182 +++++++++++++++++++++
 .../tasks/{LocatorUtil.java => StopClient.java}    |  18 +-
 .../tasks/{LocatorUtil.java => StopSniProxy.java}  |  24 ++-
 .../benchmark/tests/AbstractFunctionBenchmark.java |  13 +-
 .../AbstractPartitionedFunctionBenchmark.java      |   5 +-
 .../tests/AbstractReplicatedFunctionBenchmark.java |   5 +-
 .../geode/benchmark/tests/GeodeBenchmark.java      |  22 ++-
 .../geode/benchmark/tests/NoopBenchmark.java       |   7 +-
 .../PartitionedFunctionExecutionBenchmark.java     |   5 +-
 ...nedFunctionExecutionWithArgumentsBenchmark.java |   5 +-
 ...ionedFunctionExecutionWithFiltersBenchmark.java |   5 +-
 .../benchmark/tests/PartitionedGetBenchmark.java   |  16 +-
 .../tests/PartitionedGetLongBenchmark.java         |  16 +-
 .../tests/PartitionedIndexedQueryBenchmark.java    |  18 +-
 .../tests/PartitionedNonIndexedQueryBenchmark.java |  16 +-
 .../tests/PartitionedPutAllBenchmark.java          |  16 +-
 .../tests/PartitionedPutAllLongBenchmark.java      |  16 +-
 .../benchmark/tests/PartitionedPutBenchmark.java   |  20 +--
 .../tests/PartitionedPutLongBenchmark.java         |  17 +-
 .../ReplicatedFunctionExecutionBenchmark.java      |   5 +-
 ...tedFunctionExecutionWithArgumentsBenchmark.java |   5 +-
 ...catedFunctionExecutionWithFiltersBenchmark.java |   5 +-
 .../benchmark/tests/ReplicatedGetBenchmark.java    |  16 +-
 .../tests/ReplicatedGetLongBenchmark.java          |  16 +-
 .../tests/ReplicatedIndexedQueryBenchmark.java     |  18 +-
 .../tests/ReplicatedNonIndexedQueryBenchmark.java  |  17 +-
 .../benchmark/tests/ReplicatedPutAllBenchmark.java |  16 +-
 .../tests/ReplicatedPutAllLongBenchmark.java       |  16 +-
 .../benchmark/tests/ReplicatedPutBenchmark.java    |  16 +-
 .../tests/ReplicatedPutLongBenchmark.java          |  16 +-
 .../benchmark/topology/ClientServerTopology.java   |  81 +++------
 .../topology/ClientServerTopologyWithSNIProxy.java |  81 +++++++++
 .../benchmark/topology/HostNamingOffPlatform.java  |  20 +--
 .../Ports.java}                                    |  24 +--
 .../RoleKinds.java}                                |  15 +-
 .../Roles.java}                                    |  34 ++--
 .../parameters/GcLoggingParametersTest.java        |  36 ++--
 .../benchmark/parameters/GcParametersTest.java     |  66 ++++----
 .../benchmark/parameters/HeapParametersTest.java   |  18 +-
 .../geode/benchmark/tasks/StartSniProxyTest.java   |  64 ++++++++
 .../geode/benchmark/tests/GeodeBenchmarkTest.java  |  81 +++++++++
 .../topology/ClientServerTopologyTest.java         |  14 +-
 .../ClientServerTopologyWithSNIProxyTest.java}     |  44 ++---
 harness/build.gradle                               |   5 +
 .../main/java/org/apache/geode/perftest/Task.java  |   1 +
 .../java/org/apache/geode/perftest/TestConfig.java |  27 ---
 .../java/org/apache/geode/perftest/TestStep.java   |  62 +++++++
 .../infrastructure/local/LocalInfrastructure.java  |   2 +-
 .../apache/geode/perftest/jvms/JVMLauncher.java    |   2 +-
 .../geode/perftest/runner/DefaultTestRunner.java   |   6 +-
 .../scripts/aws/image/files/docker-compose.yml     |  26 +++
 infrastructure/scripts/aws/image/packer.json       |  31 ++++
 73 files changed, 1332 insertions(+), 473 deletions(-)

diff --git a/README.md b/README.md
index 28f1661..77d9d55 100644
--- a/README.md
+++ b/README.md
@@ -19,7 +19,7 @@ versions default to a new format. PEM format can be forced by using `-m PEM`:
 ```
 ssh-keygen -m PEM -t rsa
 ```
-While runinng a test on a single machine (i.e. localhost) add the generated key to `authorized_keys` to authorize the user:
+While running a test on a single machine (i.e. localhost) add the generated key to `authorized_keys` to authorize the user:
 ```
 cat <your_public_key_file> >> ~/.ssh/authorized_keys
 ```
@@ -147,3 +147,33 @@ public class PutTask extends BenchmarkDriverAdapter implements Serializable {
   }
 }
 ```
+
+## SNI
+
+On AWS, you can run any benchmark on a topology that routes all client-server communication through an SNI proxy (HAproxy).
+ 
+To run a test, e.g. `PartitionedGetBenchmark`, with SNI:
+
+`./run_tests.sh -t anytagname -- -PwithSniProxy '--tests=PartitionedGetBenchmark'`
+
+Since SNI is a feature of TLS, running with the SNI topology incurs TLS overheads.
+
+### TODO for SNI
+* ~~verify `StartSniProxy` runs on proxy node~~
+* ~~don't require operator to supply `-PwithSSL`/`-DwithSSL=true` when running SNI tests~~
+* ~~generate `haproxy.cfg` with client-visible SNI hostnames~~
+* ~~turn on SNI via `setPoolSocketFactory` in a new `StartClientSNI` task~~
+* ~~set `--hostname-for-clients` on locator and servers for SNI~~
+* ~~reinstate thread-per-core in `PrePopulateRegion.run()` and in `PartitionedPutBenchmark[SNI]` ya~~
+* ~~set `keyRange` back to 1e6 in `PartitionedPutBenchmark[SNI]` after client-server connections are healthy~~
+* ~~make topology orthogonal to tests so all tests can run with SNI; have a `-PwithSniProxy`/`-DwithSniProxy=true` flag~~
+* Potential performance improvement: HAproxy as configured runs one process with the max threads-per-process of 64 threads, ostensibly using 64/72 cores (89%.) We might be able to improve performance by configuring HAproxy to run in daemon mode where we can run two processes, each multithreaded, to run more than 64 threads, thereby utilizing 100% of our cores.  
+
+## TODO (General)
+* add logic to clean up existing locator.dat files before running a locator on a node
+* eliminate `harness` module dependency on Geode by moving Geode keystore/truststore setting out of `harness` module and up into `geode-benchmarks` i.e. set 'em in properties sent to `Locator.startLocatorAndDS` in `StartLocator`, `StartServer`
+* move `docker-compose.yml` distribution out of `harness` module up into `geode-benchmarks` so it gets distributed whenever it changes (without requiring rebuilding AWS AMI and cluster on AWS) 
+* generate 2048-bit keys (instead of 1024-bit ones) for TLS; will slow TLS handshakes which may necessitate a new baseline
+* make `StartServer` task use `ServerLauncher` (instead of `CacheFactory`) for symmetry with `LocatorLauncher`&mdash;also too: encapsulation!
+* `./run_tests.sh` sometimes seems to hang after benchmarks have completed, requiring operator to enter ^C to un-stick it
+* make `rsync:` Git "scheme" work in `run_tests.sh` script for benchmark repo (not just for geode repo)
diff --git a/geode-benchmarks/build.gradle b/geode-benchmarks/build.gradle
index 4078ada..c9c08d0 100644
--- a/geode-benchmarks/build.gradle
+++ b/geode-benchmarks/build.gradle
@@ -31,7 +31,12 @@ def getDate() {
 
 
 repositories {
+  /*
+   This is used in CI to benchmark various new/old versions of Geode.
+   Also useful in dev where you can clone geode and publishToMavenLocal
+   */
   mavenLocal()
+  // fall back to mavenCentral, which has lots of released versions of Geode
   mavenCentral()
 }
 
@@ -87,6 +92,7 @@ task benchmark(type: Test) {
     systemProperty 'withDuration', project.findProperty('withDuration')
   }
   systemProperty 'withSsl', project.hasProperty('withSsl')
+  systemProperty 'withSniProxy', project.hasProperty('withSniProxy')
   systemProperty 'withSecurityManager', project.hasProperty('withSecurityManager')
   systemProperty 'benchmark.profiler.argument', project.findProperty('benchmark.profiler.argument')
 
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/Config.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/Config.java
new file mode 100644
index 0000000..aa55633
--- /dev/null
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/Config.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.benchmark;
+
+import org.yardstickframework.BenchmarkDriver;
+
+import org.apache.geode.benchmark.topology.Roles;
+import org.apache.geode.perftest.Task;
+import org.apache.geode.perftest.TestConfig;
+
+/**
+ * This is a more strongly-typed interface to the (stringly-typed) TestConfig interface in the
+ * benchmark driver. This lets callers use the Roles enum instead of strings.
+ */
+public class Config {
+  private Config() {}
+
+  public static void role(final TestConfig config, final Roles role, final int numberOfInstances) {
+    config.role(role.name(), numberOfInstances);
+  }
+
+  public static void before(final TestConfig config, final Task task, final Roles role) {
+    config.before(task, role.name());
+  }
+
+  public static void workload(final TestConfig config, final BenchmarkDriver task,
+      final Roles role) {
+    config.workload(task, role.name());
+  }
+
+  public static void after(final TestConfig config, final Task task, final Roles role) {
+    config.after(task, role.name());
+  }
+
+  public static void jvmArgs(final TestConfig config, final Roles role, final String... jvmArgs) {
+    config.jvmArgs(role.name(), jvmArgs);
+  }
+}
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GcLoggingParameters.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GcLoggingParameters.java
index 9ea990f..e19482e 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GcLoggingParameters.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GcLoggingParameters.java
@@ -16,7 +16,7 @@
 package org.apache.geode.benchmark.parameters;
 
 import static org.apache.geode.benchmark.parameters.JavaVersion.v11;
-import static org.apache.geode.benchmark.parameters.Utils.configureAll;
+import static org.apache.geode.benchmark.parameters.Utils.configureGeodeProductJvms;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,9 +30,9 @@ public class GcLoggingParameters {
     final JavaVersion javaVersion = JavaVersion.current();
     logger.info("Configuring GC logging parameters for Java {}.", javaVersion);
     if (javaVersion.atLeast(v11)) {
-      configureAll(testConfig, "-Xlog:gc*:OUTPUT_DIR/gc.log");
+      configureGeodeProductJvms(testConfig, "-Xlog:gc*:OUTPUT_DIR/gc.log");
     } else {
-      configureAll(testConfig,
+      configureGeodeProductJvms(testConfig,
           "-XX:+PrintGCDetails",
           "-XX:+PrintGCTimeStamps",
           "-XX:+PrintGCDateStamps",
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GcParameters.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GcParameters.java
index 3fc1470..6239c0a 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GcParameters.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GcParameters.java
@@ -16,7 +16,7 @@
 package org.apache.geode.benchmark.parameters;
 
 import static org.apache.geode.benchmark.parameters.JavaVersion.v11;
-import static org.apache.geode.benchmark.parameters.Utils.configureAll;
+import static org.apache.geode.benchmark.parameters.Utils.configureGeodeProductJvms;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,7 +47,7 @@ public class GcParameters {
   }
 
   private static void configureShenandoah(final TestConfig testConfig) {
-    configureAll(testConfig,
+    configureGeodeProductJvms(testConfig,
         "-XX:+UnlockExperimentalVMOptions",
         "-XX:+UseShenandoahGC",
         "-XX:+AlwaysPreTouch",
@@ -59,20 +59,20 @@ public class GcParameters {
     if (javaVersion.olderThan(v11)) {
       throw new IllegalArgumentException("ZGC requires Java 11 or newer");
     }
-    configureAll(testConfig,
+    configureGeodeProductJvms(testConfig,
         "-XX:+UnlockExperimentalVMOptions",
         "-XX:+UseZGC");
   }
 
   private static void configureG1(final TestConfig testConfig) {
-    configureAll(testConfig,
+    configureGeodeProductJvms(testConfig,
         "-XX:+UseG1GC",
         "-XX:+UseNUMA",
         "-XX:+ScavengeBeforeFullGC");
   }
 
   private static void configureCms(final TestConfig testConfig) {
-    configureAll(testConfig,
+    configureGeodeProductJvms(testConfig,
         "-XX:+UseConcMarkSweepGC",
         "-XX:+UseCMSInitiatingOccupancyOnly",
         "-XX:+CMSClassUnloadingEnabled",
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GeodeProperties.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GeodeProperties.java
index f3b8231..431f117 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GeodeProperties.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GeodeProperties.java
@@ -101,9 +101,9 @@ public class GeodeProperties {
     return isPropertySet("withSsl");
   }
 
-  private static boolean isPropertySet(String withSecurityManager) {
-    String withSecurityManagerArg = System.getProperty(withSecurityManager);
-    return withSecurityManagerArg != null && withSecurityManagerArg.equals("true");
+  private static boolean isPropertySet(final String propertyName) {
+    final String propertyValue = System.getProperty(propertyName);
+    return propertyValue != null && propertyValue.equals("true");
   }
 
   private static Properties withOptions(Properties properties) {
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/HeapParameters.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/HeapParameters.java
index 0ef39c6..fd58b6a 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/HeapParameters.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/HeapParameters.java
@@ -15,7 +15,7 @@
 
 package org.apache.geode.benchmark.parameters;
 
-import static org.apache.geode.benchmark.parameters.Utils.configureAll;
+import static org.apache.geode.benchmark.parameters.Utils.configureGeodeProductJvms;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -28,7 +28,7 @@ public class HeapParameters {
   public static void configure(final TestConfig testConfig) {
     final String heap = System.getProperty("withHeap", "8g");
     logger.info("Configuring heap parameters {}.", heap);
-    configureAll(testConfig, "-Xmx" + heap, "-Xms" + heap);
+    configureGeodeProductJvms(testConfig, "-Xmx" + heap, "-Xms" + heap);
   }
 
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/JvmParameters.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/JvmParameters.java
index 3705fbe..d26a11a 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/JvmParameters.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/JvmParameters.java
@@ -15,7 +15,7 @@
 
 package org.apache.geode.benchmark.parameters;
 
-import static org.apache.geode.benchmark.parameters.Utils.configureAll;
+import static org.apache.geode.benchmark.parameters.Utils.configureGeodeProductJvms;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -28,7 +28,7 @@ public class JvmParameters {
   public static void configure(final TestConfig testConfig) {
     logger.info("Configuring JVM parameters.");
 
-    configureAll(testConfig,
+    configureGeodeProductJvms(testConfig,
         "-server",
         "-Djava.awt.headless=true",
         "-Dsun.rmi.dgc.server.gcInterval=9223372036854775806",
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/ProfilerParameters.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/ProfilerParameters.java
index 5f8fdcf..6a48ba8 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/ProfilerParameters.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/ProfilerParameters.java
@@ -16,7 +16,7 @@
 package org.apache.geode.benchmark.parameters;
 
 import static com.google.common.base.Strings.isNullOrEmpty;
-import static org.apache.geode.benchmark.parameters.Utils.configureAll;
+import static org.apache.geode.benchmark.parameters.Utils.configureGeodeProductJvms;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,7 +30,7 @@ public class ProfilerParameters {
     final String profilerArgument = System.getProperty("benchmark.profiler.argument");
     if (!isNullOrEmpty(profilerArgument)) {
       logger.info("Configuring profiler parameter. {}", profilerArgument);
-      configureAll(testConfig, profilerArgument);
+      configureGeodeProductJvms(testConfig, profilerArgument);
     }
   }
 
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/Utils.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/Utils.java
index 0b06f58..3475893 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/Utils.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/Utils.java
@@ -15,17 +15,35 @@
 
 package org.apache.geode.benchmark.parameters;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.LOCATOR;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.jvmArgs;
+import static org.apache.geode.benchmark.topology.RoleKinds.GEODE_PRODUCT;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.geode.benchmark.topology.Roles;
 import org.apache.geode.perftest.TestConfig;
 
-public interface Utils {
+public class Utils {
+
+  static final Logger logger =
+      LoggerFactory.getLogger(Utils.class);
+
+  private Utils() {}
+
+  /**
+   * We have many settings we want to apply to JVMs that are hosting Geode. Not all JVMs
+   * host Geode. This method applies the setting to only the Geode product JVMs.
+   */
+  public static void configureGeodeProductJvms(final TestConfig config, final String... args) {
+    Roles.rolesFor(GEODE_PRODUCT).forEach(role -> jvmArgs(config, role, args));
+  }
 
-  static void configureAll(TestConfig testConfig, String... args) {
-    testConfig.jvmArgs(LOCATOR, args);
-    testConfig.jvmArgs(SERVER, args);
-    testConfig.jvmArgs(CLIENT, args);
+  public static void addToTestConfig(TestConfig testConfig, String systemPropertyKey,
+      String jvmArgument) {
+    if (Boolean.getBoolean(systemPropertyKey)) {
+      logger.info("Configuring JVMs to run with " + jvmArgument);
+      configureGeodeProductJvms(testConfig, jvmArgument);
+    }
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/DefineHostNamingsOffPlatformTask.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/DefineHostNamingsOffPlatformTask.java
new file mode 100644
index 0000000..824c9c1
--- /dev/null
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/DefineHostNamingsOffPlatformTask.java
@@ -0,0 +1,69 @@
+/*
+ * 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.benchmark.tasks;
+
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.geode.benchmark.topology.HostNamingOffPlatform;
+import org.apache.geode.benchmark.topology.Roles;
+import org.apache.geode.perftest.Task;
+import org.apache.geode.perftest.TestContext;
+
+public class DefineHostNamingsOffPlatformTask implements Task {
+
+  public static final String HOST_NAMINGS_OFF_PLATFORM = "HOST_NAMINGS_OFF_PLATFORM";
+
+  public static String getOffPlatformHostName(final TestContext context,
+      final InetAddress addy) throws UnknownHostException {
+    final Map<InetAddress, String> namings =
+        (Map<InetAddress, String>) context.getAttribute(HOST_NAMINGS_OFF_PLATFORM);
+    return namings.get(addy);
+  }
+
+  @Override
+  public void run(final TestContext context) throws Exception {
+    final Map<InetAddress, String> namings =
+        Stream.concat(
+            generateHostNamingsFor(context, LOCATOR), generateHostNamingsFor(context, SERVER))
+            .collect(
+                Collectors.toMap(naming -> naming.internalAddy, naming -> naming.externalName));
+    context.setAttribute(HOST_NAMINGS_OFF_PLATFORM, namings);
+  }
+
+  private Stream<HostNamingOffPlatform> generateHostNamingsFor(final TestContext context,
+      final Roles role) {
+    final AtomicInteger i = new AtomicInteger(0);
+    final String roleName = role.name();
+    return context.getHostsForRole(roleName).stream().map(host -> new HostNamingOffPlatform(
+        formOffPlatformHostName(roleName, i.getAndIncrement()),
+        host));
+  }
+
+  private String formOffPlatformHostName(final String roleName, final int i) {
+    return roleName + "-OFF-PLATFORM-" + i;
+  }
+
+}
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
index b089006..45f16d7 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
@@ -17,6 +17,8 @@
 
 package org.apache.geode.benchmark.tasks;
 
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+
 import java.net.InetAddress;
 import java.util.Set;
 
@@ -27,7 +29,7 @@ import org.apache.geode.perftest.TestContext;
  */
 public class LocatorUtil {
   static String getLocatorString(TestContext context, int locatorPort) {
-    Set<InetAddress> locators = context.getHostsForRole("locator");
+    Set<InetAddress> locators = context.getHostsForRole(LOCATOR.name());
 
     return locators.iterator().next().getHostAddress() + "[" + locatorPort + "]";
   }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/PrePopulateRegion.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/PrePopulateRegion.java
index c56999f..faae4d7 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/PrePopulateRegion.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/PrePopulateRegion.java
@@ -16,9 +16,9 @@
  */
 package org.apache.geode.benchmark.tasks;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.LOCATOR;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import java.time.Duration;
 import java.time.Instant;
@@ -62,9 +62,9 @@ public class PrePopulateRegion implements Task {
   public void run(TestContext context) throws InterruptedException {
     final ClientCache cache = ClientCacheFactory.getAnyInstance();
     final Region<Long, Portfolio> region = cache.getRegion("region");
-    final int numLocators = context.getHostsIDsForRole(LOCATOR).size();
-    final int numServers = context.getHostsIDsForRole(SERVER).size();
-    final int numClient = context.getHostsIDsForRole(CLIENT).size();
+    final int numLocators = context.getHostsIDsForRole(LOCATOR.name()).size();
+    final int numServers = context.getHostsIDsForRole(SERVER.name()).size();
+    final int numClient = context.getHostsIDsForRole(CLIENT.name()).size();
     final int jvmID = context.getJvmID();
     final int clientIndex = jvmID - numLocators - numServers;
 
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/PrePopulateRegionLong.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/PrePopulateRegionLong.java
index 7d6564b..0874835 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/PrePopulateRegionLong.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/PrePopulateRegionLong.java
@@ -16,9 +16,9 @@
  */
 package org.apache.geode.benchmark.tasks;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.LOCATOR;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import java.time.Duration;
 import java.time.Instant;
@@ -61,9 +61,9 @@ public class PrePopulateRegionLong implements Task {
   public void run(TestContext context) throws InterruptedException {
     final ClientCache cache = ClientCacheFactory.getAnyInstance();
     final Region<Long, Long> region = cache.getRegion("region");
-    final int numLocators = context.getHostsIDsForRole(LOCATOR).size();
-    final int numServers = context.getHostsIDsForRole(SERVER).size();
-    final int numClient = context.getHostsIDsForRole(CLIENT).size();
+    final int numLocators = context.getHostsIDsForRole(LOCATOR.name()).size();
+    final int numServers = context.getHostsIDsForRole(SERVER.name()).size();
+    final int numClient = context.getHostsIDsForRole(CLIENT.name()).size();
     final int jvmID = context.getJvmID();
     final int clientIndex = jvmID - numLocators - numServers;
 
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/ProfilerParameters.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/ProcessControl.java
similarity index 60%
copy from geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/ProfilerParameters.java
copy to geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/ProcessControl.java
index 5f8fdcf..562af01 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/ProfilerParameters.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/ProcessControl.java
@@ -12,25 +12,25 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.benchmark.tasks;
 
-package org.apache.geode.benchmark.parameters;
-
-import static com.google.common.base.Strings.isNullOrEmpty;
-import static org.apache.geode.benchmark.parameters.Utils.configureAll;
+import java.io.IOException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.geode.perftest.TestConfig;
+public class ProcessControl {
 
-public class ProfilerParameters {
-  private static final Logger logger = LoggerFactory.getLogger(ProfilerParameters.class);
+  private static final Logger logger = LoggerFactory.getLogger(ProcessControl.class);
 
-  public static void configure(final TestConfig testConfig) {
-    final String profilerArgument = System.getProperty("benchmark.profiler.argument");
-    if (!isNullOrEmpty(profilerArgument)) {
-      logger.info("Configuring profiler parameter. {}", profilerArgument);
-      configureAll(testConfig, profilerArgument);
+  void runCommand(final String command) throws IOException, InterruptedException {
+    final Process startDaemon = Runtime.getRuntime().exec(command);
+    final int exitStatus = startDaemon.waitFor();
+    if (exitStatus != 0) {
+      final String msg = String.format("'%s' command exited with status %d\npwd is: %s", command,
+          exitStatus, System.getProperty("user.dir"));
+      logger.error(msg);
+      throw new IllegalStateException(msg);
     }
   }
 
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartClient.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartClient.java
index 77bb847..3c442f4 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartClient.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartClient.java
@@ -18,9 +18,12 @@
 package org.apache.geode.benchmark.tasks;
 
 import static org.apache.geode.benchmark.parameters.GeodeProperties.clientProperties;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
 
 import java.io.File;
+import java.lang.reflect.InvocationTargetException;
 import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.util.Properties;
 
 import org.apache.geode.cache.client.ClientCache;
@@ -34,7 +37,7 @@ import org.apache.geode.perftest.TestContext;
  * Task to create the client cache
  */
 public class StartClient implements Task {
-  private int locatorPort;
+  protected int locatorPort;
 
   public StartClient(int locatorPort) {
     this.locatorPort = locatorPort;
@@ -43,18 +46,33 @@ public class StartClient implements Task {
   @Override
   public void run(TestContext context) throws Exception {
 
-    InetAddress locator = context.getHostsForRole("locator").iterator().next();
+    InetAddress locator = context.getHostsForRole(LOCATOR.name()).iterator().next();
 
     String statsFile = new File(context.getOutputDir(), "stats.gfs").getAbsolutePath();
     Properties properties = clientProperties();
 
-    ClientCache clientCache = new ClientCacheFactory(properties)
-        .setPdxSerializer(new ReflectionBasedAutoSerializer("benchmark.geode.data.*"))
-        .addPoolLocator(locator.getHostAddress(), locatorPort)
-        .setPoolIdleTimeout(-1)
-        .set(ConfigurationProperties.STATISTIC_ARCHIVE_FILE, statsFile)
+    ClientCache clientCache = createClientCacheFactory(locator, statsFile, properties, context)
         .create();
 
     context.setAttribute("CLIENT_CACHE", clientCache);
   }
+
+  /**
+   * Create and configure the ClientCacheFactory.
+   *
+   * Subclasses can override this.
+   *
+   */
+  protected ClientCacheFactory createClientCacheFactory(final InetAddress locator,
+      final String statsFile,
+      final Properties properties,
+      final TestContext context)
+      throws UnknownHostException, NoSuchMethodException, InvocationTargetException,
+      IllegalAccessException, ClassNotFoundException {
+    return new ClientCacheFactory(properties)
+        .setPdxSerializer(new ReflectionBasedAutoSerializer("benchmark.geode.data.*"))
+        .setPoolIdleTimeout(-1)
+        .set(ConfigurationProperties.STATISTIC_ARCHIVE_FILE, statsFile)
+        .addPoolLocator(locator.getHostAddress(), locatorPort);
+  }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartClientSNI.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartClientSNI.java
new file mode 100644
index 0000000..d5acc7d
--- /dev/null
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartClientSNI.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.benchmark.tasks;
+
+import static org.apache.geode.benchmark.tasks.DefineHostNamingsOffPlatformTask.getOffPlatformHostName;
+import static org.apache.geode.benchmark.topology.Ports.SNI_PROXY_PORT;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.PROXY;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Properties;
+
+import org.apache.geode.cache.client.ClientCacheFactory;
+import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.pdx.ReflectionBasedAutoSerializer;
+import org.apache.geode.perftest.TestContext;
+
+public class StartClientSNI extends StartClient {
+
+  public StartClientSNI(final int locatorPort) {
+    super(locatorPort);
+  }
+
+  @Override
+  protected ClientCacheFactory createClientCacheFactory(final InetAddress locator,
+      final String statsFile,
+      final Properties properties,
+      final TestContext context)
+      throws UnknownHostException, NoSuchMethodException, InvocationTargetException,
+      IllegalAccessException, ClassNotFoundException {
+
+    final InetAddress firstLocatorAddy =
+        context.getHostsForRole(LOCATOR.name()).iterator().next();
+    final String offPlatformLocatorName =
+        getOffPlatformHostName(context, firstLocatorAddy);
+    final InetAddress proxyAddy =
+        context.getHostsForRole(PROXY.name()).iterator().next();
+
+    final ClientCacheFactory cacheFactory = new ClientCacheFactory(properties)
+        .setPdxSerializer(new ReflectionBasedAutoSerializer("benchmark.geode.data.*"))
+        .setPoolIdleTimeout(-1)
+        .set(ConfigurationProperties.STATISTIC_ARCHIVE_FILE, statsFile)
+        .addPoolLocator(offPlatformLocatorName, locatorPort);
+    final String proxyHostAddress = proxyAddy.getHostAddress();
+    return reflectivelySetSniSocketFactory(cacheFactory, proxyHostAddress);
+  }
+
+  protected ClientCacheFactory reflectivelySetSniSocketFactory(
+      final ClientCacheFactory clientCacheFactory,
+      final String proxyHostAddress)
+      throws NoSuchMethodException, InvocationTargetException, IllegalAccessException,
+      ClassNotFoundException {
+    /*
+     * We'd like to simply do the following, but that would introduce a compile-time dependency on
+     * Geode [1.13,). But we want this benchmark code to work with older Geode version. So we'll
+     * use reflection to do it.
+     *
+     * return clientCacheFactory
+     * .setPoolSocketFactory(ProxySocketFactories.sni(
+     * proxyHostAddress,
+     * SNI_PROXY_PORT));
+     */
+    final Class<?> proxySocketFactoriesClass =
+        Class.forName("org.apache.geode.cache.client.proxy.ProxySocketFactories");
+    final Method sniStaticMethod =
+        proxySocketFactoriesClass.getMethod("sni", String.class, int.class);
+
+    final Object sniSocketFactory = sniStaticMethod.invoke(null, proxyHostAddress, SNI_PROXY_PORT);
+
+    final Class<?> socketFactoryClass =
+        Class.forName("org.apache.geode.cache.client.SocketFactory");
+    final Method setPoolSocketFactoryMethod =
+        clientCacheFactory.getClass().getMethod("setPoolSocketFactory", socketFactoryClass);
+
+    return (ClientCacheFactory) setPoolSocketFactoryMethod.invoke(clientCacheFactory,
+        sniSocketFactory);
+  }
+
+}
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartLocator.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartLocator.java
index c5f53ff..d91e43d 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartLocator.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartLocator.java
@@ -20,6 +20,7 @@ package org.apache.geode.benchmark.tasks;
 import static org.apache.geode.benchmark.parameters.GeodeProperties.locatorProperties;
 
 import java.io.File;
+import java.io.IOException;
 import java.net.InetAddress;
 import java.util.Properties;
 
@@ -46,6 +47,15 @@ public class StartLocator implements Task {
     properties.setProperty(ConfigurationProperties.STATISTIC_ARCHIVE_FILE, statsFile);
 
     properties.setProperty(ConfigurationProperties.NAME, "locator-" + InetAddress.getLocalHost());
+    startLocator(properties, locatorPort, context);
+  }
+
+  /**
+   * Starts locator as a side-effect.
+   *
+   */
+  protected void startLocator(final Properties properties, final int locatorPort,
+      final TestContext context) throws IOException {
     Locator.startLocatorAndDS(locatorPort, null, properties);
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartLocatorSNI.java
similarity index 56%
copy from geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
copy to geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartLocatorSNI.java
index b089006..b54f2b0 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartLocatorSNI.java
@@ -17,18 +17,31 @@
 
 package org.apache.geode.benchmark.tasks;
 
+import static org.apache.geode.benchmark.tasks.DefineHostNamingsOffPlatformTask.getOffPlatformHostName;
+
+import java.io.IOException;
 import java.net.InetAddress;
-import java.util.Set;
+import java.util.Properties;
 
+import org.apache.geode.distributed.LocatorLauncher;
 import org.apache.geode.perftest.TestContext;
 
-/**
- * Util class to get the locator address string
- */
-public class LocatorUtil {
-  static String getLocatorString(TestContext context, int locatorPort) {
-    Set<InetAddress> locators = context.getHostsForRole("locator");
+public class StartLocatorSNI extends StartLocator {
 
-    return locators.iterator().next().getHostAddress() + "[" + locatorPort + "]";
+  public StartLocatorSNI(final int locatorPort) {
+    super(locatorPort);
   }
+
+  @Override
+  protected void startLocator(final Properties properties, final int locatorPort,
+      final TestContext context) throws IOException {
+    new LocatorLauncher.Builder()
+        .set(properties)
+        .setPort(locatorPort)
+        .setHostnameForClients(
+            getOffPlatformHostName(context, InetAddress.getLocalHost()))
+        .build()
+        .start();
+  }
+
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartServer.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartServer.java
index 1402e9d..9689261 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartServer.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartServer.java
@@ -21,6 +21,7 @@ import static org.apache.geode.benchmark.parameters.GeodeProperties.serverProper
 
 import java.io.File;
 import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.util.Properties;
 
 import org.apache.geode.cache.Cache;
@@ -57,11 +58,23 @@ public class StartServer implements Task {
         .set(ConfigurationProperties.STATISTIC_ARCHIVE_FILE, statsFile)
         .create();
     CacheServer cacheServer = cache.addCacheServer();
-    cacheServer.setPort(0);
-    cacheServer.setMaxConnections(Integer.MAX_VALUE);
+    configureCacheServer(cacheServer, context);
     cacheServer.start();
     context.setAttribute("SERVER_CACHE", cache);
 
   }
 
+  /**
+   * Configure the cache server
+   *
+   * Subclasses can override this. Call super first to inherit settings.
+   *
+   * @param cacheServer is modified by this method!
+   */
+  protected void configureCacheServer(final CacheServer cacheServer,
+      final TestContext context) throws UnknownHostException {
+    cacheServer.setMaxConnections(Integer.MAX_VALUE);
+    cacheServer.setPort(0);
+  }
+
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartServerSNI.java
similarity index 54%
copy from geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
copy to geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartServerSNI.java
index b089006..9c8f03b 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartServerSNI.java
@@ -17,18 +17,30 @@
 
 package org.apache.geode.benchmark.tasks;
 
+import static org.apache.geode.benchmark.tasks.DefineHostNamingsOffPlatformTask.getOffPlatformHostName;
+
 import java.net.InetAddress;
-import java.util.Set;
+import java.net.UnknownHostException;
 
+import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.perftest.TestContext;
 
-/**
- * Util class to get the locator address string
- */
-public class LocatorUtil {
-  static String getLocatorString(TestContext context, int locatorPort) {
-    Set<InetAddress> locators = context.getHostsForRole("locator");
+public class StartServerSNI extends StartServer {
+
+  private final int serverPortForSni;
 
-    return locators.iterator().next().getHostAddress() + "[" + locatorPort + "]";
+  public StartServerSNI(final int locatorPort, final int serverPortForSni) {
+    super(locatorPort);
+    this.serverPortForSni = serverPortForSni;
   }
+
+  @Override
+  protected void configureCacheServer(final CacheServer cacheServer, final TestContext context)
+      throws UnknownHostException {
+    cacheServer.setMaxConnections(Integer.MAX_VALUE);
+    cacheServer.setPort(serverPortForSni);
+    cacheServer.setHostnameForClients(
+        getOffPlatformHostName(context, InetAddress.getLocalHost()));
+  }
+
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartSniProxy.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartSniProxy.java
new file mode 100644
index 0000000..5ac999d
--- /dev/null
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartSniProxy.java
@@ -0,0 +1,182 @@
+/*
+ * 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.benchmark.tasks;
+
+import static org.apache.geode.benchmark.tasks.DefineHostNamingsOffPlatformTask.HOST_NAMINGS_OFF_PLATFORM;
+import static org.apache.geode.benchmark.topology.Ports.SERVER_PORT_FOR_SNI;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.geode.benchmark.topology.Roles;
+import org.apache.geode.perftest.Task;
+import org.apache.geode.perftest.TestContext;
+
+/**
+ * Task to start the SNI proxy
+ */
+public class StartSniProxy implements Task {
+  public static final String START_DOCKER_DAEMON_COMMAND = "sudo service docker start";
+  public static final String START_PROXY_COMMAND = "docker-compose up -d haproxy";
+
+  private final int locatorPort;
+
+  public StartSniProxy(int locatorPort) {
+    this.locatorPort = locatorPort;
+  }
+
+  @Override
+  public void run(TestContext context) throws Exception {
+
+    final Map<InetAddress, String> namings =
+        (Map<InetAddress, String>) context.getAttribute(HOST_NAMINGS_OFF_PLATFORM);
+
+    final String config = generateHaProxyConfig(
+        internalHostNamesFor(context, LOCATOR),
+        externalHostNamesFor(context, LOCATOR, namings),
+        internalHostNamesFor(context, SERVER),
+        externalHostNamesFor(context, SERVER, namings));
+
+    rewriteFile(config, "haproxy.cfg");
+
+    final ProcessControl processControl = new ProcessControl();
+    processControl.runCommand(START_DOCKER_DAEMON_COMMAND);
+    processControl.runCommand(START_PROXY_COMMAND);
+  }
+
+  private void rewriteFile(final String content, final String fileName) throws IOException {
+    try (final BufferedWriter writer = new BufferedWriter(new FileWriter(fileName, false))) {
+      writer.write(content);
+    }
+  }
+
+  private Stream<String> internalHostNamesFor(final TestContext context, final Roles role) {
+    return addysFor(context, role).map(InetAddress::getHostName);
+  }
+
+  private Stream<String> externalHostNamesFor(final TestContext context, final Roles role,
+      final Map<InetAddress, String> namings) {
+    return addysFor(context, role).map(addy -> namings.get(addy));
+  }
+
+  private Stream<InetAddress> addysFor(final TestContext context, final Roles role) {
+    return context.getHostsForRole(role.name()).stream();
+  }
+
+  String generateHaProxyConfig(final Stream<String> locatorsInternalStream,
+      final Stream<String> locatorsExternalStream,
+      final Stream<String> serversInternalStream,
+      final Stream<String> serversExternalStream) {
+
+    final Iterable<String> locatorsInternal = locatorsInternalStream.collect(Collectors.toList());
+    final Iterable<String> locatorsExternal = locatorsExternalStream.collect(Collectors.toList());
+    final Iterable<String> serversInternal = serversInternalStream.collect(Collectors.toList());
+    final Iterable<String> serversExternal = serversExternalStream.collect(Collectors.toList());
+
+    final StringBuilder stuff = new StringBuilder(
+        /*
+         * log to stdout per:
+         * https://www.haproxy.com/documentation/hapee/latest/administration/docker-logging/
+         */
+        "global\n"
+            + "  log stdout format raw local0 debug\n"
+            + "  maxconn 5000\n"
+            + "defaults\n"
+            + "  log global\n"
+            /*
+             * We're leaving timeouts unspecified so they are infinite. Benchmarks do bad things
+             * when the proxy breaks connections.
+             */
+            // + " timeout client 100s\n"
+            // + " timeout connect 100s\n"
+            // + " timeout server 100s\n"
+            + "frontend sniproxy\n"
+            + "  bind *:15443\n"
+            + "  mode tcp\n"
+            + "  tcp-request inspect-delay 5s\n"
+            + "  tcp-request content accept if { req_ssl_hello_type 1 }\n");
+
+    generateUseBackendRule(locatorsInternal, locatorsExternal, stuff, "locators-");
+    generateUseBackendRule(serversInternal, serversExternal, stuff, "servers-");
+
+    final String firstLocatorInternal = locatorsInternal.iterator().next();
+    stuff.append("  default_backend ").append("locators-").append(firstLocatorInternal)
+        .append("\n");
+
+    generateBackendSection(locatorsInternal, stuff, "locators-",
+        "locator1", locatorPort);
+
+    generateBackendSection(serversInternal, stuff, "servers-",
+        "server1", SERVER_PORT_FOR_SNI);
+
+    return stuff.toString();
+  }
+
+  private void generateUseBackendRule(final Iterable<String> internalsIterable,
+      final Iterable<String> externalsIterable,
+      final StringBuilder stuff,
+      final String backendNamePrefix) {
+    final Iterator<String> internals = internalsIterable.iterator();
+    final Iterator<String> externals = externalsIterable.iterator();
+    while (internals.hasNext() && externals.hasNext()) {
+      final String internal = internals.next();
+      final String external = externals.next();
+      stuff.append("  use_backend ").append(backendNamePrefix).append(internal)
+          .append(" if { req.ssl_sni -i ").append(external).append(" }\n");
+    }
+  }
+
+  private void generateBackendSection(final Iterable<String> internalsIterator,
+      final StringBuilder stuff,
+      final String backendNamePrefix,
+      final String singleHostRoleName,
+      final int port) {
+    for (final String addy : internalsIterator) {
+      stuff.append("backend ").append(backendNamePrefix).append(addy).append("\n")
+          .append("  mode tcp\n").append("  server ").append(singleHostRoleName).append(" ")
+          .append(addy)
+          .append(":").append(port).append("\n");
+    }
+  }
+
+  @Override
+  public boolean equals(final Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    final StartSniProxy that = (StartSniProxy) o;
+    return locatorPort == that.locatorPort;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(locatorPort);
+  }
+}
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StopClient.java
similarity index 70%
copy from geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
copy to geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StopClient.java
index b089006..3e796a9 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StopClient.java
@@ -17,18 +17,22 @@
 
 package org.apache.geode.benchmark.tasks;
 
-import java.net.InetAddress;
-import java.util.Set;
 
+
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.client.ClientCacheFactory;
+import org.apache.geode.perftest.Task;
 import org.apache.geode.perftest.TestContext;
 
 /**
- * Util class to get the locator address string
+ * Task to create the client cache
  */
-public class LocatorUtil {
-  static String getLocatorString(TestContext context, int locatorPort) {
-    Set<InetAddress> locators = context.getHostsForRole("locator");
+public class StopClient implements Task {
 
-    return locators.iterator().next().getHostAddress() + "[" + locatorPort + "]";
+  @Override
+  public void run(TestContext context) throws Exception {
+    ClientCache cache = ClientCacheFactory.getAnyInstance();
+    cache.close();
   }
+
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StopSniProxy.java
similarity index 60%
copy from geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
copy to geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StopSniProxy.java
index b089006..4014588 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/LocatorUtil.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StopSniProxy.java
@@ -17,18 +17,26 @@
 
 package org.apache.geode.benchmark.tasks;
 
-import java.net.InetAddress;
-import java.util.Set;
-
+import org.apache.geode.perftest.Task;
 import org.apache.geode.perftest.TestContext;
 
 /**
- * Util class to get the locator address string
+ * Task to stop the SNI proxy
  */
-public class LocatorUtil {
-  static String getLocatorString(TestContext context, int locatorPort) {
-    Set<InetAddress> locators = context.getHostsForRole("locator");
+public class StopSniProxy implements Task {
+
+  public static final String STOP_PROXY_COMMAND = "docker-compose down";
+  // TODO: fix: this exits w/ status 1
+  public static final String CAPTURE_PROXY_LOG =
+      "bash -c 'docker-compose logs haproxy > haproxy.log'";
 
-    return locators.iterator().next().getHostAddress() + "[" + locatorPort + "]";
+  public StopSniProxy() {}
+
+  @Override
+  public void run(TestContext context) throws Exception {
+    final ProcessControl processControl = new ProcessControl();
+    // processControl.runCommand(CAPTURE_PROXY_LOG);
+    processControl.runCommand(STOP_PROXY_COMMAND);
   }
+
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractFunctionBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractFunctionBenchmark.java
index 9c1761b..51d6f07 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractFunctionBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractFunctionBenchmark.java
@@ -15,8 +15,9 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import benchmark.geode.data.BenchmarkFunction;
 
@@ -24,7 +25,6 @@ import org.apache.geode.benchmark.LongRange;
 import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
 import org.apache.geode.benchmark.tasks.RegisterFunction;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 
@@ -43,11 +43,10 @@ abstract class AbstractFunctionBenchmark implements PerformanceTest {
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
     config.threads(Runtime.getRuntime().availableProcessors() * 3);
-    ClientServerTopology.configure(config);
     configureRegion(config);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegion(getKeyRange()), CLIENT);
-    config.before(new RegisterFunction(new BenchmarkFunction(getKeyRange())), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegion(getKeyRange()), CLIENT);
+    before(config, new RegisterFunction(new BenchmarkFunction(getKeyRange())), SERVER);
     return config;
   }
 
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractPartitionedFunctionBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractPartitionedFunctionBenchmark.java
index fdbd166..493fc1d 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractPartitionedFunctionBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractPartitionedFunctionBenchmark.java
@@ -15,7 +15,8 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
 import org.apache.geode.perftest.TestConfig;
@@ -23,6 +24,6 @@ import org.apache.geode.perftest.TestConfig;
 abstract class AbstractPartitionedFunctionBenchmark extends AbstractFunctionBenchmark {
   @Override
   protected void configureRegion(TestConfig config) {
-    config.before(new CreatePartitionedRegion(), SERVER);
+    before(config, new CreatePartitionedRegion(), SERVER);
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractReplicatedFunctionBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractReplicatedFunctionBenchmark.java
index 6a9fa17..0a8ee3a 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractReplicatedFunctionBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractReplicatedFunctionBenchmark.java
@@ -15,7 +15,8 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
 import org.apache.geode.perftest.TestConfig;
@@ -23,6 +24,6 @@ import org.apache.geode.perftest.TestConfig;
 abstract class AbstractReplicatedFunctionBenchmark extends AbstractFunctionBenchmark {
   @Override
   protected void configureRegion(TestConfig config) {
-    config.before(new CreateReplicatedRegion(), SERVER);
+    before(config, new CreateReplicatedRegion(), SERVER);
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/GeodeBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/GeodeBenchmark.java
index 7551eec..bf7d426 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/GeodeBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/GeodeBenchmark.java
@@ -17,6 +17,8 @@ package org.apache.geode.benchmark.tests;
 
 import static java.util.concurrent.TimeUnit.MINUTES;
 
+import org.apache.geode.benchmark.topology.ClientServerTopology;
+import org.apache.geode.benchmark.topology.ClientServerTopologyWithSNIProxy;
 import org.apache.geode.perftest.TestConfig;
 
 public class GeodeBenchmark {
@@ -38,11 +40,21 @@ public class GeodeBenchmark {
 
 
   public static TestConfig createConfig() {
-    TestConfig testConfig = new TestConfig();
-    testConfig.warmupSeconds(WARM_UP_TIME);
-    testConfig.durationSeconds(BENCHMARK_DURATION);
-    testConfig.threads(THREADS);
-    return testConfig;
+    TestConfig config = new TestConfig();
+    config.warmupSeconds(WARM_UP_TIME);
+    config.durationSeconds(BENCHMARK_DURATION);
+    config.threads(THREADS);
+
+    final String sniProp = System.getProperty("withSniProxy");
+    final boolean doSni = sniProp != null && !sniProp.equals("false");
+
+    if (doSni) {
+      ClientServerTopologyWithSNIProxy.configure(config);
+    } else {
+      ClientServerTopology.configure(config);
+    }
+
+    return config;
   }
 
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/NoopBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/NoopBenchmark.java
index 8d4e612..7cf49ab 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/NoopBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/NoopBenchmark.java
@@ -18,12 +18,12 @@
 package org.apache.geode.benchmark.tests;
 
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
 
 import org.junit.jupiter.api.Test;
 
 import org.apache.geode.benchmark.tasks.NoopTask;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -43,8 +43,7 @@ public class NoopBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
-    config.workload(new NoopTask(), CLIENT);
+    workload(config, new NoopTask(), CLIENT);
     return config;
 
   }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionBenchmark.java
index 0b964b6..381d4d6 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionBenchmark.java
@@ -14,7 +14,8 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
 
 import org.junit.jupiter.api.Test;
 
@@ -32,7 +33,7 @@ public class PartitionedFunctionExecutionBenchmark extends AbstractPartitionedFu
   @Override
   public TestConfig configure() {
     TestConfig config = super.configure();
-    config.workload(new ExecuteFunction(), CLIENT);
+    workload(config, new ExecuteFunction(), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionWithArgumentsBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionWithArgumentsBenchmark.java
index e5605d7..9de3051 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionWithArgumentsBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionWithArgumentsBenchmark.java
@@ -14,7 +14,8 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
 
 import org.junit.jupiter.api.Test;
 
@@ -35,7 +36,7 @@ public class PartitionedFunctionExecutionWithArgumentsBenchmark
   public TestConfig configure() {
     TestConfig config = super.configure();
     config.threads(Runtime.getRuntime().availableProcessors() * 4);
-    config.workload(new ExecuteParameterizedFunction(getKeyRange()), CLIENT);
+    workload(config, new ExecuteParameterizedFunction(getKeyRange()), CLIENT);
     return config;
 
   }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionWithFiltersBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionWithFiltersBenchmark.java
index cad7f8a..3a3d735 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionWithFiltersBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedFunctionExecutionWithFiltersBenchmark.java
@@ -14,7 +14,8 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
 
 import org.junit.jupiter.api.Test;
 
@@ -34,7 +35,7 @@ public class PartitionedFunctionExecutionWithFiltersBenchmark
   public TestConfig configure() {
     TestConfig config = super.configure();
     config.threads(Runtime.getRuntime().availableProcessors() * 8);
-    config.workload(new ExecuteFilteredFunction(getKeyRange()), CLIENT);
+    workload(config, new ExecuteFilteredFunction(getKeyRange()), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedGetBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedGetBenchmark.java
index 87ed609..676a1f3 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedGetBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedGetBenchmark.java
@@ -18,8 +18,10 @@
 package org.apache.geode.benchmark.tests;
 
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -28,7 +30,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
 import org.apache.geode.benchmark.tasks.GetTask;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -54,11 +55,10 @@ public class PartitionedGetBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
-    config.before(new CreatePartitionedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new GetTask(keyRange), CLIENT);
+    before(config, new CreatePartitionedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new GetTask(keyRange), CLIENT);
     return config;
 
   }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedGetLongBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedGetLongBenchmark.java
index 5b52fda..f64081d 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedGetLongBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedGetLongBenchmark.java
@@ -18,8 +18,10 @@
 package org.apache.geode.benchmark.tests;
 
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -28,7 +30,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
 import org.apache.geode.benchmark.tasks.GetTask;
 import org.apache.geode.benchmark.tasks.PrePopulateRegionLong;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -54,11 +55,10 @@ public class PartitionedGetLongBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
-    config.before(new CreatePartitionedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegionLong(keyRange), CLIENT);
-    config.workload(new GetTask(keyRange), CLIENT);
+    before(config, new CreatePartitionedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegionLong(keyRange), CLIENT);
+    workload(config, new GetTask(keyRange), CLIENT);
     return config;
 
   }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedIndexedQueryBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedIndexedQueryBenchmark.java
index 470e748..0aa42ad 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedIndexedQueryBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedIndexedQueryBenchmark.java
@@ -14,8 +14,10 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -25,7 +27,6 @@ import org.apache.geode.benchmark.tasks.CreateIndexOnID;
 import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
 import org.apache.geode.benchmark.tasks.OQLQuery;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -53,12 +54,11 @@ public class PartitionedIndexedQueryBenchmark implements PerformanceTest {
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
     config.threads(Runtime.getRuntime().availableProcessors() * 8);
-    ClientServerTopology.configure(config);
-    config.before(new CreatePartitionedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new CreateIndexOnID(), SERVER);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new OQLQuery(keyRange, queryRange), CLIENT);
+    before(config, new CreatePartitionedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new CreateIndexOnID(), SERVER);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new OQLQuery(keyRange, queryRange), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedNonIndexedQueryBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedNonIndexedQueryBenchmark.java
index 1e7ded2..06cf6ea 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedNonIndexedQueryBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedNonIndexedQueryBenchmark.java
@@ -14,8 +14,10 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -24,7 +26,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
 import org.apache.geode.benchmark.tasks.OQLQuery;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -52,11 +53,10 @@ public class PartitionedNonIndexedQueryBenchmark implements PerformanceTest {
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
     config.threads(Runtime.getRuntime().availableProcessors());
-    ClientServerTopology.configure(config);
-    config.before(new CreatePartitionedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new OQLQuery(keyRange, queryRange), CLIENT);
+    before(config, new CreatePartitionedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new OQLQuery(keyRange, queryRange), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutAllBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutAllBenchmark.java
index 79dc413..7e35bc6 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutAllBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutAllBenchmark.java
@@ -17,8 +17,10 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -27,7 +29,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
 import org.apache.geode.benchmark.tasks.PutAllTask;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -56,11 +57,10 @@ public class PartitionedPutAllBenchmark implements PerformanceTest {
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
     config.threads(Runtime.getRuntime().availableProcessors() * 2);
-    ClientServerTopology.configure(config);
-    config.before(new CreatePartitionedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new PutAllTask(keyRange, batchSize), CLIENT);
+    before(config, new CreatePartitionedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new PutAllTask(keyRange, batchSize), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutAllLongBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutAllLongBenchmark.java
index 507eece..e300de4 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutAllLongBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutAllLongBenchmark.java
@@ -17,8 +17,10 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -27,7 +29,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
 import org.apache.geode.benchmark.tasks.PrePopulateRegionLong;
 import org.apache.geode.benchmark.tasks.PutAllTask;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -56,11 +57,10 @@ public class PartitionedPutAllLongBenchmark implements PerformanceTest {
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
     config.threads(Runtime.getRuntime().availableProcessors() * 2);
-    ClientServerTopology.configure(config);
-    config.before(new CreatePartitionedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegionLong(keyRange), CLIENT);
-    config.workload(new PutAllTask(keyRange, batchSize), CLIENT);
+    before(config, new CreatePartitionedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegionLong(keyRange), CLIENT);
+    workload(config, new PutAllTask(keyRange, batchSize), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmark.java
index bef948f..8a18d73 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmark.java
@@ -17,8 +17,10 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -27,7 +29,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
 import org.apache.geode.benchmark.tasks.PutTask;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -37,7 +38,7 @@ import org.apache.geode.perftest.TestRunners;
  */
 public class PartitionedPutBenchmark implements PerformanceTest {
 
-  private LongRange keyRange = new LongRange(0, 1000000);
+  private LongRange keyRange = new LongRange(0, 1_000_000);
 
   public PartitionedPutBenchmark() {}
 
@@ -53,12 +54,11 @@ public class PartitionedPutBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
-    config.before(new CreatePartitionedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new PutTask(keyRange), CLIENT);
-    return config;
 
+    before(config, new CreatePartitionedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new PutTask(keyRange), CLIENT);
+    return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutLongBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutLongBenchmark.java
index c368071..ed33f1a 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutLongBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutLongBenchmark.java
@@ -17,8 +17,10 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -27,7 +29,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
 import org.apache.geode.benchmark.tasks.PrePopulateRegionLong;
 import org.apache.geode.benchmark.tasks.PutTask;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -53,12 +54,10 @@ public class PartitionedPutLongBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
-    config.before(new CreatePartitionedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegionLong(keyRange), CLIENT);
-    config.workload(new PutTask(keyRange), CLIENT);
+    before(config, new CreatePartitionedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegionLong(keyRange), CLIENT);
+    workload(config, new PutTask(keyRange), CLIENT);
     return config;
-
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionBenchmark.java
index e5b2cce..0b4e3c1 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionBenchmark.java
@@ -14,7 +14,8 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
 
 import org.junit.jupiter.api.Test;
 
@@ -32,7 +33,7 @@ public class ReplicatedFunctionExecutionBenchmark extends AbstractReplicatedFunc
   @Override
   public TestConfig configure() {
     TestConfig config = super.configure();
-    config.workload(new ExecuteFunction(), CLIENT);
+    workload(config, new ExecuteFunction(), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionWithArgumentsBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionWithArgumentsBenchmark.java
index faf6cb2..595e955 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionWithArgumentsBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionWithArgumentsBenchmark.java
@@ -14,7 +14,8 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
 
 import org.junit.jupiter.api.Test;
 
@@ -34,7 +35,7 @@ public class ReplicatedFunctionExecutionWithArgumentsBenchmark
   public TestConfig configure() {
     TestConfig config = super.configure();
     config.threads(Runtime.getRuntime().availableProcessors() * 16);
-    config.workload(new ExecuteParameterizedFunction(getKeyRange()), CLIENT);
+    workload(config, new ExecuteParameterizedFunction(getKeyRange()), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionWithFiltersBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionWithFiltersBenchmark.java
index 30aed9d..728fbab 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionWithFiltersBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionWithFiltersBenchmark.java
@@ -14,7 +14,8 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
 
 import org.junit.jupiter.api.Test;
 
@@ -34,7 +35,7 @@ public class ReplicatedFunctionExecutionWithFiltersBenchmark
   public TestConfig configure() {
     TestConfig config = super.configure();
     config.threads(Runtime.getRuntime().availableProcessors() * 10);
-    config.workload(new ExecuteFilteredFunction(getKeyRange()), CLIENT);
+    workload(config, new ExecuteFilteredFunction(getKeyRange()), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedGetBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedGetBenchmark.java
index 5ee0975..97b01f4 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedGetBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedGetBenchmark.java
@@ -18,8 +18,10 @@
 package org.apache.geode.benchmark.tests;
 
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -28,7 +30,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
 import org.apache.geode.benchmark.tasks.GetTask;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -54,11 +55,10 @@ public class ReplicatedGetBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
-    config.before(new CreateReplicatedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new GetTask(keyRange), CLIENT);
+    before(config, new CreateReplicatedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new GetTask(keyRange), CLIENT);
     return config;
 
   }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedGetLongBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedGetLongBenchmark.java
index 32be10b..d796638 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedGetLongBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedGetLongBenchmark.java
@@ -18,8 +18,10 @@
 package org.apache.geode.benchmark.tests;
 
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -28,7 +30,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
 import org.apache.geode.benchmark.tasks.GetTask;
 import org.apache.geode.benchmark.tasks.PrePopulateRegionLong;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -54,11 +55,10 @@ public class ReplicatedGetLongBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
-    config.before(new CreateReplicatedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegionLong(keyRange), CLIENT);
-    config.workload(new GetTask(keyRange), CLIENT);
+    before(config, new CreateReplicatedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegionLong(keyRange), CLIENT);
+    workload(config, new GetTask(keyRange), CLIENT);
     return config;
 
   }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedIndexedQueryBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedIndexedQueryBenchmark.java
index 23ca659..d0ee1b0 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedIndexedQueryBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedIndexedQueryBenchmark.java
@@ -14,8 +14,10 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -25,7 +27,6 @@ import org.apache.geode.benchmark.tasks.CreateIndexOnID;
 import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
 import org.apache.geode.benchmark.tasks.OQLQuery;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -53,12 +54,11 @@ public class ReplicatedIndexedQueryBenchmark implements PerformanceTest {
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
     config.threads(Runtime.getRuntime().availableProcessors() * 8);
-    ClientServerTopology.configure(config);
-    config.before(new CreateReplicatedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new CreateIndexOnID(), SERVER);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new OQLQuery(keyRange, queryRange), CLIENT);
+    before(config, new CreateReplicatedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new CreateIndexOnID(), SERVER);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new OQLQuery(keyRange, queryRange), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedNonIndexedQueryBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedNonIndexedQueryBenchmark.java
index ac2e167..873ef49 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedNonIndexedQueryBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedNonIndexedQueryBenchmark.java
@@ -14,8 +14,10 @@
  */
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -24,7 +26,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
 import org.apache.geode.benchmark.tasks.OQLQuery;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -52,11 +53,11 @@ public class ReplicatedNonIndexedQueryBenchmark implements PerformanceTest {
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
     config.threads(Runtime.getRuntime().availableProcessors());
-    ClientServerTopology.configure(config);
-    config.before(new CreateReplicatedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new OQLQuery(keyRange, queryRange), CLIENT);
+    before(config, new CreateReplicatedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new OQLQuery(keyRange, queryRange), CLIENT);
     return config;
   }
+
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutAllBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutAllBenchmark.java
index d05fdff..c5b4f90 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutAllBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutAllBenchmark.java
@@ -17,8 +17,10 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -27,7 +29,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
 import org.apache.geode.benchmark.tasks.PutAllTask;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -56,11 +57,10 @@ public class ReplicatedPutAllBenchmark implements PerformanceTest {
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
     config.threads(Runtime.getRuntime().availableProcessors() * 2);
-    ClientServerTopology.configure(config);
-    config.before(new CreateReplicatedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new PutAllTask(keyRange, batchSize), CLIENT);
+    before(config, new CreateReplicatedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new PutAllTask(keyRange, batchSize), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutAllLongBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutAllLongBenchmark.java
index 19e0a1f..3719f3e 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutAllLongBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutAllLongBenchmark.java
@@ -17,8 +17,10 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -27,7 +29,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
 import org.apache.geode.benchmark.tasks.PrePopulateRegionLong;
 import org.apache.geode.benchmark.tasks.PutAllTask;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -56,11 +57,10 @@ public class ReplicatedPutAllLongBenchmark implements PerformanceTest {
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
     config.threads(Runtime.getRuntime().availableProcessors() * 2);
-    ClientServerTopology.configure(config);
-    config.before(new CreateReplicatedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegionLong(keyRange), CLIENT);
-    config.workload(new PutAllTask(keyRange, batchSize), CLIENT);
+    before(config, new CreateReplicatedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegionLong(keyRange), CLIENT);
+    workload(config, new PutAllTask(keyRange, batchSize), CLIENT);
     return config;
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutBenchmark.java
index bb9378d..1768b50 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutBenchmark.java
@@ -17,8 +17,10 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -27,7 +29,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
 import org.apache.geode.benchmark.tasks.PrePopulateRegion;
 import org.apache.geode.benchmark.tasks.PutTask;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -53,11 +54,10 @@ public class ReplicatedPutBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
-    config.before(new CreateReplicatedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegion(keyRange), CLIENT);
-    config.workload(new PutTask(keyRange), CLIENT);
+    before(config, new CreateReplicatedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegion(keyRange), CLIENT);
+    workload(config, new PutTask(keyRange), CLIENT);
     return config;
 
   }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutLongBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutLongBenchmark.java
index 776e2c1..3a6b941 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutLongBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedPutLongBenchmark.java
@@ -17,8 +17,10 @@
 
 package org.apache.geode.benchmark.tests;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.workload;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.junit.jupiter.api.Test;
 
@@ -27,7 +29,6 @@ import org.apache.geode.benchmark.tasks.CreateClientProxyRegion;
 import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
 import org.apache.geode.benchmark.tasks.PrePopulateRegionLong;
 import org.apache.geode.benchmark.tasks.PutTask;
-import org.apache.geode.benchmark.topology.ClientServerTopology;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -53,11 +54,10 @@ public class ReplicatedPutLongBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
-    config.before(new CreateReplicatedRegion(), SERVER);
-    config.before(new CreateClientProxyRegion(), CLIENT);
-    config.before(new PrePopulateRegionLong(keyRange), CLIENT);
-    config.workload(new PutTask(keyRange), CLIENT);
+    before(config, new CreateReplicatedRegion(), SERVER);
+    before(config, new CreateClientProxyRegion(), CLIENT);
+    before(config, new PrePopulateRegionLong(keyRange), CLIENT);
+    workload(config, new PutTask(keyRange), CLIENT);
     return config;
 
   }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopology.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopology.java
index c04ff9c..63fa137 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopology.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopology.java
@@ -14,13 +14,13 @@
  */
 package org.apache.geode.benchmark.topology;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.LOCATOR;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
-
-import org.bouncycastle.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.role;
+import static org.apache.geode.benchmark.parameters.Utils.addToTestConfig;
+import static org.apache.geode.benchmark.topology.Ports.LOCATOR_PORT;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 
 import org.apache.geode.benchmark.parameters.GcLoggingParameters;
 import org.apache.geode.benchmark.parameters.GcParameters;
@@ -33,62 +33,29 @@ import org.apache.geode.benchmark.tasks.StartServer;
 import org.apache.geode.perftest.TestConfig;
 
 public class ClientServerTopology {
-  private static final Logger logger = LoggerFactory.getLogger(ClientServerTopology.class);
-
-  /**
-   * All roles defined for the JVMs created for the benchmark
-   */
-  public static class Roles {
-    public static final String SERVER = "server";
-    public static final String CLIENT = "client";
-    public static final String LOCATOR = "locator";
-  }
-
-  /**
-   * The port used to create the locator for the tests
-   */
-  public static final int LOCATOR_PORT = 10334;
-
-  static final int NUM_LOCATORS = 1;
-  static final int NUM_SERVERS = 2;
-  static final int NUM_CLIENTS = 1;
+  private static final int NUM_LOCATORS = 1;
+  private static final int NUM_SERVERS = 2;
+  private static final int NUM_CLIENTS = 1;
   private static final String WITH_SSL_ARGUMENT = "-DwithSsl=true";
   private static final String WITH_SECURITY_MANAGER_ARGUMENT = "-DwithSecurityManager=true";
 
-  public static void configure(TestConfig testConfig) {
-    testConfig.role(LOCATOR, NUM_LOCATORS);
-    testConfig.role(SERVER, NUM_SERVERS);
-    testConfig.role(CLIENT, NUM_CLIENTS);
+  public static void configure(TestConfig config) {
+    role(config, LOCATOR, NUM_LOCATORS);
+    role(config, SERVER, NUM_SERVERS);
+    role(config, CLIENT, NUM_CLIENTS);
 
-    JvmParameters.configure(testConfig);
-    HeapParameters.configure(testConfig);
-    GcLoggingParameters.configure(testConfig);
-    GcParameters.configure(testConfig);
-    ProfilerParameters.configure(testConfig);
+    JvmParameters.configure(config);
+    HeapParameters.configure(config);
+    GcLoggingParameters.configure(config);
+    GcParameters.configure(config);
+    ProfilerParameters.configure(config);
 
-    addToTestConfig(testConfig, "withSsl", WITH_SSL_ARGUMENT);
-    addToTestConfig(testConfig, "withSecurityManager", WITH_SECURITY_MANAGER_ARGUMENT);
+    addToTestConfig(config, "withSsl", WITH_SSL_ARGUMENT);
+    addToTestConfig(config, "withSecurityManager", WITH_SECURITY_MANAGER_ARGUMENT);
 
-    testConfig.before(new StartLocator(LOCATOR_PORT), LOCATOR);
-    testConfig.before(new StartServer(LOCATOR_PORT), SERVER);
-    testConfig.before(new StartClient(LOCATOR_PORT), CLIENT);
+    before(config, new StartLocator(LOCATOR_PORT), LOCATOR);
+    before(config, new StartServer(LOCATOR_PORT), SERVER);
+    before(config, new StartClient(LOCATOR_PORT), CLIENT);
   }
 
-  private static void addToTestConfig(TestConfig testConfig, String systemPropertyKey,
-      String jvmArgument) {
-    if (Boolean.getBoolean(systemPropertyKey)) {
-      logger.info("Configuring JVMs to run with " + jvmArgument);
-      testConfig.jvmArgs(CLIENT, jvmArgument);
-      testConfig.jvmArgs(LOCATOR, jvmArgument);
-      testConfig.jvmArgs(SERVER, jvmArgument);
-    }
-  }
-
-  private static String[] appendIfNotEmpty(String[] a, String b) {
-    if (null == b || b.length() == 0) {
-      return a;
-    }
-
-    return Arrays.append(a, b);
-  }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSNIProxy.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSNIProxy.java
new file mode 100644
index 0000000..ee5ade1
--- /dev/null
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSNIProxy.java
@@ -0,0 +1,81 @@
+/*
+ * 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.benchmark.topology;
+
+import static org.apache.geode.benchmark.Config.after;
+import static org.apache.geode.benchmark.Config.before;
+import static org.apache.geode.benchmark.Config.role;
+import static org.apache.geode.benchmark.parameters.Utils.addToTestConfig;
+import static org.apache.geode.benchmark.parameters.Utils.configureGeodeProductJvms;
+import static org.apache.geode.benchmark.topology.Ports.LOCATOR_PORT;
+import static org.apache.geode.benchmark.topology.Ports.SERVER_PORT_FOR_SNI;
+import static org.apache.geode.benchmark.topology.RoleKinds.GEODE_PRODUCT;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.PROXY;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
+
+import java.util.stream.Stream;
+
+import org.apache.geode.benchmark.parameters.GcLoggingParameters;
+import org.apache.geode.benchmark.parameters.GcParameters;
+import org.apache.geode.benchmark.parameters.HeapParameters;
+import org.apache.geode.benchmark.parameters.JvmParameters;
+import org.apache.geode.benchmark.parameters.ProfilerParameters;
+import org.apache.geode.benchmark.tasks.DefineHostNamingsOffPlatformTask;
+import org.apache.geode.benchmark.tasks.StartClientSNI;
+import org.apache.geode.benchmark.tasks.StartLocatorSNI;
+import org.apache.geode.benchmark.tasks.StartServerSNI;
+import org.apache.geode.benchmark.tasks.StartSniProxy;
+import org.apache.geode.benchmark.tasks.StopClient;
+import org.apache.geode.benchmark.tasks.StopSniProxy;
+import org.apache.geode.perftest.TestConfig;
+
+public class ClientServerTopologyWithSNIProxy {
+  private static final int NUM_LOCATORS = 1;
+  private static final int NUM_SERVERS = 2;
+  private static final int NUM_CLIENTS = 1;
+  private static final int NUM_PROXIES = 1;
+  private static final String WITH_SSL_ARGUMENT = "-DwithSsl=true";
+  private static final String WITH_SECURITY_MANAGER_ARGUMENT = "-DwithSecurityManager=true";
+
+  public static void configure(TestConfig config) {
+    role(config, LOCATOR, NUM_LOCATORS);
+    role(config, SERVER, NUM_SERVERS);
+    role(config, CLIENT, NUM_CLIENTS);
+    role(config, PROXY, NUM_PROXIES);
+
+    JvmParameters.configure(config);
+    HeapParameters.configure(config);
+    GcLoggingParameters.configure(config);
+    GcParameters.configure(config);
+    ProfilerParameters.configure(config);
+
+    configureGeodeProductJvms(config, WITH_SSL_ARGUMENT);
+    addToTestConfig(config, "withSecurityManager", WITH_SECURITY_MANAGER_ARGUMENT);
+
+    Stream.concat(Roles.rolesFor(GEODE_PRODUCT), Stream.of(PROXY))
+        .forEach(role -> before(config, new DefineHostNamingsOffPlatformTask(), role));
+
+    before(config, new StartLocatorSNI(LOCATOR_PORT), LOCATOR);
+    before(config, new StartServerSNI(LOCATOR_PORT, SERVER_PORT_FOR_SNI), SERVER);
+    before(config, new StartSniProxy(LOCATOR_PORT), PROXY);
+    before(config, new StartClientSNI(LOCATOR_PORT), CLIENT);
+
+    after(config, new StopClient(), CLIENT);
+    after(config, new StopSniProxy(), PROXY);
+  }
+
+}
diff --git a/harness/src/main/java/org/apache/geode/perftest/Task.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/HostNamingOffPlatform.java
similarity index 69%
copy from harness/src/main/java/org/apache/geode/perftest/Task.java
copy to geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/HostNamingOffPlatform.java
index a764ae5..f25b690 100644
--- a/harness/src/main/java/org/apache/geode/perftest/Task.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/HostNamingOffPlatform.java
@@ -15,18 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.geode.perftest;
+package org.apache.geode.benchmark.topology;
 
-import java.io.Serializable;
+import java.net.InetAddress;
 
-/**
- * A single task in a test, such as initializing a member
- * or doing a single operation during the workload phase.
- */
-public interface Task extends Serializable {
+public class HostNamingOffPlatform {
+  public final String externalName;
+  public final InetAddress internalAddy;
 
-  /**
-   * Execute the task on the remote machine
-   */
-  void run(TestContext context) throws Exception;
+  public HostNamingOffPlatform(final String externalName, final InetAddress internalAddy) {
+    this.externalName = externalName;
+    this.internalAddy = internalAddy;
+  }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractPartitionedFunctionBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Ports.java
similarity index 59%
copy from geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractPartitionedFunctionBenchmark.java
copy to geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Ports.java
index fdbd166..b4f0f58 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractPartitionedFunctionBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Ports.java
@@ -12,17 +12,21 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.benchmark.topology;
 
-package org.apache.geode.benchmark.tests;
+public class Ports {
+  /**
+   * The port used to create the locator for the tests
+   */
+  public static final int LOCATOR_PORT = 10334;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+  /**
+   * With an SNI proxy, both the locator ports and the server ports
+   * have to be well-known (static) since the proxy has to know them
+   * and, in general, SNI proxies don't have visibility into locator
+   * responses carrying server port numbers.
+   */
+  public static final int SERVER_PORT_FOR_SNI = 40404;
 
-import org.apache.geode.benchmark.tasks.CreatePartitionedRegion;
-import org.apache.geode.perftest.TestConfig;
-
-abstract class AbstractPartitionedFunctionBenchmark extends AbstractFunctionBenchmark {
-  @Override
-  protected void configureRegion(TestConfig config) {
-    config.before(new CreatePartitionedRegion(), SERVER);
-  }
+  public static final int SNI_PROXY_PORT = 15443;
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractReplicatedFunctionBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/RoleKinds.java
similarity index 63%
copy from geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractReplicatedFunctionBenchmark.java
copy to geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/RoleKinds.java
index 6a9fa17..2198278 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/AbstractReplicatedFunctionBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/RoleKinds.java
@@ -12,17 +12,8 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
+package org.apache.geode.benchmark.topology;
 
-package org.apache.geode.benchmark.tests;
-
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
-
-import org.apache.geode.benchmark.tasks.CreateReplicatedRegion;
-import org.apache.geode.perftest.TestConfig;
-
-abstract class AbstractReplicatedFunctionBenchmark extends AbstractFunctionBenchmark {
-  @Override
-  protected void configureRegion(TestConfig config) {
-    config.before(new CreateReplicatedRegion(), SERVER);
-  }
+public enum RoleKinds {
+  GEODE_PRODUCT, SUPPORTING;
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Roles.java
similarity index 53%
copy from geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionBenchmark.java
copy to geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Roles.java
index e5b2cce..82cb9aa 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/ReplicatedFunctionExecutionBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Roles.java
@@ -12,27 +12,31 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.benchmark.tests;
+package org.apache.geode.benchmark.topology;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.RoleKinds.GEODE_PRODUCT;
+import static org.apache.geode.benchmark.topology.RoleKinds.SUPPORTING;
 
-import org.junit.jupiter.api.Test;
+import java.util.Arrays;
+import java.util.stream.Stream;
 
-import org.apache.geode.benchmark.tasks.ExecuteFunction;
-import org.apache.geode.perftest.TestConfig;
-import org.apache.geode.perftest.TestRunners;
+/**
+ * All roles defined for the JVMs created for the benchmark
+ */
+public enum Roles {
+  SERVER(GEODE_PRODUCT),
+  CLIENT(GEODE_PRODUCT),
+  LOCATOR(GEODE_PRODUCT),
+  PROXY(SUPPORTING);
 
-public class ReplicatedFunctionExecutionBenchmark extends AbstractReplicatedFunctionBenchmark {
+  public final RoleKinds roleKind;
 
-  @Test
-  public void run() throws Exception {
-    TestRunners.defaultRunner().runTest(this);
+  public static Stream<Roles> rolesFor(final RoleKinds roleKind) {
+    return Arrays.stream(Roles.values())
+        .filter(role -> role.roleKind == roleKind);
   }
 
-  @Override
-  public TestConfig configure() {
-    TestConfig config = super.configure();
-    config.workload(new ExecuteFunction(), CLIENT);
-    return config;
+  Roles(final RoleKinds roleKind) {
+    this.roleKind = roleKind;
   }
 }
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/GcLoggingParametersTest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/GcLoggingParametersTest.java
index 4814603..f1caa05 100644
--- a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/GcLoggingParametersTest.java
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/GcLoggingParametersTest.java
@@ -15,9 +15,9 @@
 
 package org.apache.geode.benchmark.parameters;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.LOCATOR;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.util.Properties;
@@ -77,21 +77,27 @@ class GcLoggingParametersTest {
   }
 
   private void assertThatJava8GcLog(TestConfig testConfig) {
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).contains("-Xloggc:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(SERVER)).contains("-Xloggc:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).contains("-Xloggc:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).doesNotContain("-Xlog:gc:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(SERVER)).doesNotContain("-Xlog:gc:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).doesNotContain("-Xlog:gc:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-Xloggc:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).contains("-Xloggc:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).contains("-Xloggc:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name()))
+        .doesNotContain("-Xlog:gc:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(SERVER.name()))
+        .doesNotContain("-Xlog:gc:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name()))
+        .doesNotContain("-Xlog:gc:OUTPUT_DIR/gc.log");
   }
 
   private void assertThatJava9GcLog(TestConfig testConfig) {
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).contains("-Xlog:gc*:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(SERVER)).contains("-Xlog:gc*:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).contains("-Xlog:gc*:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).doesNotContain("-Xloggc:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(SERVER)).doesNotContain("-Xloggc:OUTPUT_DIR/gc.log");
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).doesNotContain("-Xloggc:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-Xlog:gc*:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).contains("-Xlog:gc*:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).contains("-Xlog:gc*:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name()))
+        .doesNotContain("-Xloggc:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(SERVER.name()))
+        .doesNotContain("-Xloggc:OUTPUT_DIR/gc.log");
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name()))
+        .doesNotContain("-Xloggc:OUTPUT_DIR/gc.log");
   }
 
 }
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/GcParametersTest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/GcParametersTest.java
index e3f4bc6..9e456fd 100644
--- a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/GcParametersTest.java
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/GcParametersTest.java
@@ -15,9 +15,9 @@
 
 package org.apache.geode.benchmark.parameters;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.LOCATOR;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
@@ -91,39 +91,45 @@ class GcParametersTest {
   }
 
   private void assertCms(TestConfig testConfig) {
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).contains(XX_USE_CONC_MARK_SWEEP_GC);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).contains(XX_USE_CONC_MARK_SWEEP_GC);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).contains(XX_USE_CONC_MARK_SWEEP_GC);
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).doesNotContain(XX_USE_G_1_GC);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).doesNotContain(XX_USE_G_1_GC);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).doesNotContain(XX_USE_G_1_GC);
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).doesNotContain(XX_USE_ZGC);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).doesNotContain(XX_USE_ZGC);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).doesNotContain(XX_USE_ZGC);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains(XX_USE_CONC_MARK_SWEEP_GC);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).contains(XX_USE_CONC_MARK_SWEEP_GC);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).contains(XX_USE_CONC_MARK_SWEEP_GC);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).doesNotContain(XX_USE_G_1_GC);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).doesNotContain(XX_USE_G_1_GC);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).doesNotContain(XX_USE_G_1_GC);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).doesNotContain(XX_USE_ZGC);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).doesNotContain(XX_USE_ZGC);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).doesNotContain(XX_USE_ZGC);
   }
 
   private void assertG1(TestConfig testConfig) {
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).contains(XX_USE_G_1_GC);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).contains(XX_USE_G_1_GC);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).contains(XX_USE_G_1_GC);
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).doesNotContain(XX_USE_ZGC);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).doesNotContain(XX_USE_ZGC);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).doesNotContain(XX_USE_ZGC);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains(XX_USE_G_1_GC);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).contains(XX_USE_G_1_GC);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).contains(XX_USE_G_1_GC);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name()))
+        .doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name()))
+        .doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name()))
+        .doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).doesNotContain(XX_USE_ZGC);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).doesNotContain(XX_USE_ZGC);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).doesNotContain(XX_USE_ZGC);
   }
 
   private void assertZ(TestConfig testConfig) {
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).contains(XX_USE_ZGC);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).contains(XX_USE_ZGC);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).contains(XX_USE_ZGC);
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).doesNotContain(XX_USE_G_1_GC);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).doesNotContain(XX_USE_G_1_GC);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).doesNotContain(XX_USE_G_1_GC);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains(XX_USE_ZGC);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).contains(XX_USE_ZGC);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).contains(XX_USE_ZGC);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name()))
+        .doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name()))
+        .doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name()))
+        .doesNotContain(XX_USE_CONC_MARK_SWEEP_GC);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).doesNotContain(XX_USE_G_1_GC);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).doesNotContain(XX_USE_G_1_GC);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).doesNotContain(XX_USE_G_1_GC);
   }
 
 }
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/HeapParametersTest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/HeapParametersTest.java
index 9a06408..4303c6b 100644
--- a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/HeapParametersTest.java
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/parameters/HeapParametersTest.java
@@ -15,9 +15,9 @@
 
 package org.apache.geode.benchmark.parameters;
 
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.CLIENT;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.LOCATOR;
-import static org.apache.geode.benchmark.topology.ClientServerTopology.Roles.SERVER;
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.apache.geode.benchmark.topology.Roles.LOCATOR;
+import static org.apache.geode.benchmark.topology.Roles.SERVER;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.util.Properties;
@@ -61,11 +61,11 @@ class HeapParametersTest {
   }
 
   private void assertHeap(final TestConfig testConfig, final String heap) {
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).contains("-Xmx" + heap);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).contains("-Xmx" + heap);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).contains("-Xmx" + heap);
-    assertThat(testConfig.getJvmArgs().get(CLIENT)).contains("-Xms" + heap);
-    assertThat(testConfig.getJvmArgs().get(SERVER)).contains("-Xms" + heap);
-    assertThat(testConfig.getJvmArgs().get(LOCATOR)).contains("-Xms" + heap);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-Xmx" + heap);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).contains("-Xmx" + heap);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).contains("-Xmx" + heap);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-Xms" + heap);
+    assertThat(testConfig.getJvmArgs().get(SERVER.name())).contains("-Xms" + heap);
+    assertThat(testConfig.getJvmArgs().get(LOCATOR.name())).contains("-Xms" + heap);
   }
 }
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tasks/StartSniProxyTest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tasks/StartSniProxyTest.java
new file mode 100644
index 0000000..2b8cd21
--- /dev/null
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tasks/StartSniProxyTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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.benchmark.tasks;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.stream.Stream;
+
+import org.junit.jupiter.api.Test;
+
+class StartSniProxyTest {
+
+  @Test
+  public void generateConfigTest() {
+    final StartSniProxy starter = new StartSniProxy(42);
+    final String config =
+        starter.generateHaProxyConfig(Stream.of("locator-one-internal"),
+            Stream.of("locator-one-external"),
+            Stream.of("server-one-internal", "server-two-internal"),
+            Stream.of("server-one-external", "server-two-external"));
+    assertThat(config).isEqualTo("global\n"
+        + "  log stdout format raw local0 debug\n"
+        + "  maxconn 5000\n"
+        + "defaults\n"
+        + "  log global\n"
+        /*
+         * We're leaving timeouts unspecified so they are infinite. Benchmarks do bad things
+         * when the proxy breaks connections.
+         */
+        // + " timeout client 100s\n"
+        // + " timeout connect 100s\n"
+        // + " timeout server 100s\n"
+        + "frontend sniproxy\n"
+        + "  bind *:15443\n"
+        + "  mode tcp\n"
+        + "  tcp-request inspect-delay 5s\n"
+        + "  tcp-request content accept if { req_ssl_hello_type 1 }\n"
+        + "  use_backend locators-locator-one-internal if { req.ssl_sni -i locator-one-external }\n"
+        + "  use_backend servers-server-one-internal if { req.ssl_sni -i server-one-external }\n"
+        + "  use_backend servers-server-two-internal if { req.ssl_sni -i server-two-external }\n"
+        + "  default_backend locators-locator-one-internal\n"
+        + "backend locators-locator-one-internal\n"
+        + "  mode tcp\n"
+        + "  server locator1 locator-one-internal:42\n"
+        + "backend servers-server-one-internal\n"
+        + "  mode tcp\n"
+        + "  server server1 server-one-internal:40404\n"
+        + "backend servers-server-two-internal\n"
+        + "  mode tcp\n"
+        + "  server server1 server-two-internal:40404\n");
+  }
+}
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/GeodeBenchmarkTest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/GeodeBenchmarkTest.java
new file mode 100644
index 0000000..6beb78c
--- /dev/null
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/GeodeBenchmarkTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.benchmark.tests;
+
+import static org.apache.geode.benchmark.topology.Ports.LOCATOR_PORT;
+import static org.apache.geode.benchmark.topology.Roles.PROXY;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import org.apache.geode.benchmark.tasks.StartSniProxy;
+import org.apache.geode.perftest.TestConfig;
+import org.apache.geode.perftest.TestStep;
+
+/**
+ * Verify that if withSniProxy system property is set at all and if it is set to anything but
+ * false, that we get an SNI proxy in our topology. If the system property is not set at all, or
+ * if it is set to (exactly) "false" then we do not get an SNI proxy in our topology.
+ */
+class GeodeBenchmarkTest {
+
+  private TestConfig config;
+  private TestStep startProxyStep;
+
+  @BeforeEach
+  public void beforeEach() {
+    startProxyStep =
+        new TestStep(new StartSniProxy(LOCATOR_PORT), new String[] {PROXY.name()});
+  }
+
+  @AfterAll
+  public static void afterAll() {
+    System.clearProperty("withSniProxy");
+  }
+
+  @Test
+  public void withoutSniProxy() {
+    System.clearProperty("withSniProxy");
+    config = GeodeBenchmark.createConfig();
+    assertThat(config.getBefore()).doesNotContain(startProxyStep);
+  }
+
+  @Test
+  public void withSniProxyFalse() {
+    System.setProperty("withSniProxy", "false");
+    config = GeodeBenchmark.createConfig();
+    assertThat(config.getBefore()).doesNotContain(startProxyStep);
+  }
+
+  @Test
+  public void withSniProxyTrue() {
+    System.setProperty("withSniProxy", "true");
+    config = GeodeBenchmark.createConfig();
+    assertThat(config.getBefore()).contains(startProxyStep);
+  }
+
+  @Test
+  public void withSniProxyNotLowercaseFalse() {
+    System.setProperty("withSniProxy", "AnythING");
+    config = GeodeBenchmark.createConfig();
+    assertThat(config.getBefore()).contains(startProxyStep);
+  }
+
+}
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyTest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyTest.java
index 684929d..4fed317 100644
--- a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyTest.java
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyTest.java
@@ -16,6 +16,7 @@
 package org.apache.geode.benchmark.topology;
 
 
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.util.Properties;
@@ -45,21 +46,22 @@ public class ClientServerTopologyTest {
     System.setProperty("withSsl", "true");
     TestConfig testConfig = new TestConfig();
     ClientServerTopology.configure(testConfig);
-    assertThat(testConfig.getJvmArgs().get("client")).contains("-DwithSsl=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSsl=true");
   }
 
   @Test
   public void configWithNoSsl() {
     TestConfig testConfig = new TestConfig();
     ClientServerTopology.configure(testConfig);
-    assertThat(testConfig.getJvmArgs().get("client")).doesNotContain("-DwithSsl=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).doesNotContain("-DwithSsl=true");
   }
 
   @Test
   public void configWithoutSecurityManager() {
     TestConfig testConfig = new TestConfig();
     ClientServerTopology.configure(testConfig);
-    assertThat(testConfig.getJvmArgs().get("client")).doesNotContain("-DwithSecurityManager=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name()))
+        .doesNotContain("-DwithSecurityManager=true");
   }
 
   @Test
@@ -67,7 +69,7 @@ public class ClientServerTopologyTest {
     System.setProperty("withSecurityManager", "true");
     TestConfig testConfig = new TestConfig();
     ClientServerTopology.configure(testConfig);
-    assertThat(testConfig.getJvmArgs().get("client")).contains("-DwithSecurityManager=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSecurityManager=true");
   }
 
   @Test
@@ -79,7 +81,7 @@ public class ClientServerTopologyTest {
 
     ClientServerTopology.configure(testConfig);
 
-    assertThat(testConfig.getJvmArgs().get("client")).contains("-DwithSecurityManager=true");
-    assertThat(testConfig.getJvmArgs().get("client")).contains("-DwithSsl=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSecurityManager=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSsl=true");
   }
 }
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/GeodeBenchmark.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSNIProxyTest.java
similarity index 52%
copy from geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/GeodeBenchmark.java
copy to geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSNIProxyTest.java
index 7551eec..51d7ee5 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/GeodeBenchmark.java
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSNIProxyTest.java
@@ -13,36 +13,40 @@
  * the License.
  */
 
-package org.apache.geode.benchmark.tests;
+package org.apache.geode.benchmark.topology;
 
-import static java.util.concurrent.TimeUnit.MINUTES;
+
+import static org.apache.geode.benchmark.topology.Roles.CLIENT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.Properties;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import org.apache.geode.perftest.TestConfig;
 
-public class GeodeBenchmark {
+public class ClientServerTopologyWithSNIProxyTest {
 
-  /**
-   * Warm up time for the benchmark running on the default runner
-   */
-  private static final long WARM_UP_TIME = MINUTES.toSeconds(1);
+  private Properties systemProperties;
 
-  /**
-   * Total duration for which the benchmark will run on the default runner
-   */
-  private static final long BENCHMARK_DURATION = MINUTES.toSeconds(5);
+  @BeforeEach
+  public void beforeEach() {
+    systemProperties = (Properties) System.getProperties().clone();
+  }
 
-  /**
-   * Number of threads to run benchmark.
-   */
-  private static final int THREADS = Runtime.getRuntime().availableProcessors() * 10;
+  @AfterEach
+  public void afterEach() {
+    System.setProperties(systemProperties);
+  }
 
 
-  public static TestConfig createConfig() {
+  @Test
+  public void configWithNoSsl() {
     TestConfig testConfig = new TestConfig();
-    testConfig.warmupSeconds(WARM_UP_TIME);
-    testConfig.durationSeconds(BENCHMARK_DURATION);
-    testConfig.threads(THREADS);
-    return testConfig;
+    ClientServerTopologyWithSNIProxy.configure(testConfig);
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSsl=true");
   }
 
 }
diff --git a/harness/build.gradle b/harness/build.gradle
index 7ee2951..e8096d3 100644
--- a/harness/build.gradle
+++ b/harness/build.gradle
@@ -26,7 +26,12 @@ def geodeVersion = project.hasProperty('geodeVersion') ? project.findProperty('g
 def isCI = project.hasProperty('ci') ? project.findProperty('ci') : 0
 
 repositories {
+  /*
+   This is used in CI to benchmark various new/old versions of Geode.
+   Also useful in dev where you can clone geode and publishToMavenLocal
+   */
   mavenLocal()
+  // fall back to mavenCentral, which has lots of released versions of Geode
   mavenCentral()
 }
 
diff --git a/harness/src/main/java/org/apache/geode/perftest/Task.java b/harness/src/main/java/org/apache/geode/perftest/Task.java
index a764ae5..58319d8 100644
--- a/harness/src/main/java/org/apache/geode/perftest/Task.java
+++ b/harness/src/main/java/org/apache/geode/perftest/Task.java
@@ -19,6 +19,7 @@ package org.apache.geode.perftest;
 
 import java.io.Serializable;
 
+
 /**
  * A single task in a test, such as initializing a member
  * or doing a single operation during the workload phase.
diff --git a/harness/src/main/java/org/apache/geode/perftest/TestConfig.java b/harness/src/main/java/org/apache/geode/perftest/TestConfig.java
index 9d66cd6..69043dd 100644
--- a/harness/src/main/java/org/apache/geode/perftest/TestConfig.java
+++ b/harness/src/main/java/org/apache/geode/perftest/TestConfig.java
@@ -140,13 +140,6 @@ public class TestConfig implements Serializable {
   }
 
   /**
-   * Return the total number of JVMs required to run this test
-   */
-  public int getTotalJVMs() {
-    return roles.values().stream().mapToInt(Integer::intValue).sum();
-  }
-
-  /**
    * Add JVM arguments used to launch JVMs for a particular role
    *
    * If multiple calls to this method are made for the same role, the new JVM arguments
@@ -161,24 +154,4 @@ public class TestConfig implements Serializable {
     return Collections.unmodifiableMap(jvmArgs);
   }
 
-  public static class TestStep {
-    private final Task task;
-    private final String[] roles;
-
-    public TestStep(Task task, String[] roles) {
-      if (roles == null || roles.length == 0) {
-        throw new IllegalStateException("Task " + task + " must be assigned to at least one role");
-      }
-      this.task = task;
-      this.roles = roles;
-    }
-
-    public Task getTask() {
-      return task;
-    }
-
-    public String[] getRoles() {
-      return roles;
-    }
-  }
 }
diff --git a/harness/src/main/java/org/apache/geode/perftest/TestStep.java b/harness/src/main/java/org/apache/geode/perftest/TestStep.java
new file mode 100644
index 0000000..32ded40
--- /dev/null
+++ b/harness/src/main/java/org/apache/geode/perftest/TestStep.java
@@ -0,0 +1,62 @@
+/*
+ * 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.perftest;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+public class TestStep {
+  private final Task task;
+  private final Set<String> roles;
+
+  public TestStep(Task task, String[] roles) {
+    if (roles == null || roles.length == 0) {
+      throw new IllegalStateException("Task " + task + " must be assigned to at least one role");
+    }
+    this.task = task;
+    this.roles = new HashSet<>(Arrays.asList(roles));
+  }
+
+  public Task getTask() {
+    return task;
+  }
+
+  public String[] getRoles() {
+    return roles.toArray(new String[0]);
+  }
+
+  @Override
+  public boolean equals(final Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    final TestStep testStep = (TestStep) o;
+    return task.equals(testStep.task) &&
+        roles.equals(testStep.roles);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(task, roles);
+  }
+}
diff --git a/harness/src/main/java/org/apache/geode/perftest/infrastructure/local/LocalInfrastructure.java b/harness/src/main/java/org/apache/geode/perftest/infrastructure/local/LocalInfrastructure.java
index 9bcf27a..596d99d 100644
--- a/harness/src/main/java/org/apache/geode/perftest/infrastructure/local/LocalInfrastructure.java
+++ b/harness/src/main/java/org/apache/geode/perftest/infrastructure/local/LocalInfrastructure.java
@@ -66,7 +66,7 @@ public class LocalInfrastructure implements Infrastructure {
     builder.inheritIO();
     builder.directory(((LocalNode) node).getWorkingDir());
 
-    System.out.println(String.format("Lauching %s>%s", ((LocalNode) node).getWorkingDir(),
+    System.out.println(String.format("Launching %s>%s", ((LocalNode) node).getWorkingDir(),
         String.join(" ", shellCommand)));
     Process process = builder.start();
     processList.add(process);
diff --git a/harness/src/main/java/org/apache/geode/perftest/jvms/JVMLauncher.java b/harness/src/main/java/org/apache/geode/perftest/jvms/JVMLauncher.java
index 6b5cfce..13c2a44 100644
--- a/harness/src/main/java/org/apache/geode/perftest/jvms/JVMLauncher.java
+++ b/harness/src/main/java/org/apache/geode/perftest/jvms/JVMLauncher.java
@@ -91,7 +91,7 @@ class JVMLauncher {
     command.add("-D" + RemoteJVMFactory.JVM_ID + "=" + jvmConfig.getId());
     command.add("-D" + RemoteJVMFactory.OUTPUT_DIR + "=" + jvmConfig.getOutputDir());
 
-    if (Boolean.getBoolean("withSsl")) {
+    if (jvmConfig.getJvmArgs().contains("-DwithSsl=true")) {
       command
           .add("-Dgemfire." + SSL_KEYSTORE + "=" + jvmConfig.getLibDir() + "/temp-self-signed.jks");
       command.add("-Dgemfire." + SSL_KEYSTORE_PASSWORD + "=123456");
diff --git a/harness/src/main/java/org/apache/geode/perftest/runner/DefaultTestRunner.java b/harness/src/main/java/org/apache/geode/perftest/runner/DefaultTestRunner.java
index 841a1c6..3f26b00 100644
--- a/harness/src/main/java/org/apache/geode/perftest/runner/DefaultTestRunner.java
+++ b/harness/src/main/java/org/apache/geode/perftest/runner/DefaultTestRunner.java
@@ -36,6 +36,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunner;
+import org.apache.geode.perftest.TestStep;
 import org.apache.geode.perftest.infrastructure.InfrastructureFactory;
 import org.apache.geode.perftest.jvms.RemoteJVMFactory;
 import org.apache.geode.perftest.jvms.RemoteJVMs;
@@ -69,7 +70,6 @@ public class DefaultTestRunner implements TestRunner {
 
   protected void runTest(TestConfig config, String testName)
       throws Exception {
-    int nodes = config.getTotalJVMs();
     File benchmarkOutput = new File(outputDir, testName);
     if (benchmarkOutput.exists()) {
       throw new IllegalStateException(
@@ -116,7 +116,7 @@ public class DefaultTestRunner implements TestRunner {
     Map<String, Integer> roles = config.getRoles();
     Map<String, List<String>> jvmArgs = config.getJvmArgs();
 
-    logger.info("Lauching JVMs...");
+    logger.info("Launching JVMs...");
     // launch JVMs in parallel, hook them up
     RemoteJVMs remoteJVMs = remoteJvmFactory.launch(roles, jvmArgs);
     try {
@@ -154,7 +154,7 @@ public class DefaultTestRunner implements TestRunner {
     return versionProperties;
   }
 
-  private void runTasks(List<TestConfig.TestStep> steps,
+  private void runTasks(List<TestStep> steps,
       RemoteJVMs remoteJVMs) {
     steps.forEach(testStep -> {
       remoteJVMs.execute(testStep.getTask(), testStep.getRoles());
diff --git a/infrastructure/scripts/aws/image/files/docker-compose.yml b/infrastructure/scripts/aws/image/files/docker-compose.yml
new file mode 100644
index 0000000..78faf55
--- /dev/null
+++ b/infrastructure/scripts/aws/image/files/docker-compose.yml
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+version: '3'
+services:
+  haproxy:
+    container_name: 'haproxy'
+    image: 'haproxy:2.1'
+    ports:
+      - "15443:15443"
+    volumes:
+      - ./haproxy.cfg:/usr/local/etc/haproxy/haproxy.cfg:ro
+
diff --git a/infrastructure/scripts/aws/image/packer.json b/infrastructure/scripts/aws/image/packer.json
index 06cec25..0375e25 100644
--- a/infrastructure/scripts/aws/image/packer.json
+++ b/infrastructure/scripts/aws/image/packer.json
@@ -40,6 +40,37 @@
       "destination": "/tmp/defaults.cfg"
     },
     {
+      "type": "file",
+      "source": "./files/docker-compose.yml",
+      "destination": "/home/geode/docker-compose.yml"
+    },
+    {
+      "type": "shell",
+      "inline": [
+        "# install Docker per https://docs.docker.com/engine/install/ubuntu/",
+        "sudo apt-get update",
+        "sudo apt-get install -y apt-transport-https ca-certificates curl gnupg-agent software-properties-common",
+        "curl -fsSL https://download.docker.com/linux/ubuntu/gpg | sudo apt-key add -",
+        "sudo apt-key fingerprint 0EBFCD88",
+        "sudo add-apt-repository \"deb [arch=amd64] https://download.docker.com/linux/ubuntu $(lsb_release -cs) stable\"",
+        "sudo apt-get update",
+        "sudo apt-get install -y docker-ce docker-ce-cli containerd.io"
+      ]
+    },
+    {
+      "type": "shell",
+      "inline": [
+        "sudo apt install -y docker-compose"
+      ]
+    },
+    {
+      "type": "shell",
+      "inline": [
+        "# permissions per https://docs.docker.com/engine/install/linux-postinstall/#manage-docker-as-a-non-root-user",
+        "sudo usermod -aG docker geode"
+      ]
+    },
+    {
       "type": "shell",
       "inline": [
         "sudo apt update",