You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by up...@apache.org on 2021/04/01 22:44:31 UTC

[geode-benchmarks] 02/05: Converting all of the withXXX properties to start with benchmark.

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

upthewaterspout pushed a commit to branch feature/redis-performance-testing
in repository https://gitbox.apache.org/repos/asf/geode-benchmarks.git

commit 2ccd1d762b761cab22dc04a8f6d0170dcab7ce0f
Author: Dan Smith <da...@vmware.com>
AuthorDate: Thu Apr 1 13:04:59 2021 -0700

    Converting all of the withXXX properties to start with benchmark.
    
    Converting all of the system properties we use in the benchmarks to start with the benchmark
    prefix. Changing the gradle build to copy all benchmark.* properties as system
    properties in the tst.
    
    For now, also copying the old set of withXXX properties to not break peoples scripts or CI.
---
 geode-benchmarks/build.gradle                      | 60 ++++++++++++++--------
 .../geode/benchmark/parameters/GcParameters.java   |  2 +-
 .../benchmark/parameters/GedisParameters.java      |  3 +-
 .../geode/benchmark/parameters/HeapParameters.java |  2 +-
 .../geode/benchmark/tests/GeodeBenchmark.java      |  2 +-
 .../benchmark/tests/redis/RedisBenchmark.java      |  4 +-
 .../benchmark/tests/redis/RedisHsetBenchmark.java  |  2 +-
 .../ClientServerTopologyWithRouterAndSniProxy.java |  4 +-
 .../topology/ClientServerTopologyWithSniProxy.java |  4 +-
 .../apache/geode/benchmark/topology/Topology.java  | 14 ++---
 .../topology/redis/ManualRedisTopology.java        |  2 +-
 .../benchmark/parameters/GcParametersTest.java     |  2 +-
 .../benchmark/parameters/HeapParametersTest.java   |  2 +-
 .../benchmark/tests/ClientServerBenchmarkTest.java | 12 ++---
 .../topology/ClientServerTopologyTest.java         | 23 +++++----
 .../ClientServerTopologyWithSniProxyTest.java      |  2 +-
 .../org/apache/geode/perftest/WorkloadConfig.java  |  6 +--
 .../apache/geode/perftest/jvms/JVMLauncher.java    |  2 +-
 .../perftest/yardstick/YardstickTaskTest.java      |  5 +-
 19 files changed, 88 insertions(+), 65 deletions(-)

