[
https://issues.apache.org/jira/browse/BEAM-2500?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16166678#comment-16166678
]
Chamikara Jayalath commented on BEAM-2500:
------------------------------------------
Thanks for looking into this Jacob. I'll try to answer some of your questions.
I agree that GCS is a good template. It's pretty well battle tested and used by
all Dataflow pipelines.
5GB limit for FileSystem.copy(), though might be a good start, might not be
enough for production use. Beam can parallelize reading of a large files. So
there might be many users who are used to reading a small number of large files
from their pipelines. Note that we use copy() operation to finalize files
written using FileBasedSink implementations. So we might need to copy large
files there as well. It'll be good if copy() can be implemented using multipart
upload as you mentioned.
FileSystem.create() is to create an empty WrittableByteChannel that will be
written to later. So we'll have to have a way to stream bytes into S3 (some
implementation of WrittableByteChannel). I'm not sure if S3 client library
already supports this.
's3' for schema sounds good to me.
For efficient parallelized reading FileSystem.open() should return an
efficiently seekable SeekableByteChannel.
Using copy() + delete() combination for rename() is fine.
We might have to address issues related to providing credentials for accessing
S3. See following JIRA where some details related to this were discussed when
[~demeshchuk] looked into adding a S3 file system for Python SDK.
https://issues.apache.org/jira/browse/BEAM-2572
> Add support for S3 as a Apache Beam FileSystem
> ----------------------------------------------
>
> Key: BEAM-2500
> URL: https://issues.apache.org/jira/browse/BEAM-2500
> Project: Beam
> Issue Type: Improvement
> Components: sdk-java-extensions
> Reporter: Luke Cwik
> Priority: Minor
> Attachments: hadoop_fs_patch.patch
>
>
> Note that this is for providing direct integration with S3 as an Apache Beam
> FileSystem.
> There is already support for using the Hadoop S3 connector by depending on
> the Hadoop File System module[1], configuring HadoopFileSystemOptions[2] with
> a S3 configuration[3].
> 1: https://github.com/apache/beam/tree/master/sdks/java/io/hadoop-file-system
> 2:
> https://github.com/apache/beam/blob/master/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptions.java#L53
> 3: https://wiki.apache.org/hadoop/AmazonS3
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)