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

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

                Author: ASF GitHub Bot
            Created on: 12/Jun/18 18:37
            Start Date: 12/Jun/18 18:37
    Worklog Time Spent: 10m 
      Work Description: jkff closed pull request #5591: [BEAM-4290] Beam File 
System based ArtifactStagingService
URL: https://github.com/apache/beam/pull/5591
 
 
   

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/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingService.java
 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingService.java
new file mode 100644
index 00000000000..48d8ad6d610
--- /dev/null
+++ 
b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingService.java
@@ -0,0 +1,302 @@
+/*
+ * 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.artifact;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.hash.Hashing;
+import com.google.protobuf.util.JsonFormat;
+import io.grpc.stub.StreamObserver;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactMetadata;
+import 
org.apache.beam.model.jobmanagement.v1.ArtifactApi.CommitManifestRequest;
+import 
org.apache.beam.model.jobmanagement.v1.ArtifactApi.CommitManifestResponse;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ProxyManifest;
+import 
org.apache.beam.model.jobmanagement.v1.ArtifactApi.ProxyManifest.Location;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactMetadata;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactRequest;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactResponse;
+import 
org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc.ArtifactStagingServiceImplBase;
+import org.apache.beam.runners.fnexecution.FnService;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.MoveOptions.StandardMoveOptions;
+import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import org.apache.beam.sdk.util.MimeTypes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This implementation is experimental.
+ *
+ * {@link ArtifactStagingServiceImplBase} based on beam file system. {@link
+ * BeamFileSystemArtifactStagingService} requires {@link StagingSessionToken} 
in every me call. The
+ * manifest is put in {@link StagingSessionToken#getBasePath()}/{@link
+ * StagingSessionToken#getSessionId()} and artifacts are put in {@link
+ * StagingSessionToken#getBasePath()}/{@link 
StagingSessionToken#getSessionId()}/{@link
+ * BeamFileSystemArtifactStagingService#ARTIFACTS}.
+ *
+ * <p>The returned token is the path to the manifest file.
+ *
+ * <p>The manifest file is encoded in {@link ProxyManifest}.
+ */
+public class BeamFileSystemArtifactStagingService extends 
ArtifactStagingServiceImplBase implements
+    FnService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(BeamFileSystemArtifactStagingService.class);
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+  // Use UTF8 for all text encoding.
+  private static final Charset CHARSET = StandardCharsets.UTF_8;
+  public static final String MANIFEST = "MANIFEST";
+  public static final String ARTIFACTS = "artifacts";
+
+  @Override
+  public StreamObserver<PutArtifactRequest> putArtifact(
+      StreamObserver<PutArtifactResponse> responseObserver) {
+    return new PutArtifactStreamObserver(responseObserver);
+  }
+
+  @Override
+  public void commitManifest(
+      CommitManifestRequest request, StreamObserver<CommitManifestResponse> 
responseObserver) {
+    try {
+      ResourceId manifestResourceId = 
getManifestFileResourceId(request.getStagingSessionToken());
+      ResourceId artifactDirResourceId = 
getArtifactDirResourceId(request.getStagingSessionToken());
+      ProxyManifest.Builder proxyManifestBuilder = ProxyManifest.newBuilder()
+          .setManifest(request.getManifest());
+      for (ArtifactMetadata artifactMetadata : 
request.getManifest().getArtifactList()) {
+        proxyManifestBuilder.addLocation(Location.newBuilder()
+            .setName(artifactMetadata.getName())
+            .setUri(artifactDirResourceId
+                .resolve(encodedFileName(artifactMetadata), 
StandardResolveOptions.RESOLVE_FILE)
+                .toString()).build());
+      }
+      try (WritableByteChannel manifestWritableByteChannel = FileSystems
+          .create(manifestResourceId, MimeTypes.TEXT)) {
+        manifestWritableByteChannel
+            
.write(CHARSET.encode(JsonFormat.printer().print(proxyManifestBuilder.build())));
+      }
+      // TODO: Validate integrity of staged files.
+      responseObserver.onNext(CommitManifestResponse.newBuilder()
+          .setRetrievalToken(manifestResourceId.toString())
+          .build());
+      responseObserver.onCompleted();
+    } catch (Exception e) {
+      // TODO: Cleanup all the artifacts.
+      LOG.error("Unable to commit manifest.", e);
+      responseObserver.onError(e);
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    // Nothing to close here.
+  }
+
+  /**
+   * Generate a stagingSessionToken compatible with {@link 
BeamFileSystemArtifactStagingService}.
+   *
+   * @param sessionId Unique sessionId for artifact staging.
+   * @param basePath Base path to upload artifacts.
+   * @return Encoded stagingSessionToken.
+   */
+  public static String generateStagingSessionToken(String sessionId, String 
basePath)
+      throws Exception {
+    StagingSessionToken stagingSessionToken = new StagingSessionToken();
+    stagingSessionToken.setSessionId(sessionId);
+    stagingSessionToken.setBasePath(basePath);
+    return encodeStagingSessionToken(stagingSessionToken);
+  }
+
+  private String encodedFileName(ArtifactMetadata artifactMetadata) {
+    return "artifact_" + 
Hashing.sha256().hashString(artifactMetadata.getName(), CHARSET)
+        .toString();
+  }
+
+  private static StagingSessionToken decodeStagingSessionToken(String 
stagingSessionToken)
+      throws Exception {
+    try {
+      return MAPPER.readValue(stagingSessionToken, StagingSessionToken.class);
+    } catch (JsonProcessingException e) {
+      LOG.error(
+          "Unable to deserialize staging token {}. Expected format {}. Error 
{}",
+          stagingSessionToken, "{\"sessionId\": \"sessionId\", \"basePath\": 
\"basePath\"",
+          e.getMessage());
+      throw e;
+    }
+  }
+
+  private static String encodeStagingSessionToken(StagingSessionToken 
stagingSessionToken)
+      throws Exception {
+    try {
+      return MAPPER.writeValueAsString(stagingSessionToken);
+    } catch (JsonProcessingException e) {
+      LOG.error("Error {} occurred while serializing {}.", e.getMessage(),
+          stagingSessionToken);
+      throw e;
+    }
+  }
+
+  private ResourceId getJobDirResourceId(String stagingSessionToken) throws 
Exception {
+    ResourceId baseResourceId;
+    StagingSessionToken parsedToken = 
decodeStagingSessionToken(stagingSessionToken);
+    // Get or Create the base path
+    baseResourceId = FileSystems
+        .matchNewResource(parsedToken.getBasePath(), true /* isDirectory */);
+    // Using sessionId as the subDir to store artifacts and manifest.
+    return baseResourceId
+        .resolve(parsedToken.getSessionId(), 
StandardResolveOptions.RESOLVE_DIRECTORY);
+  }
+
+  private ResourceId getManifestFileResourceId(String stagingSessionToken) 
throws Exception {
+    return getJobDirResourceId(stagingSessionToken)
+        .resolve(MANIFEST, StandardResolveOptions.RESOLVE_FILE);
+  }
+
+  private ResourceId getArtifactDirResourceId(String stagingSessionToken) 
throws Exception {
+    return getJobDirResourceId(stagingSessionToken)
+        .resolve(ARTIFACTS, StandardResolveOptions.RESOLVE_DIRECTORY);
+  }
+
+  private class PutArtifactStreamObserver implements 
StreamObserver<PutArtifactRequest> {
+
+    private final StreamObserver<PutArtifactResponse> outboundObserver;
+    private PutArtifactMetadata metadata;
+    private ResourceId artifactId;
+    private WritableByteChannel artifactWritableByteChannel;
+
+    PutArtifactStreamObserver(StreamObserver<PutArtifactResponse> 
outboundObserver) {
+      this.outboundObserver = outboundObserver;
+    }
+
+    @Override
+    public void onNext(PutArtifactRequest putArtifactRequest) {
+      // Create the directory structure for storing artifacts in the first 
call.
+      if (metadata == null) {
+        checkNotNull(putArtifactRequest);
+        checkNotNull(putArtifactRequest.getMetadata());
+        metadata = putArtifactRequest.getMetadata();
+        // Check the base path exists or create the base path
+        try {
+          ResourceId artifactsDirId = getArtifactDirResourceId(
+              putArtifactRequest.getMetadata().getStagingSessionToken());
+          artifactId = 
artifactsDirId.resolve(encodedFileName(metadata.getMetadata()),
+              StandardResolveOptions.RESOLVE_FILE);
+          LOG.info("Going to stage artifact {} to {}.", 
metadata.getMetadata().getName(),
+              artifactId);
+          artifactWritableByteChannel = FileSystems.create(artifactId, 
MimeTypes.BINARY);
+        } catch (Exception e) {
+          LOG.error("Staging failed for artifact {} for staging token {}",
+              encodedFileName(metadata.getMetadata()), 
metadata.getStagingSessionToken());
+          outboundObserver.onError(e);
+        }
+      } else {
+        try {
+          artifactWritableByteChannel
+              
.write(putArtifactRequest.getData().getData().asReadOnlyByteBuffer());
+        } catch (IOException e) {
+          LOG.error("Staging failed for artifact {} to file {}.", 
metadata.getMetadata().getName(),
+              artifactId);
+          outboundObserver.onError(e);
+        }
+      }
+    }
+
+    @Override
+    public void onError(Throwable throwable) {
+      // Delete the artifact.
+      LOG.error("Staging artifact failed for " + artifactId, throwable);
+      try {
+        if (artifactWritableByteChannel != null) {
+          artifactWritableByteChannel.close();
+        }
+        if (artifactId != null) {
+          FileSystems.delete(Collections.singletonList(artifactId),
+              StandardMoveOptions.IGNORE_MISSING_FILES);
+        }
+
+      } catch (IOException e) {
+        LOG.error("Unable to save artifact {}", artifactId);
+        outboundObserver.onError(e);
+        return;
+      }
+      outboundObserver.onCompleted();
+    }
+
+    @Override
+    public void onCompleted() {
+      // Close the stream.
+      LOG.info("Staging artifact completed for " + artifactId);
+      if (artifactWritableByteChannel != null) {
+        try {
+          artifactWritableByteChannel.close();
+        } catch (IOException e) {
+          onError(e);
+          return;
+        }
+      }
+      outboundObserver.onCompleted();
+    }
+  }
+
+  /**
+   * Serializable StagingSessionToken used to stage files with {@link
+   * BeamFileSystemArtifactStagingService}.
+   */
+  private static class StagingSessionToken implements Serializable {
+
+    private String sessionId;
+    private String basePath;
+
+    /**
+     * Access is public for json conversion.
+     */
+    public String getSessionId() {
+      return sessionId;
+    }
+
+    private void setSessionId(String sessionId) {
+      this.sessionId = sessionId;
+    }
+
+    /**
+     * Access is public for json conversion.
+     */
+    public String getBasePath() {
+      return basePath;
+    }
+
+    private void setBasePath(String basePath) {
+      this.basePath = basePath;
+    }
+
+    @Override
+    public String toString() {
+      return "StagingSessionToken{" + "sessionId='" + sessionId + "', " + 
"basePath='" + basePath
+          + "'" + "}";
+    }
+  }
+}
diff --git 
a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingServiceTest.java
 