diff --git a/geode-benchmarks/build.gradle b/geode-benchmarks/build.gradle
index 49b423f..49fa625 100644
--- a/geode-benchmarks/build.gradle
+++ b/geode-benchmarks/build.gradle
@@ -103,64 +103,80 @@ task benchmark(type: Test) {
   systemProperty 'TEST_HOSTS', project.findProperty('hosts')
   systemProperty 'TEST_METADATA', project.findProperty('metadata')
   systemProperty 'OUTPUT_DIR', outputDir
+
+  //Set all project properties starting with "benchmark." as system properties in the test
+  //JVM
+  project.properties.findAll {
+    it.key.startsWith("benchmark.")
+  }.each {
+    systemProperty(it.getKey(), it.getValue())
+  }
+
+  //------------------------------------------------------------
+  //Legacy properties - these properties were added before the benchmark.
+  //prefix convention. They will be passed on to the JVM for now to not break
+  //CI or peoples scripts. Remove these soon!
+  //------------------------------------------------------------
   if (project.hasProperty('withGc')) {
-    systemProperty 'withGc', project.findProperty('withGc')
+    systemProperty 'benchmark.withGc', project.findProperty('withGc')
   }
   if (project.hasProperty('withHeap')) {
-    systemProperty 'withHeap', project.findProperty('withHeap')
+    systemProperty 'benchmark.withHeap', project.findProperty('withHeap')
   }
   if (project.hasProperty('withThreads')) {
-    systemProperty 'withThreads', project.findProperty('withThreads')
+    systemProperty 'benchmark.withThreads', project.findProperty('withThreads')
   }
   if (project.hasProperty('withWarmup')) {
-    systemProperty 'withWarmup', project.findProperty('withWarmup')
+    systemProperty 'benchmark.withWarmup', project.findProperty('withWarmup')
   }
   if (project.hasProperty('withDuration')) {
-    systemProperty 'withDuration', project.findProperty('withDuration')
+    systemProperty 'benchmark.withDuration', project.findProperty('withDuration')
   }
 
   if (project.hasProperty('withMinKey')) {
-    systemProperty 'withMinKey', project.findProperty('withMinKey')
+    systemProperty 'benchmark.withMinKey', project.findProperty('withMinKey')
   }
   if (project.hasProperty('withMaxKey')) {
-    systemProperty 'withMaxKey', project.findProperty('withMaxKey')
+    systemProperty 'benchmark.withMaxKey', project.findProperty('withMaxKey')
   }
 
-  systemProperty 'withSsl', project.hasProperty('withSsl')
-  systemProperty 'withSslProtocols', project.findProperty('withSslProtocols')
-  systemProperty 'withSslCiphers', project.findProperty('withSslCiphers')
+  systemProperty 'benchmark.withSsl', project.hasProperty('withSsl')
+  systemProperty 'benchmark.withSslProtocols', project.findProperty('withSslProtocols')
+  systemProperty 'benchmark.withSslCiphers', project.findProperty('withSslCiphers')
 
   if (project.hasProperty('withSniProxy')) {
-    systemProperty 'withSniProxy', project.findProperty('withSniProxy')
+    systemProperty 'benchmark.withSniProxy', project.findProperty('withSniProxy')
   }
-  systemProperty 'withSniProxyImage', project.findProperty('withSniProxyImage')
+  systemProperty 'benchmark.withSniProxyImage', project.findProperty('withSniProxyImage')
 
   if (project.hasProperty('withRouter')) {
-    systemProperty 'withRouter', project.findProperty('withRouter')
+    systemProperty 'benchmark.withRouter', project.findProperty('withRouter')
   }
-  systemProperty 'withRouterImage', project.findProperty('withRouterImage')
+  systemProperty 'benchmark.withRouterImage', project.findProperty('withRouterImage')
 
   if (project.hasProperty('withRedisClient')) {
-    systemProperty 'withRedisClient', project.findProperty('withRedisClient')
+    systemProperty 'benchmark.withRedisClient', project.findProperty('withRedisClient')
   }
 
   if (project.hasProperty('withRedisCluster')) {
-    systemProperty 'withRedisCluster', project.findProperty('withRedisCluster')
+    systemProperty 'benchmark.withRedisCluster', project.findProperty('withRedisCluster')
   }
 
   if (project.hasProperty('withRedisServers')) {
-    systemProperty 'withRedisServers', project.findProperty('withRedisServers')
+    systemProperty 'benchmark.withRedisServers', project.findProperty('withRedisServers')
   }
 
   if (project.hasProperty('withReplicas')) {
-    systemProperty 'withReplicas', project.findProperty('withReplicas')
+    systemProperty 'benchmark.withReplicas', project.findProperty('withReplicas')
   }
 
-  systemProperty 'withValidation', project.hasProperty('withValidation')
-  systemProperty 'withAsyncReplication', project.hasProperty('withAsyncReplication')
+  systemProperty 'benchmark.withValidation', project.hasProperty('withValidation')
+  systemProperty 'benchmark.withAsyncReplication', project.hasProperty('withAsyncReplication')
 
-  systemProperty 'withSecurityManager', project.hasProperty('withSecurityManager')
-  systemProperty 'benchmark.profiler.argument', project.findProperty('benchmark.profiler.argument')
+  systemProperty 'benchmark.withSecurityManager', project.hasProperty('withSecurityManager')
+  //------------------------------------------------------------
+  //End legacy properties
+  //------------------------------------------------------------
 
 
   doFirst {
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 6239c0a..9398e5c 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
@@ -28,7 +28,7 @@ public class GcParameters {
 
   public static void configure(final TestConfig testConfig) {
     final GcImplementation gcImplementation =
-        GcImplementation.valueOf(System.getProperty("withGc", "CMS"));
+        GcImplementation.valueOf(System.getProperty("benchmark.withGc", "CMS"));
     logger.info("Configuring {} GC.", gcImplementation);
     switch (gcImplementation) {
       case CMS:
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GedisParameters.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GedisParameters.java
index c2806eb..1e641bb 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GedisParameters.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/parameters/GedisParameters.java
@@ -28,7 +28,8 @@ public class GedisParameters {
   public static void configure(final TestConfig testConfig) {
     logger.info("Configuring Gedis parameters.");
 
-    testConfig.jvmArgs(SERVER.name(), "-Dredis.replicas=" + Integer.getInteger("withReplicas", 1));
+    testConfig.jvmArgs(SERVER.name(),
+        "-Dredis.replicas=" + Integer.getInteger("benchmark.withReplicas", 1));
   }
 
 }
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 fd58b6a..91b45a0 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
@@ -26,7 +26,7 @@ public class HeapParameters {
   private static final Logger logger = LoggerFactory.getLogger(HeapParameters.class);
 
   public static void configure(final TestConfig testConfig) {
-    final String heap = System.getProperty("withHeap", "8g");
+    final String heap = System.getProperty("benchmark.withHeap", "8g");
     logger.info("Configuring heap parameters {}.", heap);
     configureGeodeProductJvms(testConfig, "-Xmx" + heap, "-Xms" + heap);
   }
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 3276b72..c090ea9 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
@@ -36,7 +36,7 @@ public class GeodeBenchmark {
    */
   private static final int THREADS = Runtime.getRuntime().availableProcessors() * 10;
 
-  public static final String WITH_VALIDATION_PROPERTY = "withValidation";
+  public static final String WITH_VALIDATION_PROPERTY = "benchmark.withValidation";
 
   public static TestConfig createConfig() {
     TestConfig config = new TestConfig();
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/redis/RedisBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/redis/RedisBenchmark.java
index f4f70b2..409b8db 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/redis/RedisBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/redis/RedisBenchmark.java
@@ -36,8 +36,8 @@ import org.apache.geode.perftest.TestConfig;
 
 public class RedisBenchmark implements PerformanceTest {
 
-  public static final String WITH_REDIS_CLIENT_PROPERTY = "withRedisClient";
-  public static final String WITH_REDIS_CLUSTER_PROPERTY = "withRedisCluster";
+  public static final String WITH_REDIS_CLIENT_PROPERTY = "benchmark.withRedisClient";
+  public static final String WITH_REDIS_CLUSTER_PROPERTY = "benchmark.withRedisCluster";
 
   public static final String REDIS_SERVERS_ATTRIBUTE = "RedisBenchmark.Servers";
 
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/redis/RedisHsetBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/redis/RedisHsetBenchmark.java
index 8cd1aca..9c296a6 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/redis/RedisHsetBenchmark.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/redis/RedisHsetBenchmark.java
@@ -37,7 +37,7 @@ import org.apache.geode.perftest.TestRunners;
 public class RedisHsetBenchmark extends RedisBenchmark {
 
   private LongRange keyRange =
-      new LongRange(getLong("withMinKey", 0), getLong("withMaxKey", 1000000));
+      new LongRange(getLong("benchmark.withMinKey", 0), getLong("benchmark.withMaxKey", 1000000));
 
   @Test
   public void run() throws Exception {
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithRouterAndSniProxy.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithRouterAndSniProxy.java
index 502df52..b5b3467 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithRouterAndSniProxy.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithRouterAndSniProxy.java
@@ -35,8 +35,8 @@ import org.apache.geode.benchmark.tasks.StopRouter;
 import org.apache.geode.perftest.TestConfig;
 
 public class ClientServerTopologyWithRouterAndSniProxy extends ClientServerTopologyWithSniProxy {
-  public static final String WITH_ROUTER_PROPERTY = "withRouter";
-  public static final String WITH_ROUTER_IMAGE_PROPERTY = "withRouterImage";
+  public static final String WITH_ROUTER_PROPERTY = "benchmark.withRouter";
+  public static final String WITH_ROUTER_IMAGE_PROPERTY = "benchmark.withRouterImage";
 
   private static final int NUM_LOCATORS = 1;
   private static final int NUM_SERVERS = 2;
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
index 57af8fe..fde623f 100644
--- 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
@@ -40,8 +40,8 @@ import org.apache.geode.benchmark.tasks.StopSniProxy;
 import org.apache.geode.perftest.TestConfig;
 
 public class ClientServerTopologyWithSniProxy extends Topology {
-  public static final String WITH_SNI_PROXY_PROPERTY = "withSniProxy";
-  public static final String WITH_SNI_PROXY_IMAGE_PROPERTY = "withSniProxyImage";
+  public static final String WITH_SNI_PROXY_PROPERTY = "benchmark.withSniProxy";
+  public static final String WITH_SNI_PROXY_IMAGE_PROPERTY = "benchmark.withSniProxyImage";
 
   private static final int NUM_LOCATORS = 1;
   private static final int NUM_SERVERS = 2;
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Topology.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Topology.java
index 3e46bdf..b3336c2 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Topology.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Topology.java
@@ -27,14 +27,14 @@ import org.apache.geode.benchmark.parameters.ProfilerParameters;
 import org.apache.geode.perftest.TestConfig;
 
 public abstract class Topology {
-  public static final String WITH_SSL_PROPERTY = "withSsl";
-  static final String WITH_SSL_ARGUMENT = "-DwithSsl=true";
+  public static final String WITH_SSL_PROPERTY = "benchmark.withSsl";
+  static final String WITH_SSL_ARGUMENT = "-Dbenchmark.withSsl=true";
 
-  public static final String WITH_SSL_PROTOCOLS_PROPERTY = "withSslProtocols";
-  public static final String WITH_SSL_CIPHERS_PROPERTY = "withSslCiphers";
+  public static final String WITH_SSL_PROTOCOLS_PROPERTY = "benchmark.withSslProtocols";
+  public static final String WITH_SSL_CIPHERS_PROPERTY = "benchmark.withSslCiphers";
 
-  public static final String WITH_SECURITY_MANAGER_PROPERTY = "withSecurityManager";
-  static final String WITH_SECURITY_MANAGER_ARGUMENT = "-DwithSecurityManager=true";
+  public static final String WITH_SECURITY_MANAGER_PROPERTY = "benchmark.withSecurityManager";
+  static final String WITH_SECURITY_MANAGER_ARGUMENT = "-Dbenchmark.withSecurityManager=true";
 
   protected static void configureCommon(TestConfig config) {
     JvmParameters.configure(config);
@@ -48,7 +48,7 @@ public abstract class Topology {
     addToTestConfig(config, WITH_SSL_CIPHERS_PROPERTY);
     addToTestConfig(config, WITH_SECURITY_MANAGER_PROPERTY, WITH_SECURITY_MANAGER_ARGUMENT);
 
-    if (getBoolean("withAsyncReplication")) {
+    if (getBoolean("benchmark.withAsyncReplication")) {
       config.jvmArgs(SERVER.name(), "-Dgemfire.disablePartitionedRegionBucketAck=true");
     }
 
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/redis/ManualRedisTopology.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/redis/ManualRedisTopology.java
index f239772..4b8eb56 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/redis/ManualRedisTopology.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/redis/ManualRedisTopology.java
@@ -33,7 +33,7 @@ import org.apache.geode.perftest.TestConfig;
 public class ManualRedisTopology extends Topology {
   private static final int NUM_CLIENTS = 4;
 
-  public static final String WITH_REDIS_SERVERS_PROPERTY = "withRedisServers";
+  public static final String WITH_REDIS_SERVERS_PROPERTY = "benchmark.withRedisServers";
 
   public static void configure(TestConfig config) {
     role(config, CLIENT, NUM_CLIENTS);
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 9e456fd..5003187 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
@@ -30,7 +30,7 @@ import org.junit.jupiter.api.Test;
 import org.apache.geode.perftest.TestConfig;
 
 class GcParametersTest {
-  private static final String WITH_GC = "withGc";
+  private static final String WITH_GC = "benchmark.withGc";
   private static final String JAVA_RUNTIME_VERSION = "java.runtime.version";
   private static final String XX_USE_ZGC = "-XX:+UseZGC";
   private static final String XX_USE_G_1_GC = "-XX:+UseG1GC";
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 4303c6b..332fc43 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
@@ -30,7 +30,7 @@ import org.apache.geode.perftest.TestConfig;
 
 class HeapParametersTest {
 
-  private static final String WITH_HEAP = "withHeap";
+  private static final String WITH_HEAP = "benchmark.withHeap";
 
   private Properties systemProperties;
 
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/ClientServerBenchmarkTest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/ClientServerBenchmarkTest.java
index 58eb5c8..0663a1d 100644
--- a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/ClientServerBenchmarkTest.java
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/ClientServerBenchmarkTest.java
@@ -56,40 +56,40 @@ class GeodeBenchmarkTest {
 
   @AfterAll
   public static void afterAll() {
-    System.clearProperty("withSniProxy");
+    System.clearProperty("benchmark.withSniProxy");
   }
 
   @Test
   public void withoutSniProxy() {
-    System.clearProperty("withSniProxy");
+    System.clearProperty("benchmark.withSniProxy");
     config = ClientServerBenchmark.createConfig();
     assertThat(config.getBefore()).doesNotContain(startHAProxyStep, startEnvoyStep);
   }
 
   @Test
   public void withSniProxyInvalid() {
-    System.setProperty("withSniProxy", "invalid");
+    System.setProperty("benchmark.withSniProxy", "invalid");
     assertThatThrownBy(() -> ClientServerBenchmark.createConfig())
         .isInstanceOf(IllegalArgumentException.class);
   }
 
   @Test
   public void withSniProxyDefault() {
-    System.setProperty("withSniProxy", "");
+    System.setProperty("benchmark.withSniProxy", "");
     config = ClientServerBenchmark.createConfig();
     assertThat(config.getBefore()).contains(startHAProxyStep).doesNotContain(startEnvoyStep);
   }
 
   @Test
   public void withSniProxyHAProxy() {
-    System.setProperty("withSniProxy", "HAProxy");
+    System.setProperty("benchmark.withSniProxy", "HAProxy");
     config = ClientServerBenchmark.createConfig();
     assertThat(config.getBefore()).contains(startHAProxyStep);
   }
 
   @Test
   public void withSniProxyEnvoy() {
-    System.setProperty("withSniProxy", "Envoy");
+    System.setProperty("benchmark.withSniProxy", "Envoy");
     config = ClientServerBenchmark.createConfig();
     assertThat(config.getBefore()).contains(startEnvoyStep);
   }
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 4fed317..63a940e 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
@@ -43,17 +43,18 @@ public class ClientServerTopologyTest {
 
   @Test
   public void configWithSsl() {
-    System.setProperty("withSsl", "true");
+    System.setProperty("benchmark.withSsl", "true");
     TestConfig testConfig = new TestConfig();
     ClientServerTopology.configure(testConfig);
-    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSsl=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-Dbenchmark.withSsl=true");
   }
 
   @Test
   public void configWithNoSsl() {
     TestConfig testConfig = new TestConfig();
     ClientServerTopology.configure(testConfig);
-    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).doesNotContain("-DwithSsl=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name()))
+        .doesNotContain("-Dbenchmark.withSsl=true");
   }
 
   @Test
@@ -61,27 +62,29 @@ public class ClientServerTopologyTest {
     TestConfig testConfig = new TestConfig();
     ClientServerTopology.configure(testConfig);
     assertThat(testConfig.getJvmArgs().get(CLIENT.name()))
-        .doesNotContain("-DwithSecurityManager=true");
+        .doesNotContain("-Dbenchmark.withSecurityManager=true");
   }
 
   @Test
   public void configWithSecurityManager() {
-    System.setProperty("withSecurityManager", "true");
+    System.setProperty("benchmark.withSecurityManager", "true");
     TestConfig testConfig = new TestConfig();
     ClientServerTopology.configure(testConfig);
-    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSecurityManager=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name()))
+        .contains("-Dbenchmark.withSecurityManager=true");
   }
 
   @Test
   public void configWithSecurityManagerAndSslAndJava11() {
-    System.setProperty("withSecurityManager", "true");
+    System.setProperty("benchmark.withSecurityManager", "true");
     System.setProperty("java.runtime.version", "11.0.4+11");
-    System.setProperty("withSsl", "true");
+    System.setProperty("benchmark.withSsl", "true");
     TestConfig testConfig = new TestConfig();
 
     ClientServerTopology.configure(testConfig);
 
-    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSecurityManager=true");
-    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSsl=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name()))
+        .contains("-Dbenchmark.withSecurityManager=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-Dbenchmark.withSsl=true");
   }
 }
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSniProxyTest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSniProxyTest.java
index 5e989c5..8938f21 100644
--- a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSniProxyTest.java
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSniProxyTest.java
@@ -51,7 +51,7 @@ public class ClientServerTopologyWithSniProxyTest {
     System.setProperty(WITH_SNI_PROXY_PROPERTY, sniProxyImplementation.name());
     final TestConfig testConfig = new TestConfig();
     ClientServerTopologyWithSniProxy.configure(testConfig);
-    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-DwithSsl=true");
+    assertThat(testConfig.getJvmArgs().get(CLIENT.name())).contains("-Dbenchmark.withSsl=true");
   }
 
 }
diff --git a/harness/src/main/java/org/apache/geode/perftest/WorkloadConfig.java b/harness/src/main/java/org/apache/geode/perftest/WorkloadConfig.java
index f62a90e..6517269 100644
--- a/harness/src/main/java/org/apache/geode/perftest/WorkloadConfig.java
+++ b/harness/src/main/java/org/apache/geode/perftest/WorkloadConfig.java
@@ -42,15 +42,15 @@ public class WorkloadConfig implements Serializable {
   public WorkloadConfig() {}
 
   public void durationSeconds(long durationSeconds) {
-    this.durationSeconds = Long.getLong("withDuration", durationSeconds);
+    this.durationSeconds = Long.getLong("benchmark.withDuration", durationSeconds);
   }
 
   public void warmupSeconds(long warmupSeconds) {
-    this.warmupSeconds = Long.getLong("withWarmup", warmupSeconds);
+    this.warmupSeconds = Long.getLong("benchmark.withWarmup", warmupSeconds);
   }
 
   public void threads(int threads) {
-    this.threads = Integer.getInteger("withThreads", threads);
+    this.threads = Integer.getInteger("benchmark.withThreads", threads);
   }
 
   public long getDurationSeconds() {
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 0d26118..a08862b 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
@@ -92,7 +92,7 @@ class JVMLauncher {
     command.add("-D" + RemoteJVMFactory.ROLE + "=" + jvmConfig.getRole());
     command.add("-D" + RemoteJVMFactory.OUTPUT_DIR + "=" + jvmConfig.getOutputDir());
 
-    if (jvmConfig.getJvmArgs().contains("-DwithSsl=true")) {
+    if (jvmConfig.getJvmArgs().contains("-Dbenchmark.withSsl=true")) {
       command
           .add("-Dgemfire." + SSL_KEYSTORE + "=" + jvmConfig.getLibDir() + "/temp-self-signed.jks");
       command.add("-Dgemfire." + SSL_KEYSTORE_PASSWORD + "=123456");
diff --git a/harness/src/test/java/org/apache/geode/perftest/yardstick/YardstickTaskTest.java b/harness/src/test/java/org/apache/geode/perftest/yardstick/YardstickTaskTest.java
index 707ca43..4643b14 100644
--- a/harness/src/test/java/org/apache/geode/perftest/yardstick/YardstickTaskTest.java
+++ b/harness/src/test/java/org/apache/geode/perftest/yardstick/YardstickTaskTest.java
@@ -18,6 +18,7 @@
 package org.apache.geode.perftest.yardstick;
 
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
 
 import java.io.File;
 import java.nio.file.Files;
@@ -30,6 +31,7 @@ import org.apache.geode.perftest.Task;
 import org.apache.geode.perftest.TestContext;
 import org.apache.geode.perftest.WorkloadConfig;
 import org.apache.geode.perftest.benchmarks.EmptyBenchmark;
+import org.apache.geode.perftest.jvms.rmi.ControllerRemote;
 import org.apache.geode.perftest.runner.DefaultTestContext;
 import org.apache.geode.perftest.yardstick.hdrhistogram.HdrHistogramWriter;
 
@@ -45,7 +47,8 @@ public class YardstickTaskTest {
     workloadConfig.threads(1);
     Task task = new YardstickTask(benchmark, workloadConfig);
     File outputDir = folder.toFile();
-    TestContext context = new DefaultTestContext(null, outputDir, 1, role, controller);
+    ControllerRemote controller = mock(ControllerRemote.class);
+    TestContext context = new DefaultTestContext(null, outputDir, 1, "role", controller);
     task.run(context);
 
     assertTrue(1 <= benchmark.getInvocations());