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/01/22 17:24:34 UTC

[GitHub] [airflow] turbaszek opened a new pull request #13847: Add ExasolToS3Operator

turbaszek opened a new pull request #13847:
URL: https://github.com/apache/airflow/pull/13847


   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   


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



[GitHub] [airflow] github-actions[bot] commented on pull request #13847: Add ExasolToS3Operator

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


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

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



[GitHub] [airflow] XD-DENG commented on a change in pull request #13847: Add ExasolToS3Operator

Posted by GitBox <gi...@apache.org>.
XD-DENG commented on a change in pull request #13847:
URL: https://github.com/apache/airflow/pull/13847#discussion_r563157195



##########
File path: airflow/providers/exasol/hooks/exasol.py
##########
@@ -108,6 +108,37 @@ def get_first(self, sql: Union[str, list], parameters: Optional[dict] = None) ->
             with closing(conn.execute(sql, parameters)) as cur:
                 return cur.fetchone()
 
+    def export_to_file(

Review comment:
       I think what @turbaszek has nicely done here is to allow Exasol users to seamlessly call what `pyexasol` supports. Please refer to https://github.com/badoo/pyexasol/blob/master/pyexasol/connection.py#L259 
   
   I don't know Exasol well, but based on the implementation here, this new method supports whatever the original method in `Pyexasol` supports. So to my understanding, the question you raised should not be a concern.
   
   Let me know if this makes sense to 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.

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



[GitHub] [airflow] JeffryMAC commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/exasol/hooks/exasol.py
##########
@@ -108,6 +108,37 @@ def get_first(self, sql: Union[str, list], parameters: Optional[dict] = None) ->
             with closing(conn.execute(sql, parameters)) as cur:
                 return cur.fetchone()
 
+    def export_to_file(

Review comment:
       does it offer the choice of file format? (json, csv, other?)
   The method name doesn't suggest it's generic so suitable for more than one file extension but there is no file extension parameter to config




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



[GitHub] [airflow] turbaszek commented on pull request #13847: Add ExasolToS3Operator

Posted by GitBox <gi...@apache.org>.
turbaszek commented on pull request #13847:
URL: https://github.com/apache/airflow/pull/13847#issuecomment-765568392


   I still need to add tests ✅ 


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



[GitHub] [airflow] github-actions[bot] commented on pull request #13847: Add ExasolToS3Operator

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/504238586) is cancelling this PR. Building image for the PR has been cancelled


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



[GitHub] [airflow] turbaszek commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/exasol/hooks/exasol.py
##########
@@ -108,6 +108,37 @@ def get_first(self, sql: Union[str, list], parameters: Optional[dict] = None) ->
             with closing(conn.execute(sql, parameters)) as cur:
                 return cur.fetchone()
 
+    def export_to_file(

Review comment:
       @XD-DENG is right - I hoped that using the `Pyexasol` method will give us most flexibility and will be easiest to maintain. Here is the exasol info:
   
   
   >The target file can either be CSV or FBV files and should comply to the format specifications in the CSV Data Format and the Fixblock Data format (FBV). File names may only consist of ASCII characters. A BOMClosed is not supported.
   
   https://docs.exasol.com/sql/export.htm#UsageNotes




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



[GitHub] [airflow] JeffryMAC commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/exasol/hooks/exasol.py
##########
@@ -108,6 +108,37 @@ def get_first(self, sql: Union[str, list], parameters: Optional[dict] = None) ->
             with closing(conn.execute(sql, parameters)) as cur:
                 return cur.fetchone()
 
+    def export_to_file(

Review comment:
       does it offer the choice of file format? (json, csv, other?)
   The method name suggest it's generic so suitable for more than one file extension but there is no file extension parameter to config




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



[GitHub] [airflow] turbaszek merged pull request #13847: Add ExasolToS3Operator

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


   


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



[GitHub] [airflow] turbaszek commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.

Review comment:
       Let me know if d76c99d look resonbale

##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.

Review comment:
       Let me know if d76c99d look resonable

##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.

Review comment:
       Let me know if d76c99d look reasonable




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



[GitHub] [airflow] turbaszek commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.

Review comment:
       Good point, I will add full path for the class




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



[GitHub] [airflow] eladkal commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of :class:`~pyexasol.connection.ExaConnection`.
+    :type query_params: dict
+    :param export_params: Extra parameters passed to underlying ``export_to_file``
+        method of :class:`~pyexasol.connection.ExaConnection`.
+    :type export_params: dict
+    """
+
+    template_fields = ('query_or_table', 'key', 'bucket_name', 'query_params', 'export_params')
+    template_fields_renderers = {"query_or_table": "sql", "query_params": "json", "export_params": "json"}
+    template_ext = ()

Review comment:
       I don't know exasol that well but shouldn't the operator allow reading query from `.sql` file?
   ```suggestion
       template_ext = ('.sql',)
   ```




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



[GitHub] [airflow] XD-DENG commented on a change in pull request #13847: Add ExasolToS3Operator

Posted by GitBox <gi...@apache.org>.
XD-DENG commented on a change in pull request #13847:
URL: https://github.com/apache/airflow/pull/13847#discussion_r563191528



##########
File path: airflow/providers/exasol/hooks/exasol.py
##########
@@ -108,6 +108,37 @@ def get_first(self, sql: Union[str, list], parameters: Optional[dict] = None) ->
             with closing(conn.execute(sql, parameters)) as cur:
                 return cur.fetchone()
 
+    def export_to_file(
+        self,
+        filename: str,
+        query_or_table: str,
+        query_params: Optional[Dict] = None,
+        export_params: Optional[Dict] = None,
+    ) -> None:
+        """
+        Exports data to a file.
+
+        :param filename: Path to the file to which the data has to be exported
+        :type filename: str
+        :param query_or_table: the sql statement to be executed or table name to export
+        :type query_or_table: str
+        :param query_params: Query parameters passed to underlying ``export_to_file``
+            method of ``ExaConnection``.
+        :type query_params: dict
+        :param export_params: Extra parameters passed to underlying ``export_to_file``
+            method of ``ExaConnection``.
+        :type export_params: dict
+        """
+        self.log.info("Getting data from exasol")

Review comment:
       Not sure if it's a good idea in this context: maybe it can make the log more meaningful by having `query_or_table` in the log.

##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.

Review comment:
       Given `ExaConnection` is a concept only existing in the underlying dependency and not in Airflow,  it took me a few moment to figure out what's this "ExaConnection".
   
   Maybe good to provide a little bit more information about `ExaConnection`, like changing this to "**Query parameters passed to underlying ``export_to_file`` method of ``pyexasol``'s ``ExaConnection``.**"




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



[GitHub] [airflow] github-actions[bot] commented on pull request #13847: Add ExasolToS3Operator

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/504308388) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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



[GitHub] [airflow] turbaszek commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/exasol/hooks/exasol.py
##########
@@ -108,6 +108,37 @@ def get_first(self, sql: Union[str, list], parameters: Optional[dict] = None) ->
             with closing(conn.execute(sql, parameters)) as cur:
                 return cur.fetchone()
 
+    def export_to_file(
+        self,
+        filename: str,
+        query_or_table: str,
+        query_params: Optional[Dict] = None,
+        export_params: Optional[Dict] = None,
+    ) -> None:
+        """
+        Exports data to a file.
+
+        :param filename: Path to the file to which the data has to be exported
+        :type filename: str
+        :param query_or_table: the sql statement to be executed or table name to export
+        :type query_or_table: str
+        :param query_params: Query parameters passed to underlying ``export_to_file``
+            method of ``ExaConnection``.
+        :type query_params: dict
+        :param export_params: Extra parameters passed to underlying ``export_to_file``
+            method of ``ExaConnection``.
+        :type export_params: dict
+        """
+        self.log.info("Getting data from exasol")

Review comment:
       @XD-DENG I thought about it - however I'm not sure if having an SQL in logs will be wise




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



[GitHub] [airflow] turbaszek commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of :class:`~pyexasol.connection.ExaConnection`.
+    :type query_params: dict
+    :param export_params: Extra parameters passed to underlying ``export_to_file``
+        method of :class:`~pyexasol.connection.ExaConnection`.
+    :type export_params: dict
+    """
+
+    template_fields = ('query_or_table', 'key', 'bucket_name', 'query_params', 'export_params')
+    template_fields_renderers = {"query_or_table": "sql", "query_params": "json", "export_params": "json"}
+    template_ext = ()

Review comment:
       Sounds good!




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



[GitHub] [airflow] turbaszek commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of :class:`~pyexasol.connection.ExaConnection`.
+    :type query_params: dict
+    :param export_params: Extra parameters passed to underlying ``export_to_file``
+        method of :class:`~pyexasol.connection.ExaConnection`.
+    :type export_params: dict
+    """
+
+    template_fields = ('query_or_table', 'key', 'bucket_name', 'query_params', 'export_params')
+    template_fields_renderers = {"query_or_table": "sql", "query_params": "json", "export_params": "json"}
+    template_ext = ()

Review comment:
       Sounds good!




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



[GitHub] [airflow] turbaszek commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.
+    :type query_params: dict
+    :param export_params: Extra parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.
+    :type export_params: dict
+    """
+
+    template_fields = ('query_or_table', 'key', 'bucket_name', 'query_params', 'export_params')
+    template_fields_renderers = {"query_or_table": "sql", "query_params": "json", "export_params": "json"}
+    template_ext = ()
+    ui_color = '#ededed'
+
+    @apply_defaults
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        *,
+        query_or_table: str,
+        key: str,
+        bucket_name: Optional[str] = None,
+        replace: bool = False,
+        encrypt: bool = False,
+        gzip: bool = False,
+        acl_policy: Optional[str] = None,
+        query_params: Optional[Dict] = None,
+        export_params: Optional[Dict] = None,
+        exasol_conn_id: str = 'exasol_default',
+        aws_conn_id: str = 'aws_default',
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.query_or_table = query_or_table
+        self.key = key
+        self.bucket_name = bucket_name
+        self.replace = replace
+        self.encrypt = encrypt
+        self.gzip = gzip
+        self.acl_policy = acl_policy
+        self.query_params = query_params
+        self.export_params = export_params
+        self.exasol_conn_id = exasol_conn_id
+        self.aws_conn_id = aws_conn_id
+
+    def execute(self, context):
+        exasol_hook = ExasolHook(exasol_conn_id=self.exasol_conn_id)
+        s3_hook = S3Hook(aws_conn_id=self.aws_conn_id)
+
+        with NamedTemporaryFile("w+") as file:
+            self.log.info("Downloading data from exasol")

Review comment:
       Thanks! Done ✅ 

##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being

Review comment:
       Thanks! Done ✅ 




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



[GitHub] [airflow] XD-DENG commented on a change in pull request #13847: Add ExasolToS3Operator

Posted by GitBox <gi...@apache.org>.
XD-DENG commented on a change in pull request #13847:
URL: https://github.com/apache/airflow/pull/13847#discussion_r562817436



##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.
+    :type query_params: dict
+    :param export_params: Extra parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.
+    :type export_params: dict
+    """
+
+    template_fields = ('query_or_table', 'key', 'bucket_name', 'query_params', 'export_params')
+    template_fields_renderers = {"query_or_table": "sql", "query_params": "json", "export_params": "json"}
+    template_ext = ()
+    ui_color = '#ededed'
+
+    @apply_defaults
+    def __init__(  # pylint: disable=too-many-arguments
+        self,
+        *,
+        query_or_table: str,
+        key: str,
+        bucket_name: Optional[str] = None,
+        replace: bool = False,
+        encrypt: bool = False,
+        gzip: bool = False,
+        acl_policy: Optional[str] = None,
+        query_params: Optional[Dict] = None,
+        export_params: Optional[Dict] = None,
+        exasol_conn_id: str = 'exasol_default',
+        aws_conn_id: str = 'aws_default',
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.query_or_table = query_or_table
+        self.key = key
+        self.bucket_name = bucket_name
+        self.replace = replace
+        self.encrypt = encrypt
+        self.gzip = gzip
+        self.acl_policy = acl_policy
+        self.query_params = query_params
+        self.export_params = export_params
+        self.exasol_conn_id = exasol_conn_id
+        self.aws_conn_id = aws_conn_id
+
+    def execute(self, context):
+        exasol_hook = ExasolHook(exasol_conn_id=self.exasol_conn_id)
+        s3_hook = S3Hook(aws_conn_id=self.aws_conn_id)
+
+        with NamedTemporaryFile("w+") as file:
+            self.log.info("Downloading data from exasol")

Review comment:
       This line may be unnecessary given you have `self.log.info("Getting data from exasol")` inside `export_to_file `

##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being

Review comment:
       Seems `acl_policy` is not passed to your S3 hook?




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



[GitHub] [airflow] turbaszek merged pull request #13847: Add ExasolToS3Operator

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


   


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



[GitHub] [airflow] turbaszek commented on a change in pull request #13847: Add ExasolToS3Operator

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



##########
File path: airflow/providers/exasol/hooks/exasol.py
##########
@@ -108,6 +108,37 @@ def get_first(self, sql: Union[str, list], parameters: Optional[dict] = None) ->
             with closing(conn.execute(sql, parameters)) as cur:
                 return cur.fetchone()
 
+    def export_to_file(

Review comment:
       @XD-DENG is right - I hoped that using the `Pyexasol` method will give us most flexibility and will be easiest to maintain. Here is the exasol info:
   
   
   >The target file can either be CSV or FBV files and should comply to the format specifications in the CSV Data Format and the Fixblock Data format (FBV). File names may only consist of ASCII characters. A BOMClosed is not supported.
   
   https://docs.exasol.com/sql/export.htm#UsageNotes
   
   Supporting formats not supported by exasol is something that we may consider adding using pandas (read_csv -> df -> save_as_format). However, I'm not sure if this is something that should happen in Airflow task. The data can be arbitrary big so keeping it in memory may be problematic.




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



[GitHub] [airflow] XD-DENG commented on a change in pull request #13847: Add ExasolToS3Operator

Posted by GitBox <gi...@apache.org>.
XD-DENG commented on a change in pull request #13847:
URL: https://github.com/apache/airflow/pull/13847#discussion_r563264489



##########
File path: airflow/providers/exasol/hooks/exasol.py
##########
@@ -108,6 +108,37 @@ def get_first(self, sql: Union[str, list], parameters: Optional[dict] = None) ->
             with closing(conn.execute(sql, parameters)) as cur:
                 return cur.fetchone()
 
+    def export_to_file(
+        self,
+        filename: str,
+        query_or_table: str,
+        query_params: Optional[Dict] = None,
+        export_params: Optional[Dict] = None,
+    ) -> None:
+        """
+        Exports data to a file.
+
+        :param filename: Path to the file to which the data has to be exported
+        :type filename: str
+        :param query_or_table: the sql statement to be executed or table name to export
+        :type query_or_table: str
+        :param query_params: Query parameters passed to underlying ``export_to_file``
+            method of ``ExaConnection``.
+        :type query_params: dict
+        :param export_params: Extra parameters passed to underlying ``export_to_file``
+            method of ``ExaConnection``.
+        :type export_params: dict
+        """
+        self.log.info("Getting data from exasol")

Review comment:
       Yep, that's why I doubted if it's a good idea as well. Feel free to ignore this then please




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



[GitHub] [airflow] XD-DENG commented on a change in pull request #13847: Add ExasolToS3Operator

Posted by GitBox <gi...@apache.org>.
XD-DENG commented on a change in pull request #13847:
URL: https://github.com/apache/airflow/pull/13847#discussion_r563265306



##########
File path: airflow/providers/amazon/aws/transfers/exasol_to_s3.py
##########
@@ -0,0 +1,117 @@
+#
+# 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.
+"""Transfers data from Exasol database into a S3 Bucket."""
+
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.providers.exasol.hooks.exasol import ExasolHook
+from airflow.utils.decorators import apply_defaults
+
+
+class ExasolToS3Operator(BaseOperator):
+    """
+    Export data from Exasol database to AWS S3 bucket.
+
+    :param query_or_table: the sql statement to be executed or table name to export
+    :type query_or_table: str
+    :param key: S3 key that will point to the file
+    :type key: str
+    :param bucket_name: Name of the bucket in which to store the file
+    :type bucket_name: str
+    :param replace: A flag to decide whether or not to overwrite the key
+        if it already exists. If replace is False and the key exists, an
+        error will be raised.
+    :type replace: bool
+    :param encrypt: If True, the file will be encrypted on the server-side
+        by S3 and will be stored in an encrypted form while at rest in S3.
+    :type encrypt: bool
+    :param gzip: If True, the file will be compressed locally
+    :type gzip: bool
+    :param acl_policy: String specifying the canned ACL policy for the file being
+        uploaded to the S3 bucket.
+    :type acl_policy: str
+    :param query_params: Query parameters passed to underlying ``export_to_file``
+        method of ``ExaConnection``.

Review comment:
       LGTM




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