You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/08/02 16:32:28 UTC

[GitHub] [airflow] denimalpaca opened a new pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

denimalpaca opened a new pull request #17382:
URL: https://github.com/apache/airflow/pull/17382


   Add new LocalFilesystemToS3Operator under Amazon provider (#17168)
   
   Currently, an S3Hook exists that allows transfer of files to S3 via `load_file()`, however there is no operator associated with it. The LocalFilesystemToS3Operator would wrap the S3 Hook, so it is not used directly.
   
   Seeing as to upload a local file to S3 using the S3 Hook, a Python task with the same functionality has to be written anyway, this could reduce a lot of redundant boiler-plate code and standardize the local file to S3 load process.
   
   closes: #17168 


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#discussion_r688843188



##########
File path: airflow/providers/amazon/aws/example_dags/example_local_to_s3.py
##########
@@ -0,0 +1,42 @@
+# 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 os
+
+from airflow import models
+from airflow.providers.amazon.aws.transfers.local_to_s3 import LocalFilesystemToS3Operator
+from airflow.utils.dates import datetime
+
+S3_BUCKET = os.environ.get("S3_BUCKET", "test-bucket")

Review comment:
       Let's do this as a separate contribution because it makes sense to standardize all the variables and write it down as a generally accepted rule to make it more permanent. Now, if we change the variable name in one place, we will fix the problem only here, but in a moment subsequent contributions may not see this convention and introduce a different naming convention.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ferruzzi commented on a change in pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#discussion_r686355631



##########
File path: airflow/providers/amazon/aws/transfers/local_to_s3.py
##########
@@ -0,0 +1,107 @@
+#
+# 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.
+from typing import Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+
+
+class LocalFilesystemToS3Operator(BaseOperator):
+    """
+    Uploads a file from a local filesystem to Amazon S3.
+
+    :param filename: Path to the local file. Path can be either absolute
+            (e.g. /path/to/file.ext) or relative (e.g. ../../foo/*/*.csv). (templated)
+    :type filename: str
+    :param dest_key: The key of the object to copy to. (templated)
+
+        It can be either full s3:// style url or relative path from root level.
+
+        When it's specified as a full s3:// url, please omit dest_bucket.
+    :type dest_key: str
+    :param dest_bucket: Name of the S3 bucket to where the object is copied. (templated)
+
+        It should be omitted when `dest_key` is provided as a full s3:// url.

Review comment:
       Nit:  I'd really prefer to see some logic to enforce this rather than a comment.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#discussion_r686491746



##########
File path: airflow/providers/amazon/aws/transfers/local_to_s3.py
##########
@@ -0,0 +1,107 @@
+#
+# 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.
+from typing import Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+
+
+class LocalFilesystemToS3Operator(BaseOperator):
+    """
+    Uploads a file from a local filesystem to Amazon S3.
+
+    :param filename: Path to the local file. Path can be either absolute
+            (e.g. /path/to/file.ext) or relative (e.g. ../../foo/*/*.csv). (templated)
+    :type filename: str
+    :param dest_key: The key of the object to copy to. (templated)
+
+        It can be either full s3:// style url or relative path from root level.
+
+        When it's specified as a full s3:// url, please omit dest_bucket.
+    :type dest_key: str
+    :param dest_bucket: Name of the S3 bucket to where the object is copied. (templated)
+
+        It should be omitted when `dest_key` is provided as a full s3:// url.

Review comment:
       +1 but I don’t think this is a nit, we should enforce this in code (e.g. raise TypeError if both are passed) and describe it here.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk merged pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #17382:
URL: https://github.com/apache/airflow/pull/17382


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#discussion_r686491746



##########
File path: airflow/providers/amazon/aws/transfers/local_to_s3.py
##########
@@ -0,0 +1,107 @@
+#
+# 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.
+from typing import Optional, Union
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+
+
+class LocalFilesystemToS3Operator(BaseOperator):
+    """
+    Uploads a file from a local filesystem to Amazon S3.
+
+    :param filename: Path to the local file. Path can be either absolute
+            (e.g. /path/to/file.ext) or relative (e.g. ../../foo/*/*.csv). (templated)
+    :type filename: str
+    :param dest_key: The key of the object to copy to. (templated)
+
+        It can be either full s3:// style url or relative path from root level.
+
+        When it's specified as a full s3:// url, please omit dest_bucket.
+    :type dest_key: str
+    :param dest_bucket: Name of the S3 bucket to where the object is copied. (templated)
+
+        It should be omitted when `dest_key` is provided as a full s3:// url.

Review comment:
       +1




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#issuecomment-898907877


   Awesome work, congrats on your first merged pull request!
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] subkanthi commented on a change in pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
subkanthi commented on a change in pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#discussion_r681881085



