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

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

                Author: ASF GitHub Bot
            Created on: 21/Feb/19 18:07
            Start Date: 21/Feb/19 18:07
    Worklog Time Spent: 10m 
      Work Description: lhaiesp commented on pull request #7907: [BEAM-6714] 
Move runner-agnostic code out of FlinkJobServerDriver
URL: https://github.com/apache/beam/pull/7907#discussion_r259049360
 
 

 ##########
 File path: 
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobServerDriver.java
 ##########
 @@ -0,0 +1,198 @@
+/*
+ * 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 java.io.IOException;
+import java.nio.file.Paths;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.apache.beam.runners.fnexecution.GrpcFnServer;
+import org.apache.beam.runners.fnexecution.ServerFactory;
+import 
org.apache.beam.runners.fnexecution.artifact.BeamFileSystemArtifactStagingService;
+import 
org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
+import org.kohsuke.args4j.Option;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Shared code for starting and serving an {@link InMemoryJobService}. */
+public abstract class JobServerDriver implements Runnable {
+
+  protected abstract JobInvoker createJobInvoker();
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(JobServerDriver.class);
+
+  @VisibleForTesting protected ServerConfiguration configuration;
+
+  private final ServerFactory jobServerFactory;
+  private final ServerFactory artifactServerFactory;
+  private volatile GrpcFnServer<InMemoryJobService> jobServer;
+  private volatile GrpcFnServer<BeamFileSystemArtifactStagingService> 
artifactStagingServer;
+
+  /** Configuration for the jobServer. */
+  public static class ServerConfiguration {
+    @Option(name = "--job-host", usage = "The job server host name")
+    String host = "localhost";
+
+    @Option(
+        name = "--job-port",
+        usage = "The job service port. 0 to use a dynamic port. (Default: 
8099)")
+    int port = 8099;
+
+    @Option(
+        name = "--artifact-port",
+        usage = "The artifact service port. 0 to use a dynamic port. (Default: 
8098)")
+    int artifactPort = 8098;
+
+    @Option(name = "--artifacts-dir", usage = "The location to store staged 
artifact files")
+    String artifactStagingPath =
+        Paths.get(System.getProperty("java.io.tmpdir"), 
"beam-artifact-staging").toString();
+
+    @Option(
+        name = "--clean-artifacts-per-job",
+        usage = "When true, remove each job's staged artifacts when it 
completes")
+    boolean cleanArtifactsPerJob = false;
+
+    @Option(
+        name = "--sdk-worker-parallelism",
+        usage = "Default parallelism for SDK worker processes (see portable 
pipeline options)")
+    Long sdkWorkerParallelism = 1L;
+
+    public Long getSdkWorkerParallelism() {
+      return this.sdkWorkerParallelism;
+    }
+  }
+
+  protected static ServerFactory createJobServerFactory(ServerConfiguration 
configuration) {
+    return ServerFactory.createWithPortSupplier(() -> configuration.port);
+  }
+
+  protected static ServerFactory 
createArtifactServerFactory(ServerConfiguration configuration) {
+    return ServerFactory.createWithPortSupplier(() -> 
configuration.artifactPort);
+  }
+
+  protected JobServerDriver(
+      ServerConfiguration configuration,
+      ServerFactory jobServerFactory,
+      ServerFactory artifactServerFactory) {
+    this.configuration = configuration;
+    this.jobServerFactory = jobServerFactory;
+    this.artifactServerFactory = artifactServerFactory;
+  }
+
+  // This method is executed by TestPortableRunner via Reflection
+  public String start() throws IOException {
+    jobServer = createJobServer();
+    return jobServer.getApiServiceDescriptor().getUrl();
+  }
+
+  @Override
+  public void run() {
+    try {
+      jobServer = createJobServer();
+      jobServer.getServer().awaitTermination();
+    } catch (InterruptedException e) {
+      LOG.warn("Job server interrupted", e);
+    } catch (Exception e) {
+      LOG.warn("Exception during job server creation", e);
+    } finally {
+      stop();
+    }
+  }
+
+  // This method is executed by TestPortableRunner via Reflection
+  // Needs to be synchronized to prevent concurrency issues in testing shutdown
+  @SuppressWarnings("WeakerAccess")
+  public synchronized void stop() {
+    if (jobServer != null) {
+      try {
+        jobServer.close();
+        LOG.info("JobServer stopped on {}", 
jobServer.getApiServiceDescriptor().getUrl());
+        jobServer = null;
+      } catch (Exception e) {
+        LOG.error("Error while closing the jobServer.", e);
+      }
+    }
+    if (artifactStagingServer != null) {
+      try {
+        artifactStagingServer.close();
+        LOG.info(
+            "ArtifactStagingServer stopped on {}",
+            artifactStagingServer.getApiServiceDescriptor().getUrl());
+        artifactStagingServer = null;
+      } catch (Exception e) {
+        LOG.error("Error while closing the artifactStagingServer.", e);
+      }
+    }
+  }
+
+  private GrpcFnServer<InMemoryJobService> createJobServer() throws 
IOException {
+    InMemoryJobService service = createJobService();
+    GrpcFnServer<InMemoryJobService> jobServiceGrpcFnServer;
+    if (configuration.port == 0) {
+      jobServiceGrpcFnServer = GrpcFnServer.allocatePortAndCreateFor(service, 
jobServerFactory);
+    } else {
+      Endpoints.ApiServiceDescriptor descriptor =
+          Endpoints.ApiServiceDescriptor.newBuilder()
+              .setUrl(configuration.host + ":" + configuration.port)
+              .build();
+      jobServiceGrpcFnServer = GrpcFnServer.create(service, descriptor, 
jobServerFactory);
+    }
+    LOG.info("JobService started on {}", 
jobServiceGrpcFnServer.getApiServiceDescriptor().getUrl());
+    return jobServiceGrpcFnServer;
+  }
+
+  private InMemoryJobService createJobService() throws IOException {
+    artifactStagingServer = createArtifactStagingService();
+    JobInvoker invoker = createJobInvoker();
+    return InMemoryJobService.create(
+        artifactStagingServer.getApiServiceDescriptor(),
+        (String session) -> {
+          try {
+            return 
BeamFileSystemArtifactStagingService.generateStagingSessionToken(
+                session, configuration.artifactStagingPath);
+          } catch (Exception exn) {
+            throw new RuntimeException(exn);
+          }
+        },
+        (String stagingSessionToken) -> {
+          if (configuration.cleanArtifactsPerJob) {
+            
artifactStagingServer.getService().removeArtifacts(stagingSessionToken);
+          }
+        },
+        invoker);
+  }
+
+  private GrpcFnServer<BeamFileSystemArtifactStagingService> 
createArtifactStagingService()
 
 Review comment:
   would it be possible to have the option to opt out of this artifact staging 
service? In Samza today we are not leveraging this service and is currently 
passing in null
 
----------------------------------------------------------------
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: 202126)
    Time Spent: 1h 40m  (was: 1.5h)

> Move runner-agnostic code out of FlinkJobServerDriver
> -----------------------------------------------------
>
>                 Key: BEAM-6714
>                 URL: https://issues.apache.org/jira/browse/BEAM-6714
>             Project: Beam
>          Issue Type: Task
>          Components: runner-flink, runner-samza, runner-spark
>            Reporter: Kyle Weaver
>            Assignee: Kyle Weaver
>            Priority: Major
>          Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> [FlinkJobServerDriver|https://github.com/apache/beam/blob/master/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobServerDriver.java]
>  contains quite a bit of code that is not actually specific to the Flink 
> runner. This runner-agnostic code should be shared so that other runners (ie 
> Spark) developing portability can leverage it.



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

Reply via email to