beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-4761) Add postCommit scripts and perfkit dashboards for nexmark on Dataflow runner
Date Fri, 03 Aug 2018 22:09:00 GMT

     [ https://issues.apache.org/jira/browse/BEAM-4761?focusedWorklogId=131104&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-131104
]

ASF GitHub Bot logged work on BEAM-4761:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 03/Aug/18 22:08
            Start Date: 03/Aug/18 22:08
    Worklog Time Spent: 10m 
      Work Description: apilloud closed pull request #6127: [BEAM-4761] Run Nexmark in thread
pool
URL: https://github.com/apache/beam/pull/6127
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow.groovy
index 3abf2b16c6f..5ff02f91a35 100644
--- a/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow.groovy
+++ b/.test-infra/jenkins/job_PostCommit_Java_Nexmark_Dataflow.groovy
@@ -39,6 +39,7 @@ NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_
               ' -Pnexmark.args="' +
               [NexmarkBigqueryProperties.nexmarkBigQueryArgs,
               '--runner=DataflowRunner',
+              '--nexmarkParallel=16',
               '--streaming=false',
               '--suite=STRESS',
               '--manageResources=false',
@@ -55,6 +56,7 @@ NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_
               ' -Pnexmark.args="' +
               [NexmarkBigqueryProperties.nexmarkBigQueryArgs,
               '--runner=DataflowRunner',
+              '--nexmarkParallel=16',
               '--streaming=true',
               '--suite=STRESS',
               '--manageResources=false',
@@ -71,6 +73,7 @@ NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_
               ' -Pnexmark.args="' +
               [NexmarkBigqueryProperties.nexmarkBigQueryArgs,
               '--runner=DataflowRunner',
+              '--nexmarkParallel=16',
               '--queryLanguage=sql',
               '--streaming=false',
               '--suite=STRESS',
@@ -88,6 +91,7 @@ NoPhraseTriggeringPostCommitBuilder.postCommitJob('beam_PostCommit_Java_Nexmark_
               ' -Pnexmark.args="' +
               [NexmarkBigqueryProperties.nexmarkBigQueryArgs,
               '--runner=DataflowRunner',
+              '--nexmarkParallel=16',
               '--queryLanguage=sql',
               '--streaming=true',
               '--suite=STRESS',
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
index 537ddd22985..ecd7b5b9bf6 100644
--- a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
@@ -33,6 +33,13 @@
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.CoderException;
@@ -74,30 +81,81 @@
  * <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
  * http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
  */
-public class Main<OptionT extends NexmarkOptions> {
+public class Main {
+
+  private static class Result {
+    private final NexmarkConfiguration configuration;
+    private final NexmarkPerf perf;
+
+    private Result(NexmarkConfiguration configuration, NexmarkPerf perf) {
+      this.configuration = configuration;
+      this.perf = perf;
+    }
+  }
+
+  private static class Run implements Callable<Result> {
+    private final NexmarkLauncher<NexmarkOptions> nexmarkLauncher;
+    private final NexmarkConfiguration configuration;
+
+    private Run(String[] args, NexmarkConfiguration configuration) {
+      NexmarkOptions options = PipelineOptionsFactory.fromArgs(args).as(NexmarkOptions.class);
+      this.nexmarkLauncher = new NexmarkLauncher<>(options);
+      this.configuration = configuration;
+    }
+
+    @Override
+    public Result call() throws IOException {
+      NexmarkPerf perf = nexmarkLauncher.run(configuration);
+      return new Result(configuration, perf);
+    }
+  }
 
   /** Entry point. */
-  void runAll(OptionT options, NexmarkLauncher nexmarkLauncher) throws IOException {
+  void runAll(String[] args) throws IOException {
     Instant start = Instant.now();
+    NexmarkOptions options =
+        PipelineOptionsFactory.fromArgs(args).withValidation().as(NexmarkOptions.class);
     Map<NexmarkConfiguration, NexmarkPerf> baseline = loadBaseline(options.getBaselineFilename());
     Map<NexmarkConfiguration, NexmarkPerf> actual = new LinkedHashMap<>();
-    Iterable<NexmarkConfiguration> configurations = options.getSuite().getConfigurations(options);
+    Set<NexmarkConfiguration> configurations = options.getSuite().getConfigurations(options);
+    int nThreads = Math.min(options.getNexmarkParallel(), configurations.size());
+    ExecutorService executor = Executors.newFixedThreadPool(nThreads);
+    CompletionService<Result> completion = new ExecutorCompletionService(executor);
 
     boolean successful = true;
     try {
-      // Run all the configurations.
+      // Schedule all the configurations.
       for (NexmarkConfiguration configuration : configurations) {
-        NexmarkPerf perf = nexmarkLauncher.run(configuration);
-        if (perf != null) {
-          if (perf.errors == null || perf.errors.size() > 0) {
-            successful = false;
+        completion.submit(new Run(args, configuration));
+      }
+
+      // Collect all the results.
+      for (int scheduled = configurations.size(); scheduled > 0; scheduled--) {
+        Result result;
+        try {
+          result = completion.take().get();
+        } catch (InterruptedException e) {
+          break;
+        } catch (ExecutionException e) {
+          Throwable t = e.getCause();
+          if (t instanceof IOException) {
+            throw new IOException(t);
+          } else {
+            throw new RuntimeException(t);
           }
-          appendPerf(options.getPerfFilename(), configuration, perf);
-          actual.put(configuration, perf);
-          // Summarize what we've run so far.
-          saveSummary(null, configurations, actual, baseline, start, options);
         }
+
+        NexmarkConfiguration configuration = result.configuration;
+        NexmarkPerf perf = result.perf;
+        if (perf.errors == null || perf.errors.size() > 0) {
+          successful = false;
+        }
+        appendPerf(options.getPerfFilename(), configuration, perf);
+        actual.put(configuration, perf);
+        // Summarize what we've run so far.
+        saveSummary(null, configurations, actual, baseline, start, options);
       }
+
       if (options.getExportSummaryToBigQuery()) {
         savePerfsToBigQuery(options, actual, null, start);
       }
@@ -107,6 +165,8 @@ void runAll(OptionT options, NexmarkLauncher nexmarkLauncher) throws IOException
         saveSummary(options.getSummaryFilename(), configurations, actual, baseline, start,
options);
         saveJavascript(options.getJavascriptFilename(), configurations, actual, baseline,
start);
       }
+
+      executor.shutdown();
     }
     if (!successful) {
       throw new RuntimeException("Execution was not successful");
@@ -412,9 +472,6 @@ private static void saveJavascript(
   }
 
   public static void main(String[] args) throws IOException {
-    NexmarkOptions options =
-        PipelineOptionsFactory.fromArgs(args).withValidation().as(NexmarkOptions.class);
-    NexmarkLauncher<NexmarkOptions> nexmarkLauncher = new NexmarkLauncher<>(options);
-    new Main<>().runAll(options, nexmarkLauncher);
+    new Main().runAll(args);
   }
 }
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
index c287bae4a25..94643655769 100644
--- a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
@@ -452,4 +452,10 @@
   int getMaxNumWorkers();
 
   void setMaxNumWorkers(int value);
+
+  @Description("Number of queries to run in parallel.")
+  @Default.Integer(1)
+  int getNexmarkParallel();
+
+  void setNexmarkParallel(int value);
 }
diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java
b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java
index 3d34a9089a0..7a6d08a1b3e 100644
--- a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java
+++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java
@@ -120,7 +120,7 @@
    * any set command line flags, except for --isStreaming which is only respected for the
{@link
    * #DEFAULT} suite.
    */
-  public Iterable<NexmarkConfiguration> getConfigurations(NexmarkOptions options) {
+  public Set<NexmarkConfiguration> getConfigurations(NexmarkOptions options) {
     Set<NexmarkConfiguration> results = new LinkedHashSet<>();
     for (NexmarkConfiguration configuration : configurations) {
       NexmarkConfiguration result = configuration.copy();


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 131104)
    Time Spent: 8h  (was: 7h 50m)

> Add postCommit scripts and perfkit dashboards for nexmark on Dataflow runner
> ----------------------------------------------------------------------------
>
>                 Key: BEAM-4761
>                 URL: https://issues.apache.org/jira/browse/BEAM-4761
>             Project: Beam
>          Issue Type: Test
>          Components: examples-nexmark
>            Reporter: Etienne Chauchot
>            Assignee: Andrew Pilloud
>            Priority: Major
>          Time Spent: 8h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message