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

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

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

 ##########
 File path: 
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobService.java
 ##########
 @@ -0,0 +1,340 @@
+/*
+ * 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.common.annotations.VisibleForTesting;
+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.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.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.apache.beam.sdk.fn.stream.SynchronizedStreamObserver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A JobService 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 JobService crashes.
+ *
+ * <p>TODO: replace in-memory job management state with persistent solution.
+ */
+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) {
+        // retry recursively in the unlikely case of a name collision.
 
 Review comment:
   This is funky, and we are forgetting to close stagingService. I recommend to 
replace nextInt() with generating a UUID so there are no collisions, then this 
issue is avoided.

----------------------------------------------------------------
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: 100945)
    Time Spent: 2h 50m  (was: 2h 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: 2h 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