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

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

                Author: ASF GitHub Bot
            Created on: 17/Apr/18 20:22
            Start Date: 17/Apr/18 20:22
    Worklog Time Spent: 10m 
      Work Description: jkff commented on a change in pull request #5150:  
[BEAM-4071] Add Portable Runner Job API shim
URL: https://github.com/apache/beam/pull/5150#discussion_r182208022
 
 

 ##########
 File path: 
runners/reference/java/src/main/java/org/apache/beam/runners/reference/PortableRunner.java
 ##########
 @@ -0,0 +1,254 @@
+/*
+ * 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.reference;
+
+import static com.google.common.base.Preconditions.checkState;
+import static 
org.apache.beam.runners.core.construction.PipelineResources.detectClassPathResourcesToStage;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Sets;
+import com.google.protobuf.ByteString;
+import io.grpc.ManagedChannel;
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+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.jobmanagement.v1.JobServiceGrpc.JobServiceBlockingStub;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
+import org.apache.beam.runners.core.construction.ArtifactServiceStager;
+import 
org.apache.beam.runners.core.construction.ArtifactServiceStager.FileToStage;
+import org.apache.beam.runners.core.construction.JavaReadViaImpulse;
+import org.apache.beam.runners.core.construction.PipelineOptionsTranslation;
+import org.apache.beam.runners.core.construction.PipelineTranslation;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.PipelineRunner;
+import org.apache.beam.sdk.fn.channel.ManagedChannelFactory;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsValidator;
+import org.apache.beam.sdk.options.PortablePipelineOptions;
+import org.apache.beam.sdk.runners.PTransformOverride;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** A {@link PipelineRunner} a {@link Pipeline} against a {@code JobService}. 
*/
+public class PortableRunner extends PipelineRunner<PipelineResult> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(PortableRunner.class);
+
+  /** Provided pipeline options. */
+  private final PipelineOptions options;
+  /** Job API endpoint. */
+  private final String endpoint;
+  /** Files to stage to artifact staging service. They will ultimately be 
added to the classpath. */
+  private final Collection<FileToStage> filesToStage;
+  /** Channel factory used to create communication channel with job and 
staging services. */
+  private final ManagedChannelFactory channelFactory;
+
+  /**
+   * Constructs a runner from the provided options.
+   *
+   * @param options Properties which configure the runner.
+   * @return The newly created runner.
+   */
+  public static PortableRunner fromOptions(PipelineOptions options) {
+    if (true) {
+      throw new UnsupportedOperationException();
+    }
+    return createInternal(options, new DirectoryZipper(), 
getChannelFactory(options));
+  }
+
+  @VisibleForTesting
+  static PortableRunner createInternal(
+      PipelineOptions options, DirectoryZipper zipper, ManagedChannelFactory 
channelFactory) {
+    PortablePipelineOptions portableOptions =
+        PipelineOptionsValidator.validate(PortablePipelineOptions.class, 
options);
+
+    String endpoint = portableOptions.getJobEndpoint();
+
+    // Deduplicate artifacts.
+    Set<String> pathsToStage = Sets.newHashSet();
+    if (portableOptions.getFilesToStage() == null) {
+      
pathsToStage.addAll(detectClassPathResourcesToStage(PortableRunner.class.getClassLoader()));
+      if (pathsToStage.isEmpty()) {
+        throw new IllegalArgumentException("No classpath elements found.");
+      }
+      LOG.debug(
+          "PortablePipelineOptions.filesToStage was not specified. "
+              + "Defaulting to files from the classpath: {}",
+          pathsToStage.size());
+    }
+
+    // Zip up directories so we can upload them to the artifact service.
+    ImmutableList.Builder<FileToStage> filesToStage = ImmutableList.builder();
+    try {
+      for (String path : pathsToStage) {
+        if (new File(path).exists()) {
+          // Spurious items get added to the classpath. Filter by just those 
that exist.
+          String zippedFile = zipper.replaceDirectoryWithZipFile(path);
+          filesToStage.add(createStagingFile(zippedFile));
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    return new PortableRunner(options, endpoint, filesToStage.build(), 
channelFactory);
+  }
+
+  private PortableRunner(
+      PipelineOptions options,
+      String endpoint,
+      Collection<FileToStage> filesToStage,
+      ManagedChannelFactory channelFactory) {
+    this.options = options;
+    this.endpoint = endpoint;
+    this.filesToStage = filesToStage;
+    this.channelFactory = channelFactory;
+  }
+
+  @Override
+  public PipelineResult run(Pipeline pipeline) {
+    replaceTransforms(pipeline, false);
+
+    LOG.debug("Initial files to stage: " + filesToStage);
+
+    PrepareJobRequest prepareJobRequest =
+        PrepareJobRequest.newBuilder()
+            .setJobName(options.getJobName())
+            .setPipeline(PipelineTranslation.toProto(pipeline))
+            .setPipelineOptions(PipelineOptionsTranslation.toProto(options))
+            .build();
+
+    ManagedChannel jobServiceChannel =
+        channelFactory.forDescriptor(getApiServiceDescriptor(endpoint));
+
+    JobServiceBlockingStub jobService = 
JobServiceGrpc.newBlockingStub(jobServiceChannel);
+    CloseableResource<JobServiceBlockingStub> wrappedJobService =
+        CloseableResource.of(jobService, (unused) -> 
jobServiceChannel.shutdown());
+
+    PrepareJobResponse prepareJobResponse = 
jobService.prepare(prepareJobRequest);
+    LOG.info("PrepareJobResponse: {}", prepareJobResponse);
+
+    ApiServiceDescriptor artifactStagingEndpoint = 
prepareJobResponse.getArtifactStagingEndpoint();
+
+    String stagingToken = null;
+    try (CloseableResource<ManagedChannel> artifactChannel =
+        CloseableResource.of(
+            channelFactory.forDescriptor(artifactStagingEndpoint), 
ManagedChannel::shutdown)) {
+      ArtifactServiceStager stager = 
ArtifactServiceStager.overChannel(artifactChannel.get());
+      LOG.debug("Actual files staged: {}", filesToStage);
+      stagingToken = stager.stage(filesToStage);
+    } catch (CloseableResource.CloseException e) {
+      LOG.warn("Error closing artifact staging channel", e);
+      // CloseExceptions should only be thrown while closing the channel.
+      checkState(stagingToken != null);
+    } catch (Exception e) {
+      throw new RuntimeException("Error staging files.", e);
+    }
+
+    RunJobRequest runJobRequest =
+        RunJobRequest.newBuilder()
+            .setPreparationId(prepareJobResponse.getPreparationId())
+            .setStagingToken(stagingToken)
+            .build();
+
+    RunJobResponse runJobResponse = jobService.run(runJobRequest);
+
+    LOG.info("RunJobResponse: {}", runJobResponse);
+    ByteString jobId = runJobResponse.getJobIdBytes();
+
+    return new JobServicePipelineResult(jobId, wrappedJobService);
+  }
+
+  private static Endpoints.ApiServiceDescriptor getApiServiceDescriptor(String 
descriptor) {
+    Endpoints.ApiServiceDescriptor.Builder apiServiceDescriptorBuilder =
+        Endpoints.ApiServiceDescriptor.newBuilder();
+    apiServiceDescriptorBuilder.setUrl(descriptor);
+    return apiServiceDescriptorBuilder.build();
+  }
+
+  private void replaceTransforms(Pipeline pipeline, boolean streaming) {
+    pipeline.replaceAll(getOverrides(streaming));
+  }
+
+  private List<PTransformOverride> getOverrides(boolean streaming) {
+    return ImmutableList.of(JavaReadViaImpulse.boundedOverride());
+  }
+
+  @Override
+  public String toString() {
+    return "PortableRunner#" + hashCode();
+  }
+
+  private static ManagedChannelFactory getChannelFactory(PipelineOptions 
options) {
+    ManagedChannelFactory channelFactory;
+    List<String> experiments = 
options.as(ExperimentalOptions.class).getExperiments();
+    if (experiments != null && experiments.contains("beam_fn_api_epoll")) {
+      channelFactory = ManagedChannelFactory.createEpoll();
+    } else {
+      channelFactory = ManagedChannelFactory.createDefault();
+    }
+    return channelFactory;
+  }
+
+  private static FileToStage createStagingFile(String path) {
+    // HACK: Encode the path name ourselves because the local artifact staging 
service currently
 
 Review comment:
   Is there a JIRA about this that you can reference?

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

> Portable Runner Job API shim
> ----------------------------
>
>                 Key: BEAM-4071
>                 URL: https://issues.apache.org/jira/browse/BEAM-4071
>             Project: Beam
>          Issue Type: New Feature
>          Components: runner-core
>            Reporter: Ben Sidhom
>            Assignee: Ben Sidhom
>            Priority: Minor
>          Time Spent: 3h
>  Remaining Estimate: 0h
>
> There needs to be a way to execute Java-SDK pipelines against a portable job 
> server. The job server itself is expected to be started up out-of-band. The 
> "PortableRunner" should take an option indicating the Job API endpoint and 
> defer other runner configurations to the backend itself.



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

Reply via email to