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/03 22:59:59 UTC

[geode-benchmarks] branch sni updated: PartitionedPutBenchmark can optionally run w/ SNI proxy via -PwithSniProxy to gradle task

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

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


The following commit(s) were added to refs/heads/sni by this push:
     new c6d7c1e  PartitionedPutBenchmark can optionally run w/ SNI proxy via -PwithSniProxy to gradle task
c6d7c1e is described below

commit c6d7c1e85e79c6ca79558247aeacb9ed2dd927e3
Author: Bill Burcham <bb...@pivotal.io>
AuthorDate: Wed Jun 3 15:59:48 2020 -0700

    PartitionedPutBenchmark can optionally run w/ SNI proxy via -PwithSniProxy to gradle task
---
 geode-benchmarks/build.gradle                      |  1 +
 .../benchmark/parameters/GeodeProperties.java      |  6 +-
 .../geode/benchmark/tasks/StartClientSNI.java      |  2 +-
 .../geode/benchmark/tasks/StartSniProxy.java       |  3 +-
 .../benchmark/tests/PartitionedPutBenchmark.java   | 12 +++-
 .../tests/PartitionedPutBenchmarkSNI.java          | 67 ----------------------
 .../topology/ClientServerTopologyWithSNIProxy.java |  4 --
 .../org/apache/geode/benchmark/topology/Ports.java |  2 +
 .../tests/PartitionedPutBenchmarkSNITest.java      | 51 ----------------
 9 files changed, 20 insertions(+), 128 deletions(-)

diff --git a/geode-benchmarks/build.gradle b/geode-benchmarks/build.gradle
index 3e58e2c..43ae57a 100644
--- a/geode-benchmarks/build.gradle
+++ b/geode-benchmarks/build.gradle
@@ -100,6 +100,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/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/tasks/StartClientSNI.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/StartClientSNI.java
index 472cde8..ef34441 100644
--- 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
@@ -1,6 +1,7 @@
 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;
 
@@ -15,7 +16,6 @@ import org.apache.geode.pdx.ReflectionBasedAutoSerializer;
 import org.apache.geode.perftest.TestContext;
 
 public class StartClientSNI extends StartClient {
-  public static final int SNI_PROXY_PORT = 15443;
 
   public StartClientSNI(final int locatorPort) {
     super(locatorPort);
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
index c974585..16e93dc 100644
--- 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
@@ -18,6 +18,7 @@
 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;
 
@@ -127,7 +128,7 @@ public class StartSniProxy implements Task {
         "locator1", locatorPort);
 
     generateBackendSection(serversInternal, stuff, "servers-",
-        "server1", 40404);
+        "server1", SERVER_PORT_FOR_SNI);
 
     return stuff.toString();
   }
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 5d8e3fc..d50a1ba 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
@@ -30,6 +30,7 @@ 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.benchmark.topology.ClientServerTopologyWithSNIProxy;
 import org.apache.geode.perftest.PerformanceTest;
 import org.apache.geode.perftest.TestConfig;
 import org.apache.geode.perftest.TestRunners;
@@ -55,7 +56,16 @@ public class PartitionedPutBenchmark implements PerformanceTest {
   @Override
   public TestConfig configure() {
     TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopology.configure(config);
+
+    final String sniProp = System.getProperty("withSniProxy");
+    final boolean doSni = sniProp != null && !sniProp.equals("false");
+
+    if (doSni) {
+      ClientServerTopologyWithSNIProxy.configure(config);
+    } else {
+      ClientServerTopology.configure(config);
+    }
+
     before(config, new CreatePartitionedRegion(), SERVER);
     before(config, new CreateClientProxyRegion(), CLIENT);
     before(config, new PrePopulateRegion(keyRange), CLIENT);
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmarkSNI.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmarkSNI.java
deleted file mode 100644
index 1c7f633..0000000
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmarkSNI.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.geode.benchmark.tests;
-
-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;
-
-import org.apache.geode.benchmark.LongRange;
-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.PrePopulateRegionLong;
-import org.apache.geode.benchmark.tasks.PutTask;
-import org.apache.geode.benchmark.topology.ClientServerTopologyWithSNIProxy;
-import org.apache.geode.perftest.PerformanceTest;
-import org.apache.geode.perftest.TestConfig;
-import org.apache.geode.perftest.TestRunners;
-
-/**
- * Benchmark of puts on a partitioned region.
- * TODO: remove this benchmark once SNI is an option for all benchmarks
- */
-public class PartitionedPutBenchmarkSNI implements PerformanceTest {
-
-  private LongRange keyRange = new LongRange(0, 1_000_000);
-
-  public PartitionedPutBenchmarkSNI() {}
-
-  public void setKeyRange(final LongRange keyRange) {
-    this.keyRange = keyRange;
-  }
-
-  @Test
-  public void run() throws Exception {
-    TestRunners.defaultRunner().runTest(this);
-  }
-
-  @Override
-  public TestConfig configure() {
-    TestConfig config = GeodeBenchmark.createConfig();
-    ClientServerTopologyWithSNIProxy.configure(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/topology/ClientServerTopologyWithSNIProxy.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/ClientServerTopologyWithSNIProxy.java
index 0dcb050..78dd787 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
@@ -67,10 +67,6 @@ public class ClientServerTopologyWithSNIProxy {
     configureGeodeProductJvms(config, WITH_SSL_ARGUMENT);
     addToTestConfig(config, "withSecurityManager", WITH_SECURITY_MANAGER_ARGUMENT);
 
-    // pass SNI proxy config to CLIENT role only
-    // TODO: peel it off over in client
-    jvmArgs(config, CLIENT, "-DwithSniProxy=hostname:port");
-
     Stream.concat(Roles.rolesFor(GEODE_PRODUCT), Stream.of(PROXY))
         .forEach(role -> before(config, new DefineHostNamingsOffPlatformTask(), role));
 
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Ports.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Ports.java
index 205481e..b4f0f58 100644
--- a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Ports.java
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/topology/Ports.java
@@ -27,4 +27,6 @@ public class Ports {
    * responses carrying server port numbers.
    */
   public static final int SERVER_PORT_FOR_SNI = 40404;
+
+  public static final int SNI_PROXY_PORT = 15443;
 }
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmarkSNITest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmarkSNITest.java
deleted file mode 100644
index 2dc0e2d..0000000
--- a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/PartitionedPutBenchmarkSNITest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.geode.benchmark.tests;
-
-
-
-import java.io.File;
-import java.nio.file.Path;
-
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.junitpioneer.jupiter.TempDirectory;
-
-import org.apache.geode.benchmark.LongRange;
-import org.apache.geode.perftest.TestRunners;
-
-@ExtendWith(TempDirectory.class)
-public class PartitionedPutBenchmarkSNITest {
-
-  private File folder;
-
-  @BeforeEach
-  void createTemporaryFolder(@TempDirectory.TempDir Path tempFolder) {
-    folder = tempFolder.toFile();
-  }
-
-  @Test
-  public void benchmarkRunsSuccessfully()
-      throws Exception {
-    final PartitionedPutBenchmarkSNI test = new PartitionedPutBenchmarkSNI();
-    test.setKeyRange(new LongRange(0, 100));
-    TestRunners.minimalRunner(folder)
-        .runTest(test);
-  }
-}