[
https://issues.apache.org/jira/browse/BEAM-2588?focusedWorklogId=100342&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-100342
]
ASF GitHub Bot logged work on BEAM-2588:
----------------------------------------
Author: ASF GitHub Bot
Created on: 10/May/18 00:12
Start Date: 10/May/18 00:12
Worklog Time Spent: 10m
Work Description: axelmagn commented on a change in pull request #5262:
[BEAM-2588] WIP Portability Runner Job Service
URL: https://github.com/apache/beam/pull/5262#discussion_r187210523
##########
File path:
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobService.java
##########
@@ -0,0 +1,309 @@
+package org.apache.beam.runners.fnexecution.jobsubmission;
+
+
+import com.google.common.collect.ImmutableList;
+import com.google.protobuf.Struct;
+import io.grpc.Status;
+import io.grpc.StatusException;
+import io.grpc.StatusRuntimeException;
+import io.grpc.stub.StreamObserver;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+import org.apache.beam.model.jobmanagement.v1.JobApi.CancelJobRequest;
+import org.apache.beam.model.jobmanagement.v1.JobApi.CancelJobResponse;
+import org.apache.beam.model.jobmanagement.v1.JobApi.GetJobStateRequest;
+import org.apache.beam.model.jobmanagement.v1.JobApi.GetJobStateResponse;
+import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessage;
+import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessagesRequest;
+import org.apache.beam.model.jobmanagement.v1.JobApi.JobMessagesResponse;
+import org.apache.beam.model.jobmanagement.v1.JobApi.JobState;
+import org.apache.beam.model.jobmanagement.v1.JobApi.PrepareJobRequest;
+import org.apache.beam.model.jobmanagement.v1.JobApi.PrepareJobResponse;
+import org.apache.beam.model.jobmanagement.v1.JobApi.RunJobRequest;
+import org.apache.beam.model.jobmanagement.v1.JobApi.RunJobResponse;
+import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc;
+import org.apache.beam.runners.fnexecution.FnService;
+import org.apache.beam.runners.fnexecution.GrpcFnServer;
+import org.apache.beam.runners.fnexecution.artifact.ArtifactStagingService;
+import
org.apache.beam.runners.fnexecution.artifact.ArtifactStagingServiceProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A JobService that prepares and runs jobs on behalf of a client using a
{@link JobInvoker}.
+ */
+public class JobService extends JobServiceGrpc.JobServiceImplBase implements
FnService {
+ private static final Logger LOG = LoggerFactory.getLogger(JobService.class);
+
+ public static JobService create(
+ ArtifactStagingServiceProvider artifactStagingServiceProvider,
JobInvoker invoker) {
+ return new JobService(artifactStagingServiceProvider, invoker);
+ }
+
+ private final ConcurrentMap<String, JobPreparation> preparations;
+ private final ConcurrentMap<String, JobInvocation> invocations;
+ private final ArtifactStagingServiceProvider artifactStagingServiceProvider;
+ private final JobInvoker invoker;
+
+ private JobService(
+ ArtifactStagingServiceProvider artifactStagingServiceProvider,
JobInvoker invoker) {
+ this.artifactStagingServiceProvider = artifactStagingServiceProvider;
+ this.invoker = invoker;
+
+ this.preparations = new ConcurrentHashMap<>();
+ this.invocations = new ConcurrentHashMap<>();
+ }
+
+ @Override
+ public void prepare(
+ PrepareJobRequest request,
+ StreamObserver<PrepareJobResponse> responseObserver) {
+ try {
+ LOG.trace("{} {}", PrepareJobRequest.class.getSimpleName(), request);
+ // insert preparation
+ String preparationId =
+ String.format("%s_%d", request.getJobName(),
ThreadLocalRandom.current().nextInt());
+ GrpcFnServer<ArtifactStagingService> stagingService =
+ artifactStagingServiceProvider.forJob(preparationId);
+ Struct pipelineOptions = request.getPipelineOptions();
+ if (pipelineOptions == null) {
+ LOG.trace("PIPELINE OPTIONS IS NULL");
+ throw new NullPointerException("Encountered null pipeline options.");
+ /*
+ LOG.debug("Encountered null pipeline options. Using default.");
+ pipelineOptions = Struct.getDefaultInstance();
+ */
+ } else {
+ LOG.trace("PIPELINE OPTIONS IS NOT NULL");
+ }
+ LOG.trace("PIPELINE OPTIONS {} {}", pipelineOptions.getClass(),
pipelineOptions);
+ JobPreparation preparation =
+ JobPreparation
+ .builder()
+ .setId(preparationId)
+ .setPipeline(request.getPipeline())
+ .setOptions(pipelineOptions)
+ .setStagingService(stagingService)
+ .build();
+ JobPreparation previous = preparations.putIfAbsent(preparationId,
preparation);
+ if (previous != null) {
+ String errMessage =
+ String.format("A job with the preparation ID \"%s\" already
exists.", preparationId);
+ StatusException exception =
Status.NOT_FOUND.withDescription(errMessage).asException();
+ responseObserver.onError(exception);
+ return;
+ }
+
+ // send response
+ PrepareJobResponse response =
+ PrepareJobResponse
+ .newBuilder()
+ .setPreparationId(preparationId)
+
.setArtifactStagingEndpoint(stagingService.getApiServiceDescriptor())
+ .build();
+ responseObserver.onNext(response);
+ responseObserver.onCompleted();
+ } catch (Exception e) {
+ LOG.error("Could not prepare job with name {}", request.getJobName(), e);
+ responseObserver.onError(Status.INTERNAL.withCause(e).asException());
+ }
+ }
+
+ @Override
+ public void run(
+ RunJobRequest request, StreamObserver<RunJobResponse> responseObserver) {
+ try {
+ LOG.trace("{} {}", RunJobRequest.class.getSimpleName(), request);
+
+ // retrieve job preparation
+ String preparationId = request.getPreparationId();
+ JobPreparation preparation = preparations.get(preparationId);
+ if (preparation == null) {
+ String errMessage = String.format("Unknown Preparation Id \"%s\".",
preparationId);
+ StatusException exception =
Status.NOT_FOUND.withDescription(errMessage).asException();
+ responseObserver.onError(exception);
+ return;
+ }
+
+ // create new invocation
+ JobInvocation invocation = invoker.invoke(preparation,
request.getStagingToken());
+ String invocationId = invocation.getId();
+ invocation.start();
+ invocations.put(invocationId, invocation);
+ RunJobResponse response =
+ RunJobResponse.newBuilder().setJobId(invocationId).build();
+ responseObserver.onNext(response);
+ responseObserver.onCompleted();
+ } catch (StatusRuntimeException e) {
+ LOG.warn("Encountered Status Exception", e);
+ responseObserver.onError(e);
+ } catch (Exception e) {
+ LOG.error("Encountered Unexpected Exception", e);
+ responseObserver.onError(Status.INTERNAL.withCause(e).asException());
+ }
+ }
+
+ @Override
+ public void getState(
+ GetJobStateRequest request, StreamObserver<GetJobStateResponse>
responseObserver) {
+ try {
+ LOG.trace("{} {}", GetJobStateRequest.class.getSimpleName(), request);
+ String invocationId = request.getJobId();
+ JobInvocation invocation = invocations.get(invocationId);
+
+ JobState.Enum state;
+ synchronized (invocation) {
+ state = invocation.getState();
+ }
+
+ GetJobStateResponse response =
GetJobStateResponse.newBuilder().setState(state).build();
+ responseObserver.onNext(response);
+ responseObserver.onCompleted();
+ } catch (Exception e) {
+ LOG.error("Encountered Unexpected Exception", e);
+ responseObserver.onError(Status.INTERNAL.withCause(e).asException());
+ }
+ }
+
+ @Override
+ public void cancel(CancelJobRequest request,
StreamObserver<CancelJobResponse> responseObserver) {
+ try {
+ LOG.trace("{} {}", CancelJobRequest.class.getSimpleName(), request);
+ String invocationId = request.getJobId();
+ JobInvocation invocation = invocations.get(invocationId);
+
+ JobState.Enum state;
+ synchronized (invocation) {
+ invocation.cancel();
+ state = invocation.getState();
+ }
+
+ CancelJobResponse response =
CancelJobResponse.newBuilder().setState(state).build();
+ responseObserver.onNext(response);
+ responseObserver.onCompleted();
+ } catch (Exception e) {
+ LOG.error("Encountered Unexpected Exception", e);
+ responseObserver.onError(Status.INTERNAL.withCause(e).asException());
+ }
+ }
+
+ @Override
+ public void getStateStream(
+ GetJobStateRequest request,
+ StreamObserver<GetJobStateResponse> responseObserver) {
+ try {
+ String invocationId = request.getJobId();
+ JobInvocation invocation = invocations.get(invocationId);
+
+ Function<JobState.Enum, GetJobStateResponse> responseFunction =
+ state -> GetJobStateResponse.newBuilder().setState(state).build();
+ TransformStreamObserver<JobState.Enum, GetJobStateResponse>
stateObserver =
+ TransformStreamObserver.create(responseFunction, responseObserver);
+ synchronized (invocation) {
Review comment:
Could you clarify what you mean by this?
----------------------------------------------------------------
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: 100342)
Time Spent: 1h (was: 50m)
> Portable Flink Runner Job API
> -----------------------------
>
> Key: BEAM-2588
> URL: https://issues.apache.org/jira/browse/BEAM-2588
> Project: Beam
> Issue Type: New Feature
> Components: runner-flink
> Reporter: Kenneth Knowles
> Assignee: Axel Magnuson
> Priority: Major
> Labels: portability
> Time Spent: 1h
> Remaining Estimate: 0h
>
> The portable Flink runner needs to be wired into a job server so that it can
> accept jobs the job api (https://s.apache.org/beam-job-api).
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)