You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by zh...@apache.org on 2019/07/31 20:58:39 UTC

[geode-benchmarks] 01/01: add PdxTypeBenchmark test case and enhance the framework to specify properties.

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

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

commit 24a7688d8d8ceba338a4def9cbaf5128f116ea42
Author: zhouxh <gz...@pivotal.io>
AuthorDate: Wed Jul 31 10:56:30 2019 -0700

    add PdxTypeBenchmark test case and enhance the framework to
    specify properties.
    
            Co-authored-by: Xiaojian Zhou <gz...@pivotal.io>
            Co-authored-by: Donal Evans <do...@pivotal.io>
---
 .../benchmark/parameters/GeodeProperties.java      | 10 +++-
 .../benchmark/tasks/CreatePdxFromJSONTask.java     | 46 ++++++++++++++++
 .../apache/geode/benchmark/tasks/StartServer.java  |  6 +-
 .../tests/CreatePdxFromJSONBenchmark.java          | 64 ++++++++++++++++++++++
 .../benchmark/topology/ClientServerTopology.java   |  2 +-
 .../tests/CreatePdxFromJSONBenchmarkTest.java      | 48 ++++++++++++++++
 .../java/org/apache/geode/perftest/TestConfig.java |  9 +++
 7 files changed, 180 insertions(+), 5 deletions(-)

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..da98573 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
@@ -39,7 +39,8 @@ import org.apache.geode.benchmark.security.ExampleAuthInit;
 
 public class GeodeProperties {
 
-  public static Properties serverProperties() {
+  public static Properties serverProperties(
+      Properties customizedProperties) {
     Properties properties = new Properties();
 
     properties.setProperty(CONSERVE_SOCKETS, "false");
@@ -57,12 +58,17 @@ public class GeodeProperties {
     properties.setProperty(USE_CLUSTER_CONFIGURATION, "false");
     properties.setProperty(SERIALIZABLE_OBJECT_FILTER, "benchmark.geode.data.**");
 
+    if (customizedProperties != null) {
+      for (String propertyName : customizedProperties.stringPropertyNames()) {
+        properties.setProperty(propertyName, customizedProperties.getProperty(propertyName));
+      }
+    }
     return withOptions(properties);
   }
 
   public static Properties locatorProperties() {
     // Locator properties are the same as the server properties right now
-    return withOptions(serverProperties());
+    return withOptions(serverProperties(null));
   }
 
   public static Properties clientProperties() {
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/CreatePdxFromJSONTask.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/CreatePdxFromJSONTask.java
new file mode 100644
index 0000000..ee9c333
--- /dev/null
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tasks/CreatePdxFromJSONTask.java
@@ -0,0 +1,46 @@
+package org.apache.geode.benchmark.tasks;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkDriverAdapter;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.pdx.JSONFormatter;
+
+public class CreatePdxFromJSONTask extends BenchmarkDriverAdapter
+    implements Serializable {
+
+  // The number of PdxTypes to create per task execution. The smaller this number, the greater the
+  // granularity of the test.
+  private int batchSize;
+
+  private int count;
+  // private ThreadLocal<Integer> rl = new ThreadLocal<Integer>();
+
+  public CreatePdxFromJSONTask(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Override
+  public void setUp(BenchmarkConfiguration cfg) throws Exception {
+    super.setUp(cfg);
+
+    Cache cache = CacheFactory.getAnyInstance();
+  }
+
+  @Override
+  public boolean test(Map<Object, Object> ctx) throws Exception {
+    for (int i = 0; i < batchSize; i++) {
+      String field =
+          "\"" + Thread.currentThread().getName() + "-" + (count + i) + "\": " +
+              i;
+      String jsonString = "{" + field + "}";
+      JSONFormatter.fromJSON(jsonString);
+    }
+    count += batchSize;
+    return true;
+  }
+}
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 99b1543..b35d0db 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
@@ -37,15 +37,17 @@ import org.apache.geode.perftest.TestContext;
 public class StartServer implements Task {
 
   private int locatorPort;
+  private Properties props;
 
-  public StartServer(int locatorPort) {
+  public StartServer(int locatorPort, Properties props) {
     this.locatorPort = locatorPort;
+    this.props = props;
   }
 
   @Override
   public void run(TestContext context) throws Exception {
 
-    Properties properties = serverProperties();
+    Properties properties = serverProperties(this.props);
 
     String locatorString = LocatorUtil.getLocatorString(context, locatorPort);
     String statsFile = new File(context.getOutputDir(), "stats.gfs").getAbsolutePath();
diff --git a/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/CreatePdxFromJSONBenchmark.java b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/CreatePdxFromJSONBenchmark.java
new file mode 100644
index 0000000..aa2e630
--- /dev/null
+++ b/geode-benchmarks/src/main/java/org/apache/geode/benchmark/tests/CreatePdxFromJSONBenchmark.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.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.distributed.ConfigurationProperties.LOG_LEVEL;
+
+import java.util.Properties;
+
+import org.junit.jupiter.api.Test;
+
+import org.apache.geode.benchmark.tasks.CreatePdxFromJSONTask;
+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;
+
+/**
+ * Benchmark of puts on a replicated region.
+ */
+public class CreatePdxFromJSONBenchmark implements PerformanceTest {
+
+  private int batchSize = 100;
+
+  public CreatePdxFromJSONBenchmark() {}
+
+  public void setBatchSize(int batchSize) {
+    this.batchSize = batchSize;
+  }
+
+  @Test
+  public void run() throws Exception {
+    TestRunners.defaultRunner().runTest(this);
+  }
+
+  @Override
+  public TestConfig configure() {
+    TestConfig config = GeodeBenchmark.createConfig();
+    config.role(CLIENT, 0);
+    Properties props = new Properties();
+    props.setProperty(LOG_LEVEL, "WARN");
+    config.setProperties(props);
+    ClientServerTopology.configure(config);
+    config.workload(new CreatePdxFromJSONTask(batchSize), SERVER);
+    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 384fa92..057d2f7 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
@@ -73,7 +73,7 @@ public class ClientServerTopology {
     addToTestConfig(testConfig, "withSecurityManager", WITH_SECURITY_MANAGER_ARGUMENT);
 
     testConfig.before(new StartLocator(LOCATOR_PORT), LOCATOR);
-    testConfig.before(new StartServer(LOCATOR_PORT), SERVER);
+    testConfig.before(new StartServer(LOCATOR_PORT, testConfig.getProperties()), SERVER);
     testConfig.before(new StartClient(LOCATOR_PORT), CLIENT);
   }
 
diff --git a/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/CreatePdxFromJSONBenchmarkTest.java b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/CreatePdxFromJSONBenchmarkTest.java
new file mode 100644
index 0000000..b8bb123
--- /dev/null
+++ b/geode-benchmarks/src/test/java/org/apache/geode/benchmark/tests/CreatePdxFromJSONBenchmarkTest.java
@@ -0,0 +1,48 @@
+/*
+ * 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.perftest.TestRunners;
+
+@ExtendWith(TempDirectory.class)
+public class CreatePdxFromJSONBenchmarkTest {
+
+  private File folder;
+
+  @BeforeEach
+  void createTemporaryFolder(@TempDirectory.TempDir Path tempFolder) {
+    folder = tempFolder.toFile();
+  }
+
+  @Test
+  public void benchmarkRunsSuccessfully()
+      throws Exception {
+    CreatePdxFromJSONBenchmark test = new CreatePdxFromJSONBenchmark();
+    test.setBatchSize(100);
+    TestRunners.minimalRunner(folder)
+        .runTest(test);
+  }
+}
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..ef4f75d 100644
--- a/harness/src/main/java/org/apache/geode/perftest/TestConfig.java
+++ b/harness/src/main/java/org/apache/geode/perftest/TestConfig.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 
 import org.yardstickframework.BenchmarkDriver;
 
@@ -43,6 +44,7 @@ public class TestConfig implements Serializable {
   private List<TestStep> before = new ArrayList<>();
   private List<TestStep> workload = new ArrayList<>();
   private List<TestStep> after = new ArrayList<>();
+  private Properties props = null;
 
   /**
    * Define a role for the test.
@@ -54,6 +56,13 @@ public class TestConfig implements Serializable {
     this.roles.put(role, numberOfJVMs);
   }
 
+  public void setProperties(Properties props) {
+    this.props = props;
+  }
+
+  public Properties getProperties() {
+    return this.props;
+  }
 
   /**
    * Add a before task to the test. Each before task is run in parallel on