You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/09/06 12:41:57 UTC

[GitHub] [beam] mosche opened a new pull request, #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

mosche opened a new pull request, #23041:
URL: https://github.com/apache/beam/pull/23041

   Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238) if `InfluxDBSettings` can be inferred from the environment:
   
   - `INFLUXDB_MEASUREMENT`: required; automatically generated based on the gradle module path; prefixed with `java_jmh_`
   - `INFLUXDB_DATABASE`: required, typically this is `beam_test_metrics`
   - `INFLUXDB_HOST`: optional host (and port), localhost otherwise
   
   Data is published as follows:
   - `measurement` name as provided
   - `tags`: `benchmark`, `mode`, `scoreUnit` and all JMH benchmark parameters
   - `fields`: `score`, `scoreMean`, `scoreMedian`, `scoreError`, `durationMs`
   - `timestamp`: start time of benchmark (precision is seconds)
   
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Mention the appropriate issue in your description (for example: `addresses #123`), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment `fixes #<ISSUE NUMBER>` instead.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/get-started-contributing/#make-the-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Go tests](https://github.com/apache/beam/workflows/Go%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on PR #23041:
URL: https://github.com/apache/beam/pull/23041#issuecomment-1243787989

   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
lukecwik commented on PR #23041:
URL: https://github.com/apache/beam/pull/23041#issuecomment-1243968679

   > R: @lukecwik The final piece missing here is the infrastructure side where (how) to run the benchmarks so that there's minimal side effects / noise. Who could help there?
   
   We would need to tag a Jenkins machine to be only used for benchmark runs and configure it to run at most one job at a time. I would reach out on the dev@ mailing list to see if there are any volunteers to help set that up.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r965037899


