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

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

                Author: ASF GitHub Bot
            Created on: 18/Jan/19 02:04
            Start Date: 18/Jan/19 02:04
    Worklog Time Spent: 10m 
      Work Description: lhaiesp commented on pull request #7321: [BEAM-6271] 
SamzaRunner: initial support for portable api in samza runner
URL: https://github.com/apache/beam/pull/7321#discussion_r248906683
 
 

 ##########
 File path: 
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobInvocation.java
 ##########
 @@ -0,0 +1,122 @@
+/*
+ * 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.samza;
+
+import static 
org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum.CANCELLED;
+import static org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum.DONE;
+import static 
org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum.FAILED;
+import static 
org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum.RUNNING;
+import static 
org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum.STARTING;
+import static 
org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum.STOPPED;
+import static 
org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum.UNRECOGNIZED;
+import static 
org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum.UPDATED;
+
+import java.util.function.Consumer;
+import org.apache.beam.model.jobmanagement.v1.JobApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser;
+import org.apache.beam.runners.fnexecution.jobsubmission.JobInvocation;
+import org.apache.beam.runners.samza.util.PortablePipelineDotRenderer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Invocation of a Samza job via {@link SamzaRunner}. */
+public class SamzaJobInvocation implements JobInvocation {
+  private static final Logger LOG = 
LoggerFactory.getLogger(SamzaJobInvocation.class);
+
+  private final SamzaPipelineOptions options;
+  private final RunnerApi.Pipeline originalPipeline;
+  private volatile SamzaPipelineResult pipelineResult;
+
+  public SamzaJobInvocation(RunnerApi.Pipeline pipeline, SamzaPipelineOptions 
options) {
+    this.originalPipeline = pipeline;
+    this.options = options;
+  }
+
+  private SamzaPipelineResult invokeSamzaJob() {
+    // Fused pipeline proto.
+    final RunnerApi.Pipeline fusedPipeline =
+        GreedyPipelineFuser.fuse(originalPipeline).toPipeline();
+    LOG.info("Portable pipeline to run:");
+    LOG.info(PortablePipelineDotRenderer.toDotString(fusedPipeline));
+    // the pipeline option coming from sdk will set the sdk specific runner 
which will break serialization
+    // so we need to reset the runner here to a valid Java runner
+    options.setRunner(SamzaRunner.class);
+    try {
+      final SamzaRunner runner = new SamzaRunner(options);
+      return runner.runPortablePipeline(fusedPipeline);
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to invoke samza job", e);
+    }
+  }
+
+  @Override
+  public void start() {
+    LOG.info("Starting job invocation {}", getId());
+    pipelineResult = invokeSamzaJob();
+  }
+
+  @Override
+  public String getId() {
+    return options.getJobName();
+  }
+
+  @Override
+  public void cancel() {
+    try {
+      if (pipelineResult != null) {
+        pipelineResult.cancel();
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to cancel job.", e);
+    }
+  }
+
+  @Override
+  public JobApi.JobState.Enum getState() {
+    if (pipelineResult == null) {
+      return STARTING;
+    }
+    switch (pipelineResult.getState()) {
 
 Review comment:
   because one is the Java PipelineResult.State, the other one is the protobuf 
version of JobApi.JobState. That's very unfortunate. 
 
----------------------------------------------------------------
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:
us...@infra.apache.org


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

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

> initial support for portable api in samza runner
> ------------------------------------------------
>
>                 Key: BEAM-6271
>                 URL: https://issues.apache.org/jira/browse/BEAM-6271
>             Project: Beam
>          Issue Type: Task
>          Components: runner-samza
>            Reporter: Hai Lu
>            Assignee: Hai Lu
>            Priority: Major
>          Time Spent: 3h 20m
>  Remaining Estimate: 0h
>
> Support portable api in Samza runner.
> This ticket tracks the initial effort to support portable api in Samza 
> runner, including job server inside samza runner, config translation for 
> portable pipeline, transform translation for portable pipeline, refactor of 
> existing codes to merge logic of portable api and java api as much as 
> possible, etc.



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

Reply via email to