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

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

                Author: ASF GitHub Bot
            Created on: 13/Jun/18 22:25
            Start Date: 13/Jun/18 22:25
    Worklog Time Spent: 10m 
      Work Description: angoenka commented on a change in pull request #5493: 
[BEAM-4130] Add job submission capabilities to Flink runner.
URL: https://github.com/apache/beam/pull/5493#discussion_r195199604
 
 

 ##########
 File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvocation.java
 ##########
 @@ -0,0 +1,186 @@
+/*
+ * 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.flink;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.function.Consumer;
+import javax.annotation.Nullable;
+import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessage;
+import org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.core.construction.PipelineOptionsTranslation;
+import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser;
+import org.apache.beam.runners.fnexecution.jobsubmission.JobInvocation;
+import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.metrics.MetricsEnvironment;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Invocation of a Flink Job via {@link FlinkRunner}.
+ */
+public class FlinkJobInvocation implements JobInvocation {
+  private static final Logger LOG = 
LoggerFactory.getLogger(FlinkJobInvocation.class);
+
+  public static FlinkJobInvocation create(String id, ListeningExecutorService 
executorService,
+      RunnerApi.Pipeline pipeline, FlinkPipelineOptions pipelineOptions) {
+    return new FlinkJobInvocation(id, executorService, pipeline, 
pipelineOptions);
+  }
+
+  private final String id;
+  private final ListeningExecutorService executorService;
+  private final RunnerApi.Pipeline pipeline;
+  private final FlinkPipelineOptions pipelineOptions;
+  private Enum jobState;
+  private List<Consumer<Enum>> stateObservers;
+
+  @Nullable
+  private ListenableFuture<PipelineResult> invocationFuture;
+
+  private FlinkJobInvocation(String id, ListeningExecutorService 
executorService,
+      RunnerApi.Pipeline pipeline, FlinkPipelineOptions pipelineOptions) {
+    this.id = id;
+    this.executorService = executorService;
+    this.pipeline = pipeline;
+    this.pipelineOptions = pipelineOptions;
+    this.invocationFuture = null;
+    this.jobState = Enum.STOPPED;
+    this.stateObservers = new ArrayList<>();
+  }
+
+  private PipelineResult runPipeline() throws Exception {
+    LOG.trace("Translating pipeline from proto");
+
+    MetricsEnvironment.setMetricsSupported(true);
+
+    LOG.info("Translating pipeline to Flink program.");
+    // Fused pipeline proto.
+    RunnerApi.Pipeline fusedPipeline = 
GreedyPipelineFuser.fuse(pipeline).toPipeline();
 
 Review comment:
   Should we pass the fuser Function in constructor parameter?

----------------------------------------------------------------
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: 111690)
    Time Spent: 1h 10m  (was: 1h)

> Portable Flink runner JobService entry point in a Docker container
> ------------------------------------------------------------------
>
>                 Key: BEAM-4130
>                 URL: https://issues.apache.org/jira/browse/BEAM-4130
>             Project: Beam
>          Issue Type: New Feature
>          Components: runner-flink
>            Reporter: Ben Sidhom
>            Priority: Minor
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> The portable Flink runner exists as a Job Service that runs somewhere. We 
> need a main entry point that itself spins up the job service (and artifact 
> staging service). The main program itself should be packaged into an uberjar 
> such that it can be run locally or submitted to a Flink deployment via `flink 
> run`.



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

Reply via email to