[ 
https://issues.apache.org/jira/browse/BEAM-2885?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16269625#comment-16269625
 ] 

ASF GitHub Bot commented on BEAM-2885:
--------------------------------------

tgroh closed pull request #4150: [BEAM-2885] Maintain In-flight preparation 
requests
URL: https://github.com/apache/beam/pull/4150
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/pom.xml b/pom.xml
index daa4b0fa5bd..a071d362dbc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -700,6 +700,12 @@
         <version>${project.version}</version>
       </dependency>
 
+      <dependency>
+        <groupId>org.apache.beam</groupId>
+        <artifactId>beam-local-artifact-service-java</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
       <dependency>
         <groupId>org.apache.beam</groupId>
         <artifactId>beam-runners-local-java-core</artifactId>
diff --git 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcFnServer.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcFnServer.java
index 9f3dd3de87e..e6a057b7bf5 100644
--- 
a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcFnServer.java
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcFnServer.java
@@ -73,6 +73,13 @@ public ServiceT getService() {
     return service;
   }
 
+  /**
+   * Get the underlying {@link Server} contained by this {@link GrpcFnServer}.
+   */
+  public Server getServer() {
+    return server;
+  }
+
   @Override
   public void close() throws Exception {
     try {
diff --git a/runners/local-artifact-service-java/build.gradle 
b/runners/local-artifact-service-java/build.gradle
index dbce60d4249..e899ec02812 100644
--- a/runners/local-artifact-service-java/build.gradle
+++ b/runners/local-artifact-service-java/build.gradle
@@ -24,6 +24,7 @@ description = "Apache Beam :: Runners :: Java Local Artifact 
Service"
 dependencies {
   compile library.java.guava
   shadow project(path: ":beam-model-parent:beam-model-job-management", 
configuration: "shadow")
+  shadow project(path: ":beam-runners-parent:beam-java-fn-execution", 
configuration: "shadow")
   shadow library.java.findbugs_jsr305
   shadow library.java.grpc_core
   shadow library.java.grpc_stub
diff --git a/runners/local-artifact-service-java/pom.xml 
b/runners/local-artifact-service-java/pom.xml
index 72490cca84d..d89e40289a2 100644
--- a/runners/local-artifact-service-java/pom.xml
+++ b/runners/local-artifact-service-java/pom.xml
@@ -56,8 +56,12 @@
       <artifactId>beam-model-job-management</artifactId>
     </dependency>
 
-    <!-- build dependencies -->
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-java-fn-execution</artifactId>
+    </dependency>
 
+    <!-- build dependencies -->
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
       <artifactId>jsr305</artifactId>
diff --git 
a/runners/local-artifact-service-java/src/main/java/org/apache/beam/artifact/local/LocalFileSystemArtifactStagerService.java
 
b/runners/local-artifact-service-java/src/main/java/org/apache/beam/artifact/local/LocalFileSystemArtifactStagerService.java
index a9f595f743c..03c03276fc5 100644
--- 
a/runners/local-artifact-service-java/src/main/java/org/apache/beam/artifact/local/LocalFileSystemArtifactStagerService.java
+++ 
b/runners/local-artifact-service-java/src/main/java/org/apache/beam/artifact/local/LocalFileSystemArtifactStagerService.java
@@ -34,12 +34,13 @@
 import javax.annotation.Nullable;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi;
 import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc;
+import org.apache.beam.runners.fnexecution.FnService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /** An {@code ArtifactStagingService} which stages files to a local temp 
directory. */
 public class LocalFileSystemArtifactStagerService
-    extends ArtifactStagingServiceGrpc.ArtifactStagingServiceImplBase {
+    extends ArtifactStagingServiceGrpc.ArtifactStagingServiceImplBase 
implements FnService {
   private static final Logger LOG =
       LoggerFactory.getLogger(LocalFileSystemArtifactStagerService.class);
 
@@ -52,7 +53,8 @@ public static LocalFileSystemArtifactStagerService 
withRootDirectory(File base)
 
   private LocalFileSystemArtifactStagerService(File stagingBase) {
     this.stagingBase = stagingBase;
-    if ((stagingBase.mkdirs() || stagingBase.exists()) && 
stagingBase.canWrite()) {
+    if (((stagingBase.exists() && stagingBase.isDirectory()) || 
stagingBase.mkdirs())
+        && stagingBase.canWrite()) {
       artifactsBase = new File(stagingBase, "artifacts");
       checkState(
           (artifactsBase.mkdir() || artifactsBase.exists()) && 
artifactsBase.canWrite(),
@@ -122,6 +124,11 @@ File getArtifactFile(String artifactName) {
     return new File(artifactsBase, artifactName);
   }
 
+  @Override
+  public void close() throws Exception {
+    // TODO: Close all active staging calls, signalling errors to the caller.
+  }
+
   private class CreateAndWriteFileObserver
       implements StreamObserver<ArtifactApi.PutArtifactRequest> {
     private final StreamObserver<ArtifactApi.PutArtifactResponse> 
responseObserver;
diff --git a/runners/reference/job-server/build.gradle 
b/runners/reference/job-server/build.gradle
index f19c82228a2..1eaee9bf29f 100644
--- a/runners/reference/job-server/build.gradle
+++ b/runners/reference/job-server/build.gradle
@@ -24,11 +24,15 @@ description = "Apache Beam :: Runners :: Reference :: Job 
Orchestrator"
 dependencies {
   shadow library.java.grpc_netty
   shadow project(path: ":beam-model-parent:beam-model-job-management", 
configuration: "shadow")
+  shadow project(path: ":beam-runners-parent:beam-java-fn-execution", 
configuration: "shadow")
+  shadow project(path: 
":beam-runners-parent:beam-local-artifact-service-java", configuration: 
"shadow")
   shadow library.java.grpc_core
   shadow library.java.grpc_stub
   shadow library.java.slf4j_api
   shadow library.java.args4j
   testCompile library.java.junit
+  shadowTest project(path: 
":beam-runners-parent:beam-runners-core-construction-java", configuration: 
"shadow")
+  shadowTest library.java.slf4j_jdk14
 }
 
 task packageTests(type: Jar) {
diff --git a/runners/reference/job-server/pom.xml 
b/runners/reference/job-server/pom.xml
index fb0f1703f10..184b959fe95 100644
--- a/runners/reference/job-server/pom.xml
+++ b/runners/reference/job-server/pom.xml
@@ -42,25 +42,39 @@
   </build>
 
   <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-model-pipeline</artifactId>
+    </dependency>
+
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-model-job-management</artifactId>
     </dependency>
 
     <dependency>
-      <groupId>io.grpc</groupId>
-      <artifactId>grpc-core</artifactId>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-java-fn-execution</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-local-artifact-service-java</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
     </dependency>
 
     <dependency>
       <groupId>io.grpc</groupId>
-      <artifactId>grpc-stub</artifactId>
+      <artifactId>grpc-core</artifactId>
     </dependency>
 
     <dependency>
       <groupId>io.grpc</groupId>
-      <artifactId>grpc-netty</artifactId>
-      <scope>runtime</scope>
+      <artifactId>grpc-stub</artifactId>
     </dependency>
 
     <dependency>
@@ -73,10 +87,40 @@
       <artifactId>args4j</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <!-- runtime dependencies -->
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+
+    <!-- test dependencies -->
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
+
+    <dependency>
+      <groupId>com.google.auto.value</groupId>
+      <artifactId>auto-value</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-core-construction-java</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git 
a/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/PreparingJob.java
 
b/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/PreparingJob.java
new file mode 100644
index 00000000000..80505cafecf
--- /dev/null
+++ 
b/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/PreparingJob.java
@@ -0,0 +1,54 @@
+/*
+ * 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.job;
+
+import com.google.auto.value.AutoValue;
+import com.google.protobuf.Struct;
+import org.apache.beam.artifact.local.LocalFileSystemArtifactStagerService;
+import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
+import org.apache.beam.runners.fnexecution.GrpcFnServer;
+
+/** A Job with a {@code prepare} call but no corresponding {@code run} call. */
+@AutoValue
+abstract class PreparingJob implements AutoCloseable {
+  public static Builder builder() {
+    return new AutoValue_PreparingJob.Builder();
+  }
+
+  abstract Pipeline getPipeline();
+  abstract Struct getOptions();
+  abstract GrpcFnServer<LocalFileSystemArtifactStagerService> 
getArtifactStagingServer();
+
+  @Override
+  public void close() throws Exception {
+    getArtifactStagingServer().close();
+  }
+
+  @AutoValue.Builder
+  public abstract static class Builder {
+    abstract Builder setPipeline(Pipeline pipeline);
+
+    abstract Builder setOptions(Struct options);
+
+    abstract Builder setArtifactStagingServer(
+        GrpcFnServer<LocalFileSystemArtifactStagerService> server);
+
+    abstract PreparingJob build();
+  }
+}
diff --git 
a/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobServer.java
 
b/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobServer.java
index 298f532177a..cbb6f5255eb 100644
--- 
a/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobServer.java
+++ 
b/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobServer.java
@@ -18,9 +18,9 @@
 
 package org.apache.beam.runners.reference.job;
 
-import io.grpc.Server;
-import io.grpc.ServerBuilder;
-import java.io.IOException;
+import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
+import org.apache.beam.runners.fnexecution.GrpcFnServer;
+import org.apache.beam.runners.fnexecution.ServerFactory;
 import org.kohsuke.args4j.CmdLineException;
 import org.kohsuke.args4j.CmdLineParser;
 import org.kohsuke.args4j.Option;
@@ -31,13 +31,13 @@
 public class ReferenceRunnerJobServer {
   private static final Logger LOG = 
LoggerFactory.getLogger(ReferenceRunnerJobService.class);
 
-  public static void main(String[] args) throws IOException, 
InterruptedException {
+  public static void main(String[] args) throws Exception {
     ServerConfiguration configuration = new ServerConfiguration();
     CmdLineParser parser = new CmdLineParser(configuration);
     try {
       parser.parseArgument(args);
     } catch (CmdLineException e) {
-      System.err.println(e);
+      e.printStackTrace(System.err);
       printUsage(parser);
       return;
     }
@@ -53,15 +53,20 @@ private static void printUsage(CmdLineParser parser) {
   }
 
   private static void runServer(ServerConfiguration configuration)
-      throws IOException, InterruptedException {
-    ReferenceRunnerJobService service = ReferenceRunnerJobService.create();
-    Server server = 
ServerBuilder.forPort(configuration.port).addService(service).build();
-    server.start();
-    System.out.println(
-        String.format(
-            "Started %s on port %s",
-            ReferenceRunnerJobService.class.getSimpleName(), 
configuration.port));
-    server.awaitTermination();
+      throws Exception {
+    ServerFactory serverFactory = ServerFactory.createDefault();
+    ReferenceRunnerJobService service = 
ReferenceRunnerJobService.create(serverFactory);
+    try (GrpcFnServer<ReferenceRunnerJobService> server =
+        GrpcFnServer.create(
+            service,
+            ApiServiceDescriptor.newBuilder().setUrl("localhost:" + 
configuration.port).build(),
+            serverFactory)) {
+      System.out.println(
+          String.format(
+              "Started %s on port %s",
+              ReferenceRunnerJobService.class.getSimpleName(), 
configuration.port));
+      server.getServer().awaitTermination();
+    }
     System.out.println("Server shut down, exiting");
   }
 
diff --git 
a/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobService.java
 
b/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobService.java
index ded09eacec2..33ccbd7b8d8 100644
--- 
a/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobService.java
+++ 
b/runners/reference/job-server/src/main/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobService.java
@@ -18,8 +18,18 @@
 
 package org.apache.beam.runners.reference.job;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.collect.ImmutableList;
 import io.grpc.Status;
 import io.grpc.stub.StreamObserver;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.beam.artifact.local.LocalFileSystemArtifactStagerService;
 import org.apache.beam.model.jobmanagement.v1.JobApi;
 import org.apache.beam.model.jobmanagement.v1.JobApi.CancelJobRequest;
 import org.apache.beam.model.jobmanagement.v1.JobApi.CancelJobResponse;
@@ -28,26 +38,69 @@
 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.JobServiceGrpc.JobServiceImplBase;
+import org.apache.beam.runners.fnexecution.FnService;
+import org.apache.beam.runners.fnexecution.GrpcFnServer;
+import org.apache.beam.runners.fnexecution.ServerFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /** The ReferenceRunner uses the portability framework to execute a Pipeline 
on a single machine. */
-public class ReferenceRunnerJobService extends JobServiceImplBase {
+public class ReferenceRunnerJobService extends JobServiceImplBase implements 
FnService {
   private static final Logger LOG = 
LoggerFactory.getLogger(ReferenceRunnerJobService.class);
 
-  public static ReferenceRunnerJobService create() {
-    return new ReferenceRunnerJobService();
+  public static ReferenceRunnerJobService create(ServerFactory serverFactory) {
+    return new ReferenceRunnerJobService(serverFactory);
   }
 
-  private ReferenceRunnerJobService() {}
+  private final ServerFactory serverFactory;
+  private final ConcurrentMap<String, PreparingJob> unpreparedJobs;
+
+  private ReferenceRunnerJobService(ServerFactory serverFactory) {
+    this.serverFactory = serverFactory;
+    unpreparedJobs = new ConcurrentHashMap<>();
+  }
 
   @Override
   public void prepare(
       JobApi.PrepareJobRequest request,
       StreamObserver<JobApi.PrepareJobResponse> responseObserver) {
-    LOG.trace("{} {}", PrepareJobResponse.class.getSimpleName(), request);
-    System.err.println("Preparation Job Blah");
-    responseObserver.onError(Status.UNIMPLEMENTED.asException());
+    try {
+      LOG.trace("{} {}", PrepareJobResponse.class.getSimpleName(), request);
+
+      String preparationId = request.getJobName() + 
ThreadLocalRandom.current().nextInt();
+      GrpcFnServer<LocalFileSystemArtifactStagerService> 
artifactStagingService =
+          createArtifactStagingService(preparationId);
+      PreparingJob previous =
+          unpreparedJobs.putIfAbsent(
+              preparationId,
+              PreparingJob.builder()
+                  .setArtifactStagingServer(artifactStagingService)
+                  .setPipeline(request.getPipeline())
+                  .setOptions(request.getPipelineOptions())
+                  .build());
+      checkArgument(
+          previous == null, "Unexpected existing job with preparation ID %s", 
preparationId);
+
+      responseObserver.onNext(
+          PrepareJobResponse.newBuilder()
+              .setPreparationId(preparationId)
+              
.setArtifactStagingEndpoint(artifactStagingService.getApiServiceDescriptor())
+              .build());
+      responseObserver.onCompleted();
+    } catch (Exception e) {
+      LOG.error("Could not prepare job with name {}", request.getJobName(), e);
+      responseObserver.onError(Status.INTERNAL.withCause(e).asException());
+    }
+  }
+
+  private GrpcFnServer<LocalFileSystemArtifactStagerService> 
createArtifactStagingService(
+      String preparationId) throws IOException {
+    Path tempDir = Files.createTempDirectory("reference-runner-staging");
+    LocalFileSystemArtifactStagerService service =
+        
LocalFileSystemArtifactStagerService.withRootDirectory(tempDir.toFile());
+    GrpcFnServer<LocalFileSystemArtifactStagerService> server =
+        GrpcFnServer.allocatePortAndCreateFor(service, serverFactory);
+    return server;
   }
 
   @Override
@@ -76,4 +129,15 @@ public void cancel(CancelJobRequest request, 
StreamObserver<CancelJobResponse> r
             .withDescription(String.format("Unknown Job ID %s", 
request.getJobId()))
             .asException());
   }
+
+  @Override
+  public void close() throws Exception {
+    for (PreparingJob preparingJob : 
ImmutableList.copyOf(unpreparedJobs.values())) {
+      try {
+        preparingJob.close();
+      } catch (Exception e) {
+        LOG.warn("Exception while closing preparing job {}", preparingJob);
+      }
+    }
+  }
 }
diff --git 
a/runners/reference/job-server/src/test/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobServiceTest.java
 
b/runners/reference/job-server/src/test/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobServiceTest.java
index 16cde116cd1..62be932fc53 100644
--- 
a/runners/reference/job-server/src/test/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobServiceTest.java
+++ 
b/runners/reference/job-server/src/test/java/org/apache/beam/runners/reference/job/ReferenceRunnerJobServiceTest.java
@@ -18,7 +18,27 @@
 
 package org.apache.beam.runners.reference.job;
 
+import com.google.common.collect.ImmutableList;
+import com.google.protobuf.Struct;
+import io.grpc.inprocess.InProcessChannelBuilder;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+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.JobServiceGrpc;
+import 
org.apache.beam.model.jobmanagement.v1.JobServiceGrpc.JobServiceBlockingStub;
+import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
+import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
+import org.apache.beam.runners.core.construction.ArtifactServiceStager;
+import org.apache.beam.runners.fnexecution.GrpcFnServer;
+import org.apache.beam.runners.fnexecution.InProcessServerFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
@@ -27,8 +47,54 @@
  */
 @RunWith(JUnit4.class)
 public class ReferenceRunnerJobServiceTest {
+  @Rule public TemporaryFolder temp = new TemporaryFolder();
+
+  private InProcessServerFactory serverFactory = 
InProcessServerFactory.create();
+  private ReferenceRunnerJobService service;
+  private GrpcFnServer<ReferenceRunnerJobService> server;
+  private JobServiceBlockingStub stub;
+
+  @Before
+  public void setup() throws Exception {
+    service = ReferenceRunnerJobService.create(serverFactory);
+    server = GrpcFnServer.allocatePortAndCreateFor(service, serverFactory);
+    stub =
+        JobServiceGrpc.newBlockingStub(
+            
InProcessChannelBuilder.forName(server.getApiServiceDescriptor().getUrl()).build());
+  }
+
+  @After
+  public void teardown() throws Exception {
+    server.close();
+  }
+
   @Test
-  public void testPrepareJob() {
-    // TODO: Implement when PrepareJob is implemented.
+  public void testPrepareJob() throws Exception {
+    PrepareJobResponse response =
+        stub.prepare(
+            PrepareJobRequest.newBuilder()
+                .setPipelineOptions(Struct.getDefaultInstance())
+                .setPipeline(Pipeline.getDefaultInstance())
+                .setJobName("myJobName")
+                .build());
+
+    ApiServiceDescriptor stagingEndpoint = 
response.getArtifactStagingEndpoint();
+    ArtifactServiceStager stager =
+        ArtifactServiceStager.overChannel(
+            InProcessChannelBuilder.forName(stagingEndpoint.getUrl()).build());
+    File foo = writeTempFile("foo", "foo, bar, baz".getBytes());
+    File bar = writeTempFile("spam", "spam, ham, eggs".getBytes());
+    stager.stage(ImmutableList.<File>of(foo, bar));
+    // TODO: 'run' the job with some sort of noop backend, to verify state is 
cleaned up.
+    // TODO: Verify that the artifacts have been staged
+  }
+
+  private File writeTempFile(String fileName, byte[] contents) throws 
Exception {
+    File file = temp.newFile(fileName);
+    try (FileOutputStream stream = new FileOutputStream(file);
+        FileChannel channel = stream.getChannel()) {
+      channel.write(ByteBuffer.wrap(contents));
+    }
+    return file;
   }
 }


 

----------------------------------------------------------------
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]


> Support job+artifact APIs locally
> ---------------------------------
>
>                 Key: BEAM-2885
>                 URL: https://issues.apache.org/jira/browse/BEAM-2885
>             Project: Beam
>          Issue Type: Sub-task
>          Components: runner-dataflow
>            Reporter: Henning Rohde
>            Assignee: Thomas Groh
>              Labels: portability
>
> As per https://s.apache.org/beam-job-api, use local support for 
> submission-side. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to