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

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

                Author: ASF GitHub Bot
            Created on: 17/May/18 23:21
            Start Date: 17/May/18 23:21
    Worklog Time Spent: 10m 
      Work Description: jkff commented on a change in pull request #5262: 
[BEAM-2588] Portability Runner Job Service
URL: https://github.com/apache/beam/pull/5262#discussion_r189128166
 
 

 ##########
 File path: 
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobService.java
 ##########
 @@ -0,0 +1,264 @@
+/*
+ * 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.fnexecution.jobsubmission;
+
+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.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Consumer;
+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.model.pipeline.v1.Endpoints;
+import org.apache.beam.runners.fnexecution.FnService;
+import org.apache.beam.sdk.fn.stream.SynchronizedStreamObserver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A InMemoryJobService that prepares and runs jobs on behalf of a client 
using a
+ * {@link JobInvoker}.
+ *
+ * <p>Job management is handled in-memory rather than any persistent storage, 
running the risk of
+ * leaking jobs if the InMemoryJobService crashes.
+ *
+ * <p>TODO: replace in-memory job management state with persistent solution.
+ */
+public class InMemoryJobService extends JobServiceGrpc.JobServiceImplBase 
implements FnService {
+  private static final Logger LOG = 
LoggerFactory.getLogger(InMemoryJobService.class);
+
+  public static InMemoryJobService create(
+      Endpoints.ApiServiceDescriptor stagingServiceDescriptor, JobInvoker 
invoker) {
+    return new InMemoryJobService(stagingServiceDescriptor, invoker);
+  }
+
+  private final ConcurrentMap<String, JobPreparation> preparations;
+  private final ConcurrentMap<String, JobInvocation> invocations;
+  private final Endpoints.ApiServiceDescriptor stagingServiceDescriptor;
+  private final JobInvoker invoker;
+
+  private InMemoryJobService(
+      Endpoints.ApiServiceDescriptor stagingServiceDescriptor, JobInvoker 
invoker) {
+    this.stagingServiceDescriptor = stagingServiceDescriptor;
+    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_%s", request.getJobName(), 
UUID.randomUUID().toString());
+      Struct pipelineOptions = request.getPipelineOptions();
+      if (pipelineOptions == null) {
+        throw new NullPointerException("Encountered null pipeline options.");
+      }
+      LOG.trace("PIPELINE OPTIONS {} {}", pipelineOptions.getClass(), 
pipelineOptions);
+      JobPreparation preparation =
+          JobPreparation
+              .builder()
+              .setId(preparationId)
+              .setPipeline(request.getPipeline())
+              .setOptions(pipelineOptions)
+              .build();
+      JobPreparation previous = preparations.putIfAbsent(preparationId, 
preparation);
+      if (previous != null) {
+        // this should never happen with a UUID
+        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(stagingServiceDescriptor)
+              .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) {
+    LOG.trace("{} {}", RunJobRequest.class.getSimpleName(), request);
+
+    String preparationId = request.getPreparationId();
+    try {
+      // retrieve job preparation
+      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.pipeline(), preparation.options(), 
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) {
+      String errMessage =
+          String.format("Encountered Unexpected Exception for Preparation %s", 
preparationId);
+      LOG.error(errMessage, e);
+      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
+    }
+  }
+
+  @Override
+  public void getState(
+      GetJobStateRequest request, StreamObserver<GetJobStateResponse> 
responseObserver) {
+    LOG.trace("{} {}", GetJobStateRequest.class.getSimpleName(), request);
+    String invocationId = request.getJobId();
+    try {
+      JobInvocation invocation = getInvocation(invocationId);
+      JobState.Enum state = invocation.getState();
+      GetJobStateResponse response = 
GetJobStateResponse.newBuilder().setState(state).build();
+      responseObserver.onNext(response);
+      responseObserver.onCompleted();
+    } catch (Exception e) {
+      String errMessage =
+          String.format("Encountered Unexpected Exception for Invocation %s", 
invocationId);
+      LOG.error(errMessage, e);
+      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
+    }
+  }
+
+  @Override
+  public void cancel(CancelJobRequest request, 
StreamObserver<CancelJobResponse> responseObserver) {
+    LOG.trace("{} {}", CancelJobRequest.class.getSimpleName(), request);
+    String invocationId = request.getJobId();
+    try {
+      JobInvocation invocation = getInvocation(invocationId);
+      invocation.cancel();
+      JobState.Enum state = invocation.getState();
+      CancelJobResponse response = 
CancelJobResponse.newBuilder().setState(state).build();
+      responseObserver.onNext(response);
+      responseObserver.onCompleted();
+    } catch (Exception e) {
+      String errMessage =
+          String.format("Encountered Unexpected Exception for Invocation %s", 
invocationId);
+      LOG.error(errMessage, e);
+      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
+    }
+  }
+
+  @Override
+  public void getStateStream(
+      GetJobStateRequest request,
+      StreamObserver<GetJobStateResponse> responseObserver) {
+    LOG.trace("{} {}", GetJobStateRequest.class.getSimpleName(), request);
+    String invocationId = request.getJobId();
+    try {
+      JobInvocation invocation = getInvocation(invocationId);
+      Function<JobState.Enum, GetJobStateResponse> responseFunction =
+          state -> GetJobStateResponse.newBuilder().setState(state).build();
+      Consumer<JobState.Enum> stateListener =
+          state -> responseObserver.onNext(responseFunction.apply(state));
 
 Review comment:
   Might as well inline the function - `state -> 
responseObserver.onNext(GetJobStateResponse.newBuilder()...)`

----------------------------------------------------------------
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: 103165)
    Time Spent: 5h 50m  (was: 5h 40m)

> 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: 5h 50m
>  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)

Reply via email to