mosche commented on code in PR #22545:
URL: https://github.com/apache/beam/pull/22545#discussion_r941113312


##########
sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBPublisher.java:
##########
@@ -64,6 +64,13 @@ public static void publishNexmarkResults(
     publishWithCheck(settings, () -> publishNexmark(results, settings, tags));
   }
 
+  public static void publishTpcdsResults(

Review Comment:
   Honestly, I'm not particularly happy to see more of these publisher 
interfaces added :/
   In my eyes it's quite a bad sign that every new use case requires adding a 
new "interface" to `InfluxDBPublisher` (not to mention conversion of datatypes 
for InfluxDB scattered around the metric collection / test logic ). Just saying 
....
   Might be worth it building this on top of 
https://github.com/apache/beam/pull/22260 or use the InfluxDB java client 
instead.



##########
sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java:
##########
@@ -306,6 +327,68 @@ public static void runUsingSqlTransform(String[] args) 
throws Exception {
 
     executor.shutdown();
 
-    printExecutionSummary(completion, queryNames.length);
+    List<TpcdsRunResult> results = collectTpcdsResults(completion, 
queryNames.length);
+
+    if (tpcdsOptions.getExportSummaryToInfluxDB()) {
+      final long timestamp = start.getMillis() / 1000; // seconds
+      savePerfsToInfluxDB(tpcdsOptions, results, timestamp);
+    }
+
+    printExecutionSummary(results);
+  }
+
+  private static void savePerfsToInfluxDB(
+      final TpcdsOptions options, final List<TpcdsRunResult> results, final 
long timestamp) {
+    final InfluxDBSettings settings = getInfluxSettings(options);
+    final Map<String, String> tags = options.getInfluxTags();
+    final String runner = options.getRunner().getSimpleName();
+    final List<Map<String, Object>> schemaResults =
+        results.stream()
+            .map(
+                entry ->
+                    getResultsFromSchema(
+                        entry, timestamp, runner, produceMeasurement(options, 
entry)))
+            .collect(toList());
+    InfluxDBPublisher.publishTpcdsResults(schemaResults, settings, tags);
+  }
+
+  private static String produceMeasurement(final TpcdsOptions options, 
TpcdsRunResult entry) {
+    return String.format(
+        "%s_%s_%s_%s",
+        options.getBaseInfluxMeasurement(),
+        entry.getQueryName(),
+        entry.getDialect(),
+        entry.getDataSize());

Review Comment:
   Wondering, particularly `dataSize` looks more like a tag than something 
making it an independent measurement. Also not sure about the others ...



##########
sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java:
##########
@@ -306,6 +327,68 @@ public static void runUsingSqlTransform(String[] args) 
throws Exception {
 
     executor.shutdown();
 
-    printExecutionSummary(completion, queryNames.length);
+    List<TpcdsRunResult> results = collectTpcdsResults(completion, 
queryNames.length);
+
+    if (tpcdsOptions.getExportSummaryToInfluxDB()) {
+      final long timestamp = start.getMillis() / 1000; // seconds
+      savePerfsToInfluxDB(tpcdsOptions, results, timestamp);
+    }
+
+    printExecutionSummary(results);
+  }
+
+  private static void savePerfsToInfluxDB(
+      final TpcdsOptions options, final List<TpcdsRunResult> results, final 
long timestamp) {
+    final InfluxDBSettings settings = getInfluxSettings(options);
+    final Map<String, String> tags = options.getInfluxTags();
+    final String runner = options.getRunner().getSimpleName();
+    final List<Map<String, Object>> schemaResults =
+        results.stream()
+            .map(
+                entry ->
+                    getResultsFromSchema(
+                        entry, timestamp, runner, produceMeasurement(options, 
entry)))
+            .collect(toList());
+    InfluxDBPublisher.publishTpcdsResults(schemaResults, settings, tags);
+  }
+
+  private static String produceMeasurement(final TpcdsOptions options, 
TpcdsRunResult entry) {
+    return String.format(
+        "%s_%s_%s_%s",
+        options.getBaseInfluxMeasurement(),
+        entry.getQueryName(),
+        entry.getDialect(),
+        entry.getDataSize());
+  }
+
+  private static Map<String, Object> getResultsFromSchema(
+      final TpcdsRunResult results,
+      final long timestamp,
+      final String runner,
+      final String measurement) {
+    final Map<String, Object> schemaResults = new HashMap<>();
+    schemaResults.put("timestamp", timestamp);
+    schemaResults.put("runner", runner);
+    schemaResults.put("measurement", measurement);
+
+    // By default, InfluxDB treats all number values as floats. We need to add 
'i' suffix to
+    // interpret the value as an integer.
+    final int runtimeMs =
+        results.getIsSuccessful()
+            ? (int) (results.getElapsedTime() * 1000)
+            : // change sec to ms
+            0;
+    schemaResults.put("runtimeMs", runtimeMs + "i");

Review Comment:
   As mentioned above, I think converting datatypes for influx DB in this place 
smells a bit :/



##########
sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java:
##########
@@ -57,6 +64,10 @@
  *
  * <p>TODO: Add tests.
  */
+@SuppressWarnings({

Review Comment:
   Please avoid using `SuppressWarnings` on new code. In case there's good 
reasons / false positives I suggest to annotate the relevant place(s) only



##########
.test-infra/jenkins/job_PostCommit_Java_Tpcds_Spark.groovy:
##########
@@ -32,73 +35,42 @@ 
NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Tpcds_Sp
 
       // Gradle goals for this job.
       steps {
-        shell('echo "*** RUN TPCDS IN BATCH MODE USING SPARK 2 RUNNER ***"')

Review Comment:
   Looks like this file contains a copy-paste error... you're running the tests 
for Spark 3 twice, but not the ones for Spark 2



##########
sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsRun.java:
##########
@@ -21,17 +21,22 @@
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.PipelineResult.State;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** To fulfill multi-threaded execution. */
 public class TpcdsRun implements Callable<TpcdsRunResult> {
   private final Pipeline pipeline;
 
+  private static final Logger LOG = LoggerFactory.getLogger(TpcdsRun.class);
+
   public TpcdsRun(Pipeline pipeline) {
     this.pipeline = pipeline;
   }
 
   @Override
   public TpcdsRunResult call() {
+    LOG.info("Run TPC-DS job: " + pipeline.getOptions().getJobName());

Review Comment:
   ```suggestion
       LOG.info("Run TPC-DS job: {}", pipeline.getOptions().getJobName());
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to