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 18:33:00 GMT

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

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

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

 ##########
 File path: sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
 ##########
 @@ -74,30 +78,97 @@
  * <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 Run implements Runnable {
+    final NexmarkLauncher<NexmarkOptions> nexmarkLauncher;
+    final NexmarkConfiguration configuration;
+    final BlockingQueue results;
+    NexmarkPerf perf = null;
+    Exception exception = null;
+
+    private Run(String[] args, NexmarkConfiguration configuration, BlockingQueue results)
{
+      NexmarkOptions options = PipelineOptionsFactory.fromArgs(args).as(NexmarkOptions.class);
+      this.nexmarkLauncher = new NexmarkLauncher<>(options);
+      this.configuration = configuration;
+      this.results = results;
+    }
+
+    @Override
+    public void run() {
+      try {
+        try {
+          perf = nexmarkLauncher.run(configuration);
+        } catch (IOException | RuntimeException e) {
+          exception = e;
+        } finally {
+          results.put(this);
+        }
+      } catch (InterruptedException ie) {
+        return;
+      }
+    }
+
+    private NexmarkPerf get() throws IOException {
+      if (exception != null) {
+        if (exception instanceof IOException) {
+          throw (IOException) exception;
+        } else if (exception instanceof RuntimeException) {
+          throw (RuntimeException) exception;
+        }
+        // This should be unreachable.
+      }
+      assert perf != null;
+      return 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);
+    ExecutorService executor = Executors.newFixedThreadPool(options.getNexmarkParallel());
+    BlockingQueue<Run> results = new LinkedBlockingQueue();
     Map<NexmarkConfiguration, NexmarkPerf> baseline = loadBaseline(options.getBaselineFilename());
     Map<NexmarkConfiguration, NexmarkPerf> actual = new LinkedHashMap<>();
     Iterable<NexmarkConfiguration> configurations = options.getSuite().getConfigurations(options);
 
     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;
+        executor.execute(new Run(args, configuration, results));
+        if (options.getNexmarkParallel() > 1) {
+          // Stagger start
+          try {
+            Thread.sleep(2000);
+          } catch (InterruptedException e) {
+            return;
           }
-          appendPerf(options.getPerfFilename(), configuration, perf);
-          actual.put(configuration, perf);
-          // Summarize what we've run so far.
-          saveSummary(null, configurations, actual, baseline, start, options);
         }
       }
+
+      // Collect all the results.
+      for (Object i : configurations) {
+        Run result;
+        try {
+          result = results.take();
 
 Review comment:
   Synced with Andrew. We have test timeout configured in test framework. Right now we rely
on that one to kill the test if it hangs for too long.
   We can add cleanup on timout logic in another PR.
   I believe this summary can be applied to other timeout-related comments on this PR.

----------------------------------------------------------------
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: 131004)
    Time Spent: 5.5h  (was: 5h 20m)

> 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: 5.5h
>  Remaining Estimate: 0h
>




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

Mime
View raw message