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

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

                Author: ASF GitHub Bot
            Created on: 21/May/20 00:24
            Start Date: 21/May/20 00:24
    Worklog Time Spent: 10m 
      Work Description: pulasthi commented on a change in pull request #10888:
URL: https://github.com/apache/beam/pull/10888#discussion_r428380317



##########
File path: 
runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2LegacyRunner.java
##########
@@ -0,0 +1,339 @@
+/*
+ * 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.runners.twister2;
+
+import static 
org.apache.beam.runners.core.construction.resources.PipelineResources.detectClassPathResourcesToStage;
+
+import edu.iu.dsc.tws.api.JobConfig;
+import edu.iu.dsc.tws.api.Twister2Job;
+import edu.iu.dsc.tws.api.config.Config;
+import edu.iu.dsc.tws.api.driver.DriverJobState;
+import edu.iu.dsc.tws.api.exceptions.Twister2RuntimeException;
+import edu.iu.dsc.tws.api.scheduler.Twister2JobState;
+import edu.iu.dsc.tws.api.tset.sets.TSet;
+import edu.iu.dsc.tws.api.tset.sets.batch.BatchTSet;
+import edu.iu.dsc.tws.local.LocalSubmitter;
+import edu.iu.dsc.tws.rsched.core.ResourceAllocator;
+import edu.iu.dsc.tws.rsched.job.Twister2Submitter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.logging.LogManager;
+import java.util.logging.Logger;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+import org.apache.beam.runners.core.construction.PTransformMatchers;
+import org.apache.beam.runners.core.construction.PTransformTranslation;
+import org.apache.beam.runners.core.construction.SplittableParDo;
+import org.apache.beam.runners.core.construction.SplittableParDoNaiveBounded;
+import org.apache.beam.runners.core.construction.resources.PipelineResources;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.runners.PTransformOverride;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+
+/**
+ * A {@link PipelineRunner} that executes the operations in the pipeline by 
first translating them
+ * to a Twister2 Plan and then executing them either locally or on a Twister2 
cluster, depending on
+ * the configuration.
+ */
+public class Twister2LegacyRunner extends PipelineRunner<PipelineResult> {
+
+  private static final Logger LOG = 
Logger.getLogger(Twister2LegacyRunner.class.getName());
+  private static final String SIDEINPUTS = "sideInputs";
+  private static final String LEAVES = "leaves";
+  private static final String GRAPH = "graph";
+  /** Provided options. */
+  private final Twister2PipelineOptions options;
+
+  public Twister2LegacyRunner(Twister2PipelineOptions options) {
+    this.options = options;
+  }
+
+  public static Twister2LegacyRunner fromOptions(PipelineOptions options) {
+    Twister2PipelineOptions pipelineOptions =
+        PipelineOptionsValidator.validate(Twister2PipelineOptions.class, 
options);
+    if (pipelineOptions.getFilesToStage() == null) {
+      pipelineOptions.setFilesToStage(
+          detectClassPathResourcesToStage(
+              Twister2LegacyRunner.class.getClassLoader(), pipelineOptions));
+      LOG.info(
+          "PipelineOptions.filesToStage was not specified. "
+              + "Defaulting to files from the classpath: will stage {} files. "
+              + "Enable logging at DEBUG level to see which files will be 
staged"
+              + pipelineOptions.getFilesToStage().size());
+    }
+    return new Twister2LegacyRunner(pipelineOptions);
+  }
+
+  @Override
+  public PipelineResult run(Pipeline pipeline) {
+    // create a worker and pass in the pipeline and then do the translation
+    Twister2PipelineExecutionEnvironment env = new 
Twister2PipelineExecutionEnvironment(options);
+    LOG.info("Translating pipeline to Twister2 program.");
+    pipeline.replaceAll(getDefaultOverrides());
+    env.translate(pipeline);
+    setupSystem(options);
+
+    Config config = ResourceAllocator.loadConfig(new HashMap<>());
+
+    JobConfig jobConfig = new JobConfig();
+    jobConfig.put(SIDEINPUTS, extractNames(env.getSideInputs()));
+    jobConfig.put(LEAVES, extractNames(env.getLeaves()));
+    jobConfig.put(GRAPH, env.getTSetGraph());
+
+    int workers = options.getParallelism();
+    Twister2Job twister2Job =
+        Twister2Job.newBuilder()
+            .setJobName(options.getJobName())
+            .setWorkerClass(BeamBatchWorker.class)
+            .addComputeResource(options.getWorkerCPUs(), 
options.getRamMegaBytes(), workers)
+            .setConfig(jobConfig)
+            .build();
+    Twister2JobState jobState = Twister2Submitter.submitJob(twister2Job, 
config);
+
+    Twister2PipelineResult result = new Twister2PipelineResult();
+    if (jobState.getJobstate() == DriverJobState.FAILED
+        || jobState.getJobstate() == DriverJobState.RUNNING) {
+      throw new RuntimeException("Pipeline execution failed", 
jobState.getCause());

Review comment:
       This is something that needs to be fixed form the Twister2 side first, 
currently the submit method returns once the job is complete, and if it returns 
with the state as "RUNNING" it means that the job terminated and did not 
complete. This can be fixed once the issue is fixed on the Twister2 side. I 
hope keeping it as is for now is OK for now. 




----------------------------------------------------------------
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.

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


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

    Worklog Id:     (was: 435785)
    Time Spent: 20h 10m  (was: 20h)

> Twister2 Beam runner
> --------------------
>
>                 Key: BEAM-7304
>                 URL: https://issues.apache.org/jira/browse/BEAM-7304
>             Project: Beam
>          Issue Type: New Feature
>          Components: runner-ideas
>            Reporter: Pulasthi Wickramasinghe
>            Assignee: Pulasthi Wickramasinghe
>            Priority: P3
>             Fix For: 2.22.0
>
>          Time Spent: 20h 10m
>  Remaining Estimate: 0h
>
> Twister2 is a big data framework which supports both batch and stream 
> processing [1] [2]. The goal is to develop an beam runner for Twister2. 
> [1] [https://github.com/DSC-SPIDAL/twister2]
> [2] [https://twister2.org/]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to