[
https://issues.apache.org/jira/browse/BEAM-2572?focusedWorklogId=342817&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-342817
]
ASF GitHub Bot logged work on BEAM-2572:
----------------------------------------
Author: ASF GitHub Bot
Created on: 13/Nov/19 19:30
Start Date: 13/Nov/19 19:30
Worklog Time Spent: 10m
Work Description: pabloem commented on pull request #9955: [BEAM-2572]
Python SDK S3 Filesystem
URL: https://github.com/apache/beam/pull/9955#discussion_r345936305
##########
File path: sdks/python/apache_beam/io/aws/s3filesystem.py
##########
@@ -0,0 +1,275 @@
+#
+# 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.
+#
+"""S3 file system implementation for accessing files on AWS S3."""
+
+from __future__ import absolute_import
+
+from future.utils import iteritems
+
+from apache_beam.io.aws import s3io
+from apache_beam.io.filesystem import BeamIOError
+from apache_beam.io.filesystem import CompressedFile
+from apache_beam.io.filesystem import CompressionTypes
+from apache_beam.io.filesystem import FileMetadata
+from apache_beam.io.filesystem import FileSystem
+
+__all__ = ['S3FileSystem']
+
+
+class S3FileSystem(FileSystem):
+ """An S3 `FileSystem` implementation for accessing files on AWS S3
+ """
+
+ CHUNK_SIZE = s3io.MAX_BATCH_OPERATION_SIZE
+ S3_PREFIX = 's3://'
+
+ @classmethod
+ def scheme(cls):
+ """URI scheme for the FileSystem
+ """
+ return 's3'
+
+ def join(self, basepath, *paths):
+ """Join two or more pathname components for the filesystem
+
+ Args:
+ basepath: string path of the first component of the path
+ paths: path components to be added
+
+ Returns: full path after combining all of the return nulled components
+ """
+ if not basepath.startswith(S3FileSystem.S3_PREFIX):
+ raise ValueError('Basepath %r must be S3 path.' % basepath)
+
+ path = basepath
+ for p in paths:
+ path = path.rstrip('/') + '/' + p.lstrip('/')
+ return path
+
+ def split(self, path):
+ """Splits the given path into two parts.
+
+ Splits the path into a pair (head, tail) such that tail contains the last
+ component of the path and head contains everything up to that.
+
+ Head will include the S3 prefix ('s3://').
+
+ Args:
+ path: path as a string
+ Returns:
+ a pair of path components as strings.
+ """
+ path = path.strip()
+ if not path.startswith(S3FileSystem.S3_PREFIX):
+ raise ValueError('Path %r must be S3 path.' % path)
+
+ prefix_len = len(S3FileSystem.S3_PREFIX)
+ last_sep = path[prefix_len:].rfind('/')
Review comment:
a lot of this code is duplicated, so it would be nice to deduplicate between
the filesystems.... but you don't need to worry about it for now : P
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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: 342817)
Time Spent: 1.5h (was: 1h 20m)
> Implement an S3 filesystem for Python SDK
> -----------------------------------------
>
> Key: BEAM-2572
> URL: https://issues.apache.org/jira/browse/BEAM-2572
> Project: Beam
> Issue Type: Task
> Components: sdk-py-core
> Reporter: Dmitry Demeshchuk
> Priority: Minor
> Labels: GSoC2019, gsoc, gsoc2019, mentor, outreachy19dec
> Time Spent: 1.5h
> Remaining Estimate: 0h
>
> There are two paths worth exploring, to my understanding:
> 1. Sticking to the HDFS-based approach (like it's done in Java).
> 2. Using boto/boto3 for accessing S3 through its common API endpoints.
> I personally prefer the second approach, for a few reasons:
> 1. In real life, HDFS and S3 have different consistency guarantees, therefore
> their behaviors may contradict each other in some edge cases (say, we write
> something to S3, but it's not immediately accessible for reading from another
> end).
> 2. There are other AWS-based sources and sinks we may want to create in the
> future: DynamoDB, Kinesis, SQS, etc.
> 3. boto3 already provides somewhat good logic for basic things like
> reattempting.
> Whatever path we choose, there's another problem related to this: we
> currently cannot pass any global settings (say, pipeline options, or just an
> arbitrary kwarg) to a filesystem. Because of that, we'd have to setup the
> runner nodes to have AWS keys set up in the environment, which is not trivial
> to achieve and doesn't look too clean either (I'd rather see one single place
> for configuring the runner options).
> Also, it's worth mentioning that I already have a janky S3 filesystem
> implementation that only supports DirectRunner at the moment (because of the
> previous paragraph). I'm perfectly fine finishing it myself, with some
> guidance from the maintainers.
> Where should I move on from here, and whose input should I be looking for?
> Thanks!
--
This message was sent by Atlassian Jira
(v8.3.4#803005)