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

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

                Author: ASF GitHub Bot
            Created on: 22/Nov/19 20:01
            Start Date: 22/Nov/19 20:01
    Worklog Time Spent: 10m 
      Work Description: tamera-lanham commented on pull request #9955: 
[BEAM-2572] Python SDK S3 Filesystem
URL: https://github.com/apache/beam/pull/9955#discussion_r349771600
 
 

 ##########
 File path: sdks/python/apache_beam/io/aws/s3io.py
 ##########
 @@ -0,0 +1,604 @@
+#
+# 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.
+#
+"""AWS S3 client
+"""
+
+from __future__ import absolute_import
+
+import errno
+import io
+import logging
+import re
+import time
+import traceback
+from builtins import object
+
+from apache_beam.io.aws.clients.s3 import messages
+from apache_beam.io.filesystemio import Downloader
+from apache_beam.io.filesystemio import DownloaderStream
+from apache_beam.io.filesystemio import Uploader
+from apache_beam.io.filesystemio import UploaderStream
+from apache_beam.utils import retry
+
+try:
+  # pylint: disable=wrong-import-order, wrong-import-position
+  # pylint: disable=ungrouped-imports
+  from apache_beam.io.aws.clients.s3 import boto3_client
+  BOTO3_INSTALLED = True
+except ImportError:
+  BOTO3_INSTALLED = False
+
+MAX_BATCH_OPERATION_SIZE = 100
+
+
+def parse_s3_path(s3_path, object_optional=False):
+  """Return the bucket and object names of the given s3:// path."""
+  match = re.match('^s3://([^/]+)/(.*)$', s3_path)
+  if match is None or (match.group(2) == '' and not object_optional):
+    raise ValueError('S3 path must be in the form s3://<bucket>/<object>.')
+  return match.group(1), match.group(2)
+
+
+class S3IO(object):
+  """S3 I/O client."""
+
+  def __init__(self, client=None):
+    if client is not None:
+      self.client = client
+    elif BOTO3_INSTALLED:
+      self.client = boto3_client.Client()
+    else:
+      message = 'AWS dependencies are not installed, and no alternative ' \
+      'client was provided to S3IO.'
+      raise RuntimeError(message)
+
+  def open(self,
+           filename,
+           mode='r',
+           read_buffer_size=16*1024*1024,
+           mime_type='application/octet-stream'):
+    """Open an S3 file path for reading or writing.
+
+    Args:
+      filename (str): S3 file path in the form ``s3://<bucket>/<object>``.
+      mode (str): ``'r'`` for reading or ``'w'`` for writing.
+      read_buffer_size (int): Buffer size to use during read operations.
+      mime_type (str): Mime type to set for write operations.
 
 Review comment:
   This is addressed with the most recent changes, so the `mime_type` that the 
user sets will be reflected in the `ContentType` of the object in S3
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 348336)
    Time Spent: 2h 10m  (was: 2h)

> 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: 2h 10m
>  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)

Reply via email to