##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing benchmarks to
+ * InfluxDB.
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:
+ *
+ * <ul>
+ *   <li>{@link #INFLUXDB_HOST}
+ *   <li>{@link #INFLUXDB_DATABASE}
+ *   <li>{@link #INFLUXDB_MEASUREMENT}
+ * </ul>
+ */
+public class Main {
+  private static final String INFLUXDB_HOST = "INFLUXDB_HOST";
+  private static final String INFLUXDB_DATABASE = "INFLUXDB_DATABASE";
+  private static final String INFLUXDB_MEASUREMENT = "INFLUXDB_MEASUREMENT";
+
+  public static void main(String[] args)
+      throws CommandLineOptionException, IOException, RunnerException {
+    final CommandLineOptions opts = new CommandLineOptions(args);
+    final InfluxDBSettings influxDB = influxDBSettings();
+
+    if (influxDB == null
+        || isSingleShotTimeOnly(opts.getBenchModes())
+        || opts.shouldHelp()
+        || opts.shouldList()
+        || opts.shouldListWithParams()
+        || opts.shouldListProfilers()
+        || opts.shouldListResultFormats()) {
+      // delegate to JMH runner
+      org.openjdk.jmh.Main.main(args);
+      return;
+    }
+
+    final Runner runner = new Runner(opts);
+    final Collection<RunResult> results = runner.run();
+
+    final Collection<DataPoint> dataPoints =
+        results.stream()
+            .filter(r -> r.getParams().getMode() != SingleShotTime)

Review Comment:
   Is that because we currently configure the `-foe` flag (fail on error)?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] github-actions[bot] commented on pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #23041:
URL: https://github.com/apache/beam/pull/23041#issuecomment-1238116798

   Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r964637840


##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -1411,8 +1413,13 @@ class BeamModulePlugin implements Plugin<Project> {
 
         project.tasks.register("jmh", JavaExec)  {
           dependsOn project.classes
-          mainClass = "org.openjdk.jmh.Main"
+          mainClass = "org.apache.beam.sdk.testutils.jmh.Main"
           classpath = project.sourceSets.main.runtimeClasspath
+
+          environment([
+            'INFLUXDB_MEASUREMENT': 'java_jmh_' + (getPath().split(':') - ['', 'sdks', 'java', 'jmh']).join('_')

Review Comment:
   Happy to discuss this... as long as benchmarks are in core (or core related modules) only I wouldn't consider this to be important. But assuming there could be benchmarks on IOs as well, it would be convenient to scope them in a more granular way. That makes it a bit easier to treat them differently.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r968618813


##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing benchmarks to
+ * InfluxDB.
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:
+ *
+ * <ul>
+ *   <li>{@link #INFLUXDB_HOST}
+ *   <li>{@link #INFLUXDB_DATABASE}
+ *   <li>{@link #INFLUXDB_MEASUREMENT}
+ * </ul>
+ */
+public class Main {
+  private static final String INFLUXDB_HOST = "INFLUXDB_HOST";
+  private static final String INFLUXDB_DATABASE = "INFLUXDB_DATABASE";
+  private static final String INFLUXDB_MEASUREMENT = "INFLUXDB_MEASUREMENT";
+
+  public static void main(String[] args)
+      throws CommandLineOptionException, IOException, RunnerException {
+    final CommandLineOptions opts = new CommandLineOptions(args);
+    final InfluxDBSettings influxDB = influxDBSettings();
+
+    if (influxDB == null
+        || isSingleShotTimeOnly(opts.getBenchModes())
+        || opts.shouldHelp()
+        || opts.shouldList()
+        || opts.shouldListWithParams()
+        || opts.shouldListProfilers()
+        || opts.shouldListResultFormats()) {
+      // delegate to JMH runner
+      org.openjdk.jmh.Main.main(args);
+      return;
+    }
+
+    final Runner runner = new Runner(opts);
+    final Collection<RunResult> results = runner.run();
+
+    final Collection<DataPoint> dataPoints =
+        results.stream()
+            .filter(r -> r.getParams().getMode() != SingleShotTime)

Review Comment:
   Looks good to me to record the sample count based upon the code that you have added instead of requiring exactly X samples. We can always update dashboards to filter on partial successes if necessary.



##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing JMH benchmark
+ * results to InfluxDB.
+ *
+ * <h3>Schema</h3>
+ *
+ * <p>The wrapper writes an aggregated InfluxDB datapoint for each benchmark to <b> measurement</b>

Review Comment:
   ```suggestion
    * <p>The wrapper writes an aggregated InfluxDB datapoint for each benchmark to <b>measurement</b>
   ```



##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing JMH benchmark
+ * results to InfluxDB.
+ *
+ * <h3>Schema</h3>
+ *
+ * <p>The wrapper writes an aggregated InfluxDB datapoint for each benchmark to <b> measurement</b>
+ * {@code {INFLUXDB_BASE_MEASUREMENT}_{mode}}. Typically this is {@code java_jmh_thrpt}.
+ *
+ * <p>The <b>timestamp</b> of the datapoint corresponds to the start time of the respective
+ * benchmark.
+ *
+ * <p>Individual timeseries are discriminated using the following <b>tags</b> including tags
+ * corresponding to additional benchmark parameters in case of parameterized benchmarks:
+ *
+ * <ul>
+ *   <li>{@code benchmark} (string) : Fully qualified name of the benchmark

Review Comment:
   ```suggestion
    *   <li>{@code benchmark} (string): Fully qualified name of the benchmark
   ```



##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing JMH benchmark
+ * results to InfluxDB.
+ *
+ * <h3>Schema</h3>
+ *
+ * <p>The wrapper writes an aggregated InfluxDB datapoint for each benchmark to <b> measurement</b>
+ * {@code {INFLUXDB_BASE_MEASUREMENT}_{mode}}. Typically this is {@code java_jmh_thrpt}.
+ *
+ * <p>The <b>timestamp</b> of the datapoint corresponds to the start time of the respective
+ * benchmark.
+ *
+ * <p>Individual timeseries are discriminated using the following <b>tags</b> including tags
+ * corresponding to additional benchmark parameters in case of parameterized benchmarks:
+ *
+ * <ul>
+ *   <li>{@code benchmark} (string) : Fully qualified name of the benchmark
+ *   <li>{@code scoreUnit} (string): JMH score unit
+ *   <li>optionally, additional parameters in case of a parameterized benchmark (string)
+ * </ul>
+ *
+ * <p>The following fields are captured for each benchmark:
+ *
+ * <ul>
+ *   <li>{@code score} (float): JMH score
+ *   <li>{@code scoreMean} (float): Mean score of all iterations
+ *   <li>{@code scoreMedian} (float): Median score of all iterations
+ *   <li>{@code scoreError} (float): Mean error of the score
+ *   <li>{@code sampleCount} (integer): Number of score samples
+ *   <li>{@code durationMs} (integer): Total benchmark duration (including warmups)
+ * </ul>
+ *
+ * <h3>Configuration</h3>
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:
+ *
+ * <ul>
+ *   <li>{@link #INFLUXDB_HOST}
+ *   <li>{@link #INFLUXDB_DATABASE}
+ *   <li>{@link #INFLUXDB_BASE_MEASUREMENT}
+ * </ul>
+ */
+public class Main {
+  private static final String INFLUXDB_HOST = "INFLUXDB_HOST";
+  private static final String INFLUXDB_DATABASE = "INFLUXDB_DATABASE";
+  private static final String INFLUXDB_BASE_MEASUREMENT = "INFLUXDB_BASE_MEASUREMENT";
+
+  public static void main(String[] args)
+      throws CommandLineOptionException, IOException, RunnerException {
+    final CommandLineOptions opts = new CommandLineOptions(args);
+    final InfluxDBSettings influxDB = influxDBSettings();
+
+    if (influxDB == null
+        || isSingleShotTimeOnly(opts.getBenchModes())
+        || opts.shouldHelp()
+        || opts.shouldList()
+        || opts.shouldListWithParams()
+        || opts.shouldListProfilers()
+        || opts.shouldListResultFormats()) {
+      // delegate to JMH runner
+      org.openjdk.jmh.Main.main(args);
+      return;
+    }
+
+    final Runner runner = new Runner(opts);
+    final Collection<RunResult> results = runner.run();
+
+    final Collection<DataPoint> dataPoints =
+        results.stream()
+            .filter(r -> r.getParams().getMode() != SingleShotTime)
+            .map(r -> dataPoint(influxDB.measurement, r))
+            .collect(toList());
+
+    InfluxDBPublisher.publish(influxDB, dataPoints);
+  }
+
+  private static boolean isSingleShotTimeOnly(Collection<Mode> modes) {
+    return !modes.isEmpty() && modes.stream().allMatch(SingleShotTime::equals);
+  }
+
+  private static DataPoint dataPoint(String baseMeasurement, RunResult run) {
+    final BenchmarkParams params = run.getParams();
+    final Result<?> result = run.getPrimaryResult();
+
+    final long startTimeMs =
+        metaDataStream(run).mapToLong(BenchmarkResultMetaData::getStartTime).min().getAsLong();
+    final long stopTimeMs =
+        metaDataStream(run).mapToLong(BenchmarkResultMetaData::getStopTime).max().getAsLong();
+
+    final String measurement =
+        String.format("%s_%s", baseMeasurement, params.getMode().shortLabel());
+
+    final Map<String, String> tags = new HashMap<>();
+    tags.put("benchmark", params.getBenchmark());
+    tags.put("scoreUnit", result.getScoreUnit());
+    // add params of parameterized benchmarks as tags
+    tags.putAll(params.getParamsKeys().stream().collect(toMap(identity(), params::getParam)));
+
+    final Map<String, Number> fields = new HashMap<>();
+    fields.put("score", result.getScore());
+    fields.put("scoreMean", result.getStatistics().getMean());
+    fields.put("scoreMedian", result.getStatistics().getPercentile(0.5));
+    if (!Double.isNaN(result.getScoreError())) {
+      fields.put("scoreError", result.getScoreError());
+    }
+    fields.put("sampleCount", result.getSampleCount());
+    fields.put("durationMs", stopTimeMs - startTimeMs);
+
+    return InfluxDBPublisher.dataPoint(
+        measurement, tags, fields, startTimeMs, TimeUnit.MILLISECONDS);
+  }
+
+  private static Stream<BenchmarkResultMetaData> metaDataStream(RunResult runResult) {
+    return runResult.getBenchmarkResults().stream()
+        .map(BenchmarkResult::getMetadata)
+        .filter(Objects::nonNull);
+  }
+
+  /** Construct InfluxDB settings from environment variables to not mess with JMH args. */
+  private static @Nullable InfluxDBSettings influxDBSettings() {
+    String host = System.getenv(INFLUXDB_HOST);
+    String database = System.getenv(INFLUXDB_DATABASE);
+    String measurement = System.getenv(INFLUXDB_BASE_MEASUREMENT);
+    if (measurement == null || database == null) {
+      return null;
+    }
+
+    InfluxDBSettings.Builder builder = InfluxDBSettings.builder();
+    if (host != null) {
+      builder.withHost(host); // default to localhost otherwise
+    }
+    return builder.withDatabase(database).withMeasurement(measurement).get();

Review Comment:
   What is the purpose of specifying the measurement here?



##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -1449,7 +1454,8 @@ class BeamModulePlugin implements Plugin<Project> {
         // Note that these tests will fail on JVMs that JMH doesn't support.
         def jmhTest = project.tasks.register("jmhTest", JavaExec) {
           dependsOn project.classes
-          mainClass = "org.openjdk.jmh.Main"
+          // Note: this will delegate to the default JMH runner

Review Comment:
   ```suggestion
             // Note: this will wrap the default JMH runner publishing results to InfluxDB
   ```



##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing benchmarks to
+ * InfluxDB.
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:

Review Comment:
   This looks great.



##########
sdks/java/testing/test-utils/build.gradle:
##########
@@ -37,6 +37,7 @@ dependencies {
   implementation library.java.http_client
   implementation library.java.http_core
   implementation library.java.slf4j_api
+  provided library.java.jmh_core

Review Comment:
   I don't understand the motivation to use `provided` over `implementation`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r965083892


##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing benchmarks to
+ * InfluxDB.
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:
+ *
+ * <ul>
+ *   <li>{@link #INFLUXDB_HOST}
+ *   <li>{@link #INFLUXDB_DATABASE}
+ *   <li>{@link #INFLUXDB_MEASUREMENT}
+ * </ul>
+ */
+public class Main {
+  private static final String INFLUXDB_HOST = "INFLUXDB_HOST";
+  private static final String INFLUXDB_DATABASE = "INFLUXDB_DATABASE";
+  private static final String INFLUXDB_MEASUREMENT = "INFLUXDB_MEASUREMENT";
+
+  public static void main(String[] args)
+      throws CommandLineOptionException, IOException, RunnerException {
+    final CommandLineOptions opts = new CommandLineOptions(args);
+    final InfluxDBSettings influxDB = influxDBSettings();
+
+    if (influxDB == null
+        || isSingleShotTimeOnly(opts.getBenchModes())
+        || opts.shouldHelp()
+        || opts.shouldList()
+        || opts.shouldListWithParams()
+        || opts.shouldListProfilers()
+        || opts.shouldListResultFormats()) {
+      // delegate to JMH runner
+      org.openjdk.jmh.Main.main(args);
+      return;
+    }
+
+    final Runner runner = new Runner(opts);
+    final Collection<RunResult> results = runner.run();
+
+    final Collection<DataPoint> dataPoints =
+        results.stream()
+            .filter(r -> r.getParams().getMode() != SingleShotTime)

Review Comment:
   yes, that's true... 
   if `-foe` is disabled `Collection<RunResult> results` will only contain results with successful forks. in case all forks for a benchmark fail it won't be included in results.
   
   it probably makes sense to add `sampleCount` as an additional field to capture this in case `-foe` is disabled. 
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r969337374


##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -1449,7 +1454,8 @@ class BeamModulePlugin implements Plugin<Project> {
         // Note that these tests will fail on JVMs that JMH doesn't support.
         def jmhTest = project.tasks.register("jmhTest", JavaExec) {
           dependsOn project.classes
-          mainClass = "org.openjdk.jmh.Main"
+          // Note: this will delegate to the default JMH runner

Review Comment:
   For `jmhTest` the code just delegates to the default JMH runner, we don't want to publish single shot times. I'll updated to comment accordingly



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r969347046


##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing JMH benchmark
+ * results to InfluxDB.
+ *
+ * <h3>Schema</h3>
+ *
+ * <p>The wrapper writes an aggregated InfluxDB datapoint for each benchmark to <b> measurement</b>
+ * {@code {INFLUXDB_BASE_MEASUREMENT}_{mode}}. Typically this is {@code java_jmh_thrpt}.
+ *
+ * <p>The <b>timestamp</b> of the datapoint corresponds to the start time of the respective
+ * benchmark.
+ *
+ * <p>Individual timeseries are discriminated using the following <b>tags</b> including tags
+ * corresponding to additional benchmark parameters in case of parameterized benchmarks:
+ *
+ * <ul>
+ *   <li>{@code benchmark} (string) : Fully qualified name of the benchmark
+ *   <li>{@code scoreUnit} (string): JMH score unit
+ *   <li>optionally, additional parameters in case of a parameterized benchmark (string)
+ * </ul>
+ *
+ * <p>The following fields are captured for each benchmark:
+ *
+ * <ul>
+ *   <li>{@code score} (float): JMH score
+ *   <li>{@code scoreMean} (float): Mean score of all iterations
+ *   <li>{@code scoreMedian} (float): Median score of all iterations
+ *   <li>{@code scoreError} (float): Mean error of the score
+ *   <li>{@code sampleCount} (integer): Number of score samples
+ *   <li>{@code durationMs} (integer): Total benchmark duration (including warmups)
+ * </ul>
+ *
+ * <h3>Configuration</h3>
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:
+ *
+ * <ul>
+ *   <li>{@link #INFLUXDB_HOST}
+ *   <li>{@link #INFLUXDB_DATABASE}
+ *   <li>{@link #INFLUXDB_BASE_MEASUREMENT}
+ * </ul>
+ */
+public class Main {
+  private static final String INFLUXDB_HOST = "INFLUXDB_HOST";
+  private static final String INFLUXDB_DATABASE = "INFLUXDB_DATABASE";
+  private static final String INFLUXDB_BASE_MEASUREMENT = "INFLUXDB_BASE_MEASUREMENT";
+
+  public static void main(String[] args)
+      throws CommandLineOptionException, IOException, RunnerException {
+    final CommandLineOptions opts = new CommandLineOptions(args);
+    final InfluxDBSettings influxDB = influxDBSettings();
+
+    if (influxDB == null
+        || isSingleShotTimeOnly(opts.getBenchModes())
+        || opts.shouldHelp()
+        || opts.shouldList()
+        || opts.shouldListWithParams()
+        || opts.shouldListProfilers()
+        || opts.shouldListResultFormats()) {
+      // delegate to JMH runner
+      org.openjdk.jmh.Main.main(args);
+      return;
+    }
+
+    final Runner runner = new Runner(opts);
+    final Collection<RunResult> results = runner.run();
+
+    final Collection<DataPoint> dataPoints =
+        results.stream()
+            .filter(r -> r.getParams().getMode() != SingleShotTime)
+            .map(r -> dataPoint(influxDB.measurement, r))
+            .collect(toList());
+
+    InfluxDBPublisher.publish(influxDB, dataPoints);
+  }
+
+  private static boolean isSingleShotTimeOnly(Collection<Mode> modes) {
+    return !modes.isEmpty() && modes.stream().allMatch(SingleShotTime::equals);
+  }
+
+  private static DataPoint dataPoint(String baseMeasurement, RunResult run) {
+    final BenchmarkParams params = run.getParams();
+    final Result<?> result = run.getPrimaryResult();
+
+    final long startTimeMs =
+        metaDataStream(run).mapToLong(BenchmarkResultMetaData::getStartTime).min().getAsLong();
+    final long stopTimeMs =
+        metaDataStream(run).mapToLong(BenchmarkResultMetaData::getStopTime).max().getAsLong();
+
+    final String measurement =
+        String.format("%s_%s", baseMeasurement, params.getMode().shortLabel());
+
+    final Map<String, String> tags = new HashMap<>();
+    tags.put("benchmark", params.getBenchmark());
+    tags.put("scoreUnit", result.getScoreUnit());
+    // add params of parameterized benchmarks as tags
+    tags.putAll(params.getParamsKeys().stream().collect(toMap(identity(), params::getParam)));
+
+    final Map<String, Number> fields = new HashMap<>();
+    fields.put("score", result.getScore());
+    fields.put("scoreMean", result.getStatistics().getMean());
+    fields.put("scoreMedian", result.getStatistics().getPercentile(0.5));
+    if (!Double.isNaN(result.getScoreError())) {
+      fields.put("scoreError", result.getScoreError());
+    }
+    fields.put("sampleCount", result.getSampleCount());
+    fields.put("durationMs", stopTimeMs - startTimeMs);
+
+    return InfluxDBPublisher.dataPoint(
+        measurement, tags, fields, startTimeMs, TimeUnit.MILLISECONDS);
+  }
+
+  private static Stream<BenchmarkResultMetaData> metaDataStream(RunResult runResult) {
+    return runResult.getBenchmarkResults().stream()
+        .map(BenchmarkResult::getMetadata)
+        .filter(Objects::nonNull);
+  }
+
+  /** Construct InfluxDB settings from environment variables to not mess with JMH args. */
+  private static @Nullable InfluxDBSettings influxDBSettings() {
+    String host = System.getenv(INFLUXDB_HOST);
+    String database = System.getenv(INFLUXDB_DATABASE);
+    String measurement = System.getenv(INFLUXDB_BASE_MEASUREMENT);
+    if (measurement == null || database == null) {
+      return null;
+    }
+
+    InfluxDBSettings.Builder builder = InfluxDBSettings.builder();
+    if (host != null) {
+      builder.withHost(host); // default to localhost otherwise
+    }
+    return builder.withDatabase(database).withMeasurement(measurement).get();

Review Comment:
   Thx for catching, fixed πŸ‘ 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] codecov[bot] commented on pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on PR #23041:
URL: https://github.com/apache/beam/pull/23041#issuecomment-1238136290

   # [Codecov](https://codecov.io/gh/apache/beam/pull/23041?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#23041](https://codecov.io/gh/apache/beam/pull/23041?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (004c600) into [master](https://codecov.io/gh/apache/beam/commit/3c91e7b24a53a6a5b929ede58231bbc57c9ddced?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3c91e7b) will **not change** coverage.
   > The diff coverage is `n/a`.
   
   ```diff
   @@           Coverage Diff           @@
   ##           master   #23041   +/-   ##
   =======================================
     Coverage   73.68%   73.68%           
   =======================================
     Files         714      714           
     Lines       95220    95220           
   =======================================
     Hits        70161    70161           
     Misses      23762    23762           
     Partials     1297     1297           
   ```
   
   
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r964629983


##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing benchmarks to
+ * InfluxDB.
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:
+ *
+ * <ul>
+ *   <li>{@link #INFLUXDB_HOST}
+ *   <li>{@link #INFLUXDB_DATABASE}
+ *   <li>{@link #INFLUXDB_MEASUREMENT}
+ * </ul>
+ */
+public class Main {
+  private static final String INFLUXDB_HOST = "INFLUXDB_HOST";
+  private static final String INFLUXDB_DATABASE = "INFLUXDB_DATABASE";
+  private static final String INFLUXDB_MEASUREMENT = "INFLUXDB_MEASUREMENT";
+
+  public static void main(String[] args)
+      throws CommandLineOptionException, IOException, RunnerException {
+    final CommandLineOptions opts = new CommandLineOptions(args);
+    final InfluxDBSettings influxDB = influxDBSettings();
+
+    if (influxDB == null
+        || isSingleShotTimeOnly(opts.getBenchModes())
+        || opts.shouldHelp()
+        || opts.shouldList()
+        || opts.shouldListWithParams()
+        || opts.shouldListProfilers()
+        || opts.shouldListResultFormats()) {
+      // delegate to JMH runner
+      org.openjdk.jmh.Main.main(args);
+      return;
+    }
+
+    final Runner runner = new Runner(opts);
+    final Collection<RunResult> results = runner.run();
+
+    final Collection<DataPoint> dataPoints =
+        results.stream()
+            .filter(r -> r.getParams().getMode() != SingleShotTime)

Review Comment:
   JMH would fail hard (exceptionally) in that case, so no results are available and nothing gets published.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on PR #23041:
URL: https://github.com/apache/beam/pull/23041#issuecomment-1238102932

   R: @lukecwik 
   The final piece missing here is the infrastructure side where (how) to run the benchmarks so that there's minimal side effects / noise. Who could help there?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r965101320


##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -1411,8 +1413,13 @@ class BeamModulePlugin implements Plugin<Project> {
 
         project.tasks.register("jmh", JavaExec)  {
           dependsOn project.classes
-          mainClass = "org.openjdk.jmh.Main"
+          mainClass = "org.apache.beam.sdk.testutils.jmh.Main"
           classpath = project.sourceSets.main.runtimeClasspath
+
+          environment([
+            'INFLUXDB_MEASUREMENT': 'java_jmh_' + (getPath().split(':') - ['', 'sdks', 'java', 'jmh']).join('_')

Review Comment:
   Ok, that sounds perfectly reasonable to me. Thanks for the suggestion πŸ‘ 
   I'll turn this into a prefix and append the label of the JMH mode, so this would basically end up being `java_jmh_thrpt`.  I was briefly considering if the measurement name should also include the `scoreUnit` (in case changed using `-tu`), but I don't think so. Regardless of the scale, semantically it's still the same type of measurement. But let me know if you disagree.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche merged pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche merged PR #23041:
URL: https://github.com/apache/beam/pull/23041


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r965059194


##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -1411,8 +1413,13 @@ class BeamModulePlugin implements Plugin<Project> {
 
         project.tasks.register("jmh", JavaExec)  {
           dependsOn project.classes
-          mainClass = "org.openjdk.jmh.Main"
+          mainClass = "org.apache.beam.sdk.testutils.jmh.Main"
           classpath = project.sourceSets.main.runtimeClasspath
+
+          environment([
+            'INFLUXDB_MEASUREMENT': 'java_jmh_' + (getPath().split(':') - ['', 'sdks', 'java', 'jmh']).join('_')

Review Comment:
   I was under the impression that if the data we collect for a measurement is the same then we should use a single measurement. For example we could have two different types of JMH runs (throughput and sample time) which we would want to store in a different measurement since they are collecting different data. In our case we are using the JMH throughput mode run for all our benchmarks currently so it would make sense to store it all in one measurement.
   
   nit: note that folks may want to have benchmarks under `runners/` like `runners/java-fn-execution` which could lead to a collision in measurement names.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r969335215


##########
sdks/java/testing/test-utils/build.gradle:
##########
@@ -37,6 +37,7 @@ dependencies {
   implementation library.java.http_client
   implementation library.java.http_core
   implementation library.java.slf4j_api
+  provided library.java.jmh_core

Review Comment:
   Motivation was to not include `jmh-core` on every classpath that depends on `test-utils`. Happy to change to `implementation` if you prefer



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r964627809


##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing benchmarks to
+ * InfluxDB.
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:

Review Comment:
   I've updated the Java docs to describe tags & fields written. Let me know if anything remains unclear.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r964637840


##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -1411,8 +1413,13 @@ class BeamModulePlugin implements Plugin<Project> {
 
         project.tasks.register("jmh", JavaExec)  {
           dependsOn project.classes
-          mainClass = "org.openjdk.jmh.Main"
+          mainClass = "org.apache.beam.sdk.testutils.jmh.Main"
           classpath = project.sourceSets.main.runtimeClasspath
+
+          environment([
+            'INFLUXDB_MEASUREMENT': 'java_jmh_' + (getPath().split(':') - ['', 'sdks', 'java', 'jmh']).join('_')

Review Comment:
   Very valid question and happy to discuss this... as long as benchmarks are in core (or core related modules) only I wouldn't consider this to be important. But assuming there could be benchmarks on IOs as well, it could be convenient to scope them in a more granular way. That makes it a bit easier to treat them differently. What do you think and what would be your preference @lukecwik ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r965105416


##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -1411,8 +1413,13 @@ class BeamModulePlugin implements Plugin<Project> {
 
         project.tasks.register("jmh", JavaExec)  {
           dependsOn project.classes
-          mainClass = "org.openjdk.jmh.Main"
+          mainClass = "org.apache.beam.sdk.testutils.jmh.Main"
           classpath = project.sourceSets.main.runtimeClasspath
+
+          environment([
+            'INFLUXDB_MEASUREMENT': 'java_jmh_' + (getPath().split(':') - ['', 'sdks', 'java', 'jmh']).join('_')

Review Comment:
   > nit: note that folks may want to have benchmarks under runners/ like runners/java-fn-execution which could lead to a collision in measurement names.
   
   Not really relevant anymore, but I can't see how to realistically generate duplicates with the above as the module path is unique and the exclusions are very specific &ast;wondering&ast;



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r970155162


##########
sdks/java/testing/test-utils/build.gradle:
##########
@@ -37,6 +37,7 @@ dependencies {
   implementation library.java.http_client
   implementation library.java.http_core
   implementation library.java.slf4j_api
+  provided library.java.jmh_core

Review Comment:
   Sounds good.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r968621896


##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing JMH benchmark
+ * results to InfluxDB.
+ *
+ * <h3>Schema</h3>
+ *
+ * <p>The wrapper writes an aggregated InfluxDB datapoint for each benchmark to <b> measurement</b>
+ * {@code {INFLUXDB_BASE_MEASUREMENT}_{mode}}. Typically this is {@code java_jmh_thrpt}.
+ *
+ * <p>The <b>timestamp</b> of the datapoint corresponds to the start time of the respective
+ * benchmark.
+ *
+ * <p>Individual timeseries are discriminated using the following <b>tags</b> including tags
+ * corresponding to additional benchmark parameters in case of parameterized benchmarks:
+ *
+ * <ul>
+ *   <li>{@code benchmark} (string) : Fully qualified name of the benchmark
+ *   <li>{@code scoreUnit} (string): JMH score unit
+ *   <li>optionally, additional parameters in case of a parameterized benchmark (string)
+ * </ul>
+ *
+ * <p>The following fields are captured for each benchmark:
+ *
+ * <ul>
+ *   <li>{@code score} (float): JMH score
+ *   <li>{@code scoreMean} (float): Mean score of all iterations
+ *   <li>{@code scoreMedian} (float): Median score of all iterations
+ *   <li>{@code scoreError} (float): Mean error of the score
+ *   <li>{@code sampleCount} (integer): Number of score samples
+ *   <li>{@code durationMs} (integer): Total benchmark duration (including warmups)
+ * </ul>
+ *
+ * <h3>Configuration</h3>
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:
+ *
+ * <ul>
+ *   <li>{@link #INFLUXDB_HOST}
+ *   <li>{@link #INFLUXDB_DATABASE}
+ *   <li>{@link #INFLUXDB_BASE_MEASUREMENT}
+ * </ul>
+ */
+public class Main {
+  private static final String INFLUXDB_HOST = "INFLUXDB_HOST";
+  private static final String INFLUXDB_DATABASE = "INFLUXDB_DATABASE";
+  private static final String INFLUXDB_BASE_MEASUREMENT = "INFLUXDB_BASE_MEASUREMENT";
+
+  public static void main(String[] args)
+      throws CommandLineOptionException, IOException, RunnerException {
+    final CommandLineOptions opts = new CommandLineOptions(args);
+    final InfluxDBSettings influxDB = influxDBSettings();
+
+    if (influxDB == null
+        || isSingleShotTimeOnly(opts.getBenchModes())
+        || opts.shouldHelp()
+        || opts.shouldList()
+        || opts.shouldListWithParams()
+        || opts.shouldListProfilers()
+        || opts.shouldListResultFormats()) {
+      // delegate to JMH runner
+      org.openjdk.jmh.Main.main(args);
+      return;
+    }
+
+    final Runner runner = new Runner(opts);
+    final Collection<RunResult> results = runner.run();
+
+    final Collection<DataPoint> dataPoints =
+        results.stream()
+            .filter(r -> r.getParams().getMode() != SingleShotTime)
+            .map(r -> dataPoint(influxDB.measurement, r))
+            .collect(toList());
+
+    InfluxDBPublisher.publish(influxDB, dataPoints);
+  }
+
+  private static boolean isSingleShotTimeOnly(Collection<Mode> modes) {
+    return !modes.isEmpty() && modes.stream().allMatch(SingleShotTime::equals);
+  }
+
+  private static DataPoint dataPoint(String baseMeasurement, RunResult run) {
+    final BenchmarkParams params = run.getParams();
+    final Result<?> result = run.getPrimaryResult();
+
+    final long startTimeMs =
+        metaDataStream(run).mapToLong(BenchmarkResultMetaData::getStartTime).min().getAsLong();
+    final long stopTimeMs =
+        metaDataStream(run).mapToLong(BenchmarkResultMetaData::getStopTime).max().getAsLong();
+
+    final String measurement =
+        String.format("%s_%s", baseMeasurement, params.getMode().shortLabel());
+
+    final Map<String, String> tags = new HashMap<>();
+    tags.put("benchmark", params.getBenchmark());
+    tags.put("scoreUnit", result.getScoreUnit());
+    // add params of parameterized benchmarks as tags
+    tags.putAll(params.getParamsKeys().stream().collect(toMap(identity(), params::getParam)));
+
+    final Map<String, Number> fields = new HashMap<>();
+    fields.put("score", result.getScore());
+    fields.put("scoreMean", result.getStatistics().getMean());
+    fields.put("scoreMedian", result.getStatistics().getPercentile(0.5));
+    if (!Double.isNaN(result.getScoreError())) {
+      fields.put("scoreError", result.getScoreError());
+    }
+    fields.put("sampleCount", result.getSampleCount());
+    fields.put("durationMs", stopTimeMs - startTimeMs);
+
+    return InfluxDBPublisher.dataPoint(
+        measurement, tags, fields, startTimeMs, TimeUnit.MILLISECONDS);
+  }
+
+  private static Stream<BenchmarkResultMetaData> metaDataStream(RunResult runResult) {
+    return runResult.getBenchmarkResults().stream()
+        .map(BenchmarkResult::getMetadata)
+        .filter(Objects::nonNull);
+  }
+
+  /** Construct InfluxDB settings from environment variables to not mess with JMH args. */
+  private static @Nullable InfluxDBSettings influxDBSettings() {
+    String host = System.getenv(INFLUXDB_HOST);
+    String database = System.getenv(INFLUXDB_DATABASE);
+    String measurement = System.getenv(INFLUXDB_BASE_MEASUREMENT);
+    if (measurement == null || database == null) {
+      return null;
+    }
+
+    InfluxDBSettings.Builder builder = InfluxDBSettings.builder();
+    if (host != null) {
+      builder.withHost(host); // default to localhost otherwise
+    }
+    return builder.withDatabase(database).withMeasurement(measurement).get();

Review Comment:
   What is the purpose of specifying the measurement here (since since each datapoint does it)?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r968610241


##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -1411,8 +1413,13 @@ class BeamModulePlugin implements Plugin<Project> {
 
         project.tasks.register("jmh", JavaExec)  {
           dependsOn project.classes
-          mainClass = "org.openjdk.jmh.Main"
+          mainClass = "org.apache.beam.sdk.testutils.jmh.Main"
           classpath = project.sourceSets.main.runtimeClasspath
+
+          environment([
+            'INFLUXDB_MEASUREMENT': 'java_jmh_' + (getPath().split(':') - ['', 'sdks', 'java', 'jmh']).join('_')

Review Comment:
   Yes very unlikely which is why I tagged it as a nit.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #23041: Publish results of JMH benchmark runs (Java SDK) to InfluxDB (#22238).

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #23041:
URL: https://github.com/apache/beam/pull/23041#discussion_r964023508


##########
buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy:
##########
@@ -1411,8 +1413,13 @@ class BeamModulePlugin implements Plugin<Project> {
 
         project.tasks.register("jmh", JavaExec)  {
           dependsOn project.classes
-          mainClass = "org.openjdk.jmh.Main"
+          mainClass = "org.apache.beam.sdk.testutils.jmh.Main"
           classpath = project.sourceSets.main.runtimeClasspath
+
+          environment([
+            'INFLUXDB_MEASUREMENT': 'java_jmh_' + (getPath().split(':') - ['', 'sdks', 'java', 'jmh']).join('_')

Review Comment:
   Why is it important to split up the JMH benchmarks under different measurements?



##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing benchmarks to
+ * InfluxDB.
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:

Review Comment:
   Can we document what attributes we are inserting into the influx db so that people who are writing a benchmark know what this will produce (particularly for folks using parameterized benchmarks) and under what generated measurement name?
   



##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/jmh/Main.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.beam.sdk.testutils.jmh;
+
+import static java.util.function.Function.identity;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static org.openjdk.jmh.annotations.Mode.SingleShotTime;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBPublisher.DataPoint;
+import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.infra.BenchmarkParams;
+import org.openjdk.jmh.results.BenchmarkResult;
+import org.openjdk.jmh.results.BenchmarkResultMetaData;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.CommandLineOptionException;
+import org.openjdk.jmh.runner.options.CommandLineOptions;
+
+/**
+ * Custom main wrapper around the {@link Runner JMH runner} that supports publishing benchmarks to
+ * InfluxDB.
+ *
+ * <p>If {@link InfluxDBSettings} can be inferred from the environment, benchmark results will be
+ * published to InfluxDB. Otherwise this will just delegate to the default {@link
+ * org.openjdk.jmh.Main JMH Main} class.
+ *
+ * <p>Use the following environment variables to configure {@link InfluxDBSettings}:
+ *
+ * <ul>
+ *   <li>{@link #INFLUXDB_HOST}
+ *   <li>{@link #INFLUXDB_DATABASE}
+ *   <li>{@link #INFLUXDB_MEASUREMENT}
+ * </ul>
+ */
+public class Main {
+  private static final String INFLUXDB_HOST = "INFLUXDB_HOST";
+  private static final String INFLUXDB_DATABASE = "INFLUXDB_DATABASE";
+  private static final String INFLUXDB_MEASUREMENT = "INFLUXDB_MEASUREMENT";
+
+  public static void main(String[] args)
+      throws CommandLineOptionException, IOException, RunnerException {
+    final CommandLineOptions opts = new CommandLineOptions(args);
+    final InfluxDBSettings influxDB = influxDBSettings();
+
+    if (influxDB == null
+        || isSingleShotTimeOnly(opts.getBenchModes())
+        || opts.shouldHelp()
+        || opts.shouldList()
+        || opts.shouldListWithParams()
+        || opts.shouldListProfilers()
+        || opts.shouldListResultFormats()) {
+      // delegate to JMH runner
+      org.openjdk.jmh.Main.main(args);
+      return;
+    }
+
+    final Runner runner = new Runner(opts);
+    final Collection<RunResult> results = runner.run();
+
+    final Collection<DataPoint> dataPoints =
+        results.stream()
+            .filter(r -> r.getParams().getMode() != SingleShotTime)

Review Comment:
   How do we filter out runs that failed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org