b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingServiceTest.java
new file mode 100644
index 00000000000..f1f31839dc9
--- /dev/null
+++ 
b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingServiceTest.java
@@ -0,0 +1,406 @@
+/*
+ * 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.artifact;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableMap;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.util.JsonFormat;
+import io.grpc.inprocess.InProcessChannelBuilder;
+import io.grpc.stub.StreamObserver;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactChunk;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactMetadata;
+import 
org.apache.beam.model.jobmanagement.v1.ArtifactApi.CommitManifestRequest;
+import 
org.apache.beam.model.jobmanagement.v1.ArtifactApi.CommitManifestResponse;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.Manifest;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ProxyManifest;
+import 
org.apache.beam.model.jobmanagement.v1.ArtifactApi.ProxyManifest.Builder;
+import 
org.apache.beam.model.jobmanagement.v1.ArtifactApi.ProxyManifest.Location;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactMetadata;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactRequest;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactResponse;
+import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc;
+import 
org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc.ArtifactStagingServiceStub;
+import org.apache.beam.runners.fnexecution.GrpcFnServer;
+import org.apache.beam.runners.fnexecution.InProcessServerFactory;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link BeamFileSystemArtifactStagingService}.
+ */
+@RunWith(JUnit4.class)
+public class BeamFileSystemArtifactStagingServiceTest {
+
+  private static final Joiner JOINER = Joiner.on("");
+  private static final Charset CHARSET = StandardCharsets.UTF_8;
+  private static final int DATA_1KB = 1 << 10;
+  private GrpcFnServer<BeamFileSystemArtifactStagingService> server;
+  private BeamFileSystemArtifactStagingService artifactStagingService;
+  private ArtifactStagingServiceStub stub;
+  private Path srcDir;
+  private Path destDir;
+
+  @Before
+  public void setUp() throws Exception {
+    artifactStagingService = new BeamFileSystemArtifactStagingService();
+    server = GrpcFnServer
+        .allocatePortAndCreateFor(artifactStagingService, 
InProcessServerFactory.create());
+    stub =
+        ArtifactStagingServiceGrpc.newStub(
+            
InProcessChannelBuilder.forName(server.getApiServiceDescriptor().getUrl()).build());
+
+    srcDir = Files.createTempDirectory("BFSTemp");
+    destDir = Files.createTempDirectory("BFDTemp");
+
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (server != null) {
+      server.close();
+    }
+    if (artifactStagingService != null) {
+      artifactStagingService.close();
+    }
+    deleteDir(srcDir, "BFSTemp");
+    deleteDir(destDir, "BFDTemp");
+    server = null;
+    artifactStagingService = null;
+    stub = null;
+  }
+
+  private void deleteDir(Path dir, String sanityString) throws IOException {
+    checkArgument(dir != null && 
dir.toAbsolutePath().toString().contains(sanityString),
+        "Invalid directory.");
+    Files.walkFileTree(dir, new SimpleFileVisitor<Path>() {
+      @Override
+      public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) 
throws IOException {
+        Files.deleteIfExists(file);
+        return FileVisitResult.CONTINUE;
+      }
+
+      @Override
+      public FileVisitResult postVisitDirectory(Path dir, IOException exc) 
throws IOException {
+        Files.deleteIfExists(dir);
+        return FileVisitResult.CONTINUE;
+      }
+    });
+  }
+
+  private void putArtifact(String stagingSessionToken, final String filePath, 
final String fileName)
+      throws Exception {
+    StreamObserver<PutArtifactRequest> outputStreamObserver = stub
+        .putArtifact(new StreamObserver<PutArtifactResponse>() {
+          @Override
+          public void onNext(PutArtifactResponse putArtifactResponse) {
+            Assert.fail("OnNext should never be called.");
+          }
+
+          @Override
+          public void onError(Throwable throwable) {
+            throwable.printStackTrace();
+            Assert.fail("OnError should never be called.");
+          }
+
+          @Override
+          public void onCompleted() {
+          }
+        });
+    outputStreamObserver.onNext(PutArtifactRequest.newBuilder().setMetadata(
+        PutArtifactMetadata.newBuilder()
+            
.setMetadata(ArtifactMetadata.newBuilder().setName(fileName).build())
+            .setStagingSessionToken(stagingSessionToken)).build());
+
+    FileInputStream fileInputStream = new FileInputStream(filePath);
+    byte[] buffer = new byte[DATA_1KB]; // 1kb chunk
+    int read = fileInputStream.read(buffer);
+    while (read != -1) {
+      outputStreamObserver.onNext(PutArtifactRequest.newBuilder().setData(
+          ArtifactChunk.newBuilder().setData(ByteString.copyFrom(buffer, 0, 
read)).build())
+          .build());
+      read = fileInputStream.read(buffer);
+    }
+    outputStreamObserver.onCompleted();
+  }
+
+  private String commitManifest(String stagingSessionToken, 
List<ArtifactMetadata> artifacts)
+      throws Exception {
+    CompletableFuture<String> stagingTokenFuture = new CompletableFuture<>();
+    stub.commitManifest(
+        
CommitManifestRequest.newBuilder().setStagingSessionToken(stagingSessionToken)
+            
.setManifest(Manifest.newBuilder().addAllArtifact(artifacts).build()).build(),
+        new StreamObserver<CommitManifestResponse>() {
+          @Override
+          public void onNext(CommitManifestResponse commitManifestResponse) {
+            
stagingTokenFuture.complete(commitManifestResponse.getRetrievalToken());
+          }
+
+          @Override
+          public void onError(Throwable throwable) {
+            Assert.fail("OnError should never be called.");
+          }
+
+          @Override
+          public void onCompleted() {
+
+          }
+        });
+    return stagingTokenFuture.get(1, TimeUnit.SECONDS);
+
+  }
+
+  @Test
+  public void generateStagingSessionTokenTest() throws Exception {
+    String basePath = destDir.toAbsolutePath().toString();
+    String stagingToken = BeamFileSystemArtifactStagingService
+        .generateStagingSessionToken("abc123", basePath);
+    Assert.assertEquals(
+        "{\"sessionId\":\"abc123\",\"basePath\":\"" + basePath + "\"}", 
stagingToken);
+  }
+
+  @Test
+  public void putArtifactsSingleSmallFileTest() throws Exception {
+    String fileName = "file1";
+    String stagingSession = "123";
+    String stagingSessionToken = BeamFileSystemArtifactStagingService
+        .generateStagingSessionToken(stagingSession, 
destDir.toUri().getPath());
+    Path srcFilePath = Paths.get(srcDir.toString(), fileName).toAbsolutePath();
+    Files.write(srcFilePath, "some_test".getBytes(CHARSET));
+    putArtifact(stagingSessionToken, srcFilePath.toString(), fileName);
+    String stagingToken = commitManifest(stagingSessionToken,
+        
Collections.singletonList(ArtifactMetadata.newBuilder().setName(fileName).build()));
+    Assert.assertEquals(
+        Paths.get(destDir.toAbsolutePath().toString(), stagingSession,
+            BeamFileSystemArtifactStagingService.MANIFEST),
+        Paths.get(stagingToken));
+    assertFiles(Collections.singleton(fileName), stagingToken);
+  }
+
+  @Test
+  public void putArtifactsMultipleFilesTest() throws Exception {
+    String stagingSession = "123";
+    Map<String, Integer> files = ImmutableMap.<String, Integer>builder()
+        .put("file5cb", (DATA_1KB / 2) /*500b*/)
+        .put("file1kb", DATA_1KB /*1 kb*/)
+        .put("file15cb", (DATA_1KB * 3) / 2  /*1.5 kb*/)
+        .put("nested/file1kb", DATA_1KB /*1 kb*/)
+        .put("file10kb", 10 * DATA_1KB /*10 kb*/)
+        .put("file100kb", 100 * DATA_1KB /*100 kb*/)
+        .build();
+
+    final String text = "abcdefghinklmop\n";
+    files.forEach((fileName, size) -> {
+      Path filePath = Paths.get(srcDir.toString(), fileName).toAbsolutePath();
+      try {
+        Files.createDirectories(filePath.getParent());
+        Files.write(filePath,
+            Strings.repeat(text, Double.valueOf(Math.ceil(size * 1.0 / 
text.length())).intValue())
+                .getBytes(CHARSET));
+      } catch (IOException ignored) {
+      }
+    });
+    String stagingSessionToken = BeamFileSystemArtifactStagingService
+        .generateStagingSessionToken(stagingSession, 
destDir.toUri().getPath());
+
+    List<ArtifactMetadata> metadata = new ArrayList<>();
+    for (String fileName : files.keySet()) {
+      putArtifact(stagingSessionToken,
+          Paths.get(srcDir.toString(), fileName).toAbsolutePath().toString(), 
fileName);
+      metadata.add(ArtifactMetadata.newBuilder().setName(fileName).build());
+    }
+
+    String stagingToken = commitManifest(stagingSessionToken, metadata);
+    Assert.assertEquals(
+        Paths.get(destDir.toAbsolutePath().toString(), stagingSession, 
"MANIFEST").toString(),
+        stagingToken);
+    assertFiles(files.keySet(), stagingToken);
+  }
+
+  @Test
+  public void putArtifactsMultipleFilesConcurrentlyTest() throws Exception {
+    String stagingSession = "123";
+    Map<String, Integer> files = ImmutableMap.<String, Integer>builder()
+        .put("file5cb", (DATA_1KB / 2) /*500b*/)
+        .put("file1kb", DATA_1KB /*1 kb*/)
+        .put("file15cb", (DATA_1KB * 3) / 2  /*1.5 kb*/)
+        .put("nested/file1kb", DATA_1KB /*1 kb*/)
+        .put("file10kb", 10 * DATA_1KB /*10 kb*/)
+        .put("file100kb", 100 * DATA_1KB /*100 kb*/)
+        .build();
+
+    final String text = "abcdefghinklmop\n";
+    files.forEach((fileName, size) -> {
+      Path filePath = Paths.get(srcDir.toString(), fileName).toAbsolutePath();
+      try {
+        Files.createDirectories(filePath.getParent());
+        Files.write(filePath,
+            Strings.repeat(text, Double.valueOf(Math.ceil(size * 1.0 / 
text.length())).intValue())
+                .getBytes(CHARSET));
+      } catch (IOException ignored) {
+      }
+    });
+    String stagingSessionToken = BeamFileSystemArtifactStagingService
+        .generateStagingSessionToken(stagingSession, 
destDir.toUri().getPath());
+
+    List<ArtifactMetadata> metadata = new ArrayList<>();
+    ExecutorService executorService = Executors.newFixedThreadPool(8);
+    try {
+      for (String fileName : files.keySet()) {
+        executorService.execute(() -> {
+          try {
+            putArtifact(stagingSessionToken,
+                Paths.get(srcDir.toString(), 
fileName).toAbsolutePath().toString(), fileName);
+          } catch (Exception e) {
+            Assert.fail(e.getMessage());
+          }
+          
metadata.add(ArtifactMetadata.newBuilder().setName(fileName).build());
+        });
+      }
+    } finally {
+      executorService.shutdown();
+      executorService.awaitTermination(2, TimeUnit.SECONDS);
+    }
+
+    String stagingToken = commitManifest(stagingSessionToken, metadata);
+    Assert.assertEquals(
+        Paths.get(destDir.toAbsolutePath().toString(), stagingSession, 
"MANIFEST").toString(),
+        stagingToken);
+    assertFiles(files.keySet(), stagingToken);
+  }
+
+  @Test
+  public void putArtifactsMultipleFilesConcurrentSessionsTest() throws 
Exception {
+    String stagingSession1 = "123";
+    String stagingSession2 = "abc";
+    Map<String, Integer> files1 = ImmutableMap.<String, Integer>builder()
+        .put("file5cb", (DATA_1KB / 2) /*500b*/)
+        .put("file1kb", DATA_1KB /*1 kb*/)
+        .put("file15cb", (DATA_1KB * 3) / 2  /*1.5 kb*/)
+        .build();
+    Map<String, Integer> files2 = ImmutableMap.<String, Integer>builder()
+        .put("nested/file1kb", DATA_1KB /*1 kb*/)
+        .put("file10kb", 10 * DATA_1KB /*10 kb*/)
+        .put("file100kb", 100 * DATA_1KB /*100 kb*/)
+        .build();
+
+    final String text = "abcdefghinklmop\n";
+    ImmutableMap.<String, 
Integer>builder().putAll(files1).putAll(files2).build()
+        .forEach((fileName, size) -> {
+          Path filePath = Paths.get(srcDir.toString(), 
fileName).toAbsolutePath();
+          try {
+            Files.createDirectories(filePath.getParent());
+            Files.write(filePath,
+                Strings
+                    .repeat(text, Double.valueOf(Math.ceil(size * 1.0 / 
text.length())).intValue())
+                    .getBytes(CHARSET));
+          } catch (IOException ignored) {
+          }
+        });
+    String stagingSessionToken1 = BeamFileSystemArtifactStagingService
+        .generateStagingSessionToken(stagingSession1, 
destDir.toUri().getPath());
+    String stagingSessionToken2 = BeamFileSystemArtifactStagingService
+        .generateStagingSessionToken(stagingSession2, 
destDir.toUri().getPath());
+
+    List<ArtifactMetadata> metadata1 = new ArrayList<>();
+    List<ArtifactMetadata> metadata2 = new ArrayList<>();
+    ExecutorService executorService = Executors.newFixedThreadPool(8);
+    try {
+      Iterator<String> iterator1 = files1.keySet().iterator();
+      Iterator<String> iterator2 = files2.keySet().iterator();
+      while (iterator1.hasNext() && iterator2.hasNext()) {
+        String fileName1 = iterator1.next();
+        String fileName2 = iterator2.next();
+        executorService.execute(() -> {
+          try {
+            putArtifact(stagingSessionToken1,
+                Paths.get(srcDir.toString(), 
fileName1).toAbsolutePath().toString(), fileName1);
+            putArtifact(stagingSessionToken2,
+                Paths.get(srcDir.toString(), 
fileName2).toAbsolutePath().toString(), fileName2);
+          } catch (Exception e) {
+            Assert.fail(e.getMessage());
+          }
+          
metadata1.add(ArtifactMetadata.newBuilder().setName(fileName1).build());
+          
metadata2.add(ArtifactMetadata.newBuilder().setName(fileName2).build());
+        });
+      }
+    } finally {
+      executorService.shutdown();
+      executorService.awaitTermination(2, TimeUnit.SECONDS);
+    }
+
+    String stagingToken1 = commitManifest(stagingSessionToken1, metadata1);
+    String stagingToken2 = commitManifest(stagingSessionToken2, metadata2);
+    Assert.assertEquals(
+        Paths.get(destDir.toAbsolutePath().toString(), stagingSession1, 
"MANIFEST").toString(),
+        stagingToken1);
+    Assert.assertEquals(
+        Paths.get(destDir.toAbsolutePath().toString(), stagingSession2, 
"MANIFEST").toString(),
+        stagingToken2);
+    assertFiles(files1.keySet(), stagingToken1);
+    assertFiles(files2.keySet(), stagingToken2);
+  }
+
+  private void assertFiles(Set<String> files, String stagingToken) throws 
IOException {
+    Builder proxyManifestBuilder = ProxyManifest.newBuilder();
+    JsonFormat.parser().merge(
+        JOINER.join(Files.readAllLines(Paths.get(stagingToken), CHARSET)),
+        proxyManifestBuilder);
+    ProxyManifest proxyManifest = proxyManifestBuilder.build();
+    Assert.assertEquals("Files in locations does not match actual file list.", 
files,
+        proxyManifest.getLocationList().stream().map(Location::getName)
+            .collect(Collectors.toSet()));
+    Assert.assertEquals("Duplicate file entries in locations.", files.size(),
+        proxyManifest.getLocationCount());
+    for (Location location : proxyManifest.getLocationList()) {
+      String expectedContent = JOINER.join(Files
+          .readAllLines(Paths.get(srcDir.toString(), location.getName()), 
CHARSET));
+      String actualContent = JOINER
+          .join(Files.readAllLines(Paths.get(location.getUri()), CHARSET));
+      Assert.assertEquals(expectedContent, actualContent);
+    }
+  }
+}


 

----------------------------------------------------------------
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: 111198)
    Time Spent: 15h 40m  (was: 15.5h)

> ArtifactStagingService that stages to a distributed filesystem
> --------------------------------------------------------------
>
>                 Key: BEAM-4290
>                 URL: https://issues.apache.org/jira/browse/BEAM-4290
>             Project: Beam
>          Issue Type: Sub-task
>          Components: runner-core
>            Reporter: Eugene Kirpichov
>            Assignee: Ankur Goenka
>            Priority: Major
>          Time Spent: 15h 40m
>  Remaining Estimate: 0h
>
> Using the job's staging directory from PipelineOptions.
> Physical layout on the distributed filesystem is TBD but it should allow for 
> arbitrary filenames and ideally for eventually avoiding uploading artifacts 
> that are already there.
> Handling credentials is TBD.



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

Reply via email to