##########
File path: airflow/providers/amazon/aws/example_dags/example_local_to_s3.py
##########
@@ -0,0 +1,40 @@
+# 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 os
+
+from airflow import models
+from airflow.providers.amazon.aws.transfers.local_to_s3 import LocalFilesystemToS3Operator
+from airflow.utils.dates import datetime
+
+S3_BUCKET = os.environ.get("S3_BUCKET", "test-bucket")
+S3_KEY = os.environ.get("S3_KEY", "key")
+
+with models.DAG(
+    "example_local_to_s3",
+    schedule_interval=None,
+    start_date=datetime(2021, 1, 1),  # Override to match your needs
+) as dag:
+    # [START howto_local_transfer_data_to_s3]
+    create_sftp_to_s3_job = LocalFilesystemToS3Operator(

Review comment:
       maybe a copy/paste, create_sftp_to_s3_job should be I guess local_to_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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#issuecomment-898907876


   The PR is likely OK to be merged with just subset of tests for default Python and Database versions without running the full matrix of tests, because it does not modify the core of Airflow. If the committers decide that the full tests matrix is needed, they will add the label 'full tests needed'. Then you should rebase to the latest main or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ferruzzi commented on a change in pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on a change in pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#discussion_r686351282



##########
File path: airflow/providers/amazon/aws/example_dags/example_local_to_s3.py
##########
@@ -0,0 +1,42 @@
+# 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 os
+
+from airflow import models
+from airflow.providers.amazon.aws.transfers.local_to_s3 import LocalFilesystemToS3Operator
+from airflow.utils.dates import datetime
+
+S3_BUCKET = os.environ.get("S3_BUCKET", "test-bucket")

Review comment:
       Nitpick for discussion, non-blocking:  There are currently four example DAGS (not counting this one) related to S3.  Two of them use the envvar "S3_BUCKET" and the other two use "BUCKET_NAME".  Should we standardize those and, if so, which way?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] boring-cyborg[bot] commented on pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#issuecomment-891165202


   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, mypy and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/main/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/main/docs/apache-airflow/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/main/BREEZE.rst) for testing locally, it’s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better 🚀.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] denimalpaca commented on a change in pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
denimalpaca commented on a change in pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#discussion_r687892436



##########
File path: airflow/providers/amazon/aws/example_dags/example_local_to_s3.py
##########
@@ -0,0 +1,42 @@
+# 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 os
+
+from airflow import models
+from airflow.providers.amazon.aws.transfers.local_to_s3 import LocalFilesystemToS3Operator
+from airflow.utils.dates import datetime
+
+S3_BUCKET = os.environ.get("S3_BUCKET", "test-bucket")

Review comment:
       I would opt for "S3_BUCKET" as the hook uses `dest_bucket`, and it seems that "S3_BUCKET" over "BUCKET_NAME" more closely follows that precedent.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] denimalpaca commented on a change in pull request #17382: Add new LocalFilesystemToS3Operator under Amazon provider (#17168)

Posted by GitBox <gi...@apache.org>.
denimalpaca commented on a change in pull request #17382:
URL: https://github.com/apache/airflow/pull/17382#discussion_r681970662



##########
File path: airflow/providers/amazon/aws/example_dags/example_local_to_s3.py
##########
@@ -0,0 +1,40 @@
+# 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 os
+
+from airflow import models
+from airflow.providers.amazon.aws.transfers.local_to_s3 import LocalFilesystemToS3Operator
+from airflow.utils.dates import datetime
+
+S3_BUCKET = os.environ.get("S3_BUCKET", "test-bucket")
+S3_KEY = os.environ.get("S3_KEY", "key")
+
+with models.DAG(
+    "example_local_to_s3",
+    schedule_interval=None,
+    start_date=datetime(2021, 1, 1),  # Override to match your needs
+) as dag:
+    # [START howto_local_transfer_data_to_s3]
+    create_sftp_to_s3_job = LocalFilesystemToS3Operator(

Review comment:
       d'oh. Yes. Thank you.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org