You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2019/07/17 19:57:46 UTC

[GitHub] [incubator-iceberg] danielcweeks commented on a change in pull request #277: Moving/Renaming hadoop module to filesystem

danielcweeks commented on a change in pull request #277: Moving/Renaming hadoop module to filesystem
URL: https://github.com/apache/incubator-iceberg/pull/277#discussion_r304525724
 
 

 ##########
 File path: python/iceberg/core/filesystem/s3_filesystem_wrapper.py
 ##########
 @@ -0,0 +1,283 @@
+# 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.
+
+import datetime
+import io
+import logging
+import time
+from urllib.parse import urlparse
+
+import boto3
+import requests
+from retrying import retry
+from s3fs import S3FileSystem
+
+from .file_status import FileStatus
+from .file_system import FileSystem
+
+_logger = logging.getLogger(__name__)
+
+
+S3_CLIENT = dict()
+BOTO_STS_CLIENT = boto3.client('sts')
+
+STS_CACHED_RESPONSE = {}
+REFRESH_THRESHOLD = 300
+
+
+@retry(wait_incrementing_start=100, wait_exponential_multiplier=4,
+       wait_exponential_max=5000, stop_max_delay=600000, stop_max_attempt_number=3)
+def get_s3(role_arn="default", obj="resource"):
+    if role_arn not in S3_CLIENT:
+        S3_CLIENT[role_arn] = dict()
+
+    if role_arn == "default":
+        S3_CLIENT["default"]["resource"] = boto3.resource('s3')
+        S3_CLIENT["default"]["client"] = boto3.client('s3')
+    else:
+        key, secret, token = get_sts_session_keys(role_arn)
+        sess = boto3.Session(aws_access_key_id=key,
+                             aws_secret_access_key=secret,
+                             aws_session_token=token)
+        S3_CLIENT[role_arn]["resource"] = sess.resource("s3")
+        S3_CLIENT[role_arn]["client"] = sess.client("s3")
+
+    return S3_CLIENT.get(role_arn, dict()).get(obj)
+
+
+def get_sts_session_keys(role_arn):
+    if role_arn is None:
+        return None, None, None
+
+    if STS_CACHED_RESPONSE.get(role_arn) is None \
+            or should_refresh_creds(role_arn):
+        STS_CACHED_RESPONSE[role_arn] = BOTO_STS_CLIENT.assume_role(
+            RoleArn=role_arn,
+            RoleSessionName='iceberg_python_client_{}'.format(int(time.time())))
+
+    return (STS_CACHED_RESPONSE[role_arn]["Credentials"]["AccessKeyId"],
+            STS_CACHED_RESPONSE[role_arn]["Credentials"]["SecretAccessKey"],
+            STS_CACHED_RESPONSE[role_arn]["Credentials"]["SessionToken"])
+
+
+def should_refresh_creds(role_arn):
+    if role_arn is None:
+        return False
+
+    if STS_CACHED_RESPONSE.get(role_arn) is None:
+        return True
+
+    expiration = STS_CACHED_RESPONSE[role_arn].get("Credentials", dict()).get("Expiration")
+    curr_dt = datetime.datetime.now(tz=datetime.timezone.utc)
+
+    return (expiration - curr_dt).total_seconds() < REFRESH_THRESHOLD
+
+
+class S3FileSystemWrapper(FileSystem):
+    _FS = dict()
+
+    def __init__(self, conf=None, anon=False):
+        self.anon = anon
+        self.conf = conf
+        self.iam_role_arn = "default"
+
+        if conf is not None:
+            self.iam_role_arn = conf.get("hive.aws_iam_role", "default")
+
+    def open(self, path, mode='rb'):
+        if mode == "rb":
+            return S3File(path, self.iam_role_arn)
+
+        elif mode == "w":
+            s3fs_obj = S3FileSystem()
+            return s3fs_obj.open(S3FileSystemWrapper.fix_s3_path(path), mode=mode)
+
+    def delete(self, path):
+        raise NotImplementedError()
+
+    def stat(self, path):
+        is_dir = False
+        st = dict()
+        try:
+            st = S3FileSystemWrapper.info(S3FileSystemWrapper.fix_s3_path(path), refresh=True)
+        except RuntimeError:
+            # must be a directory or subsequent du will fail
+            du = S3FileSystemWrapper.du(S3FileSystemWrapper.fix_s3_path(path), refresh=True)
+            if len(du) > 0:
+                is_dir = True
+            length = 0
+            for key, val in du.items():
+                length += val
+
+        return FileStatus(path=path, length=st.get("Size", length), is_dir=is_dir,
+                          blocksize=None, modification_time=st.get("LastModified"), access_time=None,
+                          permission=None, owner=None, group=None)
+
+    @staticmethod
+    def full_file(url, role_arn="default", use_requests=True):
+        if use_requests:
+            url = S3FileSystemWrapper.get_presigned_urls([url])[0]
+            stream = requests.get(url, stream=True).content
+        else:
+            stream = S3File(url, role_arn).read()
+
+        return io.BytesIO(stream)
+
+    @staticmethod
+    def get_presigned_urls(urls, role_arn="default"):
+        signed_urls = []
+        for url in urls:
+            parsed_url = urlparse(S3FileSystemWrapper.fix_s3_path(url))
+            signed_urls.append(get_s3(role_arn, "client").generate_presigned_url(ClientMethod='get_object',
+                                                                                 Params={'Bucket': parsed_url.netloc,
+                                                                                         'Key': parsed_url.path[1:]}))
+        return signed_urls
+
+    @staticmethod
+    def du(url):
+        raise NotImplementedError()
+
+    @staticmethod
+    def info(url):
+        raise NotImplementedError()
+
+    @staticmethod
+    def fix_s3_path(path):
+        if path.startswith("s3n"):
+            path = path.replace("s3n://", "s3://")
+        return path
+
+
+class S3File(object):
+    MAX_CHUNK_SIZE = 4 * 1048576
+    MIN_CHUNK_SIZE = 2 * 65536
 
 Review comment:
   `MIN_CHUNK_SIZE` doesn't appear to be used anywhere.  I'm also unclear on the reasoning behind the chunk sizes.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org