[
https://issues.apache.org/jira/browse/BEAM-6725?focusedWorklogId=205688&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-205688
]
ASF GitHub Bot logged work on BEAM-6725:
----------------------------------------
Author: ASF GitHub Bot
Created on: 28/Feb/19 09:35
Start Date: 28/Feb/19 09:35
Worklog Time Spent: 10m
Work Description: mxm commented on pull request #7941: [BEAM-6725] share
some Flink job invocation code with Samza runner
URL: https://github.com/apache/beam/pull/7941#discussion_r261111643
##########
File path:
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvocation.java
##########
@@ -17,31 +17,173 @@
*/
package org.apache.beam.runners.fnexecution.jobsubmission;
+import static
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
+import static
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Throwables.getRootCause;
+import static
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Throwables.getStackTraceAsString;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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;
import org.apache.beam.model.jobmanagement.v1.JobApi.JobState.Enum;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import
org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.FutureCallback;
+import
org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.Futures;
+import
org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.ListenableFuture;
+import
org.apache.beam.vendor.guava.v20_0.com.google.common.util.concurrent.ListeningExecutorService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/** Internal representation of a Job which has been invoked (prepared and run)
by a client. */
-public interface JobInvocation {
+public abstract class JobInvocation {
+
+ protected abstract PipelineResult run(Pipeline pipeline) throws Exception;
+
+ protected final RunnerApi.Pipeline pipeline;
+
+ private static final Logger LOG =
LoggerFactory.getLogger(JobInvocation.class);
+
+ private final String id;
+ private final String retrievalToken;
+ private final ListeningExecutorService executorService;
+ private List<Consumer<Enum>> stateObservers;
+ private List<Consumer<JobMessage>> messageObservers;
+ private JobState.Enum jobState;
+ @Nullable private ListenableFuture<PipelineResult> invocationFuture;
+
+ protected JobInvocation(
+ String id,
+ String retrievalToken,
+ ListeningExecutorService executorService,
+ Pipeline pipeline) {
+ this.id = id;
+ this.retrievalToken = retrievalToken;
+ this.executorService = executorService;
+ this.pipeline = pipeline;
+ this.stateObservers = new ArrayList<>();
+ this.messageObservers = new ArrayList<>();
+ this.invocationFuture = null;
+ this.jobState = JobState.Enum.STOPPED;
+ }
+
+ private PipelineResult runPipeline() throws Exception {
+ return run(pipeline);
+ }
/** Start the job. */
- void start();
+ public synchronized void start() {
+ LOG.info("Starting job invocation {}", getId());
+ if (getState() != JobState.Enum.STOPPED) {
+ throw new IllegalStateException(String.format("Job %s already running.",
getId()));
+ }
+ setState(JobState.Enum.STARTING);
+ invocationFuture = executorService.submit(this::runPipeline);
+ // TODO: Defer transitioning until the pipeline is up and running.
+ setState(JobState.Enum.RUNNING);
+ Futures.addCallback(
+ invocationFuture,
+ new FutureCallback<PipelineResult>() {
+ @Override
+ public void onSuccess(@Nullable PipelineResult pipelineResult) {
+ if (pipelineResult != null) {
+ checkArgument(
+ pipelineResult.getState() == PipelineResult.State.DONE,
+ "Success on non-Done state: " + pipelineResult.getState());
+ setState(JobState.Enum.DONE);
+ } else {
+ setState(JobState.Enum.UNSPECIFIED);
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable throwable) {
+ String message = String.format("Error during job invocation %s.",
getId());
+ LOG.error(message, throwable);
+ sendMessage(
+ JobMessage.newBuilder()
+ .setMessageText(getStackTraceAsString(throwable))
+
.setImportance(JobMessage.MessageImportance.JOB_MESSAGE_DEBUG)
+ .build());
+ sendMessage(
+ JobMessage.newBuilder()
+ .setMessageText(getRootCause(throwable).toString())
+
.setImportance(JobMessage.MessageImportance.JOB_MESSAGE_ERROR)
+ .build());
+ setState(JobState.Enum.FAILED);
+ }
+ },
+ executorService);
+ }
/** @return Unique identifier for the job invocation. */
- String getId();
+ public String getId() {
+ return id;
+ }
+
+ /** @return Token to retrieve artifacts staged via the artifact API. */
+ public String getRetrievalToken() {
Review comment:
If the field is `protected` then it is fine to be accessed by a subclass. In
this case it is even `final`. IMHO getters are overused by many Java developers
and getters come with their own sort of problems, e.g. when inheritance is used.
Just something to think about, we can leave it like it is.
----------------------------------------------------------------
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 205688)
Time Spent: 2h 40m (was: 2.5h)
> Move runner-agnostic code out of FlinkJobInvocation
> ---------------------------------------------------
>
> Key: BEAM-6725
> URL: https://issues.apache.org/jira/browse/BEAM-6725
> Project: Beam
> Issue Type: Task
> Components: runner-flink, runner-samza, runner-spark
> Reporter: Kyle Weaver
> Assignee: Kyle Weaver
> Priority: Major
> Labels: portability
> Time Spent: 2h 40m
> Remaining Estimate: 0h
>
> Like BEAM-6714, FlinkJobInvocation contains some code (for asynchronously
> running jobs) that isn't really Flink specific and therefore could/should be
> reused by other portable runners. I'm thinking instead of an interface,
> JobInvocation should be made an abstract class that implements the methods
> that it currently abstracts, with an abstract method `runPipeline` for the
> runner-specific details.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)