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

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

                Author: ASF GitHub Bot
            Created on: 14/Jun/18 23:36
            Start Date: 14/Jun/18 23:36
    Worklog Time Spent: 10m 
      Work Description: angoenka commented on a change in pull request #5651: 
[BEAM-4291] Implements distributed artifact retrieval
URL: https://github.com/apache/beam/pull/5651#discussion_r195600839
 
 

 ##########
 File path: 
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactRetrievalService.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.api.client.util.Base64;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import com.google.common.io.ByteStreams;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.util.JsonFormat;
+import io.grpc.stub.StreamObserver;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactMetadata;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ProxyManifest;
+import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.ResourceId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An {@link ArtifactRetrievalService} that uses {@link FileSystems} as its 
backing storage and uses
+ * the artifact layout and retrieval token format produced by {@link
+ * BeamFileSystemArtifactStagingService}.
+ */
+public class BeamFileSystemArtifactRetrievalService
+    extends ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceImplBase
+    implements ArtifactRetrievalService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(BeamFileSystemArtifactRetrievalService.class);
+
+  private static final int ARTIFACT_CHUNK_SIZE_BYTES = 2 << 20; // 2MB
+
+  public static BeamFileSystemArtifactRetrievalService create() {
+    return new BeamFileSystemArtifactRetrievalService();
+  }
+
+  @Override
+  public void getManifest(
+      ArtifactApi.GetManifestRequest request,
+      StreamObserver<ArtifactApi.GetManifestResponse> responseObserver) {
+    String token = request.getRetrievalToken();
+    LOG.info("GetManifest for {}", token);
+    try {
+      ArtifactApi.ProxyManifest proxyManifest = MANIFEST_CACHE.get(token);
+      ArtifactApi.GetManifestResponse response =
+          ArtifactApi.GetManifestResponse.newBuilder()
+              .setManifest(proxyManifest.getManifest())
+              .build();
+      LOG.info(
+          "GetManifest for {} -> {} artifacts",
+          token,
+          proxyManifest.getManifest().getArtifactCount());
+      responseObserver.onNext(response);
+      responseObserver.onCompleted();
+    } catch (Exception e) {
+      LOG.info("GetManifest for {} failed", token, e);
+      responseObserver.onError(e);
+    }
+  }
+
+  @Override
+  public void getArtifact(
+      ArtifactApi.GetArtifactRequest request,
+      StreamObserver<ArtifactApi.ArtifactChunk> responseObserver) {
+    LOG.info("GetArtifact {}", request);
+    String name = request.getName();
+    try {
+      ArtifactApi.ProxyManifest proxyManifest = 
MANIFEST_CACHE.get(request.getRetrievalToken());
+      // look for file at URI specified by proxy manifest location
+      ArtifactApi.ProxyManifest.Location location =
+          proxyManifest
+              .getLocationList()
+              .stream()
+              .filter(loc -> loc.getName().equals(name))
+              .findFirst()
+              .orElseThrow(
+                  () ->
+                      new FileNotFoundException(
+                          String.format("Artifact location not found in 
manifest: %s", name)));
+
+      List<ArtifactMetadata> existingArtifacts = 
proxyManifest.getManifest().getArtifactList();
+      ArtifactMetadata metadata =
+          existingArtifacts
+              .stream()
+              .filter(meta -> meta.getName().equals(name))
+              .findFirst()
+              .orElseThrow(
+                  () ->
+                      new FileNotFoundException(
+                          String.format("Artifact metadata not found in 
manifest: %s", name)));
+
+      ResourceId artifactResourceId =
+          FileSystems.matchNewResource(location.getUri(), false /* is 
directory */);
+      LOG.info("Artifact {} located in {}", name, artifactResourceId);
+      Hasher hasher = Hashing.md5().newHasher();
+      byte[] data = new byte[ARTIFACT_CHUNK_SIZE_BYTES];
+      try (InputStream stream = 
Channels.newInputStream(FileSystems.open(artifactResourceId))) {
+        int len;
+        while ((len = stream.read(data)) != -1) {
+          hasher.putBytes(data, 0, len);
+          responseObserver.onNext(
+              ArtifactApi.ArtifactChunk.newBuilder()
+                  .setData(ByteString.copyFrom(data, 0, len))
+                  .build());
+        }
+      }
+      if (metadata.getMd5() != null && !metadata.getMd5().isEmpty()) {
+        ByteString expected = 
ByteString.copyFrom(Base64.decodeBase64(metadata.getMd5()));
+        ByteString actual = ByteString.copyFrom(hasher.hash().asBytes());
+        if (!actual.equals(expected)) {
+          throw new IllegalStateException(
 
 Review comment:
   Should we call `responseObserver.onError`

----------------------------------------------------------------
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: 112110)
    Time Spent: 5h 20m  (was: 5h 10m)

> ArtifactRetrievalService that retrieves artifacts from a distributed 
> filesystem
> -------------------------------------------------------------------------------
>
>                 Key: BEAM-4291
>                 URL: https://issues.apache.org/jira/browse/BEAM-4291
>             Project: Beam
>          Issue Type: Sub-task
>          Components: runner-core
>            Reporter: Eugene Kirpichov
>            Assignee: Axel Magnuson
>            Priority: Major
>          Time Spent: 5h 20m
>  Remaining Estimate: 0h
>
> In agreement with how they are staged in BEAM-4290.



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

Reply via email to