[
https://issues.apache.org/jira/browse/BEAM-4291?focusedWorklogId=109922&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-109922
]
ASF GitHub Bot logged work on BEAM-4291:
----------------------------------------
Author: ASF GitHub Bot
Created on: 07/Jun/18 23:13
Start Date: 07/Jun/18 23:13
Worklog Time Spent: 10m
Work Description: jkff commented on a change in pull request #5584:
[BEAM-4291] Add distributed artifact retrieval
URL: https://github.com/apache/beam/pull/5584#discussion_r193914606
##########
File path:
runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/DfsArtifactRetrievalService.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 com.google.protobuf.ByteString;
+import io.grpc.stub.StreamObserver;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi;
+import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc;
+import org.apache.beam.sdk.io.FileSystems;
+import org.apache.beam.sdk.io.fs.ResolveOptions;
+import org.apache.beam.sdk.io.fs.ResourceId;
+
+/**
+ * An {@link ArtifactRetrievalService} that uses distributed file systems as
its backing storage.
Review comment:
So if I understand correctly this works like this, please correct me if I'm
wrong:
- It uses staging_token as the directory where the job's staging stuff is
stored - both the artifacts and the manifest
- The manifest is stored in a file $token/MANIFEST
- Artifacts are stored in $token/$artifactName
- The manifest is expected to have .metadata.name be full paths to the
artifacts, i.e. $token/$originalName
I would propose to change this a bit, considering that manifest should
reflect what artifacts do we have rather than where they are stored; i.e. I
think the name of the artifact in the manifest must be identical to its name in
PutArtifactRequest, so it can not contain the staging token.
I suggest something like the following:
- Still use staging_token as the directory where the job's staging stuff is
stored - both the artifacts and the manifest
- Still store manifest in $token/MANIFEST
- Manifest contains the original artifact names from PutArtifactRequest
- Have an "artifact map" file e.g. $token/MAP, which contains a mapping from
original artifact name to path on the storage, e.g.:
```
{
"foo.jar": "hdfs://path/to/staging/foo.jar",
"some/data.pb": "hdfs://path/to/staging/some/data.pb",
...
}
```
Then when retrieving the artifact by name, you look it up in the map and
download the file from where it points to.
This flexible mapping will also allow to later store files in something like
path/to/$md5, allowing for deduplication of artifacts (avoiding uploading
artifacts we already have).
----------------------------------------------------------------
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]
Issue Time Tracking
-------------------
Worklog Id: (was: 109922)
Time Spent: 1h 10m (was: 1h)
> 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: 1h 10m
> Remaining Estimate: 0h
>
> In agreement with how they are staged in BEAM-4290.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)